diff --git a/.github/workflows/ibis-backends.yml b/.github/workflows/ibis-backends.yml index 1de52e0b4a15f..873a3994dfebe 100644 --- a/.github/workflows/ibis-backends.yml +++ b/.github/workflows/ibis-backends.yml @@ -123,6 +123,12 @@ jobs: - postgres sys-deps: - libgeos-dev + - name: risingwave + title: Risingwave + services: + - risingwave + extras: + - risingwave - name: impala title: Impala serial: true @@ -218,6 +224,14 @@ jobs: - postgres sys-deps: - libgeos-dev + - os: windows-latest + backend: + name: risingwave + title: Risingwave + services: + - risingwave + extras: + - risingwave - os: windows-latest backend: name: postgres diff --git a/ci/schema/risingwave.sql b/ci/schema/risingwave.sql index cedfa8449d60f..251b689ada0db 100644 --- a/ci/schema/risingwave.sql +++ b/ci/schema/risingwave.sql @@ -1,27 +1,27 @@ SET RW_IMPLICIT_FLUSH=true; -DROP TABLE IF EXISTS diamonds CASCADE; - -CREATE TABLE diamonds ( - carat FLOAT, - cut TEXT, - color TEXT, - clarity TEXT, - depth FLOAT, +DROP TABLE IF EXISTS "diamonds" CASCADE; + +CREATE TABLE "diamonds" ( + "carat" FLOAT, + "cut" TEXT, + "color" TEXT, + "clarity" TEXT, + "depth" FLOAT, "table" FLOAT, - price BIGINT, - x FLOAT, - y FLOAT, - z FLOAT + "price" BIGINT, + "x" FLOAT, + "y" FLOAT, + "z" FLOAT ) WITH ( connector = 'posix_fs', match_pattern = 'diamonds.csv', posix_fs.root = '/data', ) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); -DROP TABLE IF EXISTS astronauts CASCADE; +DROP TABLE IF EXISTS "astronauts" CASCADE; -CREATE TABLE astronauts ( +CREATE TABLE "astronauts" ( "id" BIGINT, "number" BIGINT, "nationwide_number" BIGINT, @@ -52,12 +52,12 @@ CREATE TABLE astronauts ( posix_fs.root = '/data', ) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); -DROP TABLE IF EXISTS batting CASCADE; +DROP TABLE IF EXISTS "batting" CASCADE; -CREATE TABLE batting ( +CREATE TABLE "batting" ( "playerID" TEXT, "yearID" BIGINT, - stint BIGINT, + "stint" BIGINT, "teamID" TEXT, "lgID" TEXT, "G" BIGINT, @@ -83,71 +83,71 @@ CREATE TABLE batting ( posix_fs.root = '/data', ) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); -DROP TABLE IF EXISTS awards_players CASCADE; +DROP TABLE IF EXISTS "awards_players" CASCADE; -CREATE TABLE awards_players ( +CREATE TABLE "awards_players" ( "playerID" TEXT, "awardID" TEXT, "yearID" BIGINT, "lgID" TEXT, - tie TEXT, - notes TEXT + "tie" TEXT, + "notes" TEXT ) WITH ( connector = 'posix_fs', match_pattern = 'awards_players.csv', posix_fs.root = '/data', ) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); -DROP TABLE IF EXISTS functional_alltypes CASCADE; - -CREATE TABLE functional_alltypes ( - id INTEGER, - bool_col BOOLEAN, - tinyint_col SMALLINT, - smallint_col SMALLINT, - int_col INTEGER, - bigint_col BIGINT, - float_col REAL, - double_col DOUBLE PRECISION, - date_string_col TEXT, - string_col TEXT, - timestamp_col TIMESTAMP WITHOUT TIME ZONE, - year INTEGER, - month INTEGER +DROP TABLE IF EXISTS "functional_alltypes" CASCADE; + +CREATE TABLE "functional_alltypes" ( + "id" INTEGER, + "bool_col" BOOLEAN, + "tinyint_col" SMALLINT, + "smallint_col" SMALLINT, + "int_col" INTEGER, + "bigint_col" BIGINT, + "float_col" REAL, + "double_col" DOUBLE PRECISION, + "date_string_col" TEXT, + "string_col" TEXT, + "timestamp_col" TIMESTAMP WITHOUT TIME ZONE, + "year" INTEGER, + "month" INTEGER ) WITH ( connector = 'posix_fs', match_pattern = 'functional_alltypes.csv', posix_fs.root = '/data', ) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); -DROP TABLE IF EXISTS tzone CASCADE; +DROP TABLE IF EXISTS "tzone" CASCADE; -CREATE TABLE tzone ( - ts TIMESTAMP WITH TIME ZONE, - key TEXT, - value DOUBLE PRECISION +CREATE TABLE "tzone" ( + "ts" TIMESTAMP WITH TIME ZONE, + "key" TEXT, + "value" DOUBLE PRECISION ); -INSERT INTO tzone +INSERT INTO "tzone" SELECT CAST('2017-05-28 11:01:31.000400' AS TIMESTAMP WITH TIME ZONE) + - t * INTERVAL '1 day 1 second' AS ts, - CHR(97 + t) AS key, - t + t / 10.0 AS value - FROM generate_series(0, 9) AS t; - -DROP TABLE IF EXISTS array_types CASCADE; - -CREATE TABLE IF NOT EXISTS array_types ( - x BIGINT[], - y TEXT[], - z DOUBLE PRECISION[], - grouper TEXT, - scalar_column DOUBLE PRECISION, - multi_dim BIGINT[][] + t * INTERVAL '1 day 1 second' AS "ts", + CHR(97 + t) AS "key", + t + t / 10.0 AS "value" + FROM generate_series(0, 9) AS "t"; + +DROP TABLE IF EXISTS "array_types" CASCADE; + +CREATE TABLE IF NOT EXISTS "array_types" ( + "x" BIGINT[], + "y" TEXT[], + "z" DOUBLE PRECISION[], + "grouper" TEXT, + "scalar_column" DOUBLE PRECISION, + "multi_dim" BIGINT[][] ); -INSERT INTO array_types VALUES +INSERT INTO "array_types" VALUES (ARRAY[1, 2, 3], ARRAY['a', 'b', 'c'], ARRAY[1.0, 2.0, 3.0], 'a', 1.0, ARRAY[ARRAY[NULL::BIGINT, NULL, NULL], ARRAY[1, 2, 3]]), (ARRAY[4, 5], ARRAY['d', 'e'], ARRAY[4.0, 5.0], 'a', 2.0, ARRAY[]::BIGINT[][]), (ARRAY[6, NULL], ARRAY['f', NULL], ARRAY[6.0, NULL], 'a', 3.0, ARRAY[NULL, ARRAY[]::BIGINT[], NULL]), @@ -155,11 +155,11 @@ INSERT INTO array_types VALUES (ARRAY[2, NULL, 3], ARRAY['b', NULL, 'c'], NULL, 'b', 5.0, NULL), (ARRAY[4, NULL, NULL, 5], ARRAY['d', NULL, NULL, 'e'], ARRAY[4.0, NULL, NULL, 5.0], 'c', 6.0, ARRAY[ARRAY[1, 2, 3]]); -DROP TABLE IF EXISTS json_t CASCADE; +DROP TABLE IF EXISTS "json_t" CASCADE; -CREATE TABLE IF NOT EXISTS json_t (js JSONB); +CREATE TABLE IF NOT EXISTS "json_t" ("js" JSONB); -INSERT INTO json_t VALUES +INSERT INTO "json_t" VALUES ('{"a": [1,2,3,4], "b": 1}'), ('{"a":null,"b":2}'), ('{"a":"foo", "c":null}'), @@ -167,9 +167,9 @@ INSERT INTO json_t VALUES ('[42,47,55]'), ('[]'); -DROP TABLE IF EXISTS win CASCADE; -CREATE TABLE win (g TEXT, x BIGINT, y BIGINT); -INSERT INTO win VALUES +DROP TABLE IF EXISTS "win" CASCADE; +CREATE TABLE "win" ("g" TEXT, "x" BIGINT, "y" BIGINT); +INSERT INTO "win" VALUES ('a', 0, 3), ('a', 1, 2), ('a', 2, 0), diff --git a/ibis/backends/base/__init__.py b/ibis/backends/base/__init__.py index 7411077514d1a..445997da3b092 100644 --- a/ibis/backends/base/__init__.py +++ b/ibis/backends/base/__init__.py @@ -41,6 +41,7 @@ "datafusion": "postgres", # closest match see https://github.com/ibis-project/ibis/pull/7303#discussion_r1350223901 "exasol": "oracle", + "risingwave": "postgres", } _SQLALCHEMY_TO_SQLGLOT_DIALECT = { @@ -75,6 +76,7 @@ def __dir__(self) -> list[str]: ------- list[str] A list of the attributes and tables available in the database. + """ attrs = dir(type(self)) unqualified_tables = [self._unqualify(x) for x in self.tables] @@ -92,6 +94,7 @@ def __contains__(self, table: str) -> bool: ------- bool True if the given table is available in the current database. + """ return table in self.tables @@ -103,6 +106,7 @@ def tables(self) -> list[str]: ------- list[str] The list of tables in the database + """ return self.list_tables() @@ -118,6 +122,7 @@ def __getitem__(self, table: str) -> ir.Table: ------- Table Table expression + """ return self.table(table) @@ -133,6 +138,7 @@ def __getattr__(self, table: str) -> ir.Table: ------- Table Table expression + """ return self.table(table) @@ -150,6 +156,7 @@ def drop(self, force: bool = False) -> None: force If `True`, drop any objects that exist, and do not fail if the database does not exist. + """ self.client.drop_database(self.name, force=force) @@ -165,6 +172,7 @@ def table(self, name: str) -> ir.Table: ------- Table Table expression + """ qualified_name = self._qualify(name) return self.client.table(qualified_name, self.name) @@ -178,6 +186,7 @@ def list_tables(self, like=None, database=None): A pattern to use for listing tables. database The database to perform the list against + """ return self.client.list_tables(like, database=database or self.name) @@ -192,6 +201,7 @@ class TablesAccessor(collections.abc.Mapping): >>> con = ibis.sqlite.connect("example.db") >>> people = con.tables["people"] # access via index >>> people = con.tables.people # access via attribute + """ def __init__(self, backend: BaseBackend): @@ -276,6 +286,7 @@ def to_pandas( "no limit". The default is in `ibis/config.py`. kwargs Keyword arguments + """ return self.execute(expr, params=params, limit=limit, **kwargs) @@ -309,6 +320,7 @@ def to_pandas_batches( ------- Iterator[pd.DataFrame] An iterator of pandas `DataFrame`s. + """ from ibis.formats.pandas import PandasData @@ -354,6 +366,7 @@ def to_pyarrow( ------- Table A pyarrow table holding the results of the executed expression. + """ pa = self._import_pyarrow() self._run_pre_execute_hooks(expr) @@ -403,6 +416,7 @@ def to_pyarrow_batches( ------- results RecordBatchReader + """ raise NotImplementedError @@ -432,6 +446,7 @@ def to_torch( ------- dict[str, torch.Tensor] A dictionary of torch tensors, keyed by column name. + """ import torch @@ -463,6 +478,7 @@ def read_parquet( ------- ir.Table The just-registered table + """ raise NotImplementedError( f"{self.name} does not support direct registration of parquet data." @@ -487,6 +503,7 @@ def read_csv( ------- ir.Table The just-registered table + """ raise NotImplementedError( f"{self.name} does not support direct registration of CSV data." @@ -511,6 +528,7 @@ def read_json( ------- ir.Table The just-registered table + """ raise NotImplementedError( f"{self.name} does not support direct registration of JSON data." @@ -536,6 +554,7 @@ def read_delta( ------- ir.Table The just-registered table. + """ raise NotImplementedError( f"{self.name} does not support direct registration of DeltaLake tables." @@ -567,6 +586,7 @@ def to_parquet( Additional keyword arguments passed to pyarrow.parquet.ParquetWriter https://arrow.apache.org/docs/python/generated/pyarrow.parquet.ParquetWriter.html + """ self._import_pyarrow() import pyarrow.parquet as pq @@ -602,6 +622,7 @@ def to_csv( Additional keyword arguments passed to pyarrow.csv.CSVWriter https://arrow.apache.org/docs/python/generated/pyarrow.csv.CSVWriter.html + """ self._import_pyarrow() import pyarrow.csv as pcsv @@ -666,6 +687,7 @@ def list_databases(self, like: str | None = None) -> list[str]: list[str] The database names that exist in the current connection, that match the `like` pattern if provided. + """ @property @@ -685,6 +707,7 @@ def create_database(self, name: str, force: bool = False) -> None: Name of the new database. force If `False`, an exception is raised if the database already exists. + """ @abc.abstractmethod @@ -697,6 +720,7 @@ def drop_database(self, name: str, force: bool = False) -> None: Database to drop. force If `False`, an exception is raised if the database does not exist. + """ @@ -716,6 +740,7 @@ def create_schema( current database is used. force If `False`, an exception is raised if the schema exists. + """ @abc.abstractmethod @@ -733,6 +758,7 @@ def drop_schema( current database is used. force If `False`, an exception is raised if the schema does not exist. + """ @abc.abstractmethod @@ -755,6 +781,7 @@ def list_schemas( list[str] The schema names that exist in the current connection, that match the `like` pattern if provided. + """ @property @@ -814,6 +841,7 @@ def db_identity(self) -> str: ------- Hashable Database identity + """ parts = [self.__class__] parts.extend(self._con_args) @@ -844,6 +872,7 @@ def connect(self, *args, **kwargs) -> BaseBackend: ------- BaseBackend An instance of the backend + """ new_backend = self.__class__(*args, **kwargs) new_backend.reconnect() @@ -880,6 +909,7 @@ def database(self, name: str | None = None) -> Database: ------- Database A database object for the specified database. + """ return Database(name=name or self.current_database, client=self) @@ -905,6 +935,7 @@ def _filter_with_like(values: Iterable[str], like: str | None = None) -> list[st ------- list[str] Names filtered by the `like` pattern. + """ if like is None: return sorted(values) @@ -933,6 +964,7 @@ def list_tables( ------- list[str] The list of the table names that match the pattern `like`. + """ @abc.abstractmethod @@ -950,6 +982,7 @@ def table(self, name: str, database: str | None = None) -> ir.Table: ------- Table Table expression + """ @functools.cached_property @@ -963,6 +996,7 @@ def tables(self): >>> con = ibis.sqlite.connect("example.db") >>> people = con.tables["people"] # access via index >>> people = con.tables.people # access via attribute + """ return TablesAccessor(self) @@ -980,6 +1014,7 @@ def version(self) -> str: ------- str The backend version + """ @classmethod @@ -1088,6 +1123,7 @@ def create_table( ------- Table The table that was created. + """ @abc.abstractmethod @@ -1108,6 +1144,7 @@ def drop_table( Name of the database where the table exists, if not the default. force If `False`, an exception is raised if the table does not exist. + """ raise NotImplementedError( f'Backend "{self.name}" does not implement "drop_table"' @@ -1122,6 +1159,7 @@ def rename_table(self, old_name: str, new_name: str) -> None: The old name of the table. new_name The new name of the table. + """ raise NotImplementedError( f'Backend "{self.name}" does not implement "rename_table"' @@ -1154,6 +1192,7 @@ def create_view( ------- Table The view that was created. + """ @abc.abstractmethod @@ -1170,6 +1209,7 @@ def drop_view( Name of the database where the view exists, if not the default. force If `False`, an exception is raised if the view does not exist. + """ @classmethod @@ -1194,6 +1234,7 @@ def has_operation(cls, operation: type[ops.Value]) -> bool: False >>> ibis.postgres.has_operation(ops.ArrayIndex) True + """ raise NotImplementedError( f"{cls.name} backend has not implemented `has_operation` API" @@ -1228,6 +1269,7 @@ def _release_cached(self, expr: ir.CachedTable) -> None: ---------- expr Cached expression to release + """ del self._query_cache[expr.op()] @@ -1268,6 +1310,7 @@ def _get_backend_names() -> frozenset[str]: If a `set` is used, then any in-place modifications to the set are visible to every caller of this function. + """ if sys.version_info < (3, 10): @@ -1325,6 +1368,7 @@ def connect(resource: Path | str, **kwargs: Any) -> BaseBackend: >>> con = ibis.connect( ... "bigquery://my-project/my-dataset" ... ) # quartodoc: +SKIP # doctest: +SKIP + """ url = resource = str(resource) diff --git a/ibis/backends/base/sqlglot/compiler.py b/ibis/backends/base/sqlglot/compiler.py index 0f5b3e738b6f4..db90f5ea80610 100644 --- a/ibis/backends/base/sqlglot/compiler.py +++ b/ibis/backends/base/sqlglot/compiler.py @@ -141,9 +141,9 @@ def parenthesize(op, arg): C = ColGen() F = FuncGen() -NULL = sge.NULL -FALSE = sge.FALSE -TRUE = sge.TRUE +NULL = sge.Null() +FALSE = sge.false() +TRUE = sge.true() STAR = sge.Star() @@ -251,6 +251,7 @@ def translate(self, op, *, params: Mapping[ir.Value, Any]) -> sge.Expression: ------- sqlglot.expressions.Expression A sqlglot expression + """ # substitute parameters immediately to avoid having to define a # ScalarParameter translation rule diff --git a/ibis/backends/base/sqlglot/datatypes.py b/ibis/backends/base/sqlglot/datatypes.py index 370afc7a06b91..db1bae762c9aa 100644 --- a/ibis/backends/base/sqlglot/datatypes.py +++ b/ibis/backends/base/sqlglot/datatypes.py @@ -394,6 +394,24 @@ def _from_ibis_Map(cls, dtype: dt.Map) -> sge.DataType: return sge.DataType(this=typecode.HSTORE) +class RisingWaveType(PostgresType): + dialect = "risingwave" + + @classmethod + def _from_ibis_Timestamp(cls, dtype: dt.Timestamp) -> sge.DataType: + if dtype.timezone is not None: + return sge.DataType(this=typecode.TIMESTAMPTZ) + return sge.DataType(this=typecode.TIMESTAMP) + + @classmethod + def _from_ibis_Decimal(cls, dtype: dt.Decimal) -> sge.DataType: + return sge.DataType(this=typecode.DECIMAL) + + @classmethod + def _from_ibis_UUID(cls, dtype: dt.UUID) -> sge.DataType: + return sge.DataType(this=typecode.VARCHAR) + + class DataFusionType(PostgresType): unknown_type_strings = { "utf8": dt.string, diff --git a/ibis/backends/clickhouse/compiler.py b/ibis/backends/clickhouse/compiler.py index e4121a2ef9b21..6439fb99a5e27 100644 --- a/ibis/backends/clickhouse/compiler.py +++ b/ibis/backends/clickhouse/compiler.py @@ -209,24 +209,26 @@ def visit_Hash(self, op, *, arg): @visit_node.register(ops.HashBytes) def visit_HashBytes(self, op, *, arg, how): - supported_algorithms = frozenset( - ( - "MD5", - "halfMD5", - "SHA1", - "SHA224", - "SHA256", - "intHash32", - "intHash64", - "cityHash64", - "sipHash64", - "sipHash128", - ) - ) - if how not in supported_algorithms: + supported_algorithms = { + "md5": "MD5", + "MD5": "MD5", + "halfMD5": "halfMD5", + "SHA1": "SHA1", + "sha1": "SHA1", + "SHA224": "SHA224", + "sha224": "SHA224", + "SHA256": "SHA256", + "sha256": "SHA256", + "intHash32": "intHash32", + "intHash64": "intHash64", + "cityHash64": "cityHash64", + "sipHash64": "sipHash64", + "sipHash128": "sipHash128", + } + if (funcname := supported_algorithms.get(how)) is None: raise com.UnsupportedOperationError(f"Unsupported hash algorithm {how}") - return self.f[how](arg) + return self.f[funcname](arg) @visit_node.register(ops.IntervalFromInteger) def visit_IntervalFromInteger(self, op, *, arg, unit): diff --git a/ibis/backends/conftest.py b/ibis/backends/conftest.py index 949ae7752b943..5fe94de294181 100644 --- a/ibis/backends/conftest.py +++ b/ibis/backends/conftest.py @@ -26,6 +26,7 @@ if TYPE_CHECKING: from collections.abc import Iterable + from ibis.backends.tests.base import BackendTest diff --git a/ibis/backends/duckdb/compiler.py b/ibis/backends/duckdb/compiler.py index e42bb5e733b09..a21735474c6a4 100644 --- a/ibis/backends/duckdb/compiler.py +++ b/ibis/backends/duckdb/compiler.py @@ -329,10 +329,6 @@ def visit_GeoConvert(self, op, *, arg, source, target): # matches the behavior of the equivalent geopandas functionality return self.f.st_transform(arg, source, target, True) - @visit_node.register(ops.HexDigest) - def visit_HexDigest(self, op, *, arg, how): - return self.f[how](arg) - @visit_node.register(ops.TimestampNow) def visit_TimestampNow(self, op): """DuckDB current timestamp defaults to timestamp + tz.""" @@ -349,6 +345,13 @@ def visit_Quantile(self, op, *, arg, quantile, where): funcname = f"percentile_{suffix}" return self.agg[funcname](arg, quantile, where=where) + @visit_node.register(ops.HexDigest) + def visit_HexDigest(self, op, *, arg, how): + if how in ("md5", "sha256"): + return getattr(self.f, how)(arg) + else: + raise NotImplementedError(f"No available hashing function for {how}") + _SIMPLE_OPS = { ops.ArrayPosition: "list_indexof", diff --git a/ibis/backends/mssql/compiler.py b/ibis/backends/mssql/compiler.py index 4609d241610da..d0c4470d7489a 100644 --- a/ibis/backends/mssql/compiler.py +++ b/ibis/backends/mssql/compiler.py @@ -376,6 +376,36 @@ def visit_Not(self, op, *, arg): return sge.FALSE if arg == sge.TRUE else sge.TRUE return self.if_(arg, 1, 0).eq(0) + @visit_node.register(ops.HashBytes) + def visit_HashBytes(self, op, *, arg, how): + if how in ("md5", "sha1"): + return self.f.hashbytes(how, arg) + elif how == "sha256": + return self.f.hashbytes("sha2_256", arg) + elif how == "sha512": + return self.f.hashbytes("sha2_512", arg) + else: + raise NotImplementedError(how) + + @visit_node.register(ops.HexDigest) + def visit_HexDigest(self, op, *, arg, how): + if how in ("md5", "sha1"): + hashbinary = self.f.hashbytes(how, arg) + elif how == "sha256": + hashbinary = self.f.hashbytes("sha2_256", arg) + elif how == "sha512": + hashbinary = self.f.hashbytes("sha2_512", arg) + else: + raise NotImplementedError(how) + + # mssql uppercases the hexdigest which is inconsistent with several other + # implementations and inconsistent with Python, so lowercase it. + return self.f.lower( + self.f.convert( + sge.Literal(this="VARCHAR(MAX)", is_string=False), hashbinary, 2 + ) + ) + @visit_node.register(ops.Any) @visit_node.register(ops.All) @visit_node.register(ops.ApproxMedian) diff --git a/ibis/backends/postgres/__init__.py b/ibis/backends/postgres/__init__.py index 0f51218c7f7c6..10365658518b2 100644 --- a/ibis/backends/postgres/__init__.py +++ b/ibis/backends/postgres/__init__.py @@ -44,6 +44,7 @@ def _verify_source_line(func_name: str, line: str): class Backend(SQLGlotBackend): name = "postgres" + dialect = "postgres" compiler = PostgresCompiler() supports_python_udfs = True @@ -61,6 +62,7 @@ def _from_url(self, url: str, **kwargs): ------- BaseBackend A backend instance + """ url = urlparse(url) @@ -106,7 +108,7 @@ def _register_in_memory_table(self, op: ops.InMemoryTable) -> None: schema = op.schema if null_columns := [col for col, dtype in schema.items() if dtype.is_null()]: raise exc.IbisTypeError( - "Postgres cannot yet reliably handle `null` typed columns; " + f"{self.name} cannot yet reliably handle `null` typed columns; " f"got null typed columns: {null_columns}" ) @@ -137,18 +139,18 @@ def _register_in_memory_table(self, op: ops.InMemoryTable) -> None: ), properties=sg.exp.Properties(expressions=[sge.TemporaryProperty()]), ) - create_stmt_sql = create_stmt.sql(self.name) + create_stmt_sql = create_stmt.sql(self.dialect) columns = schema.keys() df = op.data.to_frame() data = df.itertuples(index=False) cols = ", ".join( - ident.sql(self.name) + ident.sql(self.dialect) for ident in map(partial(sg.to_identifier, quoted=quoted), columns) ) specs = ", ".join(repeat("%s", len(columns))) table = sg.table(name, quoted=quoted) - sql = f"INSERT INTO {table.sql(self.name)} ({cols}) VALUES ({specs})" + sql = f"INSERT INTO {table.sql(self.dialect)} ({cols}) VALUES ({specs})" with self.begin() as cur: cur.execute(create_stmt_sql) extras.execute_batch(cur, sql, data, 128) @@ -254,6 +256,7 @@ def do_connect( timestamp_col : timestamp year : int32 month : int32 + """ self.con = psycopg2.connect( @@ -291,6 +294,7 @@ def list_tables( The `schema` parameter does **not** refer to the column names and types of `table`. ::: + """ if database is not None: util.warn_deprecated( @@ -314,7 +318,7 @@ def list_tables( .from_(sg.table("tables", db="information_schema")) .distinct() .where(*conditions) - .sql(self.name) + .sql(self.dialect) ) with self._safe_raw_sql(sql) as cur: @@ -447,10 +451,10 @@ def _compile_builtin_udf(self, udf_node: ops.ScalarUDF) -> None: """No op.""" def _compile_pyarrow_udf(self, udf_node: ops.ScalarUDF) -> None: - raise NotImplementedError("pyarrow UDFs are not supported in Postgres") + raise NotImplementedError(f"pyarrow UDFs are not supported in {self.name}") def _compile_pandas_udf(self, udf_node: ops.ScalarUDF) -> str: - raise NotImplementedError("pandas UDFs are not supported in Postgres") + raise NotImplementedError(f"pandas UDFs are not supported in {self.name}") def _define_udf_translation_rules(self, expr: ir.Expr) -> None: """No-op, these are defined in the compiler.""" @@ -535,11 +539,11 @@ def _metadata(self, query: str) -> Iterable[tuple[str, dt.DataType]]: create_stmt = sge.Create( kind="VIEW", this=sg.table(name), - expression=sg.parse_one(query, read=self.name), + expression=sg.parse_one(query, read=self.dialect), properties=sge.Properties(expressions=[sge.TemporaryProperty()]), ) drop_stmt = sge.Drop(kind="VIEW", this=sg.table(name), exists=True).sql( - self.name + self.dialect ) with self._safe_raw_sql(create_stmt): @@ -555,7 +559,7 @@ def create_schema( ) -> None: if database is not None and database != self.current_database: raise exc.UnsupportedOperationError( - "Postgres does not support creating a schema in a different database" + f"{self.name} does not support creating a schema in a different database" ) sql = sge.Create( kind="SCHEMA", this=sg.table(name, catalog=database), exists=force @@ -572,7 +576,7 @@ def drop_schema( ) -> None: if database is not None and database != self.current_database: raise exc.UnsupportedOperationError( - "Postgres does not support dropping a schema in a different database" + f"{self.name} does not support dropping a schema in a different database" ) sql = sge.Drop( @@ -614,13 +618,14 @@ def create_table( overwrite If `True`, replace the table if it already exists, otherwise fail if the table exists + """ if obj is None and schema is None: raise ValueError("Either `obj` or `schema` must be specified") if database is not None and database != self.current_database: raise com.UnsupportedOperationError( - "Creating tables in other databases is not supported by Postgres" + f"Creating tables in other databases is not supported by {self.name}" ) else: database = None @@ -672,15 +677,15 @@ def create_table( this = sg.table(name, catalog=database, quoted=self.compiler.quoted) with self._safe_raw_sql(create_stmt) as cur: if query is not None: - insert_stmt = sge.Insert(this=table, expression=query).sql(self.name) + insert_stmt = sge.Insert(this=table, expression=query).sql(self.dialect) cur.execute(insert_stmt) if overwrite: cur.execute( - sge.Drop(kind="TABLE", this=this, exists=True).sql(self.name) + sge.Drop(kind="TABLE", this=this, exists=True).sql(self.dialect) ) cur.execute( - f"ALTER TABLE IF EXISTS {table.sql(self.name)} RENAME TO {this.sql(self.name)}" + f"ALTER TABLE IF EXISTS {table.sql(self.dialect)} RENAME TO {this.sql(self.dialect)}" ) if schema is None: @@ -700,7 +705,7 @@ def drop_table( ) -> None: if database is not None and database != self.current_database: raise com.UnsupportedOperationError( - "Droppping tables in other databases is not supported by Postgres" + f"Droppping tables in other databases is not supported by {self.name}" ) else: database = None @@ -721,7 +726,7 @@ def _safe_raw_sql(self, *args, **kwargs): def raw_sql(self, query: str | sg.Expression, **kwargs: Any) -> Any: with contextlib.suppress(AttributeError): - query = query.sql(dialect=self.name) + query = query.sql(dialect=self.dialect) con = self.con cursor = con.cursor() @@ -771,7 +776,8 @@ def truncate_table(self, name: str, database: str | None = None) -> None: Table name database Schema name + """ - ident = sg.table(name, db=database).sql(self.name) + ident = sg.table(name, db=database).sql(self.dialect) with self._safe_raw_sql(f"TRUNCATE TABLE {ident}"): pass diff --git a/ibis/backends/pyspark/compiler.py b/ibis/backends/pyspark/compiler.py index bc2bbf2b7584f..b4e75c959735e 100644 --- a/ibis/backends/pyspark/compiler.py +++ b/ibis/backends/pyspark/compiler.py @@ -457,6 +457,17 @@ def visit_JoinLink(self, op, **kwargs): def visit_Undefined(self, op, **_): raise com.OperationNotDefinedError(type(op).__name__) + @visit_node.register(ops.HexDigest) + def visit_HexDigest(self, op, *, arg, how): + if how == "md5": + return self.f.md5(arg) + elif how == "sha1": + return self.f.sha1(arg) + elif how in ("sha256", "sha512"): + return self.f.sha2(arg, int(how[-3:])) + else: + raise NotImplementedError(f"No available hashing function for {how}") + _SIMPLE_OPS = { ops.ArrayDistinct: "array_distinct", diff --git a/ibis/backends/risingwave/__init__.py b/ibis/backends/risingwave/__init__.py index 04de491f6dfef..996f776fd12e9 100644 --- a/ibis/backends/risingwave/__init__.py +++ b/ibis/backends/risingwave/__init__.py @@ -2,36 +2,40 @@ from __future__ import annotations -import inspect -from typing import TYPE_CHECKING, Callable, Literal - -import sqlalchemy as sa - -import ibis.common.exceptions as exc +import atexit +from functools import partial +from itertools import repeat +from typing import TYPE_CHECKING + +import psycopg2 +import sqlglot as sg +import sqlglot.expressions as sge +from psycopg2 import extras + +import ibis +import ibis.common.exceptions as com import ibis.expr.operations as ops +import ibis.expr.types as ir from ibis import util -from ibis.backends.base.sql.alchemy import AlchemyCanCreateSchema, BaseAlchemyBackend +from ibis.backends.postgres import Backend as PostgresBackend from ibis.backends.risingwave.compiler import RisingwaveCompiler -from ibis.backends.risingwave.datatypes import RisingwaveType -from ibis.common.exceptions import InvalidDecoratorError +from ibis.backends.risingwave.dialect import RisingWave as RisingWaveDialect if TYPE_CHECKING: - from collections.abc import Iterable - - import ibis.expr.datatypes as dt + import pandas as pd + import pyarrow as pa def _verify_source_line(func_name: str, line: str): if line.startswith("@"): - raise InvalidDecoratorError(func_name, line) + raise com.InvalidDecoratorError(func_name, line) return line -class Backend(BaseAlchemyBackend, AlchemyCanCreateSchema): +class Backend(PostgresBackend): name = "risingwave" - compiler = RisingwaveCompiler - supports_temporary_tables = False - supports_create_or_replace = False + dialect = RisingWaveDialect + compiler = RisingwaveCompiler() supports_python_udfs = False def do_connect( @@ -42,10 +46,8 @@ def do_connect( port: int = 5432, database: str | None = None, schema: str | None = None, - url: str | None = None, - driver: Literal["psycopg2"] = "psycopg2", ) -> None: - """Create an Ibis client connected to Risingwave database. + """Create an Ibis client connected to RisingWave database. Parameters ---------- @@ -60,13 +62,7 @@ def do_connect( database Database to connect to schema - Risingwave schema to use. If `None`, use the default `search_path`. - url - SQLAlchemy connection string. - - If passed, the other connection arguments are ignored. - driver - Database driver + RisingWave schema to use. If `None`, use the default `search_path`. Examples -------- @@ -98,185 +94,199 @@ def do_connect( timestamp_col : timestamp year : int32 month : int32 + """ - if driver != "psycopg2": - raise NotImplementedError("psycopg2 is currently the only supported driver") - alchemy_url = self._build_alchemy_url( - url=url, + self.con = psycopg2.connect( host=host, port=port, user=user, password=password, database=database, - driver=f"risingwave+{driver}", - ) - - connect_args = {} - if schema is not None: - connect_args["options"] = f"-csearch_path={schema}" - - engine = sa.create_engine( - alchemy_url, connect_args=connect_args, poolclass=sa.pool.StaticPool + options=(f"-csearch_path={schema}" * (schema is not None)) or None, ) - @sa.event.listens_for(engine, "connect") - def connect(dbapi_connection, connection_record): - with dbapi_connection.cursor() as cur: - cur.execute("SET TIMEZONE = UTC") + with self.begin() as cur: + cur.execute("SET TIMEZONE = UTC") - super().do_connect(engine) + self._temp_views = set() - def list_tables(self, like=None, schema=None): - """List the tables in the database. + def create_table( + self, + name: str, + obj: pd.DataFrame | pa.Table | ir.Table | None = None, + *, + schema: ibis.Schema | None = None, + database: str | None = None, + temp: bool = False, + overwrite: bool = False, + ): + """Create a table in Risingwave. Parameters ---------- - like - A pattern to use for listing tables. + name + Name of the table to create + obj + The data with which to populate the table; optional, but at least + one of `obj` or `schema` must be specified schema - The schema to perform the list against. - - ::: {.callout-warning} - ## `schema` refers to database hierarchy + The schema of the table to create; optional, but at least one of + `obj` or `schema` must be specified + database + The name of the database in which to create the table; if not + passed, the current database is used. + temp + Create a temporary table + overwrite + If `True`, replace the table if it already exists, otherwise fail + if the table exists - The `schema` parameter does **not** refer to the column names and - types of `table`. - ::: """ - tables = self.inspector.get_table_names(schema=schema) - views = self.inspector.get_view_names(schema=schema) - return self._filter_with_like(tables + views, like) - - def list_databases(self, like=None) -> list[str]: - # http://dba.stackexchange.com/a/1304/58517 - dbs = sa.table( - "pg_database", - sa.column("datname", sa.TEXT()), - sa.column("datistemplate", sa.BOOLEAN()), - schema="pg_catalog", + if obj is None and schema is None: + raise ValueError("Either `obj` or `schema` must be specified") + + if database is not None and database != self.current_database: + raise com.UnsupportedOperationError( + f"Creating tables in other databases is not supported by {self.name}" + ) + else: + database = None + + properties = [] + + if temp: + properties.append(sge.TemporaryProperty()) + + if obj is not None: + if not isinstance(obj, ir.Expr): + table = ibis.memtable(obj) + else: + table = obj + + self._run_pre_execute_hooks(table) + + query = self._to_sqlglot(table) + else: + query = None + + column_defs = [ + sge.ColumnDef( + this=sg.to_identifier(colname, quoted=self.compiler.quoted), + kind=self.compiler.type_mapper.from_ibis(typ), + constraints=( + None + if typ.nullable + else [sge.ColumnConstraint(kind=sge.NotNullColumnConstraint())] + ), + ) + for colname, typ in (schema or table.schema()).items() + ] + + if overwrite: + temp_name = util.gen_name(f"{self.name}_table") + else: + temp_name = name + + table = sg.table(temp_name, catalog=database, quoted=self.compiler.quoted) + target = sge.Schema(this=table, expressions=column_defs) + + create_stmt = sge.Create( + kind="TABLE", + this=target, + properties=sge.Properties(expressions=properties), ) - query = sa.select(dbs.c.datname).where(sa.not_(dbs.c.datistemplate)) - with self.begin() as con: - databases = list(con.execute(query).scalars()) - - return self._filter_with_like(databases, like) - - @property - def current_database(self) -> str: - return self._scalar_query(sa.select(sa.func.current_database())) - - @property - def current_schema(self) -> str: - return self._scalar_query(sa.select(sa.func.current_schema())) - - def function(self, name: str, *, schema: str | None = None) -> Callable: - query = sa.text( - """ -SELECT - n.nspname as schema, - pg_catalog.pg_get_function_result(p.oid) as return_type, - string_to_array(pg_catalog.pg_get_function_arguments(p.oid), ', ') as signature, - CASE p.prokind - WHEN 'a' THEN 'agg' - WHEN 'w' THEN 'window' - WHEN 'p' THEN 'proc' - ELSE 'func' - END as "Type" -FROM pg_catalog.pg_proc p -LEFT JOIN pg_catalog.pg_namespace n ON n.oid = p.pronamespace -WHERE p.proname = :name -""" - + "AND n.nspname OPERATOR(pg_catalog.~) :schema COLLATE pg_catalog.default" - * (schema is not None) - ).bindparams(name=name, schema=f"^({schema})$") - - def split_name_type(arg: str) -> tuple[str, dt.DataType]: - name, typ = arg.split(" ", 1) - return name, RisingwaveType.from_string(typ) - - with self.begin() as con: - rows = con.execute(query).mappings().fetchall() - - if not rows: - name = f"{schema}.{name}" if schema else name - raise exc.MissingUDFError(name) - elif len(rows) > 1: - raise exc.AmbiguousUDFError(name) - - [row] = rows - return_type = RisingwaveType.from_string(row["return_type"]) - signature = list(map(split_name_type, row["signature"])) - - # dummy callable - def fake_func(*args, **kwargs): - ... - - fake_func.__name__ = name - fake_func.__signature__ = inspect.Signature( - [ - inspect.Parameter( - name, kind=inspect.Parameter.POSITIONAL_OR_KEYWORD, annotation=typ + + this = sg.table(name, catalog=database, quoted=self.compiler.quoted) + with self._safe_raw_sql(create_stmt) as cur: + if query is not None: + insert_stmt = sge.Insert(this=table, expression=query).sql(self.dialect) + cur.execute(insert_stmt) + + if overwrite: + cur.execute( + sge.Drop(kind="TABLE", this=this, exists=True).sql(self.dialect) ) - for name, typ in signature - ], - return_annotation=return_type, - ) - fake_func.__annotations__ = {"return": return_type, **dict(signature)} - op = ops.udf.scalar.builtin(fake_func, schema=schema) - return op - - def _metadata(self, query: str) -> Iterable[tuple[str, dt.DataType]]: - name = util.gen_name("risingwave_metadata") - type_info_sql = """\ - SELECT - attname, - format_type(atttypid, atttypmod) AS type - FROM pg_attribute - WHERE attrelid = CAST(:name AS regclass) - AND attnum > 0 - AND NOT attisdropped - ORDER BY attnum""" - if self.inspector.has_table(query): - query = f"TABLE {query}" - - text = sa.text(type_info_sql).bindparams(name=name) - with self.begin() as con: - con.exec_driver_sql(f"CREATE VIEW IF NOT EXISTS {name} AS {query}") - try: - yield from ( - (col, RisingwaveType.from_string(typestr)) - for col, typestr in con.execute(text) + cur.execute( + f"ALTER TABLE {table.sql(self.dialect)} RENAME TO {this.sql(self.dialect)}" ) - finally: - con.exec_driver_sql(f"DROP VIEW IF EXISTS {name}") - def _get_temp_view_definition( - self, name: str, definition: sa.sql.compiler.Compiled - ) -> str: - yield f"DROP VIEW IF EXISTS {name}" - yield f"CREATE TEMPORARY VIEW {name} AS {definition}" - - def create_schema( - self, name: str, database: str | None = None, force: bool = False - ) -> None: - if database is not None and database != self.current_database: - raise exc.UnsupportedOperationError( - "Risingwave does not support creating a schema in a different database" + if schema is None: + return self.table(name, schema=database) + + # preserve the input schema if it was provided + return ops.DatabaseTable( + name, schema=schema, source=self, namespace=ops.Namespace(database=database) + ).to_expr() + + def _get_temp_view_definition(self, name: str, definition): + drop = sge.Drop( + kind="VIEW", exists=True, this=sg.table(name), cascade=True + ).sql(self.dialect) + + create = sge.Create( + this=sg.to_identifier(name, quoted=self.compiler.quoted), + kind="VIEW", + expression=definition, + replace=False, + ).sql(self.dialect) + + atexit.register(self._clean_up_tmp_view, name) + return f"{drop}; {create}" + + def _clean_up_tmp_view(self, name: str) -> None: + drop = sge.Drop( + kind="VIEW", exists=True, this=sg.table(name), cascade=True + ).sql(self.dialect) + with self.begin() as bind: + bind.execute(drop) + + def _register_in_memory_table(self, op: ops.InMemoryTable) -> None: + schema = op.schema + if null_columns := [col for col, dtype in schema.items() if dtype.is_null()]: + raise com.IbisTypeError( + f"{self.name} cannot yet reliably handle `null` typed columns; " + f"got null typed columns: {null_columns}" ) - if_not_exists = "IF NOT EXISTS " * force - name = self._quote(name) - with self.begin() as con: - con.exec_driver_sql(f"CREATE SCHEMA {if_not_exists}{name}") - def drop_schema( - self, name: str, database: str | None = None, force: bool = False - ) -> None: - if database is not None and database != self.current_database: - raise exc.UnsupportedOperationError( - "Risingwave does not support dropping a schema in a different database" + # only register if we haven't already done so + if (name := op.name) not in self.list_tables(): + quoted = self.compiler.quoted + column_defs = [ + sg.exp.ColumnDef( + this=sg.to_identifier(colname, quoted=quoted), + kind=self.compiler.type_mapper.from_ibis(typ), + constraints=( + None + if typ.nullable + else [ + sg.exp.ColumnConstraint( + kind=sg.exp.NotNullColumnConstraint() + ) + ] + ), + ) + for colname, typ in schema.items() + ] + + create_stmt = sg.exp.Create( + kind="TABLE", + this=sg.exp.Schema( + this=sg.to_identifier(name, quoted=quoted), expressions=column_defs + ), + ) + create_stmt_sql = create_stmt.sql(self.dialect) + + columns = schema.keys() + df = op.data.to_frame() + data = df.itertuples(index=False) + cols = ", ".join( + ident.sql(self.dialect) + for ident in map(partial(sg.to_identifier, quoted=quoted), columns) ) - name = self._quote(name) - if_exists = "IF EXISTS " * force - with self.begin() as con: - con.exec_driver_sql(f"DROP SCHEMA {if_exists}{name}") + specs = ", ".join(repeat("%s", len(columns))) + table = sg.table(name, quoted=quoted) + sql = f"INSERT INTO {table.sql(self.dialect)} ({cols}) VALUES ({specs})" + with self.begin() as cur: + cur.execute(create_stmt_sql) + extras.execute_batch(cur, sql, data, 128) diff --git a/ibis/backends/risingwave/compiler.py b/ibis/backends/risingwave/compiler.py index b4bcd9c0b9d5c..5bc7bfef2f5bc 100644 --- a/ibis/backends/risingwave/compiler.py +++ b/ibis/backends/risingwave/compiler.py @@ -1,34 +1,104 @@ from __future__ import annotations +from functools import singledispatchmethod + +import sqlglot.expressions as sge +from public import public + +import ibis.common.exceptions as com +import ibis.expr.datashape as ds +import ibis.expr.datatypes as dt import ibis.expr.operations as ops -from ibis.backends.base.sql.alchemy import AlchemyCompiler, AlchemyExprTranslator -from ibis.backends.risingwave.datatypes import RisingwaveType -from ibis.backends.risingwave.registry import operation_registry -from ibis.expr.rewrites import rewrite_sample +from ibis.backends.base.sqlglot.datatypes import RisingWaveType +from ibis.backends.postgres.compiler import PostgresCompiler +from ibis.backends.risingwave.dialect import RisingWave # noqa: F401 + + +@public +class RisingwaveCompiler(PostgresCompiler): + __slots__ = () + + dialect = "risingwave" + name = "risingwave" + type_mapper = RisingWaveType + + @singledispatchmethod + def visit_node(self, op, **kwargs): + return super().visit_node(op, **kwargs) + + @visit_node.register(ops.Correlation) + def visit_Correlation(self, op, *, left, right, how, where): + if how == "sample": + raise com.UnsupportedOperationError( + f"{self.name} only implements `pop` correlation coefficient" + ) + return super().visit_Correlation( + op, left=left, right=right, how=how, where=where + ) + + @visit_node.register(ops.TimestampTruncate) + @visit_node.register(ops.DateTruncate) + @visit_node.register(ops.TimeTruncate) + def visit_TimestampTruncate(self, op, *, arg, unit): + unit_mapping = { + "Y": "year", + "Q": "quarter", + "M": "month", + "W": "week", + "D": "day", + "h": "hour", + "m": "minute", + "s": "second", + "ms": "milliseconds", + "us": "microseconds", + } + + if (unit := unit_mapping.get(unit.short)) is None: + raise com.UnsupportedOperationError(f"Unsupported truncate unit {unit}") + + return self.f.date_trunc(unit, arg) + @visit_node.register(ops.IntervalFromInteger) + def visit_IntervalFromInteger(self, op, *, arg, unit): + if op.arg.shape == ds.scalar: + return sge.Interval(this=arg, unit=self.v[unit.name]) + elif op.arg.shape == ds.columnar: + return arg * sge.Interval(this=sge.convert(1), unit=self.v[unit.name]) + else: + raise ValueError("Invalid shape for converting to interval") -class RisingwaveExprTranslator(AlchemyExprTranslator): - _registry = operation_registry.copy() - _rewrites = AlchemyExprTranslator._rewrites.copy() - _has_reduction_filter_syntax = True - _supports_tuple_syntax = True - _dialect_name = "risingwave" + def visit_NonNullLiteral(self, op, *, value, dtype): + if dtype.is_binary(): + return self.cast("".join(map(r"\x{:0>2x}".format, value)), dt.binary) + elif dtype.is_date(): + return self.cast(value.isoformat(), dtype) + elif dtype.is_json(): + return sge.convert(str(value)) + return None - # it does support it, but we can't use it because of support for pivot - supports_unnest_in_select = False + @visit_node.register(ops.DateFromYMD) + @visit_node.register(ops.Mode) + def visit_Undefined(self, op, **_): + raise com.OperationNotDefinedError(type(op).__name__) - type_mapper = RisingwaveType +_SIMPLE_OPS = { + ops.First: "first_value", + ops.Last: "last_value", +} -rewrites = RisingwaveExprTranslator.rewrites +for _op, _name in _SIMPLE_OPS.items(): + assert isinstance(type(_op), type), type(_op) + if issubclass(_op, ops.Reduction): + @RisingwaveCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, where, **kw): + return self.agg[_name](*kw.values(), where=where) -@rewrites(ops.Any) -@rewrites(ops.All) -def _any_all_no_op(expr): - return expr + else: + @RisingwaveCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, **kw): + return self.f[_name](*kw.values()) -class RisingwaveCompiler(AlchemyCompiler): - translator_class = RisingwaveExprTranslator - rewrites = AlchemyCompiler.rewrites | rewrite_sample + setattr(RisingwaveCompiler, f"visit_{_op.__name__}", _fmt) diff --git a/ibis/backends/risingwave/dialect.py b/ibis/backends/risingwave/dialect.py new file mode 100644 index 0000000000000..2237c2a4d188c --- /dev/null +++ b/ibis/backends/risingwave/dialect.py @@ -0,0 +1,35 @@ +from __future__ import annotations + +import sqlglot.expressions as sge +from sqlglot import generator +from sqlglot.dialects import Postgres + + +class RisingWave(Postgres): + # Need to disable timestamp precision + # No "or replace" allowed in create statements + # no "not null" clause for column constraints + + class Generator(generator.Generator): + SINGLE_STRING_INTERVAL = True + RENAME_TABLE_WITH_DB = False + LOCKING_READS_SUPPORTED = True + JOIN_HINTS = False + TABLE_HINTS = False + QUERY_HINTS = False + NVL2_SUPPORTED = False + PARAMETER_TOKEN = "$" + TABLESAMPLE_SIZE_IS_ROWS = False + TABLESAMPLE_SEED_KEYWORD = "REPEATABLE" + SUPPORTS_SELECT_INTO = True + JSON_TYPE_REQUIRED_FOR_EXTRACTION = True + SUPPORTS_UNLOGGED_TABLES = True + + TYPE_MAPPING = { + **Postgres.Generator.TYPE_MAPPING, + sge.DataType.Type.TIMESTAMPTZ: "TIMESTAMPTZ", + } + + TRANSFORMS = { + **Postgres.Generator.TRANSFORMS, + } diff --git a/ibis/backends/risingwave/tests/conftest.py b/ibis/backends/risingwave/tests/conftest.py index 35cfe6b8e1dbd..4ffb2ab857223 100644 --- a/ibis/backends/risingwave/tests/conftest.py +++ b/ibis/backends/risingwave/tests/conftest.py @@ -4,10 +4,8 @@ from typing import TYPE_CHECKING, Any import pytest -import sqlalchemy as sa import ibis -from ibis.backends.conftest import init_database from ibis.backends.tests.base import ServiceBackendTest if TYPE_CHECKING: @@ -35,23 +33,14 @@ class TestConf(ServiceBackendTest): supports_structs = False rounding_method = "half_to_even" service_name = "risingwave" - deps = "psycopg2", "sqlalchemy" + deps = ("psycopg2",) @property def test_files(self) -> Iterable[Path]: return self.data_dir.joinpath("csv").glob("*.csv") - def _load_data( - self, - *, - user: str = PG_USER, - password: str = PG_PASS, - host: str = PG_HOST, - port: int = PG_PORT, - database: str = IBIS_TEST_RISINGWAVE_DB, - **_: Any, - ) -> None: - """Load test data into a Risingwave backend instance. + def _load_data(self, **_: Any) -> None: + """Load test data into a PostgreSQL backend instance. Parameters ---------- @@ -60,15 +49,8 @@ def _load_data( script_dir Location of scripts defining schemas """ - init_database( - url=sa.engine.make_url( - f"risingwave://{user}:{password}@{host}:{port:d}/{database}" - ), - database=database, - schema=self.ddl_script, - isolation_level="AUTOCOMMIT", - recreate=False, - ) + with self.connection._safe_raw_sql(";".join(self.ddl_script)): + pass @staticmethod def connect(*, tmpdir, worker_id, port: int | None = None, **kw): @@ -91,13 +73,8 @@ def con(tmp_path_factory, data_dir, worker_id): @pytest.fixture(scope="module") -def db(con): - return con.database() - - -@pytest.fixture(scope="module") -def alltypes(db): - return db.functional_alltypes +def alltypes(con): + return con.tables.functional_alltypes @pytest.fixture(scope="module") @@ -105,20 +82,6 @@ def df(alltypes): return alltypes.execute() -@pytest.fixture(scope="module") -def alltypes_sqla(con, alltypes): - name = alltypes.op().name - return con._get_sqla_table(name) - - @pytest.fixture(scope="module") def intervals(con): return con.table("intervals") - - -@pytest.fixture -def translate(): - from ibis.backends.risingwave import Backend - - context = Backend.compiler.make_context() - return lambda expr: Backend.compiler.translator_class(expr, context).get_result() diff --git a/ibis/backends/risingwave/tests/snapshots/test_client/test_compile_toplevel/out.sql b/ibis/backends/risingwave/tests/snapshots/test_client/test_compile_toplevel/out.sql index cfbcf133a863b..c0b4a0b83304f 100644 --- a/ibis/backends/risingwave/tests/snapshots/test_client/test_compile_toplevel/out.sql +++ b/ibis/backends/risingwave/tests/snapshots/test_client/test_compile_toplevel/out.sql @@ -1,2 +1,3 @@ -SELECT sum(t0.foo) AS "Sum(foo)" -FROM t0 AS t0 \ No newline at end of file +SELECT + SUM("t0"."foo") AS "Sum(foo)" +FROM "t0" AS "t0" \ No newline at end of file diff --git a/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/False/out.sql b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/False/out.sql index 34761d9a76e0d..f0366d83444de 100644 --- a/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/False/out.sql +++ b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/False/out.sql @@ -1 +1 @@ -WITH anon_2 AS (SELECT t2.string_col AS string_col, sum(t2.double_col) AS metric FROM functional_alltypes AS t2 GROUP BY 1), anon_3 AS (SELECT t3.string_col AS string_col, sum(t3.double_col) AS metric FROM functional_alltypes AS t3 GROUP BY 1), anon_1 AS (SELECT t2.string_col AS string_col, t2.metric AS metric FROM (SELECT anon_2.string_col AS string_col, anon_2.metric AS metric FROM anon_2 UNION ALL SELECT anon_3.string_col AS string_col, anon_3.metric AS metric FROM anon_3) AS t2), anon_4 AS (SELECT t3.string_col AS string_col, sum(t3.double_col) AS metric FROM functional_alltypes AS t3 GROUP BY 1) SELECT t1.string_col, t1.metric FROM (SELECT anon_1.string_col AS string_col, anon_1.metric AS metric FROM anon_1 UNION ALL SELECT anon_4.string_col AS string_col, anon_4.metric AS metric FROM anon_4) AS t1 \ No newline at end of file +WITH "t1" AS ( SELECT "t0"."string_col", SUM("t0"."double_col") AS "metric" FROM "functional_alltypes" AS "t0" GROUP BY 1 ) SELECT "t7"."string_col", "t7"."metric" FROM ( SELECT "t5"."string_col", "t5"."metric" FROM ( SELECT * FROM "t1" AS "t2" UNION ALL SELECT * FROM "t1" AS "t4" ) AS "t5" UNION ALL SELECT * FROM "t1" AS "t3" ) AS "t7" \ No newline at end of file diff --git a/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/True/out.sql b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/True/out.sql index 6ce31e7468bb8..5a873785e92bb 100644 --- a/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/True/out.sql +++ b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/True/out.sql @@ -1 +1 @@ -WITH anon_2 AS (SELECT t2.string_col AS string_col, sum(t2.double_col) AS metric FROM functional_alltypes AS t2 GROUP BY 1), anon_3 AS (SELECT t3.string_col AS string_col, sum(t3.double_col) AS metric FROM functional_alltypes AS t3 GROUP BY 1), anon_1 AS (SELECT t2.string_col AS string_col, t2.metric AS metric FROM (SELECT anon_2.string_col AS string_col, anon_2.metric AS metric FROM anon_2 UNION SELECT anon_3.string_col AS string_col, anon_3.metric AS metric FROM anon_3) AS t2), anon_4 AS (SELECT t3.string_col AS string_col, sum(t3.double_col) AS metric FROM functional_alltypes AS t3 GROUP BY 1) SELECT t1.string_col, t1.metric FROM (SELECT anon_1.string_col AS string_col, anon_1.metric AS metric FROM anon_1 UNION SELECT anon_4.string_col AS string_col, anon_4.metric AS metric FROM anon_4) AS t1 \ No newline at end of file +WITH "t1" AS ( SELECT "t0"."string_col", SUM("t0"."double_col") AS "metric" FROM "functional_alltypes" AS "t0" GROUP BY 1 ) SELECT "t7"."string_col", "t7"."metric" FROM ( SELECT "t5"."string_col", "t5"."metric" FROM ( SELECT * FROM "t1" AS "t2" UNION SELECT * FROM "t1" AS "t4" ) AS "t5" UNION SELECT * FROM "t1" AS "t3" ) AS "t7" \ No newline at end of file diff --git a/ibis/backends/risingwave/tests/test_client.py b/ibis/backends/risingwave/tests/test_client.py index b5c7cfa985609..918b648b7bc8b 100644 --- a/ibis/backends/risingwave/tests/test_client.py +++ b/ibis/backends/risingwave/tests/test_client.py @@ -4,17 +4,15 @@ import pandas as pd import pytest +import sqlglot as sg from pytest import param import ibis import ibis.expr.datatypes as dt import ibis.expr.types as ir -from ibis.tests.util import assert_equal +from ibis.util import gen_name pytest.importorskip("psycopg2") -sa = pytest.importorskip("sqlalchemy") - -from sqlalchemy.dialects import postgresql # noqa: E402 RISINGWAVE_TEST_DB = os.environ.get("IBIS_TEST_RISINGWAVE_DATABASE", "dev") IBIS_RISINGWAVE_HOST = os.environ.get("IBIS_TEST_RISINGWAVE_HOST", "localhost") @@ -64,47 +62,15 @@ def test_list_databases(con): assert RISINGWAVE_TEST_DB in con.list_databases() -def test_schema_type_conversion(con): - typespec = [ - # name, type, nullable - ("jsonb", postgresql.JSONB, True, dt.JSON), - ] - - sqla_types = [] - ibis_types = [] - for name, t, nullable, ibis_type in typespec: - sqla_types.append(sa.Column(name, t, nullable=nullable)) - ibis_types.append((name, ibis_type(nullable=nullable))) - - # Create a table with placeholder stubs for JSON, JSONB, and UUID. - table = sa.Table("tname", sa.MetaData(), *sqla_types) - - # Check that we can correctly create a schema with dt.any for the - # missing types. - schema = con._schema_from_sqla_table(table) - expected = ibis.schema(ibis_types) - - assert_equal(schema, expected) +def test_create_and_drop_table(con, temp_table): + sch = ibis.schema([("first_name", "string")]) + con.create_table(temp_table, schema=sch) + assert con.table(temp_table) is not None -@pytest.mark.parametrize("params", [{}, {"database": RISINGWAVE_TEST_DB}]) -def test_create_and_drop_table(con, temp_table, params): - sch = ibis.schema( - [ - ("first_name", "string"), - ("last_name", "string"), - ("department_name", "string"), - ("salary", "float64"), - ] - ) - - con.create_table(temp_table, schema=sch, **params) - assert con.table(temp_table, **params) is not None - - con.drop_table(temp_table, **params) + con.drop_table(temp_table) - with pytest.raises(sa.exc.NoSuchTableError): - con.table(temp_table, **params) + assert temp_table not in con.list_tables() @pytest.mark.parametrize( @@ -124,8 +90,8 @@ def test_create_and_drop_table(con, temp_table, params): ("date", dt.date), ("time", dt.time), ("time without time zone", dt.time), - ("timestamp without time zone", dt.timestamp), - ("timestamp with time zone", dt.Timestamp("UTC")), + ("timestamp without time zone", dt.Timestamp(scale=6)), + ("timestamp with time zone", dt.Timestamp("UTC", scale=6)), ("interval", dt.Interval("s")), ("numeric", dt.decimal), ("jsonb", dt.json), @@ -133,17 +99,16 @@ def test_create_and_drop_table(con, temp_table, params): ], ) def test_get_schema_from_query(con, pg_type, expected_type): - name = con._quote(ibis.util.guid()) + name = sg.table(gen_name("risingwave_temp_table"), quoted=True) with con.begin() as c: - c.exec_driver_sql(f"CREATE TABLE {name} (x {pg_type}, y {pg_type}[])") + c.execute(f"CREATE TABLE {name} (x {pg_type}, y {pg_type}[])") expected_schema = ibis.schema(dict(x=expected_type, y=dt.Array(expected_type))) result_schema = con._get_schema_using_query(f"SELECT x, y FROM {name}") assert result_schema == expected_schema with con.begin() as c: - c.exec_driver_sql(f"DROP TABLE {name}") + c.execute(f"DROP TABLE {name}") -@pytest.mark.xfail(reason="unsupported insert with CTEs") def test_insert_with_cte(con): X = con.create_table("X", schema=ibis.schema(dict(id="int")), temp=False) expr = X.join(X.mutate(a=X["id"] + 1), ["id"]) @@ -151,8 +116,3 @@ def test_insert_with_cte(con): assert Y.execute().empty con.drop_table("Y") con.drop_table("X") - - -def test_connect_url_with_empty_host(): - con = ibis.connect("risingwave:///dev") - assert con.con.url.host is None diff --git a/ibis/backends/risingwave/tests/test_functions.py b/ibis/backends/risingwave/tests/test_functions.py index c8874e390c608..d680fb3190f90 100644 --- a/ibis/backends/risingwave/tests/test_functions.py +++ b/ibis/backends/risingwave/tests/test_functions.py @@ -1,7 +1,6 @@ from __future__ import annotations import operator -import string import warnings from datetime import datetime @@ -13,104 +12,9 @@ import ibis import ibis.expr.datatypes as dt -import ibis.expr.types as ir -from ibis import config from ibis import literal as L pytest.importorskip("psycopg2") -sa = pytest.importorskip("sqlalchemy") - -from sqlalchemy.dialects import postgresql # noqa: E402 - - -@pytest.mark.parametrize( - ("left_func", "right_func"), - [ - param( - lambda t: t.double_col.cast("int8"), - lambda at: sa.cast(at.c.double_col, sa.SMALLINT), - id="double_to_int8", - ), - param( - lambda t: t.double_col.cast("int16"), - lambda at: sa.cast(at.c.double_col, sa.SMALLINT), - id="double_to_int16", - ), - param( - lambda t: t.string_col.cast("double"), - lambda at: sa.cast(at.c.string_col, postgresql.DOUBLE_PRECISION), - id="string_to_double", - ), - param( - lambda t: t.string_col.cast("float32"), - lambda at: sa.cast(at.c.string_col, postgresql.REAL), - id="string_to_float", - ), - param( - lambda t: t.string_col.cast("decimal"), - lambda at: sa.cast(at.c.string_col, sa.NUMERIC()), - id="string_to_decimal_no_params", - ), - param( - lambda t: t.string_col.cast("decimal(9, 3)"), - lambda at: sa.cast(at.c.string_col, sa.NUMERIC(9, 3)), - id="string_to_decimal_params", - ), - ], -) -def test_cast(alltypes, alltypes_sqla, translate, left_func, right_func): - left = left_func(alltypes) - right = right_func(alltypes_sqla.alias("t0")) - assert str(translate(left.op()).compile()) == str(right.compile()) - - -def test_date_cast(alltypes, alltypes_sqla, translate): - result = alltypes.date_string_col.cast("date") - expected = sa.cast(alltypes_sqla.alias("t0").c.date_string_col, sa.DATE) - assert str(translate(result.op())) == str(expected) - - -@pytest.mark.parametrize( - "column", - [ - "id", - "bool_col", - "tinyint_col", - "smallint_col", - "int_col", - "bigint_col", - "float_col", - "double_col", - "date_string_col", - "string_col", - "timestamp_col", - "year", - "month", - ], -) -def test_noop_cast(alltypes, alltypes_sqla, translate, column): - col = alltypes[column] - result = col.cast(col.type()) - expected = alltypes_sqla.alias("t0").c[column] - assert result.equals(col) - assert str(translate(result.op())) == str(expected) - - -def test_timestamp_cast_noop(alltypes, alltypes_sqla, translate): - # See GH #592 - result1 = alltypes.timestamp_col.cast("timestamp") - result2 = alltypes.int_col.cast("timestamp") - - assert isinstance(result1, ir.TimestampColumn) - assert isinstance(result2, ir.TimestampColumn) - - expected1 = alltypes_sqla.alias("t0").c.timestamp_col - expected2 = sa.cast( - sa.func.to_timestamp(alltypes_sqla.alias("t0").c.int_col), sa.TIMESTAMP() - ) - - assert str(translate(result1.op())) == str(expected1) - assert str(translate(result2.op())) == str(expected2) @pytest.mark.parametrize(("value", "expected"), [(0, None), (5.5, 5.5)]) @@ -427,12 +331,7 @@ def test_union_cte(alltypes, distinct, snapshot): expr2 = expr1.view() expr3 = expr1.view() expr = expr1.union(expr2, distinct=distinct).union(expr3, distinct=distinct) - result = " ".join( - line.strip() - for line in str( - expr.compile().compile(compile_kwargs={"literal_binds": True}) - ).splitlines() - ) + result = " ".join(line.strip() for line in expr.compile().splitlines()) snapshot.assert_match(result, "out.sql") @@ -568,18 +467,6 @@ def test_not_exists(alltypes, df): tm.assert_frame_equal(result, expected, check_index_type=False, check_dtype=False) -def test_interactive_repr_shows_error(alltypes): - # #591. Doing this in Postgres because so many built-in functions are - # not available - - expr = alltypes.int_col.convert_base(10, 2) - - with config.option_context("interactive", True): - result = repr(expr) - - assert "no translation rule" in result.lower() - - def test_subquery(alltypes, df): t = alltypes @@ -758,9 +645,6 @@ def array_types(con): return con.table("array_types") -@pytest.mark.xfail( - reason="Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype" -) def test_array_length(array_types): expr = array_types.select( array_types.x.length().name("x_length"), @@ -861,60 +745,6 @@ def test_timestamp_with_timezone(con): assert str(result.dtype.tz) -@pytest.fixture( - params=[ - None, - "UTC", - "America/New_York", - "America/Los_Angeles", - "Europe/Paris", - "Chile/Continental", - "Asia/Tel_Aviv", - "Asia/Tokyo", - "Africa/Nairobi", - "Australia/Sydney", - ] -) -def tz(request): - return request.param - - -@pytest.fixture -def tzone_compute(con, temp_table, tz): - schema = ibis.schema([("ts", dt.Timestamp(tz)), ("b", "double"), ("c", "string")]) - con.create_table(temp_table, schema=schema, temp=False) - t = con.table(temp_table) - - n = 10 - df = pd.DataFrame( - { - "ts": pd.date_range("2017-04-01", periods=n, tz=tz).values, - "b": np.arange(n).astype("float64"), - "c": list(string.ascii_lowercase[:n]), - } - ) - - df.to_sql( - temp_table, - con.con, - index=False, - if_exists="append", - dtype={"ts": sa.TIMESTAMP(timezone=True), "b": sa.FLOAT, "c": sa.TEXT}, - ) - - yield t - con.drop_table(temp_table) - - -def test_ts_timezone_is_preserved(tzone_compute, tz): - assert dt.Timestamp(tz).equals(tzone_compute.ts.type()) - - -def test_timestamp_with_timezone_select(tzone_compute, tz): - ts = tzone_compute.ts.execute() - assert str(getattr(ts.dtype, "tz", None)) == str(tz) - - @pytest.mark.parametrize( ("left", "right", "type"), [ @@ -1010,8 +840,8 @@ def test_string_to_binary_cast(con): "FROM functional_alltypes LIMIT 10" ) with con.begin() as c: - cur = c.exec_driver_sql(sql_string) - raw_data = [row[0][0] for row in cur] + c.execute(sql_string) + raw_data = [row[0][0] for row in c.fetchall()] expected = pd.Series(raw_data, name=name) tm.assert_series_equal(result, expected) @@ -1027,6 +857,6 @@ def test_string_to_binary_round_trip(con): "FROM functional_alltypes LIMIT 10" ) with con.begin() as c: - cur = c.exec_driver_sql(sql_string) - expected = pd.Series([row[0][0] for row in cur], name=name) + c.execute(sql_string) + expected = pd.Series([row[0][0] for row in c.fetchall()], name=name) tm.assert_series_equal(result, expected) diff --git a/ibis/backends/tests/errors.py b/ibis/backends/tests/errors.py index a314b4f7543ce..e9a8347ab094a 100644 --- a/ibis/backends/tests/errors.py +++ b/ibis/backends/tests/errors.py @@ -92,14 +92,18 @@ try: from psycopg2.errors import DivisionByZero as PsycoPg2DivisionByZero from psycopg2.errors import IndeterminateDatatype as PsycoPg2IndeterminateDatatype + from psycopg2.errors import InternalError_ as PsycoPg2InternalError from psycopg2.errors import ( InvalidTextRepresentation as PsycoPg2InvalidTextRepresentation, ) + from psycopg2.errors import ProgrammingError as PsycoPg2ProgrammingError from psycopg2.errors import SyntaxError as PsycoPg2SyntaxError except ImportError: PsycoPg2SyntaxError = ( PsycoPg2IndeterminateDatatype - ) = PsycoPg2InvalidTextRepresentation = PsycoPg2DivisionByZero = None + ) = ( + PsycoPg2InvalidTextRepresentation + ) = PsycoPg2DivisionByZero = PsycoPg2InternalError = PsycoPg2ProgrammingError = None try: from pymysql.err import NotSupportedError as MySQLNotSupportedError diff --git a/ibis/backends/tests/snapshots/test_dot_sql/test_cte/risingwave/out.sql b/ibis/backends/tests/snapshots/test_dot_sql/test_cte/risingwave/out.sql new file mode 100644 index 0000000000000..efc0daaef0d6f --- /dev/null +++ b/ibis/backends/tests/snapshots/test_dot_sql/test_cte/risingwave/out.sql @@ -0,0 +1,8 @@ +WITH "foo" AS ( + SELECT + * + FROM "test_risingwave_temp_mem_t_for_cte" AS "t0" +) +SELECT + COUNT(*) AS "x" +FROM "foo" \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_default_limit/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/risingwave/out.sql new file mode 100644 index 0000000000000..b309cd65374d5 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/risingwave/out.sql @@ -0,0 +1,5 @@ +SELECT + "t0"."id", + "t0"."bool_col" +FROM "functional_alltypes" AS "t0" +LIMIT 11 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/risingwave/out.sql new file mode 100644 index 0000000000000..b309cd65374d5 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/risingwave/out.sql @@ -0,0 +1,5 @@ +SELECT + "t0"."id", + "t0"."bool_col" +FROM "functional_alltypes" AS "t0" +LIMIT 11 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/risingwave/out.sql new file mode 100644 index 0000000000000..6bd0ba8c995d3 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/risingwave/out.sql @@ -0,0 +1,3 @@ +SELECT + SUM("t0"."bigint_col") AS "Sum(bigint_col)" +FROM "functional_alltypes" AS "t0" \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/risingwave/out.sql new file mode 100644 index 0000000000000..97338646649f0 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/risingwave/out.sql @@ -0,0 +1,10 @@ +SELECT + * +FROM ( + SELECT + "t0"."id", + "t0"."bool_col" + FROM "functional_alltypes" AS "t0" + LIMIT 10 +) AS "t2" +LIMIT 11 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/risingwave/out.sql b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/risingwave/out.sql new file mode 100644 index 0000000000000..d3969647c9ea1 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/risingwave/out.sql @@ -0,0 +1,22 @@ +SELECT + CASE "t0"."continent" + WHEN 'NA' + THEN 'North America' + WHEN 'SA' + THEN 'South America' + WHEN 'EU' + THEN 'Europe' + WHEN 'AF' + THEN 'Africa' + WHEN 'AS' + THEN 'Asia' + WHEN 'OC' + THEN 'Oceania' + WHEN 'AN' + THEN 'Antarctica' + ELSE 'Unknown continent' + END AS "cont", + SUM("t0"."population") AS "total_pop" +FROM "countries" AS "t0" +GROUP BY + 1 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_isin_bug/risingwave/out.sql b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/risingwave/out.sql new file mode 100644 index 0000000000000..c1611d8cecc33 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/risingwave/out.sql @@ -0,0 +1,9 @@ +SELECT + "t0"."x" IN ( + SELECT + "t0"."x" + FROM "t" AS "t0" + WHERE + "t0"."x" > 2 + ) AS "InSubquery(x)" +FROM "t" AS "t0" \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/risingwave/out.sql b/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/risingwave/out.sql new file mode 100644 index 0000000000000..b7508b9ef535f --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/risingwave/out.sql @@ -0,0 +1,60 @@ +WITH "t5" AS ( + SELECT + "t4"."field_of_study", + FIRST("t4"."diff") AS "diff" + FROM ( + SELECT + "t3"."field_of_study", + "t3"."years", + "t3"."degrees", + "t3"."earliest_degrees", + "t3"."latest_degrees", + "t3"."latest_degrees" - "t3"."earliest_degrees" AS "diff" + FROM ( + SELECT + "t2"."field_of_study", + "t2"."years", + "t2"."degrees", + FIRST_VALUE("t2"."degrees") OVER (PARTITION BY "t2"."field_of_study" ORDER BY "t2"."years" ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS "earliest_degrees", + LAST_VALUE("t2"."degrees") OVER (PARTITION BY "t2"."field_of_study" ORDER BY "t2"."years" ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS "latest_degrees" + FROM ( + SELECT + "t1"."field_of_study", + CAST(TO_JSONB("t1"."__pivoted__") -> 'f1' AS VARCHAR) AS "years", + CAST(TO_JSONB("t1"."__pivoted__") -> 'f2' AS BIGINT) AS "degrees" + FROM ( + SELECT + "t0"."field_of_study", + UNNEST( + ARRAY[ROW(CAST('1970-71' AS VARCHAR), CAST("t0"."1970-71" AS BIGINT)), ROW(CAST('1975-76' AS VARCHAR), CAST("t0"."1975-76" AS BIGINT)), ROW(CAST('1980-81' AS VARCHAR), CAST("t0"."1980-81" AS BIGINT)), ROW(CAST('1985-86' AS VARCHAR), CAST("t0"."1985-86" AS BIGINT)), ROW(CAST('1990-91' AS VARCHAR), CAST("t0"."1990-91" AS BIGINT)), ROW(CAST('1995-96' AS VARCHAR), CAST("t0"."1995-96" AS BIGINT)), ROW(CAST('2000-01' AS VARCHAR), CAST("t0"."2000-01" AS BIGINT)), ROW(CAST('2005-06' AS VARCHAR), CAST("t0"."2005-06" AS BIGINT)), ROW(CAST('2010-11' AS VARCHAR), CAST("t0"."2010-11" AS BIGINT)), ROW(CAST('2011-12' AS VARCHAR), CAST("t0"."2011-12" AS BIGINT)), ROW(CAST('2012-13' AS VARCHAR), CAST("t0"."2012-13" AS BIGINT)), ROW(CAST('2013-14' AS VARCHAR), CAST("t0"."2013-14" AS BIGINT)), ROW(CAST('2014-15' AS VARCHAR), CAST("t0"."2014-15" AS BIGINT)), ROW(CAST('2015-16' AS VARCHAR), CAST("t0"."2015-16" AS BIGINT)), ROW(CAST('2016-17' AS VARCHAR), CAST("t0"."2016-17" AS BIGINT)), ROW(CAST('2017-18' AS VARCHAR), CAST("t0"."2017-18" AS BIGINT)), ROW(CAST('2018-19' AS VARCHAR), CAST("t0"."2018-19" AS BIGINT)), ROW(CAST('2019-20' AS VARCHAR), CAST("t0"."2019-20" AS BIGINT))] + ) AS "__pivoted__" + FROM "humanities" AS "t0" + ) AS "t1" + ) AS "t2" + ) AS "t3" + ) AS "t4" + GROUP BY + 1 +) +SELECT + "t11"."field_of_study", + "t11"."diff" +FROM ( + SELECT + "t6"."field_of_study", + "t6"."diff" + FROM "t5" AS "t6" + ORDER BY + "t6"."diff" DESC NULLS LAST + LIMIT 10 + UNION ALL + SELECT + "t6"."field_of_study", + "t6"."diff" + FROM "t5" AS "t6" + WHERE + "t6"."diff" < 0 + ORDER BY + "t6"."diff" ASC + LIMIT 10 +) AS "t11" \ No newline at end of file diff --git a/ibis/backends/tests/test_aggregation.py b/ibis/backends/tests/test_aggregation.py index 2ae1aa8b4be47..63c40a7a24a70 100644 --- a/ibis/backends/tests/test_aggregation.py +++ b/ibis/backends/tests/test_aggregation.py @@ -22,6 +22,7 @@ MySQLNotSupportedError, OracleDatabaseError, PolarsInvalidOperationError, + PsycoPg2InternalError, Py4JError, PyDruidProgrammingError, PyODBCProgrammingError, @@ -92,6 +93,7 @@ def mean_udf(s): "druid", "oracle", "flink", + "risingwave", "exasol", ], raises=com.OperationNotDefinedError, @@ -439,6 +441,7 @@ def mean_and_std(v): "oracle", "exasol", "flink", + "risingwave", ], raises=com.OperationNotDefinedError, ), @@ -537,7 +540,7 @@ def mean_and_std(v): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, ), ], ), @@ -562,7 +565,7 @@ def mean_and_std(v): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, ), ], ), @@ -592,7 +595,7 @@ def mean_and_std(v): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, ), ], ), @@ -649,7 +652,7 @@ def mean_and_std(v): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, ), ], ), @@ -664,7 +667,7 @@ def mean_and_std(v): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, ), ], ), @@ -771,21 +774,25 @@ def mean_and_std(v): param( lambda t: t.string_col.isin(["1", "7"]), lambda t: t.string_col.isin(["1", "7"]), - marks=pytest.mark.notimpl( - ["exasol"], - raises=(com.OperationNotDefinedError, ExaQueryError), - strict=False, - ), + marks=[ + pytest.mark.notimpl( + ["exasol"], + raises=(com.OperationNotDefinedError, ExaQueryError), + strict=False, + ), + ], id="is_in", ), param( lambda _: ibis._.string_col.isin(["1", "7"]), lambda t: t.string_col.isin(["1", "7"]), - marks=pytest.mark.notimpl( - ["exasol"], - raises=(com.OperationNotDefinedError, ExaQueryError), - strict=False, - ), + marks=[ + pytest.mark.notimpl( + ["exasol"], + raises=(com.OperationNotDefinedError, ExaQueryError), + strict=False, + ), + ], id="is_in_deferred", ), ], @@ -939,7 +946,7 @@ def test_count_distinct_star(alltypes, df, ibis_cond, pandas_cond): pytest.mark.broken( ["risingwave"], reason="Invalid input syntax: direct arg in `percentile_cont` must be castable to float64", - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, ), ], ), @@ -954,7 +961,14 @@ def test_count_distinct_star(alltypes, df, ibis_cond, pandas_cond): lambda t: t.string_col.isin(["1", "7"]), id="is_in", marks=[ - pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) + pytest.mark.notimpl( + ["datafusion"], raises=com.OperationNotDefinedError + ), + pytest.mark.notimpl( + "risingwave", + raises=PsycoPg2InternalError, + reason="probably incorrect filter syntax but not sure", + ), ], ), ], @@ -991,7 +1005,7 @@ def test_quantile( ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function covar_pop(integer, integer) does not exist", ), ], @@ -1011,7 +1025,7 @@ def test_quantile( ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function covar_pop(integer, integer) does not exist", ), ], @@ -1036,7 +1050,7 @@ def test_quantile( ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function covar_pop(integer, integer) does not exist", ), ], @@ -1051,7 +1065,7 @@ def test_quantile( raises=com.OperationNotDefinedError, ), pytest.mark.notyet( - ["postgres", "duckdb", "snowflake"], + ["postgres", "duckdb", "snowflake", "risingwave"], raises=com.UnsupportedOperationError, reason="backend only implements population correlation coefficient", ), @@ -1095,7 +1109,7 @@ def test_quantile( ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function covar_pop(integer, integer) does not exist", ), ], @@ -1124,7 +1138,7 @@ def test_quantile( ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function covar_pop(integer, integer) does not exist", ), ], @@ -1608,7 +1622,9 @@ def test_grouped_case(backend, con): @pytest.mark.notyet(["oracle"], raises=OracleDatabaseError) @pytest.mark.notyet(["pyspark"], raises=PySparkAnalysisException) @pytest.mark.notyet(["mssql"], raises=PyODBCProgrammingError) +@pytest.mark.notyet(["risingwave"], raises=AssertionError, strict=False) def test_group_concat_over_window(backend, con): + # TODO: this test is flaky on risingwave and I DO NOT LIKE IT input_df = pd.DataFrame( { "s": ["a|b|c", "b|a|c", "b|b|b|c|a"], diff --git a/ibis/backends/tests/test_array.py b/ibis/backends/tests/test_array.py index 2586782954ba0..08485208bd3ae 100644 --- a/ibis/backends/tests/test_array.py +++ b/ibis/backends/tests/test_array.py @@ -8,7 +8,6 @@ import pandas.testing as tm import pytest import pytz -import sqlalchemy as sa import toolz from pytest import param @@ -24,6 +23,8 @@ MySQLOperationalError, PolarsComputeError, PsycoPg2IndeterminateDatatype, + PsycoPg2InternalError, + PsycoPg2ProgrammingError, PsycoPg2SyntaxError, Py4JJavaError, PySparkAnalysisException, @@ -83,7 +84,19 @@ def test_array_column(backend, alltypes, df): backend.assert_series_equal(result, expected, check_names=False) -def test_array_scalar(con): +ARRAY_BACKEND_TYPES = { + "clickhouse": "Array(Float64)", + "snowflake": "ARRAY", + "trino": "array(double)", + "bigquery": "ARRAY", + "duckdb": "DOUBLE[]", + "postgres": "numeric[]", + "risingwave": "numeric[]", + "flink": "ARRAY NOT NULL", +} + + +def test_array_scalar(con, backend): expr = ibis.array([1.0, 2.0, 3.0]) assert isinstance(expr, ir.ArrayScalar) @@ -126,11 +139,6 @@ def test_array_concat_variadic(con): # Issues #2370 @pytest.mark.notimpl(["flink"], raises=com.OperationNotDefinedError) -@pytest.mark.notyet( - ["risingwave"], - raises=sa.exc.InternalError, - reason="Bind error: cannot determine type of empty array", -) @pytest.mark.notyet(["trino"], raises=TrinoUserError) def test_array_concat_some_empty(con): left = ibis.literal([]) @@ -210,7 +218,7 @@ def test_array_index(con, idx): ) @pytest.mark.notimpl( ["risingwave"], - raises=ValueError, + raises=AssertionError, reason="Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", ) @pytest.mark.never( @@ -243,10 +251,11 @@ def test_array_discovery(backend): raises=GoogleBadRequest, ) @pytest.mark.notimpl(["dask"], raises=ValueError) -@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=ValueError, + # TODO: valueerror -> assertion error + raises=AssertionError, reason="Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", ) def test_unnest_simple(backend): @@ -266,11 +275,6 @@ def test_unnest_simple(backend): @builtin_array @pytest.mark.notimpl("dask", raises=com.OperationNotDefinedError) @pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["risingwave"], - raises=ValueError, - reason="ValueError: Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", -) def test_unnest_complex(backend): array_types = backend.array_types df = array_types.execute() @@ -309,11 +313,6 @@ def test_unnest_complex(backend): ) @pytest.mark.notimpl(["dask"], raises=ValueError) @pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["risingwave"], - raises=ValueError, - reason="Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", -) def test_unnest_idempotent(backend): array_types = backend.array_types df = array_types.execute() @@ -335,11 +334,6 @@ def test_unnest_idempotent(backend): @builtin_array @pytest.mark.notimpl("dask", raises=ValueError) @pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["risingwave"], - raises=ValueError, - reason="ValueError: Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", -) def test_unnest_no_nulls(backend): array_types = backend.array_types df = array_types.execute() @@ -366,17 +360,8 @@ def test_unnest_no_nulls(backend): @builtin_array @pytest.mark.notimpl("dask", raises=ValueError) -@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["risingwave"], - raises=ValueError, - reason="ValueError: Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", -) -@pytest.mark.broken( - ["pandas"], - raises=ValueError, - reason="all the input arrays must have same number of dimensions", -) +@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) +@pytest.mark.broken(["risingwave"], raises=AssertionError) def test_unnest_default_name(backend): array_types = backend.array_types df = array_types.execute() @@ -426,10 +411,11 @@ def test_unnest_default_name(backend): ["datafusion"], raises=Exception, reason="array_types table isn't defined" ) @pytest.mark.notimpl(["dask"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( +@pytest.mark.broken( ["risingwave"], - raises=ValueError, - reason="ValueError: Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", + raises=AssertionError, + reason="not broken; row ordering is not guaranteed and sometimes this test will pass", + strict=False, ) def test_array_slice(backend, start, stop): array_types = backend.array_types @@ -452,6 +438,11 @@ def test_array_slice(backend, start, stop): ], raises=com.OperationNotDefinedError, ) +@pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="TODO(Kexiang): seems a bug", +) @pytest.mark.notimpl( ["dask", "pandas"], raises=com.OperationNotDefinedError, @@ -480,7 +471,7 @@ def test_array_slice(backend, start, stop): ) @pytest.mark.broken( ["risingwave"], - raises=AssertionError, + raises=PsycoPg2InternalError, reason="TODO(Kexiang): seems a bug", ) def test_array_map(con, input, output): @@ -539,6 +530,11 @@ def test_array_map(con, input, output): param({"a": [[1, 2], [4]]}, {"a": [[2], [4]]}, id="no_nulls"), ], ) +@pytest.mark.notyet( + "risingwave", + raises=PsycoPg2InternalError, + reason="no support for not null column constraint", +) def test_array_filter(con, input, output): t = ibis.memtable(input, schema=ibis.schema(dict(a="!array"))) expected = pd.Series(output["a"]) @@ -559,15 +555,11 @@ def test_array_filter(con, input, output): @builtin_array @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl(["dask"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["risingwave"], - raises=ValueError, - reason="ValueError: Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", -) @pytest.mark.broken( - ["flink"], - raises=Py4JJavaError, - reason="Caused by: java.lang.NullPointerException", + ["risingwave"], + raises=AssertionError, + reason="not broken; row ordering is not guaranteed and sometimes this test will pass", + strict=False, ) def test_array_contains(backend, con): t = backend.array_types @@ -617,11 +609,6 @@ def test_array_position(backend, con, a, expected_array): @builtin_array @pytest.mark.notimpl(["dask", "polars"], raises=com.OperationNotDefinedError) -@pytest.mark.broken( - ["risingwave"], - raises=AssertionError, - reason="TODO(Kexiang): seems a bug", -) @pytest.mark.parametrize( ("a"), [ @@ -712,13 +699,13 @@ def test_array_unique(con, input, expected): raises=AssertionError, reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14735", ) -def test_array_sort(con): - t = ibis.memtable({"a": [[3, 2], [], [42, 42], []]}) - expr = t.a.sort() +def test_array_sort(backend, con): + t = ibis.memtable({"a": [[3, 2], [], [42, 42], []], "id": range(4)}) + expr = t.mutate(a=t.a.sort()).order_by("id") result = con.execute(expr) expected = pd.Series([[2, 3], [], [42, 42], []], dtype="object") - assert frozenset(map(tuple, result.values)) == frozenset( + assert frozenset(map(tuple, result["a"].values)) == frozenset( map(tuple, expected.values) ) @@ -818,9 +805,9 @@ def test_array_intersect(con, data): raises=ClickHouseDatabaseError, reason="ClickHouse won't accept dicts for struct type values", ) -@pytest.mark.notimpl(["risingwave"], raises=sa.exc.ProgrammingError) @pytest.mark.notimpl(["postgres"], raises=PsycoPg2SyntaxError) -@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["risingwave"], raises=PsycoPg2InternalError) +@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) @pytest.mark.broken( ["trino"], reason="inserting maps into structs doesn't work", raises=TrinoUserError ) @@ -841,7 +828,6 @@ def test_unnest_struct(con): "dask", "datafusion", "druid", - "flink", "oracle", "pandas", "polars", @@ -852,7 +838,7 @@ def test_unnest_struct(con): ) @pytest.mark.notimpl( ["risingwave"], - raises=ValueError, + raises=com.OperationNotDefinedError, reason="Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", ) def test_zip(backend): @@ -879,9 +865,9 @@ def test_zip(backend): raises=ClickHouseDatabaseError, reason="https://github.com/ClickHouse/ClickHouse/issues/41112", ) -@pytest.mark.notimpl(["risingwave"], raises=sa.exc.ProgrammingError) @pytest.mark.notimpl(["postgres"], raises=PsycoPg2SyntaxError) -@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["risingwave"], raises=PsycoPg2ProgrammingError) +@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["polars"], raises=com.OperationNotDefinedError, @@ -940,7 +926,11 @@ def flatten_data(): @pytest.mark.notyet( ["postgres", "risingwave"], reason="Postgres doesn't truly support arrays of arrays", - raises=(com.OperationNotDefinedError, PsycoPg2IndeterminateDatatype), + raises=( + com.OperationNotDefinedError, + PsycoPg2IndeterminateDatatype, + PsycoPg2InternalError, + ), ) @pytest.mark.parametrize( ("column", "expected"), @@ -1057,7 +1047,7 @@ def test_range_start_stop_step(con, start, stop, step): @pytest.mark.notimpl(["flink", "dask"], raises=com.OperationNotDefinedError) @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Invalid parameter step: step size cannot equal zero", ) def test_range_start_stop_step_zero(con, start, stop): @@ -1096,6 +1086,11 @@ def test_unnest_empty_array(con): raises=com.OperationNotDefinedError, ) @pytest.mark.notimpl(["sqlite"], raises=com.UnsupportedBackendType) +@pytest.mark.notyet( + "risingwave", + raises=PsycoPg2InternalError, + reason="no support for not null column constraint", +) def test_array_map_with_conflicting_names(backend, con): t = ibis.memtable({"x": [[1, 2]]}, schema=ibis.schema(dict(x="!array"))) expr = t.select(a=t.x.map(lambda x: x + 1)).select( @@ -1184,7 +1179,7 @@ def swap(token): id="pos", marks=pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function make_interval() does not exist", ), ), @@ -1200,7 +1195,7 @@ def swap(token): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function neg(interval) does not exist", ), ], @@ -1220,7 +1215,7 @@ def swap(token): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function neg(interval) does not exist", ), ], @@ -1252,7 +1247,7 @@ def test_timestamp_range(con, start, stop, step, freq, tzinfo): pytest.mark.notyet(["polars"], raises=PolarsComputeError), pytest.mark.notyet( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function make_interval() does not exist", ), ], @@ -1271,7 +1266,7 @@ def test_timestamp_range(con, start, stop, step, freq, tzinfo): ), pytest.mark.notyet( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function neg(interval) does not exist", ), ], @@ -1301,24 +1296,14 @@ def test_repr_timestamp_array(con, monkeypatch): assert ibis.options.default_backend is con expr = ibis.array(pd.date_range("2010-01-01", "2010-01-03", freq="D").tolist()) - assert "No translation rule" not in repr(expr) - assert "OperationNotDefinedError" not in repr(expr) + assert "Translation to backend failed" not in repr(expr) @pytest.mark.notyet( ["dask", "datafusion", "flink", "polars"], raises=com.OperationNotDefinedError, ) -@pytest.mark.broken( - ["risingwave"], - raises=sa.exc.OperationalError, - reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14734", -) -@pytest.mark.broken( - ["pandas"], - raises=ValueError, - reason="cannot reindex on an axis with duplicate labels", -) +@pytest.mark.broken(["pandas"], raises=ValueError, reason="reindex on duplicate values") def test_unnest_range(con): expr = ibis.range(2).unnest().name("x").as_table().mutate({"y": 1.0}) result = con.execute(expr) diff --git a/ibis/backends/tests/test_asof_join.py b/ibis/backends/tests/test_asof_join.py index ffe86146b65f4..2a1901efc520b 100644 --- a/ibis/backends/tests/test_asof_join.py +++ b/ibis/backends/tests/test_asof_join.py @@ -95,6 +95,7 @@ def time_keyed_right(time_keyed_df2): "oracle", "mssql", "sqlite", + "risingwave", ] ) def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op): @@ -135,6 +136,7 @@ def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op "oracle", "mssql", "sqlite", + "risingwave", ] ) def test_keyed_asof_join_with_tolerance( diff --git a/ibis/backends/tests/test_benchmarks.py b/ibis/backends/tests/test_benchmarks.py deleted file mode 100644 index 3234d3c8693f2..0000000000000 --- a/ibis/backends/tests/test_benchmarks.py +++ /dev/null @@ -1,900 +0,0 @@ -from __future__ import annotations - -import copy -import functools -import inspect -import itertools -import os -import string - -import numpy as np -import pandas as pd -import pytest -import sqlalchemy as sa -from packaging.version import parse as vparse - -import ibis -import ibis.expr.datatypes as dt -import ibis.expr.operations as ops -import ibis.expr.types as ir -from ibis.backends.base import _get_backend_names - -# from ibis.backends.pandas.udf import udf - -# FIXME(kszucs): pytestmark = pytest.mark.benchmark -pytestmark = pytest.mark.skip(reason="the backends must be rewritten first") - - -def make_t(): - return ibis.table( - [ - ("_timestamp", "int32"), - ("dim1", "int32"), - ("dim2", "int32"), - ("valid_seconds", "int32"), - ("meas1", "int32"), - ("meas2", "int32"), - ("year", "int32"), - ("month", "int32"), - ("day", "int32"), - ("hour", "int32"), - ("minute", "int32"), - ], - name="t", - ) - - -@pytest.fixture(scope="module") -def t(): - return make_t() - - -def make_base(t): - return t[ - ( - (t.year > 2016) - | ((t.year == 2016) & (t.month > 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day > 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day == 6) & (t.hour > 6)) - | ( - (t.year == 2016) - & (t.month == 6) - & (t.day == 6) - & (t.hour == 6) - & (t.minute >= 5) - ) - ) - & ( - (t.year < 2016) - | ((t.year == 2016) & (t.month < 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day < 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day == 6) & (t.hour < 6)) - | ( - (t.year == 2016) - & (t.month == 6) - & (t.day == 6) - & (t.hour == 6) - & (t.minute <= 5) - ) - ) - ] - - -@pytest.fixture(scope="module") -def base(t): - return make_base(t) - - -def make_large_expr(base): - src_table = base - src_table = src_table.mutate( - _timestamp=(src_table["_timestamp"] - src_table["_timestamp"] % 3600) - .cast("int32") - .name("_timestamp"), - valid_seconds=300, - ) - - aggs = [] - for meas in ["meas1", "meas2"]: - aggs.append(src_table[meas].sum().cast("float").name(meas)) - src_table = src_table.aggregate( - aggs, by=["_timestamp", "dim1", "dim2", "valid_seconds"] - ) - - part_keys = ["year", "month", "day", "hour", "minute"] - ts_col = src_table["_timestamp"].cast("timestamp") - new_cols = {} - for part_key in part_keys: - part_col = getattr(ts_col, part_key)() - new_cols[part_key] = part_col - src_table = src_table.mutate(**new_cols) - return src_table[ - [ - "_timestamp", - "dim1", - "dim2", - "meas1", - "meas2", - "year", - "month", - "day", - "hour", - "minute", - ] - ] - - -@pytest.fixture(scope="module") -def large_expr(base): - return make_large_expr(base) - - -@pytest.mark.benchmark(group="construction") -@pytest.mark.parametrize( - "construction_fn", - [ - pytest.param(lambda *_: make_t(), id="small"), - pytest.param(lambda t, *_: make_base(t), id="medium"), - pytest.param(lambda _, base: make_large_expr(base), id="large"), - ], -) -def test_construction(benchmark, construction_fn, t, base): - benchmark(construction_fn, t, base) - - -@pytest.mark.benchmark(group="builtins") -@pytest.mark.parametrize( - "expr_fn", - [ - pytest.param(lambda t, _base, _large_expr: t, id="small"), - pytest.param(lambda _t, base, _large_expr: base, id="medium"), - pytest.param(lambda _t, _base, large_expr: large_expr, id="large"), - ], -) -@pytest.mark.parametrize("builtin", [hash, str]) -def test_builtins(benchmark, expr_fn, builtin, t, base, large_expr): - expr = expr_fn(t, base, large_expr) - benchmark(builtin, expr) - - -_backends = set(_get_backend_names()) -# compile is a no-op -_backends.remove("pandas") - -_XFAIL_COMPILE_BACKENDS = {"dask", "pyspark", "polars", "risingwave"} - - -@pytest.mark.benchmark(group="compilation") -@pytest.mark.parametrize( - "module", - [ - pytest.param( - mod, - marks=pytest.mark.xfail( - condition=mod in _XFAIL_COMPILE_BACKENDS, - reason=f"{mod} backend doesn't support compiling UnboundTable", - ), - ) - for mod in _backends - ], -) -@pytest.mark.parametrize( - "expr_fn", - [ - pytest.param(lambda t, _base, _large_expr: t, id="small"), - pytest.param(lambda _t, base, _large_expr: base, id="medium"), - pytest.param(lambda _t, _base, large_expr: large_expr, id="large"), - ], -) -def test_compile(benchmark, module, expr_fn, t, base, large_expr): - try: - mod = getattr(ibis, module) - except (AttributeError, ImportError) as e: - pytest.skip(str(e)) - else: - expr = expr_fn(t, base, large_expr) - try: - benchmark(mod.compile, expr) - except (sa.exc.NoSuchModuleError, ImportError) as e: # delayed imports - pytest.skip(str(e)) - - -@pytest.fixture(scope="module") -def pt(): - n = 60_000 - data = pd.DataFrame( - { - "key": np.random.choice(16000, size=n), - "low_card_key": np.random.choice(30, size=n), - "value": np.random.rand(n), - "timestamps": pd.date_range( - start="2023-05-05 16:37:57", periods=n, freq="s" - ).values, - "timestamp_strings": pd.date_range( - start="2023-05-05 16:37:39", periods=n, freq="s" - ).values.astype(str), - "repeated_timestamps": pd.date_range(start="2018-09-01", periods=30).repeat( - int(n / 30) - ), - } - ) - - return ibis.pandas.connect(dict(df=data)).table("df") - - -def high_card_group_by(t): - return t.group_by(t.key).aggregate(avg_value=t.value.mean()) - - -def cast_to_dates(t): - return t.timestamps.cast(dt.date) - - -def cast_to_dates_from_strings(t): - return t.timestamp_strings.cast(dt.date) - - -def multikey_group_by_with_mutate(t): - return ( - t.mutate(dates=t.timestamps.cast("date")) - .group_by(["low_card_key", "dates"]) - .aggregate(avg_value=lambda t: t.value.mean()) - ) - - -def simple_sort(t): - return t.order_by([t.key]) - - -def simple_sort_projection(t): - return t[["key", "value"]].order_by(["key"]) - - -def multikey_sort(t): - return t.order_by(["low_card_key", "key"]) - - -def multikey_sort_projection(t): - return t[["low_card_key", "key", "value"]].order_by(["low_card_key", "key"]) - - -def low_card_rolling_window(t): - return ibis.trailing_range_window( - ibis.interval(days=2), - order_by=t.repeated_timestamps, - group_by=t.low_card_key, - ) - - -def low_card_grouped_rolling(t): - return t.value.mean().over(low_card_rolling_window(t)) - - -def high_card_rolling_window(t): - return ibis.trailing_range_window( - ibis.interval(days=2), - order_by=t.repeated_timestamps, - group_by=t.key, - ) - - -def high_card_grouped_rolling(t): - return t.value.mean().over(high_card_rolling_window(t)) - - -# @udf.reduction(["double"], "double") -# def my_mean(series): -# return series.mean() - - -def low_card_grouped_rolling_udf_mean(t): - return my_mean(t.value).over(low_card_rolling_window(t)) - - -def high_card_grouped_rolling_udf_mean(t): - return my_mean(t.value).over(high_card_rolling_window(t)) - - -# @udf.analytic(["double"], "double") -# def my_zscore(series): -# return (series - series.mean()) / series.std() - - -def low_card_window(t): - return ibis.window(group_by=t.low_card_key) - - -def high_card_window(t): - return ibis.window(group_by=t.key) - - -def low_card_window_analytics_udf(t): - return my_zscore(t.value).over(low_card_window(t)) - - -def high_card_window_analytics_udf(t): - return my_zscore(t.value).over(high_card_window(t)) - - -# @udf.reduction(["double", "double"], "double") -# def my_wm(v, w): -# return np.average(v, weights=w) - - -def low_card_grouped_rolling_udf_wm(t): - return my_wm(t.value, t.value).over(low_card_rolling_window(t)) - - -def high_card_grouped_rolling_udf_wm(t): - return my_wm(t.value, t.value).over(low_card_rolling_window(t)) - - -broken_pandas_grouped_rolling = pytest.mark.xfail( - condition=vparse("1.4") <= vparse(pd.__version__) < vparse("1.4.2"), - raises=ValueError, - reason="https://github.com/pandas-dev/pandas/pull/44068", -) - - -@pytest.mark.benchmark(group="execution") -@pytest.mark.parametrize( - "expression_fn", - [ - pytest.param(high_card_group_by, id="high_card_group_by"), - pytest.param(cast_to_dates, id="cast_to_dates"), - pytest.param(cast_to_dates_from_strings, id="cast_to_dates_from_strings"), - pytest.param(multikey_group_by_with_mutate, id="multikey_group_by_with_mutate"), - pytest.param(simple_sort, id="simple_sort"), - pytest.param(simple_sort_projection, id="simple_sort_projection"), - pytest.param(multikey_sort, id="multikey_sort"), - pytest.param(multikey_sort_projection, id="multikey_sort_projection"), - pytest.param( - low_card_grouped_rolling, - id="low_card_grouped_rolling", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - high_card_grouped_rolling, - id="high_card_grouped_rolling", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - low_card_grouped_rolling_udf_mean, - id="low_card_grouped_rolling_udf_mean", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - high_card_grouped_rolling_udf_mean, - id="high_card_grouped_rolling_udf_mean", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param(low_card_window_analytics_udf, id="low_card_window_analytics_udf"), - pytest.param( - high_card_window_analytics_udf, id="high_card_window_analytics_udf" - ), - pytest.param( - low_card_grouped_rolling_udf_wm, - id="low_card_grouped_rolling_udf_wm", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - high_card_grouped_rolling_udf_wm, - id="high_card_grouped_rolling_udf_wm", - marks=[broken_pandas_grouped_rolling], - ), - ], -) -def test_execute(benchmark, expression_fn, pt): - expr = expression_fn(pt) - benchmark(expr.execute) - - -@pytest.fixture(scope="module") -def part(): - return ibis.table( - dict( - p_partkey="int64", - p_size="int64", - p_type="string", - p_mfgr="string", - ), - name="part", - ) - - -@pytest.fixture(scope="module") -def supplier(): - return ibis.table( - dict( - s_suppkey="int64", - s_nationkey="int64", - s_name="string", - s_acctbal="decimal(15, 3)", - s_address="string", - s_phone="string", - s_comment="string", - ), - name="supplier", - ) - - -@pytest.fixture(scope="module") -def partsupp(): - return ibis.table( - dict( - ps_partkey="int64", - ps_suppkey="int64", - ps_supplycost="decimal(15, 3)", - ), - name="partsupp", - ) - - -@pytest.fixture(scope="module") -def nation(): - return ibis.table( - dict(n_nationkey="int64", n_regionkey="int64", n_name="string"), - name="nation", - ) - - -@pytest.fixture(scope="module") -def region(): - return ibis.table(dict(r_regionkey="int64", r_name="string"), name="region") - - -@pytest.fixture(scope="module") -def tpc_h02(part, supplier, partsupp, nation, region): - REGION = "EUROPE" - SIZE = 25 - TYPE = "BRASS" - - expr = ( - part.join(partsupp, part.p_partkey == partsupp.ps_partkey) - .join(supplier, supplier.s_suppkey == partsupp.ps_suppkey) - .join(nation, supplier.s_nationkey == nation.n_nationkey) - .join(region, nation.n_regionkey == region.r_regionkey) - ) - - subexpr = ( - partsupp.join(supplier, supplier.s_suppkey == partsupp.ps_suppkey) - .join(nation, supplier.s_nationkey == nation.n_nationkey) - .join(region, nation.n_regionkey == region.r_regionkey) - ) - - subexpr = subexpr[ - (subexpr.r_name == REGION) & (expr.p_partkey == subexpr.ps_partkey) - ] - - filters = [ - expr.p_size == SIZE, - expr.p_type.like(f"%{TYPE}"), - expr.r_name == REGION, - expr.ps_supplycost == subexpr.ps_supplycost.min(), - ] - q = expr.filter(filters) - - q = q.select( - [ - q.s_acctbal, - q.s_name, - q.n_name, - q.p_partkey, - q.p_mfgr, - q.s_address, - q.s_phone, - q.s_comment, - ] - ) - - return q.order_by( - [ - ibis.desc(q.s_acctbal), - q.n_name, - q.s_name, - q.p_partkey, - ] - ).limit(100) - - -@pytest.mark.benchmark(group="repr") -def test_repr_tpc_h02(benchmark, tpc_h02): - benchmark(repr, tpc_h02) - - -@pytest.mark.benchmark(group="repr") -def test_repr_huge_union(benchmark): - n = 10 - raw_types = [ - "int64", - "float64", - "string", - "array, b: map>>>", - ] - tables = [ - ibis.table( - list(zip(string.ascii_letters, itertools.cycle(raw_types))), - name=f"t{i:d}", - ) - for i in range(n) - ] - expr = functools.reduce(ir.Table.union, tables) - benchmark(repr, expr) - - -@pytest.mark.benchmark(group="node_args") -def test_op_argnames(benchmark): - t = ibis.table([("a", "int64")]) - expr = t[["a"]] - benchmark(lambda op: op.argnames, expr.op()) - - -@pytest.mark.benchmark(group="node_args") -def test_op_args(benchmark): - t = ibis.table([("a", "int64")]) - expr = t[["a"]] - benchmark(lambda op: op.args, expr.op()) - - -@pytest.mark.benchmark(group="datatype") -def test_complex_datatype_parse(benchmark): - type_str = "array, b: map>>>" - expected = dt.Array( - dt.Struct(dict(a=dt.Array(dt.string), b=dt.Map(dt.string, dt.Array(dt.int64)))) - ) - assert dt.parse(type_str) == expected - benchmark(dt.parse, type_str) - - -@pytest.mark.benchmark(group="datatype") -@pytest.mark.parametrize("func", [str, hash]) -def test_complex_datatype_builtins(benchmark, func): - datatype = dt.Array( - dt.Struct(dict(a=dt.Array(dt.string), b=dt.Map(dt.string, dt.Array(dt.int64)))) - ) - benchmark(func, datatype) - - -@pytest.mark.benchmark(group="equality") -def test_large_expr_equals(benchmark, tpc_h02): - benchmark(ir.Expr.equals, tpc_h02, copy.deepcopy(tpc_h02)) - - -@pytest.mark.benchmark(group="datatype") -@pytest.mark.parametrize( - "dtypes", - [ - pytest.param( - [ - obj - for _, obj in inspect.getmembers( - dt, - lambda obj: isinstance(obj, dt.DataType), - ) - ], - id="singletons", - ), - pytest.param( - dt.Array( - dt.Struct( - dict( - a=dt.Array(dt.string), - b=dt.Map(dt.string, dt.Array(dt.int64)), - ) - ) - ), - id="complex", - ), - ], -) -def test_eq_datatypes(benchmark, dtypes): - def eq(a, b): - assert a == b - - benchmark(eq, dtypes, copy.deepcopy(dtypes)) - - -def multiple_joins(table, num_joins): - for _ in range(num_joins): - table = table.mutate(dummy=ibis.literal("")) - table = table.left_join(table, ["dummy"])[[table]] - - -@pytest.mark.parametrize("num_joins", [1, 10]) -@pytest.mark.parametrize("num_columns", [1, 10, 100]) -def test_multiple_joins(benchmark, num_joins, num_columns): - table = ibis.table( - {f"col_{i:d}": "string" for i in range(num_columns)}, - name="t", - ) - benchmark(multiple_joins, table, num_joins) - - -@pytest.fixture -def customers(): - return ibis.table( - dict( - customerid="int32", - name="string", - address="string", - citystatezip="string", - birthdate="date", - phone="string", - timezone="string", - lat="float64", - long="float64", - ), - name="customers", - ) - - -@pytest.fixture -def orders(): - return ibis.table( - dict( - orderid="int32", - customerid="int32", - ordered="timestamp", - shipped="timestamp", - items="string", - total="float64", - ), - name="orders", - ) - - -@pytest.fixture -def orders_items(): - return ibis.table( - dict(orderid="int32", sku="string", qty="int32", unit_price="float64"), - name="orders_items", - ) - - -@pytest.fixture -def products(): - return ibis.table( - dict( - sku="string", - desc="string", - weight_kg="float64", - cost="float64", - dims_cm="string", - ), - name="products", - ) - - -@pytest.mark.benchmark(group="compilation") -@pytest.mark.parametrize( - "module", - [ - pytest.param( - mod, - marks=pytest.mark.xfail( - condition=mod in _XFAIL_COMPILE_BACKENDS, - reason=f"{mod} backend doesn't support compiling UnboundTable", - ), - ) - for mod in _backends - ], -) -def test_compile_with_drops( - benchmark, module, customers, orders, orders_items, products -): - expr = ( - customers.join(orders, "customerid") - .join(orders_items, "orderid") - .join(products, "sku") - .drop("customerid", "qty", "total", "items") - .drop("dims_cm", "cost") - .mutate(o_date=lambda t: t.shipped.date()) - .filter(lambda t: t.ordered == t.shipped) - ) - - try: - mod = getattr(ibis, module) - except (AttributeError, ImportError) as e: - pytest.skip(str(e)) - else: - try: - benchmark(mod.compile, expr) - except sa.exc.NoSuchModuleError as e: - pytest.skip(str(e)) - - -def test_repr_join(benchmark, customers, orders, orders_items, products): - expr = ( - customers.join(orders, "customerid") - .join(orders_items, "orderid") - .join(products, "sku") - .drop("customerid", "qty", "total", "items") - ) - op = expr.op() - benchmark(repr, op) - - -@pytest.mark.parametrize("overwrite", [True, False], ids=["overwrite", "no_overwrite"]) -def test_insert_duckdb(benchmark, overwrite, tmp_path): - pytest.importorskip("duckdb") - - n_rows = int(1e4) - table_name = "t" - schema = ibis.schema(dict(a="int64", b="int64", c="int64")) - t = ibis.memtable(dict.fromkeys(list("abc"), range(n_rows)), schema=schema) - - con = ibis.duckdb.connect(tmp_path / "test_insert.ddb") - con.create_table(table_name, schema=schema) - benchmark(con.insert, table_name, t, overwrite=overwrite) - - -def test_snowflake_medium_sized_to_pandas(benchmark): - pytest.importorskip("snowflake.connector") - - if (url := os.environ.get("SNOWFLAKE_URL")) is None: - pytest.skip("SNOWFLAKE_URL environment variable not set") - - con = ibis.connect(url) - - # LINEITEM at scale factor 1 is around 6MM rows, but we limit to 1,000,000 - # to make the benchmark fast enough for development, yet large enough to show a - # difference if there's a performance hit - lineitem = con.table("LINEITEM", schema="SNOWFLAKE_SAMPLE_DATA.TPCH_SF1").limit( - 1_000_000 - ) - - benchmark.pedantic(lineitem.to_pandas, rounds=5, iterations=1, warmup_rounds=1) - - -def test_parse_many_duckdb_types(benchmark): - parse = pytest.importorskip("ibis.backends.duckdb.datatypes").DuckDBType.from_string - - def parse_many(types): - list(map(parse, types)) - - types = ["VARCHAR", "INTEGER", "DOUBLE", "BIGINT"] * 1000 - benchmark(parse_many, types) - - -@pytest.fixture(scope="session") -def sql() -> str: - return """ - SELECT t1.id as t1_id, x, t2.id as t2_id, y - FROM t1 INNER JOIN t2 - ON t1.id = t2.id - """ - - -@pytest.fixture(scope="session") -def ddb(tmp_path_factory): - duckdb = pytest.importorskip("duckdb") - - N = 20_000_000 - - con = duckdb.connect() - - path = str(tmp_path_factory.mktemp("duckdb") / "data.ddb") - sql = ( - lambda var, table, n=N: f""" - CREATE TABLE {table} AS - SELECT ROW_NUMBER() OVER () AS id, {var} - FROM ( - SELECT {var} - FROM RANGE({n}) _ ({var}) - ORDER BY RANDOM() - ) - """ - ) - - with duckdb.connect(path) as con: - con.execute(sql("x", table="t1")) - con.execute(sql("y", table="t2")) - return path - - -def test_duckdb_to_pyarrow(benchmark, sql, ddb) -> None: - # yes, we're benchmarking duckdb here, not ibis - # - # we do this to get a baseline for comparison - duckdb = pytest.importorskip("duckdb") - con = duckdb.connect(ddb, read_only=True) - - benchmark(lambda sql: con.sql(sql).to_arrow_table(), sql) - - -def test_ibis_duckdb_to_pyarrow(benchmark, sql, ddb) -> None: - pytest.importorskip("duckdb") - - con = ibis.duckdb.connect(ddb, read_only=True) - - expr = con.sql(sql) - benchmark(expr.to_pyarrow) - - -@pytest.fixture -def diffs(): - return ibis.table( - { - "id": "int64", - "validation_name": "string", - "difference": "float64", - "pct_difference": "float64", - "pct_threshold": "float64", - "validation_status": "string", - }, - name="diffs", - ) - - -@pytest.fixture -def srcs(): - return ibis.table( - { - "id": "int64", - "validation_name": "string", - "validation_type": "string", - "aggregation_type": "string", - "table_name": "string", - "column_name": "string", - "primary_keys": "string", - "num_random_rows": "string", - "agg_value": "float64", - }, - name="srcs", - ) - - -@pytest.fixture -def nrels(): - return 300 - - -def make_big_union(t, nrels): - return ibis.union(*[t] * nrels) - - -@pytest.fixture -def src(srcs, nrels): - return make_big_union(srcs, nrels) - - -@pytest.fixture -def diff(diffs, nrels): - return make_big_union(diffs, nrels) - - -def test_big_eq_expr(benchmark, src, diff): - benchmark(ops.core.Node.equals, src.op(), diff.op()) - - -def test_big_join_expr(benchmark, src, diff): - benchmark(ir.Table.join, src, diff, ["validation_name"], how="outer") - - -def test_big_join_execute(benchmark, nrels): - pytest.importorskip("duckdb") - - con = ibis.duckdb.connect() - - # cache to avoid a request-per-union operand - src = make_big_union( - con.read_csv( - "https://github.com/ibis-project/ibis/files/12580336/source_pivot.csv" - ) - .rename(id="column0") - .cache(), - nrels, - ) - - diff = make_big_union( - con.read_csv( - "https://github.com/ibis-project/ibis/files/12580340/differences_pivot.csv" - ) - .rename(id="column0") - .cache(), - nrels, - ) - - expr = src.join(diff, ["validation_name"], how="outer") - t = benchmark.pedantic(expr.to_pyarrow, rounds=1, iterations=1, warmup_rounds=1) - assert len(t) diff --git a/ibis/backends/tests/test_client.py b/ibis/backends/tests/test_client.py index 19868964eb270..8bad125da763a 100644 --- a/ibis/backends/tests/test_client.py +++ b/ibis/backends/tests/test_client.py @@ -25,7 +25,11 @@ import ibis.expr.datatypes as dt import ibis.expr.operations as ops from ibis.backends.conftest import ALL_BACKENDS -from ibis.backends.tests.errors import Py4JJavaError, PyDruidProgrammingError +from ibis.backends.tests.errors import ( + PsycoPg2InternalError, + Py4JJavaError, + PyDruidProgrammingError, +) from ibis.util import gen_name, guid if TYPE_CHECKING: @@ -115,7 +119,8 @@ def test_create_table(backend, con, temp_table, lamduh, sch): marks=[ pytest.mark.notyet(["clickhouse"], reason="Can't specify both"), pytest.mark.notyet( - ["pyspark", "trino", "exasol"], reason="No support for temp tables" + ["pyspark", "trino", "exasol", "risingwave"], + reason="No support for temp tables", ), pytest.mark.never(["polars"], reason="Everything in-memory is temp"), pytest.mark.broken(["mssql"], reason="Incorrect temp table syntax"), @@ -132,7 +137,8 @@ def test_create_table(backend, con, temp_table, lamduh, sch): id="temp, no overwrite", marks=[ pytest.mark.notyet( - ["pyspark", "trino", "exasol"], reason="No support for temp tables" + ["pyspark", "trino", "exasol", "risingwave"], + reason="No support for temp tables", ), pytest.mark.never(["polars"], reason="Everything in-memory is temp"), pytest.mark.broken(["mssql"], reason="Incorrect temp table syntax"), @@ -308,7 +314,7 @@ def tmpcon(alchemy_con): @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", ) def test_create_temporary_table_from_schema(tmpcon, new_schema): @@ -375,7 +381,7 @@ def test_rename_table(con, temp_table, temp_table_orig): ) @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason='Feature is not yet implemented: column constraints "NOT NULL"', ) def test_nullable_input_output(con, temp_table): @@ -719,11 +725,6 @@ def test_unsigned_integer_type(alchemy_con, alchemy_temp_table): marks=mark.postgres, id="postgresql", ), - param( - "postgresql://root:@localhost:4566/dev", - marks=mark.risingwave, - id="risingwave", - ), param( "pyspark://?spark.app.name=test-pyspark", marks=[ @@ -1120,11 +1121,6 @@ def test_set_backend_name(name, monkeypatch): marks=mark.postgres, id="postgres", ), - param( - "postgres://root:@localhost:4566/dev", - marks=mark.risingwave, - id="risingwave", - ), ], ) def test_set_backend_url(url, monkeypatch): @@ -1188,7 +1184,7 @@ def test_create_table_timestamp(con, temp_table): @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", ) def test_persist_expression_ref_count(backend, con, alltypes): @@ -1213,7 +1209,7 @@ def test_persist_expression_ref_count(backend, con, alltypes): @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", ) def test_persist_expression(backend, alltypes): @@ -1232,7 +1228,7 @@ def test_persist_expression(backend, alltypes): @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", ) def test_persist_expression_contextmanager(backend, alltypes): @@ -1253,7 +1249,7 @@ def test_persist_expression_contextmanager(backend, alltypes): @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", ) def test_persist_expression_contextmanager_ref_count(backend, con, alltypes): @@ -1276,7 +1272,7 @@ def test_persist_expression_contextmanager_ref_count(backend, con, alltypes): ) @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", ) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") @@ -1318,7 +1314,7 @@ def test_persist_expression_multiple_refs(backend, con, alltypes): @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") @pytest.mark.never( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", ) def test_persist_expression_repeated_cache(alltypes): @@ -1345,6 +1341,11 @@ def test_persist_expression_repeated_cache(alltypes): ["oracle"], reason="Oracle error message for a missing table/view doesn't include the name of the table", ) +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_persist_expression_release(con, alltypes): non_cached_table = alltypes.mutate( test_column="calculation", other_column="big calc 3" @@ -1431,7 +1432,7 @@ def test_create_schema(con_create_schema): @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: information_schema.schemata is not supported,", ) def test_list_schemas(con_create_schema): diff --git a/ibis/backends/tests/test_dot_sql.py b/ibis/backends/tests/test_dot_sql.py index f2621791442a7..f938ea0143b7b 100644 --- a/ibis/backends/tests/test_dot_sql.py +++ b/ibis/backends/tests/test_dot_sql.py @@ -231,7 +231,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", "risingwave"} +_NO_SQLGLOT_DIALECT = {"pandas", "dask", "druid", "flink"} no_sqlglot_dialect = sorted( # TODO(cpcloud): remove the strict=False hack once backends are ported to # sqlglot @@ -244,11 +244,6 @@ def test_dot_sql_reuse_alias_with_different_types(backend, alltypes, df): "dialect", [*sorted(_get_backend_names() - _NO_SQLGLOT_DIALECT), *no_sqlglot_dialect], ) -@pytest.mark.notyet( - ["risingwave"], - raises=ValueError, - reason="risingwave doesn't support sqlglot.dialects.dialect.Dialect", -) @pytest.mark.notyet(["polars"], raises=PolarsComputeError) @dot_sql_notimpl @dot_sql_never @@ -276,11 +271,6 @@ def test_table_dot_sql_transpile(backend, alltypes, dialect, df): ["druid"], raises=AttributeError, reason="druid doesn't respect column names" ) @pytest.mark.notyet(["snowflake", "bigquery"]) -@pytest.mark.notyet( - ["risingwave"], - raises=ValueError, - reason="risingwave doesn't support sqlglot.dialects.dialect.Dialect", -) @dot_sql_notimpl @dot_sql_never def test_con_dot_sql_transpile(backend, con, dialect, df): @@ -300,11 +290,6 @@ def test_con_dot_sql_transpile(backend, con, dialect, df): @dot_sql_never @pytest.mark.notimpl(["druid", "flink", "polars", "exasol"]) @pytest.mark.notyet(["snowflake"], reason="snowflake column names are case insensitive") -@pytest.mark.notyet( - ["risingwave"], - raises=ValueError, - reason="risingwave doesn't support sqlglot.dialects.dialect.Dialect", -) def test_order_by_no_projection(backend): con = backend.connection expr = ( diff --git a/ibis/backends/tests/test_export.py b/ibis/backends/tests/test_export.py index b3ce4d4cfaf03..02eefb296c3a8 100644 --- a/ibis/backends/tests/test_export.py +++ b/ibis/backends/tests/test_export.py @@ -4,7 +4,6 @@ import pyarrow as pa import pyarrow.csv as pcsv import pytest -import sqlalchemy as sa from pytest import param import ibis @@ -342,11 +341,6 @@ def test_table_to_csv_writer_kwargs(delimiter, tmp_path, awards_players): id="decimal128", marks=[ pytest.mark.notyet(["flink"], raises=NotImplementedError), - pytest.mark.notyet( - ["risingwave"], - raises=sa.exc.DBAPIError, - reason="Feature is not yet implemented: unsupported data type: NUMERIC(38,9)", - ), pytest.mark.notyet(["exasol"], raises=ExaQueryError), ], ), @@ -367,11 +361,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( - ["risingwave"], - raises=sa.exc.DBAPIError, - reason="Feature is not yet implemented: unsupported data type: NUMERIC(76,38)", - ), pytest.mark.notyet(["flink"], raises=NotImplementedError), pytest.mark.notyet(["exasol"], raises=ExaQueryError), ], @@ -495,16 +484,7 @@ def test_to_pandas_batches_empty_table(backend, con): @pytest.mark.parametrize( "n", [ - param( - None, - marks=[ - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="risingwave doesn't support limit null", - ), - ], - ), + None, 1, ], ) @@ -516,19 +496,11 @@ 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", [ - param( - None, - marks=[ - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="risingwave doesn't support limit null", - ), - ], - ), + None, 0, 1, 2, diff --git a/ibis/backends/tests/test_generic.py b/ibis/backends/tests/test_generic.py index f4065eb9058f8..8ffd569a71a9d 100644 --- a/ibis/backends/tests/test_generic.py +++ b/ibis/backends/tests/test_generic.py @@ -9,7 +9,6 @@ import numpy as np import pandas as pd import pytest -import sqlalchemy as sa import toolz from pytest import param @@ -26,6 +25,7 @@ ImpalaHiveServer2Error, MySQLProgrammingError, OracleDatabaseError, + PsycoPg2InternalError, Py4JJavaError, PyDruidProgrammingError, PyODBCDataError, @@ -548,7 +548,7 @@ def test_order_by(backend, alltypes, df, key, df_kwargs): @pytest.mark.notimpl(["dask", "pandas", "polars", "mssql", "druid"]) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function random() does not exist", ) def test_order_by_random(alltypes): @@ -852,12 +852,12 @@ def test_typeof(con): @pytest.mark.notimpl(["datafusion", "druid"]) @pytest.mark.notimpl(["pyspark"], condition=is_older_than("pyspark", "3.5.0")) @pytest.mark.notyet(["dask"], reason="not supported by the backend") +@pytest.mark.notyet(["exasol"], raises=ExaQueryError, reason="not supported by exasol") @pytest.mark.broken( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="https://github.com/risingwavelabs/risingwave/issues/1343", ) -@pytest.mark.notyet(["exasol"], raises=ExaQueryError, reason="not supported by exasol") def test_isin_uncorrelated( backend, batting, awards_players, batting_df, awards_players_df ): @@ -1037,11 +1037,6 @@ def query(t, group_cols): reason="backend doesn't support arrays and we don't implement pivot_longer with unions yet", raises=com.OperationNotDefinedError, ) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason='sql parser error: Expected ), found: TEXT at line:3, column:219 Near "))]) AS anon_1(f1"', -) @pytest.mark.broken( ["trino"], reason="invalid code generated for unnesting a struct", @@ -1163,7 +1158,7 @@ def test_pivot_wider(backend): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function last(double precision) does not exist, do you mean left or least", ) def test_distinct_on_keep(backend, on, keep): @@ -1233,7 +1228,7 @@ def test_distinct_on_keep(backend, on, keep): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function first(double precision) does not exist", ) def test_distinct_on_keep_is_none(backend, on): @@ -1287,8 +1282,6 @@ def test_hash_consistent(backend, alltypes): "pyspark", "risingwave", "sqlite", - "clickhouse", - "mssql", ] ) def test_hashbytes(backend, alltypes): @@ -1322,8 +1315,6 @@ def hash_256(col): "risingwave", "snowflake", "trino", - "pyspark", - "mssql", ] ) @pytest.mark.notyet( @@ -1352,7 +1343,6 @@ def hash_256(col): "pandas", "dask", "oracle", - "risingwave", "snowflake", "sqlite", ] @@ -1513,26 +1503,12 @@ def test_try_cast_func(con, from_val, to_type, func): param( slice(None, None), lambda t: t.count().to_pandas(), - marks=[ - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="risingwave doesn't support limit/offset", - ), - ], id="[:]", ), param(slice(0, 0), lambda _: 0, id="[0:0]"), param( slice(0, None), lambda t: t.count().to_pandas(), - marks=[ - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="risingwave doesn't support limit/offset", - ), - ], id="[0:]", ), # positive stop @@ -1588,11 +1564,6 @@ def test_try_cast_func(con, from_val, to_type, func): raises=ImpalaHiveServer2Error, reason="impala doesn't support OFFSET without ORDER BY", ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="risingwave doesn't support limit/offset", - ), pytest.mark.notyet(["oracle"], raises=com.UnsupportedArgumentError), ], ), @@ -1680,16 +1651,16 @@ def test_static_table_slice(backend, slc, expected_count_fn): raises=com.UnsupportedArgumentError, reason="Removed half-baked dynamic offset functionality for now", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="risingwave doesn't support limit/offset", +) @pytest.mark.notyet( ["trino"], raises=TrinoUserError, reason="backend doesn't support dynamic limit/offset", ) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="risingwave doesn't support limit/offset", -) @pytest.mark.notimpl(["exasol"], raises=ExaQueryError) @pytest.mark.notyet( ["clickhouse"], @@ -1770,16 +1741,16 @@ def test_dynamic_table_slice(backend, slc, expected_count_fn): ) @pytest.mark.notyet(["pyspark"], reason="pyspark doesn't support dynamic limit/offset") @pytest.mark.notyet(["flink"], reason="flink doesn't support dynamic limit/offset") -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="risingwave doesn't support limit/offset", -) @pytest.mark.notyet( ["mssql"], reason="doesn't support dynamic limit/offset; compiles incorrectly in sqlglot", raises=AssertionError, ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="risingwave doesn't support limit/offset", +) def test_dynamic_table_slice_with_computed_offset(backend): t = backend.functional_alltypes @@ -1798,17 +1769,10 @@ 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", "flink", "polars", "snowflake"]) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function random() does not exist", ) def test_sample(backend): @@ -1826,17 +1790,10 @@ def test_sample(backend): backend.assert_frame_equal(empty, df.iloc[:0]) -@pytest.mark.notimpl( - [ - "druid", - "flink", - "polars", - "snowflake", - ] -) +@pytest.mark.notimpl(["druid", "flink", "polars", "snowflake"]) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function random() does not exist", ) def test_sample_memtable(con, backend): @@ -1895,11 +1852,6 @@ def test_substitute(backend): ["dask", "pandas", "polars"], raises=NotImplementedError, reason="not a SQL backend" ) @pytest.mark.notimpl(["flink"], reason="no sqlglot dialect", raises=ValueError) -@pytest.mark.notimpl( - ["risingwave"], - raises=ValueError, - reason="risingwave doesn't support sqlglot.dialects.dialect.Dialect", -) 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 a20cd61a98d38..887c10547b6d5 100644 --- a/ibis/backends/tests/test_join.py +++ b/ibis/backends/tests/test_join.py @@ -198,7 +198,7 @@ def test_semi_join_topk(con, batting, awards_players, func): @pytest.mark.notimpl(["dask", "druid", "exasol", "oracle"]) @pytest.mark.notimpl( - ["postgres", "mssql"], + ["postgres", "mssql", "risingwave"], raises=com.IbisTypeError, reason="postgres can't handle null types columns", ) diff --git a/ibis/backends/tests/test_map.py b/ibis/backends/tests/test_map.py index 4aa30d0796208..74d62db9a8b54 100644 --- a/ibis/backends/tests/test_map.py +++ b/ibis/backends/tests/test_map.py @@ -3,13 +3,12 @@ import numpy as np import pandas as pd import pytest -import sqlalchemy as sa from pytest import param import ibis import ibis.common.exceptions as exc import ibis.expr.datatypes as dt -from ibis.backends.tests.errors import Py4JJavaError +from ibis.backends.tests.errors import PsycoPg2InternalError, Py4JJavaError pytestmark = [ pytest.mark.never( @@ -38,7 +37,7 @@ def test_map_table(backend): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_column_map_values(backend): @@ -73,7 +72,7 @@ def test_column_map_merge(backend): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_literal_map_keys(con): @@ -93,7 +92,7 @@ def test_literal_map_keys(con): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_literal_map_values(con): @@ -145,7 +144,7 @@ def test_map_scalar_contains_key_scalar(con): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_map_scalar_contains_key_column(backend, alltypes, df): @@ -215,7 +214,7 @@ def test_literal_map_merge(con): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_literal_map_getitem_broadcast(backend, alltypes, df): @@ -237,7 +236,7 @@ def test_literal_map_getitem_broadcast(backend, alltypes, df): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_literal_map_get_broadcast(backend, alltypes, df): @@ -269,7 +268,7 @@ def test_literal_map_get_broadcast(backend, alltypes, df): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_map_construct_dict(con, keys, values): @@ -361,7 +360,7 @@ def test_map_get_with_null_on_not_nullable(con, null_value): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_map_get_with_null_on_null_type_with_null(con, null_value): @@ -392,7 +391,7 @@ def test_map_get_with_null_on_null_type_with_non_null(con): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_map_create_table(con, temp_table): @@ -410,7 +409,7 @@ def test_map_create_table(con, temp_table): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function hstore(character varying[], character varying[]) does not exist", ) def test_map_length(con): diff --git a/ibis/backends/tests/test_numeric.py b/ibis/backends/tests/test_numeric.py index 76ffe94aae62c..b92abea1470cf 100644 --- a/ibis/backends/tests/test_numeric.py +++ b/ibis/backends/tests/test_numeric.py @@ -9,7 +9,6 @@ import numpy as np import pandas as pd import pytest -import sqlalchemy as sa from pytest import param import ibis @@ -25,6 +24,7 @@ MySQLOperationalError, OracleDatabaseError, PsycoPg2DivisionByZero, + PsycoPg2InternalError, Py4JError, PyDruidProgrammingError, PyODBCDataError, @@ -254,9 +254,9 @@ def test_numeric_literal(con, backend, expr, expected_types): "dask": decimal.Decimal("1.1"), "exasol": decimal.Decimal("1"), "duckdb": decimal.Decimal("1.1"), - "risingwave": 1.1, "impala": decimal.Decimal("1"), "postgres": decimal.Decimal("1.1"), + "risingwave": decimal.Decimal("1.1"), "pandas": decimal.Decimal("1.1"), "pyspark": decimal.Decimal("1.1"), "mysql": decimal.Decimal("1"), @@ -296,9 +296,9 @@ def test_numeric_literal(con, backend, expr, expected_types): "sqlite": decimal.Decimal("1.1"), "trino": decimal.Decimal("1.1"), "duckdb": decimal.Decimal("1.100000000"), - "risingwave": 1.1, "impala": decimal.Decimal("1.1"), "postgres": decimal.Decimal("1.1"), + "risingwave": decimal.Decimal("1.1"), "pandas": decimal.Decimal("1.1"), "pyspark": decimal.Decimal("1.1"), "mysql": decimal.Decimal("1.1"), @@ -332,8 +332,8 @@ def test_numeric_literal(con, backend, expr, expected_types): "bigquery": decimal.Decimal("1.1"), "sqlite": decimal.Decimal("1.1"), "dask": decimal.Decimal("1.1"), - "risingwave": 1.1, "postgres": decimal.Decimal("1.1"), + "risingwave": decimal.Decimal("1.1"), "pandas": decimal.Decimal("1.1"), "pyspark": decimal.Decimal("1.1"), "clickhouse": decimal.Decimal( @@ -384,10 +384,10 @@ def test_numeric_literal(con, backend, expr, expected_types): ibis.literal(decimal.Decimal("Infinity"), type=dt.decimal), # TODO(krzysztof-kwitt): Should we unify it? { - "risingwave": float("nan"), "bigquery": float("inf"), "sqlite": decimal.Decimal("Infinity"), "postgres": decimal.Decimal("Infinity"), + "risingwave": decimal.Decimal("Infinity"), "pandas": decimal.Decimal("Infinity"), "dask": decimal.Decimal("Infinity"), "pyspark": decimal.Decimal("Infinity"), @@ -406,13 +406,6 @@ def test_numeric_literal(con, backend, expr, expected_types): "Unsupported precision. Supported values: [1 : 76]. Current value: None", raises=NotImplementedError, ), - pytest.mark.broken( - ["trino"], - "(trino.exceptions.TrinoUserError) TrinoUserError(type=USER_ERROR, name=INVALID_LITERAL, " - "message=\"line 1:51: 'Infinity' is not a valid decimal literal\", " - "query_id=20230128_024107_01084_y8zm3)", - raises=sa.exc.ProgrammingError, - ), pytest.mark.notyet( ["mysql", "impala"], raises=com.UnsupportedOperationError ), @@ -455,10 +448,10 @@ def test_numeric_literal(con, backend, expr, expected_types): ibis.literal(decimal.Decimal("-Infinity"), type=dt.decimal), # TODO(krzysztof-kwitt): Should we unify it? { - "risingwave": float("nan"), "bigquery": float("-inf"), "sqlite": decimal.Decimal("-Infinity"), "postgres": decimal.Decimal("-Infinity"), + "risingwave": decimal.Decimal("-Infinity"), "pandas": decimal.Decimal("-Infinity"), "dask": decimal.Decimal("-Infinity"), "pyspark": decimal.Decimal("-Infinity"), @@ -477,13 +470,6 @@ def test_numeric_literal(con, backend, expr, expected_types): "Unsupported precision. Supported values: [1 : 76]. Current value: None", raises=NotImplementedError, ), - pytest.mark.broken( - ["trino"], - "(trino.exceptions.TrinoUserError) TrinoUserError(type=USER_ERROR, name=INVALID_LITERAL, " - "message=\"line 1:51: '-Infinity' is not a valid decimal literal\", " - "query_id=20230128_024107_01084_y8zm3)", - raises=sa.exc.ProgrammingError, - ), pytest.mark.notyet( ["mysql", "impala"], raises=com.UnsupportedOperationError ), @@ -551,13 +537,6 @@ def test_numeric_literal(con, backend, expr, expected_types): "Unsupported precision. Supported values: [1 : 76]. Current value: None", raises=NotImplementedError, ), - pytest.mark.broken( - ["trino"], - "(trino.exceptions.TrinoUserError) TrinoUserError(type=USER_ERROR, name=INVALID_LITERAL, " - "message=\"line 1:51: 'NaN' is not a valid decimal literal\", " - "query_id=20230128_024107_01084_y8zm3)", - raises=sa.exc.ProgrammingError, - ), pytest.mark.notyet( ["mysql", "impala"], raises=com.UnsupportedOperationError ), @@ -754,12 +733,12 @@ def test_isnan_isinf( math.log(5.556, 2), id="log-base", marks=[ + pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function log10(numeric, numeric) does not exist", ), - pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), ], ), param( @@ -773,14 +752,34 @@ def test_isnan_isinf( math.log(5.556, 2), id="log2", marks=[ + pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function log10(numeric, numeric) does not exist", ), - pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), ], ), + param( + L(5.556).log10(), + math.log10(5.556), + id="log10", + ), + param( + L(5.556).radians(), + math.radians(5.556), + id="radians", + ), + param( + L(5.556).degrees(), + math.degrees(5.556), + id="degrees", + ), + param( + L(11) % 3, + 11 % 3, + id="mod", + ), param(L(5.556).log10(), math.log10(5.556), id="log10"), param( L(5.556).radians(), @@ -929,12 +928,12 @@ def test_simple_math_functions_columns( lambda t: t.double_col.add(1).log(2), lambda t: np.log2(t.double_col + 1), marks=[ + pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function log10(numeric, numeric) does not exist", ), - pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), ], id="log2", ), @@ -971,7 +970,7 @@ def test_simple_math_functions_columns( pytest.mark.notimpl(["polars"], raises=com.UnsupportedArgumentError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function log10(numeric, numeric) does not exist", ), ], @@ -1197,7 +1196,6 @@ 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(["risingwave"], raises=sa.exc.InternalError), ], ), param( @@ -1210,7 +1208,6 @@ 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(["risingwave"], raises=sa.exc.InternalError), ], ), param( @@ -1223,7 +1220,6 @@ 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(["risingwave"], raises=sa.exc.InternalError), ], ), param( @@ -1236,7 +1232,6 @@ 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(["risingwave"], raises=sa.exc.InternalError), ], ), param( @@ -1319,6 +1314,7 @@ def test_divide_by_zero(backend, alltypes, df, column, denominator): "snowflake", "trino", "postgres", + "risingwave", "mysql", "druid", "mssql", @@ -1326,11 +1322,6 @@ def test_divide_by_zero(backend, alltypes, df, column, denominator): ], reason="Not SQLAlchemy backends", ) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="Feature is not yet implemented: unsupported data type: NUMERIC(5)", -) def test_sa_default_numeric_precision_and_scale( con, backend, default_precisions, default_scales, temp_table ): @@ -1364,13 +1355,13 @@ def test_sa_default_numeric_precision_and_scale( assert_equal(schema, expected) +@pytest.mark.notimpl(["dask", "pandas", "polars"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function random() does not exist", ) -@pytest.mark.notimpl(["dask", "pandas", "polars"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError) def test_random(con): expr = ibis.random() result = con.execute(expr) @@ -1481,7 +1472,7 @@ def test_constants(con, const): param(lambda t: t.int_col, lambda _: 3, id="col_scalar"), ], ) -@pytest.mark.notimpl(["exasol"], raises=(sa.exc.DBAPIError, ExaQueryError)) +@pytest.mark.notimpl(["exasol"], raises=(ExaQueryError)) @flink_no_bitwise def test_bitwise_columns(backend, con, alltypes, df, op, left_fn, right_fn): expr = op(left_fn(alltypes), right_fn(alltypes)).name("tmp") @@ -1518,7 +1509,7 @@ def test_bitwise_columns(backend, con, alltypes, df, op, left_fn, right_fn): ], ) @pytest.mark.notimpl(["oracle"], raises=OracleDatabaseError) -@pytest.mark.notimpl(["exasol"], raises=(sa.exc.DBAPIError, ExaQueryError)) +@pytest.mark.notimpl(["exasol"], raises=(ExaQueryError)) @flink_no_bitwise def test_bitwise_shift(backend, alltypes, df, op, left_fn, right_fn): expr = op(left_fn(alltypes), right_fn(alltypes)).name("tmp") diff --git a/ibis/backends/tests/test_param.py b/ibis/backends/tests/test_param.py index a72b7c140b222..67c7b51232812 100644 --- a/ibis/backends/tests/test_param.py +++ b/ibis/backends/tests/test_param.py @@ -6,13 +6,16 @@ import numpy as np import pandas as pd import pytest -import sqlalchemy as sa from pytest import param import ibis import ibis.expr.datatypes as dt from ibis import _ -from ibis.backends.tests.errors import OracleDatabaseError, Py4JJavaError +from ibis.backends.tests.errors import ( + OracleDatabaseError, + PsycoPg2InternalError, + Py4JJavaError, +) @pytest.mark.parametrize( @@ -38,11 +41,6 @@ def test_floating_scalar_parameter(backend, alltypes, df, column, raw_value): [("2009-03-01", "2010-07-03"), ("2014-12-01", "2017-01-05")], ) @pytest.mark.notimpl(["trino", "druid"]) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function make_date(integer, integer, integer) does not exist", -) @pytest.mark.broken(["oracle"], raises=OracleDatabaseError) def test_date_scalar_parameter(backend, alltypes, start_string, end_string): start, end = ibis.param(dt.date), ibis.param(dt.date) @@ -116,7 +114,7 @@ def test_scalar_param_struct(con): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function make_date(integer, integer, integer) does not exist", ) def test_scalar_param_map(con): @@ -179,11 +177,6 @@ def test_scalar_param(backend, alltypes, df, value, dtype, col): ids=["string", "date", "datetime"], ) @pytest.mark.notimpl(["druid", "oracle"]) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function make_date(integer, integer, integer) does not exist", -) def test_scalar_param_date(backend, alltypes, value): param = ibis.param("date") ds_col = alltypes.date_string_col diff --git a/ibis/backends/tests/test_register.py b/ibis/backends/tests/test_register.py index f1ecdd6bdab63..64824b6124629 100644 --- a/ibis/backends/tests/test_register.py +++ b/ibis/backends/tests/test_register.py @@ -409,12 +409,7 @@ def test_register_garbage(con, monkeypatch): ], ) @pytest.mark.notyet( - ["impala", "mssql", "mysql", "postgres", "risingwave", "sqlite", "trino"] -) -@pytest.mark.notimpl( - ["flink"], - raises=ValueError, - reason="read_parquet() missing required argument: 'schema'", + ["flink", "impala", "mssql", "mysql", "postgres", "risingwave", "sqlite", "trino"] ) def test_read_parquet(con, tmp_path, data_dir, fname, in_table_name): pq = pytest.importorskip("pyarrow.parquet") @@ -445,12 +440,17 @@ def ft_data(data_dir): @pytest.mark.notyet( - ["impala", "mssql", "mysql", "pandas", "postgres", "risingwave", "sqlite", "trino"] -) -@pytest.mark.notimpl( - ["flink"], - raises=ValueError, - reason="read_parquet() missing required argument: 'schema'", + [ + "flink", + "impala", + "mssql", + "mysql", + "pandas", + "postgres", + "risingwave", + "sqlite", + "trino", + ] ) def test_read_parquet_glob(con, tmp_path, ft_data): pq = pytest.importorskip("pyarrow.parquet") @@ -469,12 +469,17 @@ def test_read_parquet_glob(con, tmp_path, ft_data): @pytest.mark.notyet( - ["impala", "mssql", "mysql", "pandas", "postgres", "risingwave", "sqlite", "trino"] -) -@pytest.mark.notimpl( - ["flink"], - raises=ValueError, - reason="read_csv() missing required argument: 'schema'", + [ + "flink", + "impala", + "mssql", + "mysql", + "pandas", + "postgres", + "risingwave", + "sqlite", + "trino", + ] ) def test_read_csv_glob(con, tmp_path, ft_data): pc = pytest.importorskip("pyarrow.csv") @@ -556,12 +561,7 @@ def num_diamonds(data_dir): [param(None, id="default"), param("fancy_stones", id="file_name")], ) @pytest.mark.notyet( - ["impala", "mssql", "mysql", "postgres", "risingwave", "sqlite", "trino"] -) -@pytest.mark.notimpl( - ["flink"], - raises=ValueError, - reason="read_csv() missing required argument: 'schema'", + ["flink", "impala", "mssql", "mysql", "postgres", "risingwave", "sqlite", "trino"] ) def test_read_csv(con, data_dir, in_table_name, num_diamonds): fname = "diamonds.csv" diff --git a/ibis/backends/tests/test_set_ops.py b/ibis/backends/tests/test_set_ops.py index 41102559ad9c6..4df076da7f978 100644 --- a/ibis/backends/tests/test_set_ops.py +++ b/ibis/backends/tests/test_set_ops.py @@ -4,14 +4,13 @@ import pandas as pd import pytest -import sqlalchemy as sa from pytest import param import ibis import ibis.common.exceptions as com import ibis.expr.types as ir from ibis import _ -from ibis.backends.tests.errors import PyDruidProgrammingError +from ibis.backends.tests.errors import PsycoPg2InternalError, PyDruidProgrammingError @pytest.fixture @@ -84,7 +83,7 @@ def test_union_mixed_distinct(backend, union_subsets): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: INTERSECT all", ), ], @@ -138,7 +137,7 @@ def test_intersect(backend, alltypes, df, distinct): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: EXCEPT all", ), ], @@ -223,7 +222,7 @@ def test_top_level_union(backend, con, alltypes, distinct): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: INTERSECT all", ), ], diff --git a/ibis/backends/tests/test_sql.py b/ibis/backends/tests/test_sql.py index 4cd7d0d8ffa24..0db31eb5662db 100644 --- a/ibis/backends/tests/test_sql.py +++ b/ibis/backends/tests/test_sql.py @@ -8,10 +8,9 @@ from ibis import _ from ibis.backends.conftest import _get_backends_to_test -sa = pytest.importorskip("sqlalchemy") sg = pytest.importorskip("sqlglot") -pytestmark = pytest.mark.notimpl(["flink", "risingwave"]) +pytestmark = pytest.mark.notimpl(["flink"]) simple_literal = param(ibis.literal(1), id="simple_literal") array_literal = param( @@ -27,7 +26,7 @@ ) no_structs = pytest.mark.never( ["impala", "mysql", "sqlite", "mssql", "exasol"], - raises=(NotImplementedError, sa.exc.CompileError, exc.UnsupportedBackendType), + raises=(NotImplementedError, exc.UnsupportedBackendType), reason="structs not supported in the backend", ) no_struct_literals = pytest.mark.notimpl( @@ -62,9 +61,6 @@ def test_literal(backend, expr): @pytest.mark.never(["pandas", "dask", "polars"], reason="not SQL") -@pytest.mark.xfail_version( - mssql=["sqlalchemy>=2"], reason="sqlalchemy 2 prefixes literals with `N`" -) def test_group_by_has_index(backend, snapshot): countries = ibis.table( dict(continent="string", population="int64"), name="countries" diff --git a/ibis/backends/tests/test_string.py b/ibis/backends/tests/test_string.py index e25ab5f62d131..8e4e8d5fc0c48 100644 --- a/ibis/backends/tests/test_string.py +++ b/ibis/backends/tests/test_string.py @@ -5,7 +5,6 @@ import numpy as np import pandas as pd import pytest -import sqlalchemy as sa from pytest import param import ibis @@ -14,6 +13,7 @@ from ibis.backends.tests.errors import ( ClickHouseDatabaseError, OracleDatabaseError, + PsycoPg2InternalError, PyDruidProgrammingError, PyODBCProgrammingError, ) @@ -62,7 +62,7 @@ ), pytest.mark.broken( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason='sql parser error: Expected end of statement, found: "NG\'" at line:1, column:31 Near "SELECT \'STRI"NG\' AS "\'STRI""', ), ], @@ -90,7 +90,7 @@ ), pytest.mark.broken( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason='sql parser error: Expected end of statement, found: "NG\'" at line:1, column:31 Near "SELECT \'STRI"NG\' AS "\'STRI""', ), ], @@ -233,11 +233,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -248,11 +243,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -268,11 +258,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["druid"], reason="No posix support", raises=AssertionError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -283,11 +268,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -300,11 +280,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -317,11 +292,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -334,11 +304,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -349,11 +314,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -364,11 +324,6 @@ def uses_java_re(t): pytest.mark.notimpl( ["mssql", "exasol"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function textregexeq(character varying, character varying) does not exist", - ), ], ), param( @@ -990,7 +945,7 @@ def test_multiple_subs(con): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function levenshtein(character varying, character varying) does not exist", ) @pytest.mark.parametrize( diff --git a/ibis/backends/tests/test_struct.py b/ibis/backends/tests/test_struct.py index 95f7df9f4ea50..f2b8c99fc73b8 100644 --- a/ibis/backends/tests/test_struct.py +++ b/ibis/backends/tests/test_struct.py @@ -79,7 +79,7 @@ def test_literal(backend, con, field): backend.assert_series_equal(result, expected.astype(dtype)) -@pytest.mark.notimpl(["postgres", "risingwave"]) +@pytest.mark.notimpl(["postgres"]) @pytest.mark.parametrize("field", ["a", "b", "c"]) @pytest.mark.notyet( ["clickhouse"], reason="clickhouse doesn't support nullable nested types" diff --git a/ibis/backends/tests/test_temporal.py b/ibis/backends/tests/test_temporal.py index 68a9306c26d53..3573a598ddf60 100644 --- a/ibis/backends/tests/test_temporal.py +++ b/ibis/backends/tests/test_temporal.py @@ -9,7 +9,6 @@ import numpy as np import pandas as pd import pytest -import sqlalchemy as sa import sqlglot as sg from pytest import param @@ -30,6 +29,7 @@ OracleDatabaseError, PolarsComputeError, PolarsPanicException, + PsycoPg2InternalError, Py4JJavaError, PyDruidProgrammingError, PyODBCProgrammingError, @@ -152,6 +152,11 @@ def test_timestamp_extract(backend, alltypes, df, attr): raises=AssertionError, reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", ), + pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", + ), ], ), ], @@ -634,7 +639,7 @@ def test_date_truncate(backend, alltypes, df, unit): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Bind error: Invalid unit: week", ), ], @@ -657,7 +662,7 @@ def test_date_truncate(backend, alltypes, df, unit): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Bind error: Invalid unit: millisecond", ), ], @@ -681,7 +686,7 @@ def test_date_truncate(backend, alltypes, df, unit): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Bind error: Invalid unit: microsecond", ), ], @@ -738,7 +743,7 @@ def convert_to_offset(offset, displacement_type=displacement_type): pytest.mark.notyet(["trino"], raises=com.UnsupportedOperationError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Bind error: Invalid unit: week", ), ], @@ -831,7 +836,7 @@ def convert_to_offset(x): id="timestamp-add-interval-binop", marks=[ pytest.mark.notimpl( - ["dask", "risingwave", "snowflake", "sqlite", "bigquery", "exasol"], + ["dask", "snowflake", "sqlite", "bigquery", "exasol"], raises=com.OperationNotDefinedError, ), pytest.mark.notimpl(["impala"], raises=com.UnsupportedOperationError), @@ -851,14 +856,7 @@ def convert_to_offset(x): id="timestamp-add-interval-binop-different-units", marks=[ pytest.mark.notimpl( - [ - "sqlite", - "risingwave", - "polars", - "snowflake", - "bigquery", - "exasol", - ], + ["sqlite", "polars", "snowflake", "bigquery", "exasol"], raises=com.OperationNotDefinedError, ), pytest.mark.notimpl(["impala"], raises=com.UnsupportedOperationError), @@ -982,11 +980,6 @@ def convert_to_offset(x): raises=AttributeError, reason="'StringColumn' object has no attribute 'date'", ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function make_date(integer, integer, integer) does not exist", - ), pytest.mark.broken( ["flink"], raises=com.UnsupportedOperationError, @@ -1428,13 +1421,6 @@ def test_interval_add_cast_column(backend, alltypes, df): ), "%Y%m%d", marks=[ - pytest.mark.notimpl( - [ - "risingwave", - ], - raises=AttributeError, - reason="Neither 'concat' object nor 'Comparator' object has an attribute 'value'", - ), pytest.mark.notimpl( [ "polars", @@ -1617,7 +1603,6 @@ def test_integer_to_timestamp(backend, con, unit): [ "dask", "pandas", - "risingwave", "clickhouse", "sqlite", "datafusion", @@ -1723,6 +1708,11 @@ def test_day_of_week_column(backend, alltypes, df): "Ref: https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/dev/table/functions/systemfunctions/#temporal-functions" ), ), + pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", + ), ], ), ], @@ -1803,7 +1793,7 @@ def test_now_from_projection(alltypes): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=com.OperationNotDefinedError, reason="function make_date(integer, integer, integer) does not exist", ) def test_date_literal(con, backend): @@ -1837,7 +1827,7 @@ def test_date_literal(con, backend): @pytest.mark.notyet(["impala"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function make_timestamp(integer, integer, integer, integer, integer, integer) does not exist", ) def test_timestamp_literal(con, backend): @@ -1895,7 +1885,7 @@ def test_timestamp_literal(con, backend): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function make_timestamp(integer, integer, integer, integer, integer, integer) does not exist", ) def test_timestamp_with_timezone_literal(con, timezone, expected): @@ -1928,7 +1918,7 @@ def test_timestamp_with_timezone_literal(con, timezone, expected): @pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function make_time(integer, integer, integer) does not exist", ) def test_time_literal(con, backend): @@ -2078,7 +2068,7 @@ def test_interval_literal(con, backend): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=com.OperationNotDefinedError, reason="function make_date(integer, integer, integer) does not exist", ) def test_date_column_from_ymd(backend, con, alltypes, df): @@ -2100,12 +2090,12 @@ def test_date_column_from_ymd(backend, con, alltypes, df): raises=AttributeError, reason="StringColumn' object has no attribute 'year'", ) +@pytest.mark.notyet(["impala", "oracle"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function make_timestamp(smallint, smallint, smallint, smallint, smallint, smallint) does not exist", ) -@pytest.mark.notyet(["impala", "oracle"], raises=com.OperationNotDefinedError) def test_timestamp_column_from_ymdhms(backend, con, alltypes, df): c = alltypes.timestamp_col expr = ibis.timestamp( @@ -2234,11 +2224,6 @@ def build_date_col(t): param(lambda _: DATE, build_date_col, id="date_column"), ], ) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function make_date(integer, integer, integer) does not exist", -) def test_timestamp_date_comparison(backend, alltypes, df, left_fn, right_fn): left = left_fn(alltypes) right = right_fn(alltypes) @@ -2360,12 +2345,12 @@ def test_large_timestamp(con): reason="assert Timestamp('2023-01-07 13:20:05.561000') == Timestamp('2023-01-07 13:20:05.561000231')", raises=AssertionError, ), + pytest.mark.notimpl(["exasol"], raises=AssertionError), pytest.mark.notyet( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Parse error: timestamp without time zone Can't cast string to timestamp (expected format is YYYY-MM-DD HH:MM:SS[.D+{up to 6 digits}] or YYYY-MM-DD HH:MM or YYYY-MM-DD or ISO 8601 format)", ), - pytest.mark.notimpl(["exasol"], raises=AssertionError), ], ), ], @@ -2395,11 +2380,6 @@ def test_timestamp_precision_output(con, ts, scale, unit): ], raises=com.OperationNotDefinedError, ) -@pytest.mark.notyet( - ["risingwave"], - reason="risingwave doesn't have any easy way to accurately compute the delta in specific units", - raises=com.OperationNotDefinedError, -) @pytest.mark.parametrize( ("start", "end", "unit", "expected"), [ @@ -2416,7 +2396,7 @@ def test_timestamp_precision_output(con, ts, scale, unit): reason="time types not yet implemented in ibis for the clickhouse backend", ), pytest.mark.notyet( - ["postgres"], + ["postgres", "risingwave"], reason="postgres doesn't have any easy way to accurately compute the delta in specific units", raises=com.OperationNotDefinedError, ), @@ -2565,7 +2545,7 @@ def test_delta(con, start, end, unit, expected): @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function date_bin(interval, timestamp without time zone, timestamp without time zone) does not exist", ) def test_timestamp_bucket(backend, kws, pd_freq): @@ -2604,7 +2584,7 @@ def test_timestamp_bucket(backend, kws, pd_freq): @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="function date_bin(interval, timestamp without time zone, timestamp without time zone) does not exist", ) def test_timestamp_bucket_offset(backend, offset_mins): @@ -2717,11 +2697,6 @@ def test_time_literal_sql(dialect, snapshot, micros): param(datetime.date.fromisoformat, id="fromstring"), ], ) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="function make_date(integer, integer, integer) does not exist", -) def test_date_scalar(con, value, func): expr = ibis.date(func(value)).name("tmp") diff --git a/ibis/backends/tests/test_uuid.py b/ibis/backends/tests/test_uuid.py index 5802727f205dd..7b427bea91737 100644 --- a/ibis/backends/tests/test_uuid.py +++ b/ibis/backends/tests/test_uuid.py @@ -4,7 +4,6 @@ import uuid import pytest -import sqlalchemy.exc import ibis import ibis.common.exceptions as com @@ -21,6 +20,7 @@ "flink": "CHAR(36) NOT NULL", "impala": "STRING", "postgres": "uuid", + "risingwave": "character varying", "snowflake": "VARCHAR", "sqlite": "text", "trino": "uuid", @@ -28,11 +28,6 @@ @pytest.mark.notimpl(["datafusion", "polars"], raises=NotImplementedError) -@pytest.mark.notimpl( - ["risingwave"], - raises=sqlalchemy.exc.InternalError, - reason="Feature is not yet implemented: unsupported data type: UUID", -) @pytest.mark.notimpl(["polars"], raises=NotImplementedError) @pytest.mark.notimpl(["datafusion"], raises=Exception) def test_uuid_literal(con, backend): diff --git a/ibis/backends/tests/test_window.py b/ibis/backends/tests/test_window.py index e68e70773ad78..1841a1ac02875 100644 --- a/ibis/backends/tests/test_window.py +++ b/ibis/backends/tests/test_window.py @@ -6,7 +6,6 @@ import numpy as np import pandas as pd import pytest -import sqlalchemy as sa from pytest import param import ibis @@ -19,6 +18,7 @@ ImpalaHiveServer2Error, MySQLOperationalError, OracleDatabaseError, + PsycoPg2InternalError, Py4JJavaError, PyDruidProgrammingError, PyODBCProgrammingError, @@ -148,7 +148,7 @@ def calc_zscore(s): pytest.mark.notimpl(["dask"], raises=NotImplementedError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Unrecognized window function: percent_rank", ), ], @@ -165,7 +165,7 @@ def calc_zscore(s): pytest.mark.notimpl(["dask"], raises=NotImplementedError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Unrecognized window function: cume_dist", ), ], @@ -196,7 +196,7 @@ def calc_zscore(s): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Unrecognized window function: ntile", ), ], @@ -236,12 +236,8 @@ def calc_zscore(s): ["impala", "mssql"], raises=com.OperationNotDefinedError ), pytest.mark.notimpl(["dask"], raises=NotImplementedError), - pytest.mark.notimpl( - ["flink"], - raises=com.OperationNotDefinedError, - reason="No translation rule for ", - ), - pytest.mark.notimpl(["risingwave"], raises=sa.exc.InternalError), + pytest.mark.notimpl(["flink"], raises=com.OperationNotDefinedError), + pytest.mark.notimpl(["risingwave"], raises=PsycoPg2InternalError), ], ), param( @@ -407,7 +403,7 @@ def test_grouped_bounded_expanding_window( pytest.mark.notimpl(["dask"], raises=NotImplementedError), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ), ], @@ -667,14 +663,10 @@ 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, - reason="OVER RANGE FOLLOWING windows are not supported in Flink yet", -) +@pytest.mark.notimpl(["flink"], raises=com.UnsupportedOperationError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ) def test_simple_ungrouped_unbound_following_window( @@ -706,7 +698,7 @@ def test_simple_ungrouped_unbound_following_window( @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ) def test_simple_ungrouped_window_with_scalar_order_by(alltypes): @@ -739,7 +731,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ), ], @@ -771,16 +763,16 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): pytest.mark.notimpl( ["pandas", "dask"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="Feature is not yet implemented: Unrecognized window function: ntile", - ), pytest.mark.notimpl( ["flink"], raises=Py4JJavaError, reason="CalciteContextException: Argument to function 'NTILE' must be a literal", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Unrecognized window function: ntile", + ), ], ), param( @@ -857,7 +849,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): marks=[ pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ), ], @@ -892,7 +884,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ), ], @@ -905,7 +897,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): marks=[ pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ), ], @@ -943,7 +935,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): ), pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ), ], @@ -1060,17 +1052,17 @@ def test_ungrouped_unbounded_window( reason="RANGE OFFSET frame for 'DB::ColumnNullable' ORDER BY column is not implemented", raises=ClickHouseDatabaseError, ) -@pytest.mark.notimpl( - ["risingwave"], - raises=sa.exc.InternalError, - reason="Feature is not yet implemented: window frame in `RANGE` mode is not supported yet", -) @pytest.mark.notyet(["mssql"], raises=PyODBCProgrammingError) @pytest.mark.broken( ["mysql"], raises=MySQLOperationalError, reason="https://github.com/tobymao/sqlglot/issues/2779", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: window frame in `RANGE` mode is not supported yet", +) def test_grouped_bounded_range_window(backend, alltypes, df): # Explanation of the range window spec below: # @@ -1128,7 +1120,7 @@ def gb_fn(df): ) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Unrecognized window function: percent_rank", ) def test_percent_rank_whole_table_no_order_by(backend, alltypes, df): @@ -1179,7 +1171,7 @@ def agg(df): @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ) def test_mutate_window_filter(backend, alltypes): @@ -1256,16 +1248,16 @@ def test_first_last(backend): ["mssql"], raises=PyODBCProgrammingError, reason="not support by the backend" ) @pytest.mark.broken(["flink"], raises=Py4JJavaError, reason="bug in Flink") -@pytest.mark.broken( - ["risingwave"], - raises=sa.exc.InternalError, - reason="sql parser error: Expected literal int, found: INTERVAL at line:1, column:99", -) @pytest.mark.broken( ["exasol"], raises=ExaQueryError, reason="database can't handle UTC timestamps in DataFrames", ) +@pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="sql parser error: Expected literal int, found: INTERVAL at line:1, column:99", +) def test_range_expression_bounds(backend): t = ibis.memtable( { @@ -1312,7 +1304,7 @@ def test_range_expression_bounds(backend): ) @pytest.mark.broken( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Unrecognized window function: percent_rank", ) def test_rank_followed_by_over_call_merge_frames(backend, alltypes, df): @@ -1347,7 +1339,7 @@ def test_rank_followed_by_over_call_merge_frames(backend, alltypes, df): @pytest.mark.notyet(["flink"], raises=com.UnsupportedOperationError) @pytest.mark.notimpl( ["risingwave"], - raises=sa.exc.InternalError, + raises=PsycoPg2InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ) def test_windowed_order_by_sequence_is_preserved(con): diff --git a/poetry.lock b/poetry.lock index 1120abb6e22ec..83cf246a8e5c7 100644 --- a/poetry.lock +++ b/poetry.lock @@ -2,87 +2,87 @@ [[package]] name = "aiohttp" -version = "3.9.1" +version = "3.9.3" description = "Async http client/server framework (asyncio)" optional = true python-versions = ">=3.8" files = [ - {file = "aiohttp-3.9.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e1f80197f8b0b846a8d5cf7b7ec6084493950d0882cc5537fb7b96a69e3c8590"}, - {file = "aiohttp-3.9.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c72444d17777865734aa1a4d167794c34b63e5883abb90356a0364a28904e6c0"}, - {file = "aiohttp-3.9.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9b05d5cbe9dafcdc733262c3a99ccf63d2f7ce02543620d2bd8db4d4f7a22f83"}, - {file = "aiohttp-3.9.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c4fa235d534b3547184831c624c0b7c1e262cd1de847d95085ec94c16fddcd5"}, - {file = "aiohttp-3.9.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:289ba9ae8e88d0ba16062ecf02dd730b34186ea3b1e7489046fc338bdc3361c4"}, - {file = "aiohttp-3.9.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bff7e2811814fa2271be95ab6e84c9436d027a0e59665de60edf44e529a42c1f"}, - {file = "aiohttp-3.9.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:81b77f868814346662c96ab36b875d7814ebf82340d3284a31681085c051320f"}, - {file = "aiohttp-3.9.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3b9c7426923bb7bd66d409da46c41e3fb40f5caf679da624439b9eba92043fa6"}, - {file = "aiohttp-3.9.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:8d44e7bf06b0c0a70a20f9100af9fcfd7f6d9d3913e37754c12d424179b4e48f"}, - {file = "aiohttp-3.9.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:22698f01ff5653fe66d16ffb7658f582a0ac084d7da1323e39fd9eab326a1f26"}, - {file = "aiohttp-3.9.1-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:ca7ca5abfbfe8d39e653870fbe8d7710be7a857f8a8386fc9de1aae2e02ce7e4"}, - {file = "aiohttp-3.9.1-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:8d7f98fde213f74561be1d6d3fa353656197f75d4edfbb3d94c9eb9b0fc47f5d"}, - {file = "aiohttp-3.9.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:5216b6082c624b55cfe79af5d538e499cd5f5b976820eac31951fb4325974501"}, - {file = "aiohttp-3.9.1-cp310-cp310-win32.whl", hash = "sha256:0e7ba7ff228c0d9a2cd66194e90f2bca6e0abca810b786901a569c0de082f489"}, - {file = "aiohttp-3.9.1-cp310-cp310-win_amd64.whl", hash = "sha256:c7e939f1ae428a86e4abbb9a7c4732bf4706048818dfd979e5e2839ce0159f23"}, - {file = "aiohttp-3.9.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:df9cf74b9bc03d586fc53ba470828d7b77ce51b0582d1d0b5b2fb673c0baa32d"}, - {file = "aiohttp-3.9.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ecca113f19d5e74048c001934045a2b9368d77b0b17691d905af18bd1c21275e"}, - {file = "aiohttp-3.9.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:8cef8710fb849d97c533f259103f09bac167a008d7131d7b2b0e3a33269185c0"}, - {file = "aiohttp-3.9.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bea94403a21eb94c93386d559bce297381609153e418a3ffc7d6bf772f59cc35"}, - {file = "aiohttp-3.9.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:91c742ca59045dce7ba76cab6e223e41d2c70d79e82c284a96411f8645e2afff"}, - {file = "aiohttp-3.9.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6c93b7c2e52061f0925c3382d5cb8980e40f91c989563d3d32ca280069fd6a87"}, - {file = "aiohttp-3.9.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ee2527134f95e106cc1653e9ac78846f3a2ec1004cf20ef4e02038035a74544d"}, - {file = "aiohttp-3.9.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:11ff168d752cb41e8492817e10fb4f85828f6a0142b9726a30c27c35a1835f01"}, - {file = "aiohttp-3.9.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:b8c3a67eb87394386847d188996920f33b01b32155f0a94f36ca0e0c635bf3e3"}, - {file = "aiohttp-3.9.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:c7b5d5d64e2a14e35a9240b33b89389e0035e6de8dbb7ffa50d10d8b65c57449"}, - {file = "aiohttp-3.9.1-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:69985d50a2b6f709412d944ffb2e97d0be154ea90600b7a921f95a87d6f108a2"}, - {file = "aiohttp-3.9.1-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:c9110c06eaaac7e1f5562caf481f18ccf8f6fdf4c3323feab28a93d34cc646bd"}, - {file = "aiohttp-3.9.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d737e69d193dac7296365a6dcb73bbbf53bb760ab25a3727716bbd42022e8d7a"}, - {file = "aiohttp-3.9.1-cp311-cp311-win32.whl", hash = "sha256:4ee8caa925aebc1e64e98432d78ea8de67b2272252b0a931d2ac3bd876ad5544"}, - {file = "aiohttp-3.9.1-cp311-cp311-win_amd64.whl", hash = "sha256:a34086c5cc285be878622e0a6ab897a986a6e8bf5b67ecb377015f06ed316587"}, - {file = "aiohttp-3.9.1-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:f800164276eec54e0af5c99feb9494c295118fc10a11b997bbb1348ba1a52065"}, - {file = "aiohttp-3.9.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:500f1c59906cd142d452074f3811614be04819a38ae2b3239a48b82649c08821"}, - {file = "aiohttp-3.9.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0b0a6a36ed7e164c6df1e18ee47afbd1990ce47cb428739d6c99aaabfaf1b3af"}, - {file = "aiohttp-3.9.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:69da0f3ed3496808e8cbc5123a866c41c12c15baaaead96d256477edf168eb57"}, - {file = "aiohttp-3.9.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:176df045597e674fa950bf5ae536be85699e04cea68fa3a616cf75e413737eb5"}, - {file = "aiohttp-3.9.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b796b44111f0cab6bbf66214186e44734b5baab949cb5fb56154142a92989aeb"}, - {file = "aiohttp-3.9.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f27fdaadce22f2ef950fc10dcdf8048407c3b42b73779e48a4e76b3c35bca26c"}, - {file = "aiohttp-3.9.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:bcb6532b9814ea7c5a6a3299747c49de30e84472fa72821b07f5a9818bce0f66"}, - {file = "aiohttp-3.9.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:54631fb69a6e44b2ba522f7c22a6fb2667a02fd97d636048478db2fd8c4e98fe"}, - {file = "aiohttp-3.9.1-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:4b4c452d0190c5a820d3f5c0f3cd8a28ace48c54053e24da9d6041bf81113183"}, - {file = "aiohttp-3.9.1-cp312-cp312-musllinux_1_1_ppc64le.whl", hash = "sha256:cae4c0c2ca800c793cae07ef3d40794625471040a87e1ba392039639ad61ab5b"}, - {file = "aiohttp-3.9.1-cp312-cp312-musllinux_1_1_s390x.whl", hash = "sha256:565760d6812b8d78d416c3c7cfdf5362fbe0d0d25b82fed75d0d29e18d7fc30f"}, - {file = "aiohttp-3.9.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:54311eb54f3a0c45efb9ed0d0a8f43d1bc6060d773f6973efd90037a51cd0a3f"}, - {file = "aiohttp-3.9.1-cp312-cp312-win32.whl", hash = "sha256:85c3e3c9cb1d480e0b9a64c658cd66b3cfb8e721636ab8b0e746e2d79a7a9eed"}, - {file = "aiohttp-3.9.1-cp312-cp312-win_amd64.whl", hash = "sha256:11cb254e397a82efb1805d12561e80124928e04e9c4483587ce7390b3866d213"}, - {file = "aiohttp-3.9.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:8a22a34bc594d9d24621091d1b91511001a7eea91d6652ea495ce06e27381f70"}, - {file = "aiohttp-3.9.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:598db66eaf2e04aa0c8900a63b0101fdc5e6b8a7ddd805c56d86efb54eb66672"}, - {file = "aiohttp-3.9.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:2c9376e2b09895c8ca8b95362283365eb5c03bdc8428ade80a864160605715f1"}, - {file = "aiohttp-3.9.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:41473de252e1797c2d2293804e389a6d6986ef37cbb4a25208de537ae32141dd"}, - {file = "aiohttp-3.9.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9c5857612c9813796960c00767645cb5da815af16dafb32d70c72a8390bbf690"}, - {file = "aiohttp-3.9.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ffcd828e37dc219a72c9012ec44ad2e7e3066bec6ff3aaa19e7d435dbf4032ca"}, - {file = "aiohttp-3.9.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:219a16763dc0294842188ac8a12262b5671817042b35d45e44fd0a697d8c8361"}, - {file = "aiohttp-3.9.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f694dc8a6a3112059258a725a4ebe9acac5fe62f11c77ac4dcf896edfa78ca28"}, - {file = "aiohttp-3.9.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:bcc0ea8d5b74a41b621ad4a13d96c36079c81628ccc0b30cfb1603e3dfa3a014"}, - {file = "aiohttp-3.9.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:90ec72d231169b4b8d6085be13023ece8fa9b1bb495e4398d847e25218e0f431"}, - {file = "aiohttp-3.9.1-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:cf2a0ac0615842b849f40c4d7f304986a242f1e68286dbf3bd7a835e4f83acfd"}, - {file = "aiohttp-3.9.1-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:0e49b08eafa4f5707ecfb321ab9592717a319e37938e301d462f79b4e860c32a"}, - {file = "aiohttp-3.9.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:2c59e0076ea31c08553e868cec02d22191c086f00b44610f8ab7363a11a5d9d8"}, - {file = "aiohttp-3.9.1-cp38-cp38-win32.whl", hash = "sha256:4831df72b053b1eed31eb00a2e1aff6896fb4485301d4ccb208cac264b648db4"}, - {file = "aiohttp-3.9.1-cp38-cp38-win_amd64.whl", hash = "sha256:3135713c5562731ee18f58d3ad1bf41e1d8883eb68b363f2ffde5b2ea4b84cc7"}, - {file = "aiohttp-3.9.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:cfeadf42840c1e870dc2042a232a8748e75a36b52d78968cda6736de55582766"}, - {file = "aiohttp-3.9.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:70907533db712f7aa791effb38efa96f044ce3d4e850e2d7691abd759f4f0ae0"}, - {file = "aiohttp-3.9.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:cdefe289681507187e375a5064c7599f52c40343a8701761c802c1853a504558"}, - {file = "aiohttp-3.9.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d7481f581251bb5558ba9f635db70908819caa221fc79ee52a7f58392778c636"}, - {file = "aiohttp-3.9.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:49f0c1b3c2842556e5de35f122fc0f0b721334ceb6e78c3719693364d4af8499"}, - {file = "aiohttp-3.9.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0d406b01a9f5a7e232d1b0d161b40c05275ffbcbd772dc18c1d5a570961a1ca4"}, - {file = "aiohttp-3.9.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8d8e4450e7fe24d86e86b23cc209e0023177b6d59502e33807b732d2deb6975f"}, - {file = "aiohttp-3.9.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3c0266cd6f005e99f3f51e583012de2778e65af6b73860038b968a0a8888487a"}, - {file = "aiohttp-3.9.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:ab221850108a4a063c5b8a70f00dd7a1975e5a1713f87f4ab26a46e5feac5a0e"}, - {file = "aiohttp-3.9.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:c88a15f272a0ad3d7773cf3a37cc7b7d077cbfc8e331675cf1346e849d97a4e5"}, - {file = "aiohttp-3.9.1-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:237533179d9747080bcaad4d02083ce295c0d2eab3e9e8ce103411a4312991a0"}, - {file = "aiohttp-3.9.1-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:02ab6006ec3c3463b528374c4cdce86434e7b89ad355e7bf29e2f16b46c7dd6f"}, - {file = "aiohttp-3.9.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:04fa38875e53eb7e354ece1607b1d2fdee2d175ea4e4d745f6ec9f751fe20c7c"}, - {file = "aiohttp-3.9.1-cp39-cp39-win32.whl", hash = "sha256:82eefaf1a996060602f3cc1112d93ba8b201dbf5d8fd9611227de2003dddb3b7"}, - {file = "aiohttp-3.9.1-cp39-cp39-win_amd64.whl", hash = "sha256:9b05d33ff8e6b269e30a7957bd3244ffbce2a7a35a81b81c382629b80af1a8bf"}, - {file = "aiohttp-3.9.1.tar.gz", hash = "sha256:8fc49a87ac269d4529da45871e2ffb6874e87779c3d0e2ccd813c0899221239d"}, + {file = "aiohttp-3.9.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:939677b61f9d72a4fa2a042a5eee2a99a24001a67c13da113b2e30396567db54"}, + {file = "aiohttp-3.9.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:1f5cd333fcf7590a18334c90f8c9147c837a6ec8a178e88d90a9b96ea03194cc"}, + {file = "aiohttp-3.9.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:82e6aa28dd46374f72093eda8bcd142f7771ee1eb9d1e223ff0fa7177a96b4a5"}, + {file = "aiohttp-3.9.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f56455b0c2c7cc3b0c584815264461d07b177f903a04481dfc33e08a89f0c26b"}, + {file = "aiohttp-3.9.3-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:bca77a198bb6e69795ef2f09a5f4c12758487f83f33d63acde5f0d4919815768"}, + {file = "aiohttp-3.9.3-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e083c285857b78ee21a96ba1eb1b5339733c3563f72980728ca2b08b53826ca5"}, + {file = "aiohttp-3.9.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ab40e6251c3873d86ea9b30a1ac6d7478c09277b32e14745d0d3c6e76e3c7e29"}, + {file = "aiohttp-3.9.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:df822ee7feaaeffb99c1a9e5e608800bd8eda6e5f18f5cfb0dc7eeb2eaa6bbec"}, + {file = "aiohttp-3.9.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:acef0899fea7492145d2bbaaaec7b345c87753168589cc7faf0afec9afe9b747"}, + {file = "aiohttp-3.9.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:cd73265a9e5ea618014802ab01babf1940cecb90c9762d8b9e7d2cc1e1969ec6"}, + {file = "aiohttp-3.9.3-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:a78ed8a53a1221393d9637c01870248a6f4ea5b214a59a92a36f18151739452c"}, + {file = "aiohttp-3.9.3-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:6b0e029353361f1746bac2e4cc19b32f972ec03f0f943b390c4ab3371840aabf"}, + {file = "aiohttp-3.9.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:7cf5c9458e1e90e3c390c2639f1017a0379a99a94fdfad3a1fd966a2874bba52"}, + {file = "aiohttp-3.9.3-cp310-cp310-win32.whl", hash = "sha256:3e59c23c52765951b69ec45ddbbc9403a8761ee6f57253250c6e1536cacc758b"}, + {file = "aiohttp-3.9.3-cp310-cp310-win_amd64.whl", hash = "sha256:055ce4f74b82551678291473f66dc9fb9048a50d8324278751926ff0ae7715e5"}, + {file = "aiohttp-3.9.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:6b88f9386ff1ad91ace19d2a1c0225896e28815ee09fc6a8932fded8cda97c3d"}, + {file = "aiohttp-3.9.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c46956ed82961e31557b6857a5ca153c67e5476972e5f7190015018760938da2"}, + {file = "aiohttp-3.9.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:07b837ef0d2f252f96009e9b8435ec1fef68ef8b1461933253d318748ec1acdc"}, + {file = "aiohttp-3.9.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dad46e6f620574b3b4801c68255492e0159d1712271cc99d8bdf35f2043ec266"}, + {file = "aiohttp-3.9.3-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5ed3e046ea7b14938112ccd53d91c1539af3e6679b222f9469981e3dac7ba1ce"}, + {file = "aiohttp-3.9.3-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:039df344b45ae0b34ac885ab5b53940b174530d4dd8a14ed8b0e2155b9dddccb"}, + {file = "aiohttp-3.9.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7943c414d3a8d9235f5f15c22ace69787c140c80b718dcd57caaade95f7cd93b"}, + {file = "aiohttp-3.9.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:84871a243359bb42c12728f04d181a389718710129b36b6aad0fc4655a7647d4"}, + {file = "aiohttp-3.9.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:5eafe2c065df5401ba06821b9a054d9cb2848867f3c59801b5d07a0be3a380ae"}, + {file = "aiohttp-3.9.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:9d3c9b50f19704552f23b4eaea1fc082fdd82c63429a6506446cbd8737823da3"}, + {file = "aiohttp-3.9.3-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:f033d80bc6283092613882dfe40419c6a6a1527e04fc69350e87a9df02bbc283"}, + {file = "aiohttp-3.9.3-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:2c895a656dd7e061b2fd6bb77d971cc38f2afc277229ce7dd3552de8313a483e"}, + {file = "aiohttp-3.9.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1f5a71d25cd8106eab05f8704cd9167b6e5187bcdf8f090a66c6d88b634802b4"}, + {file = "aiohttp-3.9.3-cp311-cp311-win32.whl", hash = "sha256:50fca156d718f8ced687a373f9e140c1bb765ca16e3d6f4fe116e3df7c05b2c5"}, + {file = "aiohttp-3.9.3-cp311-cp311-win_amd64.whl", hash = "sha256:5fe9ce6c09668063b8447f85d43b8d1c4e5d3d7e92c63173e6180b2ac5d46dd8"}, + {file = "aiohttp-3.9.3-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:38a19bc3b686ad55804ae931012f78f7a534cce165d089a2059f658f6c91fa60"}, + {file = "aiohttp-3.9.3-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:770d015888c2a598b377bd2f663adfd947d78c0124cfe7b959e1ef39f5b13869"}, + {file = "aiohttp-3.9.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:ee43080e75fc92bf36219926c8e6de497f9b247301bbf88c5c7593d931426679"}, + {file = "aiohttp-3.9.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:52df73f14ed99cee84865b95a3d9e044f226320a87af208f068ecc33e0c35b96"}, + {file = "aiohttp-3.9.3-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:dc9b311743a78043b26ffaeeb9715dc360335e5517832f5a8e339f8a43581e4d"}, + {file = "aiohttp-3.9.3-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b955ed993491f1a5da7f92e98d5dad3c1e14dc175f74517c4e610b1f2456fb11"}, + {file = "aiohttp-3.9.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:504b6981675ace64c28bf4a05a508af5cde526e36492c98916127f5a02354d53"}, + {file = "aiohttp-3.9.3-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a6fe5571784af92b6bc2fda8d1925cccdf24642d49546d3144948a6a1ed58ca5"}, + {file = "aiohttp-3.9.3-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:ba39e9c8627edc56544c8628cc180d88605df3892beeb2b94c9bc857774848ca"}, + {file = "aiohttp-3.9.3-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:e5e46b578c0e9db71d04c4b506a2121c0cb371dd89af17a0586ff6769d4c58c1"}, + {file = "aiohttp-3.9.3-cp312-cp312-musllinux_1_1_ppc64le.whl", hash = "sha256:938a9653e1e0c592053f815f7028e41a3062e902095e5a7dc84617c87267ebd5"}, + {file = "aiohttp-3.9.3-cp312-cp312-musllinux_1_1_s390x.whl", hash = "sha256:c3452ea726c76e92f3b9fae4b34a151981a9ec0a4847a627c43d71a15ac32aa6"}, + {file = "aiohttp-3.9.3-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:ff30218887e62209942f91ac1be902cc80cddb86bf00fbc6783b7a43b2bea26f"}, + {file = "aiohttp-3.9.3-cp312-cp312-win32.whl", hash = "sha256:38f307b41e0bea3294a9a2a87833191e4bcf89bb0365e83a8be3a58b31fb7f38"}, + {file = "aiohttp-3.9.3-cp312-cp312-win_amd64.whl", hash = "sha256:b791a3143681a520c0a17e26ae7465f1b6f99461a28019d1a2f425236e6eedb5"}, + {file = "aiohttp-3.9.3-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:0ed621426d961df79aa3b963ac7af0d40392956ffa9be022024cd16297b30c8c"}, + {file = "aiohttp-3.9.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:7f46acd6a194287b7e41e87957bfe2ad1ad88318d447caf5b090012f2c5bb528"}, + {file = "aiohttp-3.9.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:feeb18a801aacb098220e2c3eea59a512362eb408d4afd0c242044c33ad6d542"}, + {file = "aiohttp-3.9.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f734e38fd8666f53da904c52a23ce517f1b07722118d750405af7e4123933511"}, + {file = "aiohttp-3.9.3-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b40670ec7e2156d8e57f70aec34a7216407848dfe6c693ef131ddf6e76feb672"}, + {file = "aiohttp-3.9.3-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fdd215b7b7fd4a53994f238d0f46b7ba4ac4c0adb12452beee724ddd0743ae5d"}, + {file = "aiohttp-3.9.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:017a21b0df49039c8f46ca0971b3a7fdc1f56741ab1240cb90ca408049766168"}, + {file = "aiohttp-3.9.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e99abf0bba688259a496f966211c49a514e65afa9b3073a1fcee08856e04425b"}, + {file = "aiohttp-3.9.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:648056db9a9fa565d3fa851880f99f45e3f9a771dd3ff3bb0c048ea83fb28194"}, + {file = "aiohttp-3.9.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:8aacb477dc26797ee089721536a292a664846489c49d3ef9725f992449eda5a8"}, + {file = "aiohttp-3.9.3-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:522a11c934ea660ff8953eda090dcd2154d367dec1ae3c540aff9f8a5c109ab4"}, + {file = "aiohttp-3.9.3-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:5bce0dc147ca85caa5d33debc4f4d65e8e8b5c97c7f9f660f215fa74fc49a321"}, + {file = "aiohttp-3.9.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:4b4af9f25b49a7be47c0972139e59ec0e8285c371049df1a63b6ca81fdd216a2"}, + {file = "aiohttp-3.9.3-cp38-cp38-win32.whl", hash = "sha256:298abd678033b8571995650ccee753d9458dfa0377be4dba91e4491da3f2be63"}, + {file = "aiohttp-3.9.3-cp38-cp38-win_amd64.whl", hash = "sha256:69361bfdca5468c0488d7017b9b1e5ce769d40b46a9f4a2eed26b78619e9396c"}, + {file = "aiohttp-3.9.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:0fa43c32d1643f518491d9d3a730f85f5bbaedcbd7fbcae27435bb8b7a061b29"}, + {file = "aiohttp-3.9.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:835a55b7ca49468aaaac0b217092dfdff370e6c215c9224c52f30daaa735c1c1"}, + {file = "aiohttp-3.9.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:06a9b2c8837d9a94fae16c6223acc14b4dfdff216ab9b7202e07a9a09541168f"}, + {file = "aiohttp-3.9.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:abf151955990d23f84205286938796c55ff11bbfb4ccfada8c9c83ae6b3c89a3"}, + {file = "aiohttp-3.9.3-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:59c26c95975f26e662ca78fdf543d4eeaef70e533a672b4113dd888bd2423caa"}, + {file = "aiohttp-3.9.3-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f95511dd5d0e05fd9728bac4096319f80615aaef4acbecb35a990afebe953b0e"}, + {file = "aiohttp-3.9.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:595f105710293e76b9dc09f52e0dd896bd064a79346234b521f6b968ffdd8e58"}, + {file = "aiohttp-3.9.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c7c8b816c2b5af5c8a436df44ca08258fc1a13b449393a91484225fcb7545533"}, + {file = "aiohttp-3.9.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:f1088fa100bf46e7b398ffd9904f4808a0612e1d966b4aa43baa535d1b6341eb"}, + {file = "aiohttp-3.9.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:f59dfe57bb1ec82ac0698ebfcdb7bcd0e99c255bd637ff613760d5f33e7c81b3"}, + {file = "aiohttp-3.9.3-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:361a1026c9dd4aba0109e4040e2aecf9884f5cfe1b1b1bd3d09419c205e2e53d"}, + {file = "aiohttp-3.9.3-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:363afe77cfcbe3a36353d8ea133e904b108feea505aa4792dad6585a8192c55a"}, + {file = "aiohttp-3.9.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8e2c45c208c62e955e8256949eb225bd8b66a4c9b6865729a786f2aa79b72e9d"}, + {file = "aiohttp-3.9.3-cp39-cp39-win32.whl", hash = "sha256:f7217af2e14da0856e082e96ff637f14ae45c10a5714b63c77f26d8884cf1051"}, + {file = "aiohttp-3.9.3-cp39-cp39-win_amd64.whl", hash = "sha256:27468897f628c627230dba07ec65dc8d0db566923c48f29e084ce382119802bc"}, + {file = "aiohttp-3.9.3.tar.gz", hash = "sha256:90842933e5d1ff760fae6caca4b2b3edba53ba8f4b71e95dacf2818a2aca06f7"}, ] [package.dependencies] @@ -259,37 +259,40 @@ tests-no-zope = ["attrs[tests-mypy]", "cloudpickle", "hypothesis", "pympler", "p [[package]] name = "beartype" -version = "0.16.4" +version = "0.17.0" description = "Unbearably fast runtime type checking in pure Python." optional = false python-versions = ">=3.8.0" files = [ - {file = "beartype-0.16.4-py3-none-any.whl", hash = "sha256:64865952f9dff1e17f22684b3c7286fc79754553b47eaefeb1286224ae8c1bd9"}, - {file = "beartype-0.16.4.tar.gz", hash = "sha256:1ada89cf2d6eb30eb6e156eed2eb5493357782937910d74380918e53c2eae0bf"}, + {file = "beartype-0.17.0-py3-none-any.whl", hash = "sha256:fa84b77a8d037f2a39c4aa2f3dc71854afc7d79312e55a66b338da68fdd48c60"}, + {file = "beartype-0.17.0.tar.gz", hash = "sha256:3226fbba8c53b4e698acdb47dcaf3c0640151c4d405618c281e6631f4112947d"}, ] [package.extras] all = ["typing-extensions (>=3.10.0.0)"] -dev = ["autoapi (>=0.9.0)", "coverage (>=5.5)", "mypy (>=0.800)", "numpy", "pandera", "pydata-sphinx-theme (<=0.7.2)", "pytest (>=4.0.0)", "sphinx", "sphinx (>=4.2.0,<6.0.0)", "sphinxext-opengraph (>=0.7.5)", "tox (>=3.20.1)", "typing-extensions (>=3.10.0.0)"] +dev = ["autoapi (>=0.9.0)", "coverage (>=5.5)", "equinox", "mypy (>=0.800)", "numpy", "pandera", "pydata-sphinx-theme (<=0.7.2)", "pytest (>=4.0.0)", "sphinx", "sphinx (>=4.2.0,<6.0.0)", "sphinxext-opengraph (>=0.7.5)", "torch", "tox (>=3.20.1)", "typing-extensions (>=3.10.0.0)"] doc-rtd = ["autoapi (>=0.9.0)", "pydata-sphinx-theme (<=0.7.2)", "sphinx (>=4.2.0,<6.0.0)", "sphinxext-opengraph (>=0.7.5)"] -test-tox = ["mypy (>=0.800)", "numpy", "pandera", "pytest (>=4.0.0)", "sphinx", "typing-extensions (>=3.10.0.0)"] +test-tox = ["equinox", "mypy (>=0.800)", "numpy", "pandera", "pytest (>=4.0.0)", "sphinx", "torch", "typing-extensions (>=3.10.0.0)"] test-tox-coverage = ["coverage (>=5.5)"] [[package]] name = "beautifulsoup4" -version = "4.12.2" +version = "4.12.3" description = "Screen-scraping library" optional = false python-versions = ">=3.6.0" files = [ - {file = "beautifulsoup4-4.12.2-py3-none-any.whl", hash = "sha256:bd2520ca0d9d7d12694a53d44ac482d181b4ec1888909b035a3dbf40d0f57d4a"}, - {file = "beautifulsoup4-4.12.2.tar.gz", hash = "sha256:492bbc69dca35d12daac71c4db1bfff0c876c00ef4a2ffacce226d4638eb72da"}, + {file = "beautifulsoup4-4.12.3-py3-none-any.whl", hash = "sha256:b80878c9f40111313e55da8ba20bdba06d8fa3969fc68304167741bbf9e082ed"}, + {file = "beautifulsoup4-4.12.3.tar.gz", hash = "sha256:74e3d1928edc070d21748185c46e3fb33490f22f52a3addee9aee0f4f7781051"}, ] [package.dependencies] soupsieve = ">1.2" [package.extras] +cchardet = ["cchardet"] +chardet = ["chardet"] +charset-normalizer = ["charset-normalizer"] html5lib = ["html5lib"] lxml = ["lxml"] @@ -442,33 +445,33 @@ files = [ [[package]] name = "black" -version = "23.12.1" +version = "24.1.1" description = "The uncompromising code formatter." optional = true python-versions = ">=3.8" files = [ - {file = "black-23.12.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e0aaf6041986767a5e0ce663c7a2f0e9eaf21e6ff87a5f95cbf3675bfd4c41d2"}, - {file = "black-23.12.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c88b3711d12905b74206227109272673edce0cb29f27e1385f33b0163c414bba"}, - {file = "black-23.12.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a920b569dc6b3472513ba6ddea21f440d4b4c699494d2e972a1753cdc25df7b0"}, - {file = "black-23.12.1-cp310-cp310-win_amd64.whl", hash = "sha256:3fa4be75ef2a6b96ea8d92b1587dd8cb3a35c7e3d51f0738ced0781c3aa3a5a3"}, - {file = "black-23.12.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:8d4df77958a622f9b5a4c96edb4b8c0034f8434032ab11077ec6c56ae9f384ba"}, - {file = "black-23.12.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:602cfb1196dc692424c70b6507593a2b29aac0547c1be9a1d1365f0d964c353b"}, - {file = "black-23.12.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c4352800f14be5b4864016882cdba10755bd50805c95f728011bcb47a4afd59"}, - {file = "black-23.12.1-cp311-cp311-win_amd64.whl", hash = "sha256:0808494f2b2df923ffc5723ed3c7b096bd76341f6213989759287611e9837d50"}, - {file = "black-23.12.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:25e57fd232a6d6ff3f4478a6fd0580838e47c93c83eaf1ccc92d4faf27112c4e"}, - {file = "black-23.12.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:2d9e13db441c509a3763a7a3d9a49ccc1b4e974a47be4e08ade2a228876500ec"}, - {file = "black-23.12.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6d1bd9c210f8b109b1762ec9fd36592fdd528485aadb3f5849b2740ef17e674e"}, - {file = "black-23.12.1-cp312-cp312-win_amd64.whl", hash = "sha256:ae76c22bde5cbb6bfd211ec343ded2163bba7883c7bc77f6b756a1049436fbb9"}, - {file = "black-23.12.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1fa88a0f74e50e4487477bc0bb900c6781dbddfdfa32691e780bf854c3b4a47f"}, - {file = "black-23.12.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:a4d6a9668e45ad99d2f8ec70d5c8c04ef4f32f648ef39048d010b0689832ec6d"}, - {file = "black-23.12.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b18fb2ae6c4bb63eebe5be6bd869ba2f14fd0259bda7d18a46b764d8fb86298a"}, - {file = "black-23.12.1-cp38-cp38-win_amd64.whl", hash = "sha256:c04b6d9d20e9c13f43eee8ea87d44156b8505ca8a3c878773f68b4e4812a421e"}, - {file = "black-23.12.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3e1b38b3135fd4c025c28c55ddfc236b05af657828a8a6abe5deec419a0b7055"}, - {file = "black-23.12.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:4f0031eaa7b921db76decd73636ef3a12c942ed367d8c3841a0739412b260a54"}, - {file = "black-23.12.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:97e56155c6b737854e60a9ab1c598ff2533d57e7506d97af5481141671abf3ea"}, - {file = "black-23.12.1-cp39-cp39-win_amd64.whl", hash = "sha256:dd15245c8b68fe2b6bd0f32c1556509d11bb33aec9b5d0866dd8e2ed3dba09c2"}, - {file = "black-23.12.1-py3-none-any.whl", hash = "sha256:78baad24af0f033958cad29731e27363183e140962595def56423e626f4bee3e"}, - {file = "black-23.12.1.tar.gz", hash = "sha256:4ce3ef14ebe8d9509188014d96af1c456a910d5b5cbf434a09fef7e024b3d0d5"}, + {file = "black-24.1.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2588021038bd5ada078de606f2a804cadd0a3cc6a79cb3e9bb3a8bf581325a4c"}, + {file = "black-24.1.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1a95915c98d6e32ca43809d46d932e2abc5f1f7d582ffbe65a5b4d1588af7445"}, + {file = "black-24.1.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2fa6a0e965779c8f2afb286f9ef798df770ba2b6cee063c650b96adec22c056a"}, + {file = "black-24.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:5242ecd9e990aeb995b6d03dc3b2d112d4a78f2083e5a8e86d566340ae80fec4"}, + {file = "black-24.1.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:fc1ec9aa6f4d98d022101e015261c056ddebe3da6a8ccfc2c792cbe0349d48b7"}, + {file = "black-24.1.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:0269dfdea12442022e88043d2910429bed717b2d04523867a85dacce535916b8"}, + {file = "black-24.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b3d64db762eae4a5ce04b6e3dd745dcca0fb9560eb931a5be97472e38652a161"}, + {file = "black-24.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:5d7b06ea8816cbd4becfe5f70accae953c53c0e53aa98730ceccb0395520ee5d"}, + {file = "black-24.1.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:e2c8dfa14677f90d976f68e0c923947ae68fa3961d61ee30976c388adc0b02c8"}, + {file = "black-24.1.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:a21725862d0e855ae05da1dd25e3825ed712eaaccef6b03017fe0853a01aa45e"}, + {file = "black-24.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:07204d078e25327aad9ed2c64790d681238686bce254c910de640c7cc4fc3aa6"}, + {file = "black-24.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:a83fe522d9698d8f9a101b860b1ee154c1d25f8a82ceb807d319f085b2627c5b"}, + {file = "black-24.1.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:08b34e85170d368c37ca7bf81cf67ac863c9d1963b2c1780c39102187ec8dd62"}, + {file = "black-24.1.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7258c27115c1e3b5de9ac6c4f9957e3ee2c02c0b39222a24dc7aa03ba0e986f5"}, + {file = "black-24.1.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:40657e1b78212d582a0edecafef133cf1dd02e6677f539b669db4746150d38f6"}, + {file = "black-24.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:e298d588744efda02379521a19639ebcd314fba7a49be22136204d7ed1782717"}, + {file = "black-24.1.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:34afe9da5056aa123b8bfda1664bfe6fb4e9c6f311d8e4a6eb089da9a9173bf9"}, + {file = "black-24.1.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:854c06fb86fd854140f37fb24dbf10621f5dab9e3b0c29a690ba595e3d543024"}, + {file = "black-24.1.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3897ae5a21ca132efa219c029cce5e6bfc9c3d34ed7e892113d199c0b1b444a2"}, + {file = "black-24.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:ecba2a15dfb2d97105be74bbfe5128bc5e9fa8477d8c46766505c1dda5883aac"}, + {file = "black-24.1.1-py3-none-any.whl", hash = "sha256:5cdc2e2195212208fbcae579b931407c1fa9997584f0a415421748aeafff1168"}, + {file = "black-24.1.1.tar.gz", hash = "sha256:48b5760dcbfe5cf97fd4fba23946681f3a81514c6ab8a45b50da67ac8fbc6c7b"}, ] [package.dependencies] @@ -506,17 +509,17 @@ traittypes = ">=0.0.6" [[package]] name = "branca" -version = "0.7.0" +version = "0.7.1" description = "Generate complex HTML+JS pages with Python" optional = false python-versions = ">=3.7" files = [ - {file = "branca-0.7.0-py3-none-any.whl", hash = "sha256:c653d9a3fef1e6cd203757c77d3eb44810f11998506451f9a27d52b983500c16"}, - {file = "branca-0.7.0.tar.gz", hash = "sha256:503ccb589a9ee9464cb7b5b17e5ffd8d5082c5c28624197f58f20d4d377a68bb"}, + {file = "branca-0.7.1-py3-none-any.whl", hash = "sha256:70515944ed2d1ed2784c552508df58037ca19402a8a1069d57f9113e3e012f51"}, + {file = "branca-0.7.1.tar.gz", hash = "sha256:e6b6f37a37bc0abffd960c68c045a7fe025d628eff87fedf6ab6ca814812110c"}, ] [package.dependencies] -jinja2 = "*" +jinja2 = ">=3" [[package]] name = "build" @@ -818,91 +821,77 @@ dev = ["coveralls", "pytest (>=3.6)", "pytest-cov", "wheel"] [[package]] name = "clickhouse-connect" -version = "0.6.23" +version = "0.7.0" description = "ClickHouse Database Core Driver for Python, Pandas, and Superset" optional = true -python-versions = "~=3.7" -files = [ - {file = "clickhouse-connect-0.6.23.tar.gz", hash = "sha256:a74d01349390c0e2713603539927f105c1b9772cd81f850deb1d1ec57e4d9cfc"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:cfc8e1cd68f7824e90a3492bf64f66934ad95529fac282cf96bc5a50255a5932"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f0726b5f19343dde5b337e8495713a28e0449f42504ea47a691a5a39768ccd79"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e35c767497b22f4c9069f99c24f965ae266b2448e7d2c69407d82f632616bbf7"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c677748c2fb6087fce8fb185980cb539887db2253d0f81900c4a21ef38d7cb89"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:106c680114649d5bfacf76e26bcc87df8d07141b1a3c944099ba0ce297694c7e"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:4cd72d85d246fe000377035b1858720b12045f3df1042cc03a5fad4dba6b7f78"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:4f7c68297ac933603768f5dd4f49c94f88dacd9344e099b0221ead6b9914643e"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:cff31e4fd52517ad79c6d50037c1da7fcaf1270d4c1c70a52852701ff415d86b"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-win32.whl", hash = "sha256:931d9bf3ecd212107e43dd8ed735a79b840b8577d4508b2cf866b1503350b415"}, - {file = "clickhouse_connect-0.6.23-cp310-cp310-win_amd64.whl", hash = "sha256:a9e55a50fb165a7be30d335da84e04ec025b2783999312917df86815dc8be3af"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:4d3a7734e48f0494764ef481c694e02bc78415df60a49d5119c032e75b5e1f8b"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:c5a66bee81dcbdea969f39a7f75b11225e985cfa752dccd148f54bacac800f72"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d62335114e5792fa95548465d831bb33a1b226c85b87b075c7e6c692a5edc77"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:24b9fa3eb7d8cbc87f635f7942cb6817a38c6491c8b40cfb6a7c0a6a8e0d59e4"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ac7c459641dd99fa7e48921d2c4148d8c0cb171697aa487b55364b0b9081bf07"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:746be95964105fa2eca05ab2dab02b1319e9c94f4a9599b4d3c2894f9090c9bc"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:f2593096fc0af049dcf55e03aaf3858bbc94bedddd1fd504087127ec48b68c7b"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:38f480f264b648333f90a9f715f2357bf6ec284a9152d3a4a249dea87c797a60"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-win32.whl", hash = "sha256:f9793b8ae15ca93f7ae5d2c96c7de79ab7f6cf311b0c115d9e7948f0887086a0"}, - {file = "clickhouse_connect-0.6.23-cp311-cp311-win_amd64.whl", hash = "sha256:e6301202780893d5e3f2f62874670790a450bcbf8009d48ca360d04197205efa"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:66883e21a1598688c2a32f46a3ab9a858eca609bcd6fa6e4e0e758993027f356"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:257482716a9563cd2c964543e46af01848779fcb3665063c30b49e13f82ad27a"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a7489202b7eec462e40b52066393f5ec62c82f1e514013a4e9e5f5eab962ad61"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8e297da4ab46a405ce3555b89573cd256c80efc505130e08bac673095d731c6d"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6f5d0097ae7ef1ff13afb2d56e5b93dfca515480d491f280315b0c16ce58c93c"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:1dbd63e6bd2189259a5a9506e8efe5d8117f3d8b114c8d76bb4397eed4751927"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:1d861a3b7b877227fe136e6648f3aca070a69ed407fd77c49722ad6b8f3a5aa2"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:e5912503717237e0d4cafee19fbe579442484bfe6923063053e21e42c952a043"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-win32.whl", hash = "sha256:d288cf60ef846720fa8b2d2758b72dc488072121d331d7c4b27547e935129472"}, - {file = "clickhouse_connect-0.6.23-cp312-cp312-win_amd64.whl", hash = "sha256:4948ca8f292984d1d0cd7ea3bd9aa909101bf62e7d0d7902546d19b3f965f3be"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:ad213ef12b0c5a474e38a13b95113aa3aafe1a35d7e81035b4c1bcc2d8d50e93"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ed9ea8f2be2dcc4cfda1d33ce07d61467c93f1dbb7a98f09d69464b991dcaba0"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e3fd4dcdefcfa2a7175c3c97c53bf66c38544ef84a276932fae4ffcb4c273998"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:102a04bf1eb7612e0924031c751e31dd4bad58e79543c13e8805fddcecbbfe45"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:ac3943d4d522bcb1a4becb8850cdb3bfba07713178e84e4cadcd955b9002e28c"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:7fe4d55c6f8b72eeedce439ed1287ea1971e30b54879df6059dc87f73bf472d2"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:e2dc8127159d5415859af6ac8b42bc70b71709d834477a1dd82e5b147de66e82"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-win32.whl", hash = "sha256:854fcd6cbf7919aa2b9e9f92c52cb5b2d1c976c4e2145193910662237a8d879c"}, - {file = "clickhouse_connect-0.6.23-cp37-cp37m-win_amd64.whl", hash = "sha256:24c17054e395908eb483fad3dd899748a4d919e5b4e0db2a31c56df503f0921d"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:3af001ef95f8dcb572f5cb4518d367b449fa6baf2b8fccc0e6112e04f1932b2b"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:9ad33e0949dd8842beb34e6c5f01fac293bfa7701a2697f64d400d30b2858fe0"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1b73130d09bb6eddf57d94d9c3bca4953af03231cc14a6757fddd9d3839720c3"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f27d725f054c54d6d8340b5545c8713ca6af640c75ade9eb0eef8b441ec37d66"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7ac6bcbf730538106c376d029a0e9aa3e155490cae326e7256a51804d9576345"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:9a7b35ccc8526456ad2794ab6af014506cb4472eed7f864d4d7d58bc4acf3b83"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:3ac5fe6ac94ca77eed1ba440df81b5f4ff99fa85120afe46676e185f3f6f840d"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6eb8576ab497f9425e1dc3595e0fbe6c97615ae5dc7d184f2f65df874bb31539"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-win32.whl", hash = "sha256:c936e1adf8257acfc459282477ad65e2ef38eba26f72d58187154e8cbeaa1641"}, - {file = "clickhouse_connect-0.6.23-cp38-cp38-win_amd64.whl", hash = "sha256:07756dd0c610765000e9d7e202557cb6a06d1e0fd007234458d752998cd8c6da"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:bcc1e103b1af2ae8b0485d1c09a55f8e9cb80e02fdaf8a039b813d07950a9586"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:10e6d20b61e5342fed752fb5221c10517af92182072fc792c5252541850d7340"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3ce7caa2ceff666aaa86da11b9f42ddd09ae7ffe727db1617f352991f7a67667"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a7cafc6ed2214321f3363fe5f23cf9880544ba05dc1820a994f0975bdd7e31d9"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7f7e5ba4c78ef49354fac3d91eb951611430c8be2298f1dc2a1516be3149a41c"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:a3691ed25e971bbf62c8bed843e80eecc0e4cb9b990e662e0916e8f2b05cd4ec"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:8b43948da665bbcb5c60e3871e878e068384cd9d2f845bc02fc4c22c934831cd"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:9d1b7fb932e45482199f4abce61c95ead8e8628cf1fb882c2b28dc11318742da"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-win32.whl", hash = "sha256:4315c7718a9cc8eedc4b40b53a954e2d5a5dfb705c4a659e3a167537889268da"}, - {file = "clickhouse_connect-0.6.23-cp39-cp39-win_amd64.whl", hash = "sha256:040307cad9c11f503290e75d14c0b402454502fa7ab3c742ad8dac1a419219eb"}, - {file = "clickhouse_connect-0.6.23-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:086599316e5a532c34dadcf9fa2ea19a923d0acdcc891a829b3cc5cc061fd26a"}, - {file = "clickhouse_connect-0.6.23-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6bb73a0ee0a0161fce7c38b7f8f66e3c5f0598b8d1f3c30e24ccd17ba1c117b3"}, - {file = "clickhouse_connect-0.6.23-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e4b0111ed72058836a44313369dd05e7c550da8e8ca486834c599ae81c7cd6b"}, - {file = "clickhouse_connect-0.6.23-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e70e97eb15e89995a220fdec19b448b48f8ea65a014e71bc1cc17763a7d8fd0e"}, - {file = "clickhouse_connect-0.6.23-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:8e541c2f8921cd48dc57551391441b35af5991ae49f1d221ba77950ad195b807"}, - {file = "clickhouse_connect-0.6.23-pp37-pypy37_pp73-macosx_10_9_x86_64.whl", hash = "sha256:c7301862b6eb87aeb48f257932f60b3141777cae317217185279b7a036840e07"}, - {file = "clickhouse_connect-0.6.23-pp37-pypy37_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f06348ecd72036d22d65d528221010c86559bdfc048f1633c5cd009993219a0c"}, - {file = "clickhouse_connect-0.6.23-pp37-pypy37_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b40d1ec596f7c3ecf1e0d07916ab8c4f7ee52eb867758977335b285c4916e585"}, - {file = "clickhouse_connect-0.6.23-pp37-pypy37_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:09753a2ce1dfbe0a54fe8a7a464f67b2e0f01c1731f06d68a3ec821a00985d88"}, - {file = "clickhouse_connect-0.6.23-pp37-pypy37_pp73-win_amd64.whl", hash = "sha256:a8ff9600c9721a574c7716a2ad6b436fd043eb05a544ed08d895504d18fb6d5d"}, - {file = "clickhouse_connect-0.6.23-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:7d5ec217ae361c8c18c605030f9d601d861e23dc23af502d9608e769f3e467b4"}, - {file = "clickhouse_connect-0.6.23-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2de6bc3e4ac94545a6f80c49f49ad9a9945efe1084ecd89804ebbb048b022699"}, - {file = "clickhouse_connect-0.6.23-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:76d6277c7ec66585e1f03da95617de172aeb38232d8da8a3e69f67b4f7149017"}, - {file = "clickhouse_connect-0.6.23-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ee1cb7130f0d5e03066e9d4b94ae2052dd96329466c20a3f8688eeebc6f16033"}, - {file = "clickhouse_connect-0.6.23-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:aa7009a68df2f464861c694a15f06221f308ee9758a6848c4541936d0b5a9448"}, - {file = "clickhouse_connect-0.6.23-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:019bf068b38cb0b94fda3cb51d776f556911777197d1a3d0e73eb41259449879"}, - {file = "clickhouse_connect-0.6.23-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a378b6c6b0c092b18e0169b0b94a1f709b80d192e8e6275cfe6eff9c3cb26df0"}, - {file = "clickhouse_connect-0.6.23-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bd2c4356a7a496d8920c756b0ddac916d7a9a902e6183fe4da67c86a6bf19b34"}, - {file = "clickhouse_connect-0.6.23-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:075acc6085c8ece277ce91688a739cbfd54c48de2c839d554045457defdbb81c"}, - {file = "clickhouse_connect-0.6.23-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:7fca7189195785f5ff2a0805e52dd68f684bd5e4f475ba5ade06c5c6769d562e"}, +python-versions = "~=3.8" +files = [ + {file = "clickhouse-connect-0.7.0.tar.gz", hash = "sha256:4fc0c7c58632237d91a26691507ab37dc28233461f6bbe42699a4f36bba86181"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0ca2e06e033afe8bbf5bad97b5766501f11886414b2f5b504a15cf703ad2d774"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:96480e2c36b265ec1b619e610e3d691be33327a0accb8ba4b4e9b3e6e0437e6f"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a8f990b247caa6560f7b5d266d86364c68dbace9d44515c77b62bbd9bdeb8011"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6de3043b32f40d3300a0931ef91d783eba2d67e12437747a59dfda72b796987d"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:80545c875038830bb57f28c37c0d0924f99d149cea8c603528abb37022b66ac5"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:01f8a33949d42085207638ed21d7e5442794680f276f9b3903511f6abe08bdce"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:860e117edfca7b4bdc89aa5f8dd89fc1625f90ec0ced0eccf5572bec205fb2c0"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:32a048eab8d415318c5983db7dfeb73dc431f1a53e2e253fffe795906bed1cc6"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-win32.whl", hash = "sha256:f26d9bc7a25193e4e27e636a8b3162ffd67c29c49945e0087ff27a0fbc87a980"}, + {file = "clickhouse_connect-0.7.0-cp310-cp310-win_amd64.whl", hash = "sha256:ac6a5bc0fb93e003291a22c74802560dc7b47ac8e17c400014728072f3296ce4"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3d5a037afd82a3ea21106f0de0b556e2ec619b2d836af5268381f939f8a78c2d"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:8f4560a6eed525ce02aaa42891876e6566a59427a5c8709533fca3fabd49b308"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4f03e79bce8535936a938eb5c6bb5d83ae51d70f2f8ecc09c9b6b56c55141b36"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7aac5cc6f769ba2b5fc9da6e331cdfe6d1586e3a2af59b28ff9b0408ddd4bf61"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2dedf464abe72e1807b5fc86761760e5e736730c2ca2185ef2931b6d2fac860c"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:3c3af22a296caec7680a1e6a94eccb78e2aabccd33dd5d5f37187566f6072cb2"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:9c0a1e26584bce35642632c28aef99e9a19502ce1148ca78974f0e84fdfe2676"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:a37ba5fe6d9dfde5299b6a04e2c4086ebe6c3b6652e5841de2b12fea51694d99"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-win32.whl", hash = "sha256:739779d942f2103991d85f0d0297a05e17f0ee46dbc370d1420590eb836a44b1"}, + {file = "clickhouse_connect-0.7.0-cp311-cp311-win_amd64.whl", hash = "sha256:b9f2a19a2e53463694046e835dea78dfb1ab1891115148020568dc3d18f40e64"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:6b9e1a818dcb2287c327f2ae5e103094bbeb50656b21f7e1536551f668a6348a"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:5cba0f75c07d8ee8b8acc974134b04184a9d971511a0cd0cc794d4de0edd4786"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ab38f3cd100c1f97f24d12b41a97f18117f4e77e2b00d35e92898a115a328bef"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:73a95a3a32f036aff1ce4616082bcb1b2246de36aad13dc60641fa592f7bbcee"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:71e54b434cf7905957338b8db8e2a9981a6d9bb0a8fa8ee32d6ce30a8f2e7996"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:dd3b4aec4cb695eacbcbbe6a3d88aef7b72e4829d5b1003edf87a4b0bebb17a1"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:ca0eb351d1b9e913887a340878bc444cfd1c4ded337446bf014c281a7254c089"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:0625fe98c746f3d66baf30630863f61c1decd2e86dba8d024c7bc3175728794c"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-win32.whl", hash = "sha256:9b9b83e8e630564c4045ebf9ff38d6d5cef5fd9fb669ab7357dd981cd58959b4"}, + {file = "clickhouse_connect-0.7.0-cp312-cp312-win_amd64.whl", hash = "sha256:ca0e3b7ece52cb24bee459b42eb2bc9d2460b53c5de47e99f89454f197509f3e"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:46558c4f54149fb82d06977e536ca19ee5d343ea77cdffbdd1398f534cb5b9a9"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:6267326bf1a4e4f6803bead7a28fc148c499e5e4aec5aff9a142bde7a4b269b6"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:31019259fdc38759884efaf7e5b5ea6b3612767ac52934f1f4e79913e66ddc09"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:be688d9a885035c1604f846ea44d400af7d7e14c49b72ec04ee932216860755d"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b99319b8b08e4696e4011f8c8e3e5a5041a9f98920e8e2abf8c444e9e2d1aae2"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:1ffc7465c509bb10c6d8f8d66b31298a203b6a85c137d2cd21195e86243eaf70"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:0d3a2e600b50360ac36bb6b97ac44d4851ef2144a3c055df19fff2f48e84ab3e"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:85fedc194b35b7676660bbe510b6eb0fd8ae6c78dca4038dec09a93a227168e8"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-win32.whl", hash = "sha256:61b140694779843f6c2110f1068fb4acbcb3601599d9a721c897605e5939e3ac"}, + {file = "clickhouse_connect-0.7.0-cp38-cp38-win_amd64.whl", hash = "sha256:29afdd5edc77dec17db140df4f1fde66ccffd384011627ce96cd73f0c67ed75f"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:d8f14faeafcf9add0eeeb4781d4a5aa8e3b83648db401c5e76237b7a2c631141"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:85a12697c0c3ebcc24bc2c4f5636f986a36f040b28e079b90c7974e12db3424f"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:db497029d455e07278b4f714d63936d4462e63554d68c3285f3e0a01e5f7aaa1"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0b5462bbd9d776c899a16d17ec49ca4c43793565f5a6956fd64272eecb6bfc55"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d641717b0f675c6cd7c93dc452863a1eac6cf91d637b483a9c42d23b5617ec23"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:a9531260d1ef35119ed9d909fda51578279270e38ecf54ff5f1d9d6b6a2b39f8"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:fa88a35cb47e38f4ce3d1c3dbc61656537de22c84a5d751f775b974a4efd2e32"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:3eb7e0dc1c87e4e9126b2bc614e312022fa741f53f003d98b2f240e6ce8c1e1c"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-win32.whl", hash = "sha256:f479e6422578bc4fb7828f22b882e5294fe9ac9f9af8c660d216c24746bb2be0"}, + {file = "clickhouse_connect-0.7.0-cp39-cp39-win_amd64.whl", hash = "sha256:b1c81338664b2457fae97c1334182e81b77ec057ea9ec3a47d682d14a03b6573"}, + {file = "clickhouse_connect-0.7.0-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:7f167de0f3639aaa0807d011e175ff33be86e2727a4644da65a019306ff3f021"}, + {file = "clickhouse_connect-0.7.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:185ee65eab42bdd545e00c8244a72c797d1961173b78e37b0ea7b130ef0d9c73"}, + {file = "clickhouse_connect-0.7.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:48dde9b8cd9daf7ec0e4baa72e94e40cdd749ae7aef1dfbe7c7d22af53dae8f4"}, + {file = "clickhouse_connect-0.7.0-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3d75b1a01770c04650a7dd3ebbee21369939b00125fbb70c02067ac779f523c8"}, + {file = "clickhouse_connect-0.7.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:9f895290de6fa8347114a361cc944ade1ddeba895f936752533b85984d4d183e"}, + {file = "clickhouse_connect-0.7.0-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:359eab438b3d6f20090b592084493e04dac369e65d0cf4e1da3ecc84750b52c4"}, + {file = "clickhouse_connect-0.7.0-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:efc6b0af171681844fbb39d7b756235aea5b416ce7fca163834e068afbd3f833"}, + {file = "clickhouse_connect-0.7.0-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7cc1a64bc8fb91e594efbd5b4560e6c661ebf75a11d37e08d48c45f3f4e439f7"}, + {file = "clickhouse_connect-0.7.0-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:99c2d25ceb1fbf187c7b9815373dbcdfc04d1b233dafb3547b56dfeca6904584"}, + {file = "clickhouse_connect-0.7.0-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:95a1e292b4cf230c2944bdc31c19c4e8fcbcd5609e24322ff5211af357beb26a"}, + {file = "clickhouse_connect-0.7.0-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:e94de62c3dacfb7b6874f6593ad4268d38c17a1117847acdc1ad0c7b34c4e373"}, + {file = "clickhouse_connect-0.7.0-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a34a3f713f0148e30ddaf431af671ed16baf732eff7437ff2c7519adeda2f9c9"}, + {file = "clickhouse_connect-0.7.0-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b9d08af8c5383d377a12e576106d7c3e0de0d03a3cbc6b9de89932e4b40f550d"}, + {file = "clickhouse_connect-0.7.0-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e06b70e4a4a54810ee293875febf71562c346688e2bc517c141958ef1c2af710"}, + {file = "clickhouse_connect-0.7.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:5c1bc46af3e0eca5a580aaecffd7dc47d541173d3189f250c59ffdd9d1cb0dd1"}, ] [package.dependencies] @@ -1078,63 +1067,63 @@ test-no-images = ["pytest", "pytest-cov", "pytest-xdist", "wurlitzer"] [[package]] name = "coverage" -version = "7.4.0" +version = "7.4.1" description = "Code coverage measurement for Python" optional = false python-versions = ">=3.8" files = [ - {file = "coverage-7.4.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:36b0ea8ab20d6a7564e89cb6135920bc9188fb5f1f7152e94e8300b7b189441a"}, - {file = "coverage-7.4.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0676cd0ba581e514b7f726495ea75aba3eb20899d824636c6f59b0ed2f88c471"}, - {file = "coverage-7.4.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d0ca5c71a5a1765a0f8f88022c52b6b8be740e512980362f7fdbb03725a0d6b9"}, - {file = "coverage-7.4.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a7c97726520f784239f6c62506bc70e48d01ae71e9da128259d61ca5e9788516"}, - {file = "coverage-7.4.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:815ac2d0f3398a14286dc2cea223a6f338109f9ecf39a71160cd1628786bc6f5"}, - {file = "coverage-7.4.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:80b5ee39b7f0131ebec7968baa9b2309eddb35b8403d1869e08f024efd883566"}, - {file = "coverage-7.4.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:5b2ccb7548a0b65974860a78c9ffe1173cfb5877460e5a229238d985565574ae"}, - {file = "coverage-7.4.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:995ea5c48c4ebfd898eacb098164b3cc826ba273b3049e4a889658548e321b43"}, - {file = "coverage-7.4.0-cp310-cp310-win32.whl", hash = "sha256:79287fd95585ed36e83182794a57a46aeae0b64ca53929d1176db56aacc83451"}, - {file = "coverage-7.4.0-cp310-cp310-win_amd64.whl", hash = "sha256:5b14b4f8760006bfdb6e08667af7bc2d8d9bfdb648351915315ea17645347137"}, - {file = "coverage-7.4.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:04387a4a6ecb330c1878907ce0dc04078ea72a869263e53c72a1ba5bbdf380ca"}, - {file = "coverage-7.4.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ea81d8f9691bb53f4fb4db603203029643caffc82bf998ab5b59ca05560f4c06"}, - {file = "coverage-7.4.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:74775198b702868ec2d058cb92720a3c5a9177296f75bd97317c787daf711505"}, - {file = "coverage-7.4.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:76f03940f9973bfaee8cfba70ac991825611b9aac047e5c80d499a44079ec0bc"}, - {file = "coverage-7.4.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:485e9f897cf4856a65a57c7f6ea3dc0d4e6c076c87311d4bc003f82cfe199d25"}, - {file = "coverage-7.4.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:6ae8c9d301207e6856865867d762a4b6fd379c714fcc0607a84b92ee63feff70"}, - {file = "coverage-7.4.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:bf477c355274a72435ceb140dc42de0dc1e1e0bf6e97195be30487d8eaaf1a09"}, - {file = "coverage-7.4.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:83c2dda2666fe32332f8e87481eed056c8b4d163fe18ecc690b02802d36a4d26"}, - {file = "coverage-7.4.0-cp311-cp311-win32.whl", hash = "sha256:697d1317e5290a313ef0d369650cfee1a114abb6021fa239ca12b4849ebbd614"}, - {file = "coverage-7.4.0-cp311-cp311-win_amd64.whl", hash = "sha256:26776ff6c711d9d835557ee453082025d871e30b3fd6c27fcef14733f67f0590"}, - {file = "coverage-7.4.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:13eaf476ec3e883fe3e5fe3707caeb88268a06284484a3daf8250259ef1ba143"}, - {file = "coverage-7.4.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:846f52f46e212affb5bcf131c952fb4075b55aae6b61adc9856222df89cbe3e2"}, - {file = "coverage-7.4.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:26f66da8695719ccf90e794ed567a1549bb2644a706b41e9f6eae6816b398c4a"}, - {file = "coverage-7.4.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:164fdcc3246c69a6526a59b744b62e303039a81e42cfbbdc171c91a8cc2f9446"}, - {file = "coverage-7.4.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:316543f71025a6565677d84bc4df2114e9b6a615aa39fb165d697dba06a54af9"}, - {file = "coverage-7.4.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:bb1de682da0b824411e00a0d4da5a784ec6496b6850fdf8c865c1d68c0e318dd"}, - {file = "coverage-7.4.0-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:0e8d06778e8fbffccfe96331a3946237f87b1e1d359d7fbe8b06b96c95a5407a"}, - {file = "coverage-7.4.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:a56de34db7b7ff77056a37aedded01b2b98b508227d2d0979d373a9b5d353daa"}, - {file = "coverage-7.4.0-cp312-cp312-win32.whl", hash = "sha256:51456e6fa099a8d9d91497202d9563a320513fcf59f33991b0661a4a6f2ad450"}, - {file = "coverage-7.4.0-cp312-cp312-win_amd64.whl", hash = "sha256:cd3c1e4cb2ff0083758f09be0f77402e1bdf704adb7f89108007300a6da587d0"}, - {file = "coverage-7.4.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:e9d1bf53c4c8de58d22e0e956a79a5b37f754ed1ffdbf1a260d9dcfa2d8a325e"}, - {file = "coverage-7.4.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:109f5985182b6b81fe33323ab4707011875198c41964f014579cf82cebf2bb85"}, - {file = "coverage-7.4.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3cc9d4bc55de8003663ec94c2f215d12d42ceea128da8f0f4036235a119c88ac"}, - {file = "coverage-7.4.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cc6d65b21c219ec2072c1293c505cf36e4e913a3f936d80028993dd73c7906b1"}, - {file = "coverage-7.4.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5a10a4920def78bbfff4eff8a05c51be03e42f1c3735be42d851f199144897ba"}, - {file = "coverage-7.4.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:b8e99f06160602bc64da35158bb76c73522a4010f0649be44a4e167ff8555952"}, - {file = "coverage-7.4.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:7d360587e64d006402b7116623cebf9d48893329ef035278969fa3bbf75b697e"}, - {file = "coverage-7.4.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:29f3abe810930311c0b5d1a7140f6395369c3db1be68345638c33eec07535105"}, - {file = "coverage-7.4.0-cp38-cp38-win32.whl", hash = "sha256:5040148f4ec43644702e7b16ca864c5314ccb8ee0751ef617d49aa0e2d6bf4f2"}, - {file = "coverage-7.4.0-cp38-cp38-win_amd64.whl", hash = "sha256:9864463c1c2f9cb3b5db2cf1ff475eed2f0b4285c2aaf4d357b69959941aa555"}, - {file = "coverage-7.4.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:936d38794044b26c99d3dd004d8af0035ac535b92090f7f2bb5aa9c8e2f5cd42"}, - {file = "coverage-7.4.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:799c8f873794a08cdf216aa5d0531c6a3747793b70c53f70e98259720a6fe2d7"}, - {file = "coverage-7.4.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e7defbb9737274023e2d7af02cac77043c86ce88a907c58f42b580a97d5bcca9"}, - {file = "coverage-7.4.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a1526d265743fb49363974b7aa8d5899ff64ee07df47dd8d3e37dcc0818f09ed"}, - {file = "coverage-7.4.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bf635a52fc1ea401baf88843ae8708591aa4adff875e5c23220de43b1ccf575c"}, - {file = "coverage-7.4.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:756ded44f47f330666843b5781be126ab57bb57c22adbb07d83f6b519783b870"}, - {file = "coverage-7.4.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:0eb3c2f32dabe3a4aaf6441dde94f35687224dfd7eb2a7f47f3fd9428e421058"}, - {file = "coverage-7.4.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:bfd5db349d15c08311702611f3dccbef4b4e2ec148fcc636cf8739519b4a5c0f"}, - {file = "coverage-7.4.0-cp39-cp39-win32.whl", hash = "sha256:53d7d9158ee03956e0eadac38dfa1ec8068431ef8058fe6447043db1fb40d932"}, - {file = "coverage-7.4.0-cp39-cp39-win_amd64.whl", hash = "sha256:cfd2a8b6b0d8e66e944d47cdec2f47c48fef2ba2f2dff5a9a75757f64172857e"}, - {file = "coverage-7.4.0-pp38.pp39.pp310-none-any.whl", hash = "sha256:c530833afc4707fe48524a44844493f36d8727f04dcce91fb978c414a8556cc6"}, - {file = "coverage-7.4.0.tar.gz", hash = "sha256:707c0f58cb1712b8809ece32b68996ee1e609f71bd14615bd8f87a1293cb610e"}, + {file = "coverage-7.4.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:077d366e724f24fc02dbfe9d946534357fda71af9764ff99d73c3c596001bbd7"}, + {file = "coverage-7.4.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0193657651f5399d433c92f8ae264aff31fc1d066deee4b831549526433f3f61"}, + {file = "coverage-7.4.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d17bbc946f52ca67adf72a5ee783cd7cd3477f8f8796f59b4974a9b59cacc9ee"}, + {file = "coverage-7.4.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a3277f5fa7483c927fe3a7b017b39351610265308f5267ac6d4c2b64cc1d8d25"}, + {file = "coverage-7.4.1-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6dceb61d40cbfcf45f51e59933c784a50846dc03211054bd76b421a713dcdf19"}, + {file = "coverage-7.4.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:6008adeca04a445ea6ef31b2cbaf1d01d02986047606f7da266629afee982630"}, + {file = "coverage-7.4.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:c61f66d93d712f6e03369b6a7769233bfda880b12f417eefdd4f16d1deb2fc4c"}, + {file = "coverage-7.4.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:b9bb62fac84d5f2ff523304e59e5c439955fb3b7f44e3d7b2085184db74d733b"}, + {file = "coverage-7.4.1-cp310-cp310-win32.whl", hash = "sha256:f86f368e1c7ce897bf2457b9eb61169a44e2ef797099fb5728482b8d69f3f016"}, + {file = "coverage-7.4.1-cp310-cp310-win_amd64.whl", hash = "sha256:869b5046d41abfea3e381dd143407b0d29b8282a904a19cb908fa24d090cc018"}, + {file = "coverage-7.4.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b8ffb498a83d7e0305968289441914154fb0ef5d8b3157df02a90c6695978295"}, + {file = "coverage-7.4.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:3cacfaefe6089d477264001f90f55b7881ba615953414999c46cc9713ff93c8c"}, + {file = "coverage-7.4.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5d6850e6e36e332d5511a48a251790ddc545e16e8beaf046c03985c69ccb2676"}, + {file = "coverage-7.4.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:18e961aa13b6d47f758cc5879383d27b5b3f3dcd9ce8cdbfdc2571fe86feb4dd"}, + {file = "coverage-7.4.1-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dfd1e1b9f0898817babf840b77ce9fe655ecbe8b1b327983df485b30df8cc011"}, + {file = "coverage-7.4.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:6b00e21f86598b6330f0019b40fb397e705135040dbedc2ca9a93c7441178e74"}, + {file = "coverage-7.4.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:536d609c6963c50055bab766d9951b6c394759190d03311f3e9fcf194ca909e1"}, + {file = "coverage-7.4.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:7ac8f8eb153724f84885a1374999b7e45734bf93a87d8df1e7ce2146860edef6"}, + {file = "coverage-7.4.1-cp311-cp311-win32.whl", hash = "sha256:f3771b23bb3675a06f5d885c3630b1d01ea6cac9e84a01aaf5508706dba546c5"}, + {file = "coverage-7.4.1-cp311-cp311-win_amd64.whl", hash = "sha256:9d2f9d4cc2a53b38cabc2d6d80f7f9b7e3da26b2f53d48f05876fef7956b6968"}, + {file = "coverage-7.4.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:f68ef3660677e6624c8cace943e4765545f8191313a07288a53d3da188bd8581"}, + {file = "coverage-7.4.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:23b27b8a698e749b61809fb637eb98ebf0e505710ec46a8aa6f1be7dc0dc43a6"}, + {file = "coverage-7.4.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3e3424c554391dc9ef4a92ad28665756566a28fecf47308f91841f6c49288e66"}, + {file = "coverage-7.4.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e0860a348bf7004c812c8368d1fc7f77fe8e4c095d661a579196a9533778e156"}, + {file = "coverage-7.4.1-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fe558371c1bdf3b8fa03e097c523fb9645b8730399c14fe7721ee9c9e2a545d3"}, + {file = "coverage-7.4.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:3468cc8720402af37b6c6e7e2a9cdb9f6c16c728638a2ebc768ba1ef6f26c3a1"}, + {file = "coverage-7.4.1-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:02f2edb575d62172aa28fe00efe821ae31f25dc3d589055b3fb64d51e52e4ab1"}, + {file = "coverage-7.4.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:ca6e61dc52f601d1d224526360cdeab0d0712ec104a2ce6cc5ccef6ed9a233bc"}, + {file = "coverage-7.4.1-cp312-cp312-win32.whl", hash = "sha256:ca7b26a5e456a843b9b6683eada193fc1f65c761b3a473941efe5a291f604c74"}, + {file = "coverage-7.4.1-cp312-cp312-win_amd64.whl", hash = "sha256:85ccc5fa54c2ed64bd91ed3b4a627b9cce04646a659512a051fa82a92c04a448"}, + {file = "coverage-7.4.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:8bdb0285a0202888d19ec6b6d23d5990410decb932b709f2b0dfe216d031d218"}, + {file = "coverage-7.4.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:918440dea04521f499721c039863ef95433314b1db00ff826a02580c1f503e45"}, + {file = "coverage-7.4.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:379d4c7abad5afbe9d88cc31ea8ca262296480a86af945b08214eb1a556a3e4d"}, + {file = "coverage-7.4.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b094116f0b6155e36a304ff912f89bbb5067157aff5f94060ff20bbabdc8da06"}, + {file = "coverage-7.4.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f2f5968608b1fe2a1d00d01ad1017ee27efd99b3437e08b83ded9b7af3f6f766"}, + {file = "coverage-7.4.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:10e88e7f41e6197ea0429ae18f21ff521d4f4490aa33048f6c6f94c6045a6a75"}, + {file = "coverage-7.4.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:a4a3907011d39dbc3e37bdc5df0a8c93853c369039b59efa33a7b6669de04c60"}, + {file = "coverage-7.4.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6d224f0c4c9c98290a6990259073f496fcec1b5cc613eecbd22786d398ded3ad"}, + {file = "coverage-7.4.1-cp38-cp38-win32.whl", hash = "sha256:23f5881362dcb0e1a92b84b3c2809bdc90db892332daab81ad8f642d8ed55042"}, + {file = "coverage-7.4.1-cp38-cp38-win_amd64.whl", hash = "sha256:a07f61fc452c43cd5328b392e52555f7d1952400a1ad09086c4a8addccbd138d"}, + {file = "coverage-7.4.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8e738a492b6221f8dcf281b67129510835461132b03024830ac0e554311a5c54"}, + {file = "coverage-7.4.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:46342fed0fff72efcda77040b14728049200cbba1279e0bf1188f1f2078c1d70"}, + {file = "coverage-7.4.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9641e21670c68c7e57d2053ddf6c443e4f0a6e18e547e86af3fad0795414a628"}, + {file = "coverage-7.4.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:aeb2c2688ed93b027eb0d26aa188ada34acb22dceea256d76390eea135083950"}, + {file = "coverage-7.4.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d12c923757de24e4e2110cf8832d83a886a4cf215c6e61ed506006872b43a6d1"}, + {file = "coverage-7.4.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:0491275c3b9971cdbd28a4595c2cb5838f08036bca31765bad5e17edf900b2c7"}, + {file = "coverage-7.4.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:8dfc5e195bbef80aabd81596ef52a1277ee7143fe419efc3c4d8ba2754671756"}, + {file = "coverage-7.4.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:1a78b656a4d12b0490ca72651fe4d9f5e07e3c6461063a9b6265ee45eb2bdd35"}, + {file = "coverage-7.4.1-cp39-cp39-win32.whl", hash = "sha256:f90515974b39f4dea2f27c0959688621b46d96d5a626cf9c53dbc653a895c05c"}, + {file = "coverage-7.4.1-cp39-cp39-win_amd64.whl", hash = "sha256:64e723ca82a84053dd7bfcc986bdb34af8d9da83c521c19d6b472bc6880e191a"}, + {file = "coverage-7.4.1-pp38.pp39.pp310-none-any.whl", hash = "sha256:32a8d985462e37cfdab611a6f95b09d7c091d07668fdc26e47a725ee575fe166"}, + {file = "coverage-7.4.1.tar.gz", hash = "sha256:1ed4b95480952b1a26d863e546fa5094564aa0065e1e5f0d4d0041f293251d04"}, ] [package.dependencies] @@ -1216,13 +1205,13 @@ tests = ["pytest", "pytest-cov", "pytest-xdist"] [[package]] name = "dask" -version = "2024.1.0" +version = "2024.1.1" description = "Parallel PyData with Task Scheduling" optional = false python-versions = ">=3.9" files = [ - {file = "dask-2024.1.0-py3-none-any.whl", hash = "sha256:717102ef7c309297291095a0061d374f3b72e11ce4e1115ab9faff940e274b4b"}, - {file = "dask-2024.1.0.tar.gz", hash = "sha256:f24fdc7a07e59a1403bf6903e6d8dc15ed6f8607d3311b4f00f88d8a2ac63e49"}, + {file = "dask-2024.1.1-py3-none-any.whl", hash = "sha256:860ce2797905095beff0187c214840b80c77d752dcb9098a8283e3655a762bf5"}, + {file = "dask-2024.1.1.tar.gz", hash = "sha256:d0dc92e81ce68594a0a0ce23ba33f4d648f2c2f4217ab9b79068b7ecfb0416c7"}, ] [package.dependencies] @@ -1242,7 +1231,7 @@ array = ["numpy (>=1.21)"] complete = ["dask[array,dataframe,diagnostics,distributed]", "lz4 (>=4.3.2)", "pyarrow (>=7.0)", "pyarrow-hotfix"] dataframe = ["dask[array]", "pandas (>=1.3)"] diagnostics = ["bokeh (>=2.4.2)", "jinja2 (>=2.10.3)"] -distributed = ["distributed (==2024.1.0)"] +distributed = ["distributed (==2024.1.1)"] test = ["pandas[test]", "pre-commit", "pytest", "pytest-cov", "pytest-rerunfailures", "pytest-timeout", "pytest-xdist"] [[package]] @@ -1355,19 +1344,19 @@ files = [ [[package]] name = "distributed" -version = "2024.1.0" +version = "2024.1.1" description = "Distributed scheduler for Dask" optional = false python-versions = ">=3.9" files = [ - {file = "distributed-2024.1.0-py3-none-any.whl", hash = "sha256:b552c9331350ba0e7cb8eccb1da8942b44997ccb680338f61c43fe9843c69988"}, - {file = "distributed-2024.1.0.tar.gz", hash = "sha256:f1d0e2dd5249085e32c6ff5c0ce0521c7e844dd52337683a69363a6bb1799a30"}, + {file = "distributed-2024.1.1-py3-none-any.whl", hash = "sha256:cf05d3b38e1700339b3e36395729ab62110e723efefaecc21a8260fdc7555cf9"}, + {file = "distributed-2024.1.1.tar.gz", hash = "sha256:28cf5e9f4f07197b03ea8e5272e374ce2b9e9dc6742f6c9b525fd81645213c67"}, ] [package.dependencies] click = ">=8.0" cloudpickle = ">=1.5.0" -dask = "2024.1.0" +dask = "2024.1.1" jinja2 = ">=2.10.3" locket = ">=1.0.0" msgpack = ">=1.0.0" @@ -1884,22 +1873,24 @@ gcsfuse = ["fusepy"] [[package]] name = "gdown" -version = "4.7.1" -description = "Google Drive direct download of big files." +version = "5.0.1" +description = "Google Drive Public File/Folder Downloader" optional = false -python-versions = "*" +python-versions = ">=3.8" files = [ - {file = "gdown-4.7.1-py3-none-any.whl", hash = "sha256:65d495699e7c2c61af0d0e9c32748fb4f79abaf80d747a87456c7be14aac2560"}, - {file = "gdown-4.7.1.tar.gz", hash = "sha256:347f23769679aaf7efa73e5655270fcda8ca56be65eb84a4a21d143989541045"}, + {file = "gdown-5.0.1-py3-none-any.whl", hash = "sha256:3f595fcfd4b1bccd5cf73453f60984c5fa1c18eed499277a52b23337238c2670"}, + {file = "gdown-5.0.1.tar.gz", hash = "sha256:173557b4d33aad9f7dc75ce2ff963d8b313f36371e15da4b5ebb35ac6c7d5af6"}, ] [package.dependencies] beautifulsoup4 = "*" filelock = "*" requests = {version = "*", extras = ["socks"]} -six = "*" tqdm = "*" +[package.extras] +test = ["build", "mypy", "pytest", "ruff", "twine", "types-requests"] + [[package]] name = "geojson" version = "3.1.0" @@ -1913,13 +1904,13 @@ files = [ [[package]] name = "geopandas" -version = "0.14.2" +version = "0.14.3" description = "Geographic pandas extensions" optional = false python-versions = ">=3.9" files = [ - {file = "geopandas-0.14.2-py3-none-any.whl", hash = "sha256:0efa61235a68862c1c6be89fc3707cdeba67667d5676bb19e24f3c57a8c2f723"}, - {file = "geopandas-0.14.2.tar.gz", hash = "sha256:6e71d57b8376f9fdc9f1c3aa3170e7e420e91778de854f51013ae66fd371ccdb"}, + {file = "geopandas-0.14.3-py3-none-any.whl", hash = "sha256:41b31ad39e21bc9e8c4254f78f8dc4ce3d33d144e22e630a00bb336c83160204"}, + {file = "geopandas-0.14.3.tar.gz", hash = "sha256:748af035d4a068a4ae00cab384acb61d387685c833b0022e0729aa45216b23ac"}, ] [package.dependencies] @@ -1931,13 +1922,13 @@ shapely = ">=1.8.0" [[package]] name = "google-api-core" -version = "2.15.0" +version = "2.16.1" description = "Google API client core library" optional = false python-versions = ">=3.7" files = [ - {file = "google-api-core-2.15.0.tar.gz", hash = "sha256:abc978a72658f14a2df1e5e12532effe40f94f868f6e23d95133bd6abcca35ca"}, - {file = "google_api_core-2.15.0-py3-none-any.whl", hash = "sha256:2aa56d2be495551e66bbff7f729b790546f87d5c90e74781aa77233bcb395a8a"}, + {file = "google-api-core-2.16.1.tar.gz", hash = "sha256:7f668ffa3d5b9f3c6930407e5f5d691c05a376050a5a5fd772b9dc32e70a0c30"}, + {file = "google_api_core-2.16.1-py3-none-any.whl", hash = "sha256:257e9e152cd18da0c6701113c122ade04dca04731e179fc5c7dca48e1396ec4c"}, ] [package.dependencies] @@ -1961,13 +1952,13 @@ grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.dev0)"] [[package]] name = "google-auth" -version = "2.26.2" +version = "2.27.0" description = "Google Authentication Library" optional = false python-versions = ">=3.7" files = [ - {file = "google-auth-2.26.2.tar.gz", hash = "sha256:97327dbbf58cccb58fc5a1712bba403ae76668e64814eb30f7316f7e27126b81"}, - {file = "google_auth-2.26.2-py2.py3-none-any.whl", hash = "sha256:3f445c8ce9b61ed6459aad86d8ccdba4a9afed841b2d1451a11ef4db08957424"}, + {file = "google-auth-2.27.0.tar.gz", hash = "sha256:e863a56ccc2d8efa83df7a80272601e43487fa9a728a376205c86c26aaefa821"}, + {file = "google_auth-2.27.0-py2.py3-none-any.whl", hash = "sha256:8e4bad367015430ff253fe49d500fdc3396c1a434db5740828c728e45bcce245"}, ] [package.dependencies] @@ -2002,13 +1993,13 @@ tool = ["click (>=6.0.0)"] [[package]] name = "google-cloud-bigquery" -version = "3.16.0" +version = "3.17.1" description = "Google BigQuery API client library" optional = true python-versions = ">=3.7" files = [ - {file = "google-cloud-bigquery-3.16.0.tar.gz", hash = "sha256:1d6abf4b1d740df17cb43a078789872af8059a0b1dd999f32ea69ebc6f7ba7ef"}, - {file = "google_cloud_bigquery-3.16.0-py2.py3-none-any.whl", hash = "sha256:8bac7754f92bf87ee81f38deabb7554d82bb9591fbe06a5c82f33e46e5a482f9"}, + {file = "google-cloud-bigquery-3.17.1.tar.gz", hash = "sha256:0ae07b90d5052ba3a296a2210a2144c28469300d71f6f455881f94c2df543057"}, + {file = "google_cloud_bigquery-3.17.1-py2.py3-none-any.whl", hash = "sha256:7a9a92c7b1f6a6bf8b4c05c150e49f4ad1a03dd591dbd4522381b3f23bf07c73"}, ] [package.dependencies] @@ -2298,13 +2289,13 @@ test = ["objgraph", "psutil"] [[package]] name = "griffe" -version = "0.38.1" +version = "0.40.0" description = "Signatures for entire Python programs. Extract the structure, the frame, the skeleton of your project, to generate API documentation or find breaking changes in your API." optional = false python-versions = ">=3.8" files = [ - {file = "griffe-0.38.1-py3-none-any.whl", hash = "sha256:334c79d3b5964ade65c05dfcaf53518c576dedd387aaba5c9fd71212f34f1483"}, - {file = "griffe-0.38.1.tar.gz", hash = "sha256:bd68d7da7f3d87bc57eb9962b250db123efd9bbcc06c11c1a91b6e583b2a9361"}, + {file = "griffe-0.40.0-py3-none-any.whl", hash = "sha256:db1da6d1d8e08cbb20f1a7dee8c09da940540c2d4c1bfa26a9091cf6fc36a9ec"}, + {file = "griffe-0.40.0.tar.gz", hash = "sha256:76c4439eaa2737af46ae003c331ab6ca79c5365b552f7b5aed263a3b4125735b"}, ] [package.dependencies] @@ -2312,84 +2303,84 @@ colorama = ">=0.4" [[package]] name = "grpcio" -version = "1.60.0" +version = "1.60.1" description = "HTTP/2-based RPC framework" optional = true python-versions = ">=3.7" files = [ - {file = "grpcio-1.60.0-cp310-cp310-linux_armv7l.whl", hash = "sha256:d020cfa595d1f8f5c6b343530cd3ca16ae5aefdd1e832b777f9f0eb105f5b139"}, - {file = "grpcio-1.60.0-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:b98f43fcdb16172dec5f4b49f2fece4b16a99fd284d81c6bbac1b3b69fcbe0ff"}, - {file = "grpcio-1.60.0-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:20e7a4f7ded59097c84059d28230907cd97130fa74f4a8bfd1d8e5ba18c81491"}, - {file = "grpcio-1.60.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:452ca5b4afed30e7274445dd9b441a35ece656ec1600b77fff8c216fdf07df43"}, - {file = "grpcio-1.60.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:43e636dc2ce9ece583b3e2ca41df5c983f4302eabc6d5f9cd04f0562ee8ec1ae"}, - {file = "grpcio-1.60.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:6e306b97966369b889985a562ede9d99180def39ad42c8014628dd3cc343f508"}, - {file = "grpcio-1.60.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:f897c3b127532e6befdcf961c415c97f320d45614daf84deba0a54e64ea2457b"}, - {file = "grpcio-1.60.0-cp310-cp310-win32.whl", hash = "sha256:b87efe4a380887425bb15f220079aa8336276398dc33fce38c64d278164f963d"}, - {file = "grpcio-1.60.0-cp310-cp310-win_amd64.whl", hash = "sha256:a9c7b71211f066908e518a2ef7a5e211670761651039f0d6a80d8d40054047df"}, - {file = "grpcio-1.60.0-cp311-cp311-linux_armv7l.whl", hash = "sha256:fb464479934778d7cc5baf463d959d361954d6533ad34c3a4f1d267e86ee25fd"}, - {file = "grpcio-1.60.0-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:4b44d7e39964e808b071714666a812049765b26b3ea48c4434a3b317bac82f14"}, - {file = "grpcio-1.60.0-cp311-cp311-manylinux_2_17_aarch64.whl", hash = "sha256:90bdd76b3f04bdb21de5398b8a7c629676c81dfac290f5f19883857e9371d28c"}, - {file = "grpcio-1.60.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:91229d7203f1ef0ab420c9b53fe2ca5c1fbeb34f69b3bc1b5089466237a4a134"}, - {file = "grpcio-1.60.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b36a2c6d4920ba88fa98075fdd58ff94ebeb8acc1215ae07d01a418af4c0253"}, - {file = "grpcio-1.60.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:297eef542156d6b15174a1231c2493ea9ea54af8d016b8ca7d5d9cc65cfcc444"}, - {file = "grpcio-1.60.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:87c9224acba0ad8bacddf427a1c2772e17ce50b3042a789547af27099c5f751d"}, - {file = "grpcio-1.60.0-cp311-cp311-win32.whl", hash = "sha256:95ae3e8e2c1b9bf671817f86f155c5da7d49a2289c5cf27a319458c3e025c320"}, - {file = "grpcio-1.60.0-cp311-cp311-win_amd64.whl", hash = "sha256:467a7d31554892eed2aa6c2d47ded1079fc40ea0b9601d9f79204afa8902274b"}, - {file = "grpcio-1.60.0-cp312-cp312-linux_armv7l.whl", hash = "sha256:a7152fa6e597c20cb97923407cf0934e14224af42c2b8d915f48bc3ad2d9ac18"}, - {file = "grpcio-1.60.0-cp312-cp312-macosx_10_10_universal2.whl", hash = "sha256:7db16dd4ea1b05ada504f08d0dca1cd9b926bed3770f50e715d087c6f00ad748"}, - {file = "grpcio-1.60.0-cp312-cp312-manylinux_2_17_aarch64.whl", hash = "sha256:b0571a5aef36ba9177e262dc88a9240c866d903a62799e44fd4aae3f9a2ec17e"}, - {file = "grpcio-1.60.0-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6fd9584bf1bccdfff1512719316efa77be235469e1e3295dce64538c4773840b"}, - {file = "grpcio-1.60.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d6a478581b1a1a8fdf3318ecb5f4d0cda41cacdffe2b527c23707c9c1b8fdb55"}, - {file = "grpcio-1.60.0-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:77c8a317f0fd5a0a2be8ed5cbe5341537d5c00bb79b3bb27ba7c5378ba77dbca"}, - {file = "grpcio-1.60.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:1c30bb23a41df95109db130a6cc1b974844300ae2e5d68dd4947aacba5985aa5"}, - {file = "grpcio-1.60.0-cp312-cp312-win32.whl", hash = "sha256:2aef56e85901c2397bd557c5ba514f84de1f0ae5dd132f5d5fed042858115951"}, - {file = "grpcio-1.60.0-cp312-cp312-win_amd64.whl", hash = "sha256:e381fe0c2aa6c03b056ad8f52f8efca7be29fb4d9ae2f8873520843b6039612a"}, - {file = "grpcio-1.60.0-cp37-cp37m-linux_armv7l.whl", hash = "sha256:92f88ca1b956eb8427a11bb8b4a0c0b2b03377235fc5102cb05e533b8693a415"}, - {file = "grpcio-1.60.0-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:e278eafb406f7e1b1b637c2cf51d3ad45883bb5bd1ca56bc05e4fc135dfdaa65"}, - {file = "grpcio-1.60.0-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:a48edde788b99214613e440fce495bbe2b1e142a7f214cce9e0832146c41e324"}, - {file = "grpcio-1.60.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:de2ad69c9a094bf37c1102b5744c9aec6cf74d2b635558b779085d0263166454"}, - {file = "grpcio-1.60.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:073f959c6f570797272f4ee9464a9997eaf1e98c27cb680225b82b53390d61e6"}, - {file = "grpcio-1.60.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c826f93050c73e7769806f92e601e0efdb83ec8d7c76ddf45d514fee54e8e619"}, - {file = "grpcio-1.60.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:9e30be89a75ee66aec7f9e60086fadb37ff8c0ba49a022887c28c134341f7179"}, - {file = "grpcio-1.60.0-cp37-cp37m-win_amd64.whl", hash = "sha256:b0fb2d4801546598ac5cd18e3ec79c1a9af8b8f2a86283c55a5337c5aeca4b1b"}, - {file = "grpcio-1.60.0-cp38-cp38-linux_armv7l.whl", hash = "sha256:9073513ec380434eb8d21970e1ab3161041de121f4018bbed3146839451a6d8e"}, - {file = "grpcio-1.60.0-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:74d7d9fa97809c5b892449b28a65ec2bfa458a4735ddad46074f9f7d9550ad13"}, - {file = "grpcio-1.60.0-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:1434ca77d6fed4ea312901122dc8da6c4389738bf5788f43efb19a838ac03ead"}, - {file = "grpcio-1.60.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e61e76020e0c332a98290323ecfec721c9544f5b739fab925b6e8cbe1944cf19"}, - {file = "grpcio-1.60.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:675997222f2e2f22928fbba640824aebd43791116034f62006e19730715166c0"}, - {file = "grpcio-1.60.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:5208a57eae445ae84a219dfd8b56e04313445d146873117b5fa75f3245bc1390"}, - {file = "grpcio-1.60.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:428d699c8553c27e98f4d29fdc0f0edc50e9a8a7590bfd294d2edb0da7be3629"}, - {file = "grpcio-1.60.0-cp38-cp38-win32.whl", hash = "sha256:83f2292ae292ed5a47cdcb9821039ca8e88902923198f2193f13959360c01860"}, - {file = "grpcio-1.60.0-cp38-cp38-win_amd64.whl", hash = "sha256:705a68a973c4c76db5d369ed573fec3367d7d196673fa86614b33d8c8e9ebb08"}, - {file = "grpcio-1.60.0-cp39-cp39-linux_armv7l.whl", hash = "sha256:c193109ca4070cdcaa6eff00fdb5a56233dc7610216d58fb81638f89f02e4968"}, - {file = "grpcio-1.60.0-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:676e4a44e740deaba0f4d95ba1d8c5c89a2fcc43d02c39f69450b1fa19d39590"}, - {file = "grpcio-1.60.0-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:5ff21e000ff2f658430bde5288cb1ac440ff15c0d7d18b5fb222f941b46cb0d2"}, - {file = "grpcio-1.60.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4c86343cf9ff7b2514dd229bdd88ebba760bd8973dac192ae687ff75e39ebfab"}, - {file = "grpcio-1.60.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0fd3b3968ffe7643144580f260f04d39d869fcc2cddb745deef078b09fd2b328"}, - {file = "grpcio-1.60.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:30943b9530fe3620e3b195c03130396cd0ee3a0d10a66c1bee715d1819001eaf"}, - {file = "grpcio-1.60.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:b10241250cb77657ab315270b064a6c7f1add58af94befa20687e7c8d8603ae6"}, - {file = "grpcio-1.60.0-cp39-cp39-win32.whl", hash = "sha256:79a050889eb8d57a93ed21d9585bb63fca881666fc709f5d9f7f9372f5e7fd03"}, - {file = "grpcio-1.60.0-cp39-cp39-win_amd64.whl", hash = "sha256:8a97a681e82bc11a42d4372fe57898d270a2707f36c45c6676e49ce0d5c41353"}, - {file = "grpcio-1.60.0.tar.gz", hash = "sha256:2199165a1affb666aa24adf0c97436686d0a61bc5fc113c037701fb7c7fceb96"}, -] - -[package.extras] -protobuf = ["grpcio-tools (>=1.60.0)"] + {file = "grpcio-1.60.1-cp310-cp310-linux_armv7l.whl", hash = "sha256:14e8f2c84c0832773fb3958240c69def72357bc11392571f87b2d7b91e0bb092"}, + {file = "grpcio-1.60.1-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:33aed0a431f5befeffd9d346b0fa44b2c01aa4aeae5ea5b2c03d3e25e0071216"}, + {file = "grpcio-1.60.1-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:fead980fbc68512dfd4e0c7b1f5754c2a8e5015a04dea454b9cada54a8423525"}, + {file = "grpcio-1.60.1-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:082081e6a36b6eb5cf0fd9a897fe777dbb3802176ffd08e3ec6567edd85bc104"}, + {file = "grpcio-1.60.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:55ccb7db5a665079d68b5c7c86359ebd5ebf31a19bc1a91c982fd622f1e31ff2"}, + {file = "grpcio-1.60.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:9b54577032d4f235452f77a83169b6527bf4b77d73aeada97d45b2aaf1bf5ce0"}, + {file = "grpcio-1.60.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:7d142bcd604166417929b071cd396aa13c565749a4c840d6c702727a59d835eb"}, + {file = "grpcio-1.60.1-cp310-cp310-win32.whl", hash = "sha256:2a6087f234cb570008a6041c8ffd1b7d657b397fdd6d26e83d72283dae3527b1"}, + {file = "grpcio-1.60.1-cp310-cp310-win_amd64.whl", hash = "sha256:f2212796593ad1d0235068c79836861f2201fc7137a99aa2fea7beeb3b101177"}, + {file = "grpcio-1.60.1-cp311-cp311-linux_armv7l.whl", hash = "sha256:79ae0dc785504cb1e1788758c588c711f4e4a0195d70dff53db203c95a0bd303"}, + {file = "grpcio-1.60.1-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:4eec8b8c1c2c9b7125508ff7c89d5701bf933c99d3910e446ed531cd16ad5d87"}, + {file = "grpcio-1.60.1-cp311-cp311-manylinux_2_17_aarch64.whl", hash = "sha256:8c9554ca8e26241dabe7951aa1fa03a1ba0856688ecd7e7bdbdd286ebc272e4c"}, + {file = "grpcio-1.60.1-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:91422ba785a8e7a18725b1dc40fbd88f08a5bb4c7f1b3e8739cab24b04fa8a03"}, + {file = "grpcio-1.60.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cba6209c96828711cb7c8fcb45ecef8c8859238baf15119daa1bef0f6c84bfe7"}, + {file = "grpcio-1.60.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:c71be3f86d67d8d1311c6076a4ba3b75ba5703c0b856b4e691c9097f9b1e8bd2"}, + {file = "grpcio-1.60.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:af5ef6cfaf0d023c00002ba25d0751e5995fa0e4c9eec6cd263c30352662cbce"}, + {file = "grpcio-1.60.1-cp311-cp311-win32.whl", hash = "sha256:a09506eb48fa5493c58f946c46754ef22f3ec0df64f2b5149373ff31fb67f3dd"}, + {file = "grpcio-1.60.1-cp311-cp311-win_amd64.whl", hash = "sha256:49c9b6a510e3ed8df5f6f4f3c34d7fbf2d2cae048ee90a45cd7415abab72912c"}, + {file = "grpcio-1.60.1-cp312-cp312-linux_armv7l.whl", hash = "sha256:b58b855d0071575ea9c7bc0d84a06d2edfbfccec52e9657864386381a7ce1ae9"}, + {file = "grpcio-1.60.1-cp312-cp312-macosx_10_10_universal2.whl", hash = "sha256:a731ac5cffc34dac62053e0da90f0c0b8560396a19f69d9703e88240c8f05858"}, + {file = "grpcio-1.60.1-cp312-cp312-manylinux_2_17_aarch64.whl", hash = "sha256:cf77f8cf2a651fbd869fbdcb4a1931464189cd210abc4cfad357f1cacc8642a6"}, + {file = "grpcio-1.60.1-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c557e94e91a983e5b1e9c60076a8fd79fea1e7e06848eb2e48d0ccfb30f6e073"}, + {file = "grpcio-1.60.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:069fe2aeee02dfd2135d562d0663fe70fbb69d5eed6eb3389042a7e963b54de8"}, + {file = "grpcio-1.60.1-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:cb0af13433dbbd1c806e671d81ec75bd324af6ef75171fd7815ca3074fe32bfe"}, + {file = "grpcio-1.60.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:2f44c32aef186bbba254129cea1df08a20be414144ac3bdf0e84b24e3f3b2e05"}, + {file = "grpcio-1.60.1-cp312-cp312-win32.whl", hash = "sha256:a212e5dea1a4182e40cd3e4067ee46be9d10418092ce3627475e995cca95de21"}, + {file = "grpcio-1.60.1-cp312-cp312-win_amd64.whl", hash = "sha256:6e490fa5f7f5326222cb9f0b78f207a2b218a14edf39602e083d5f617354306f"}, + {file = "grpcio-1.60.1-cp37-cp37m-linux_armv7l.whl", hash = "sha256:4216e67ad9a4769117433814956031cb300f85edc855252a645a9a724b3b6594"}, + {file = "grpcio-1.60.1-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:73e14acd3d4247169955fae8fb103a2b900cfad21d0c35f0dcd0fdd54cd60367"}, + {file = "grpcio-1.60.1-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:6ecf21d20d02d1733e9c820fb5c114c749d888704a7ec824b545c12e78734d1c"}, + {file = "grpcio-1.60.1-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:33bdea30dcfd4f87b045d404388469eb48a48c33a6195a043d116ed1b9a0196c"}, + {file = "grpcio-1.60.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:53b69e79d00f78c81eecfb38f4516080dc7f36a198b6b37b928f1c13b3c063e9"}, + {file = "grpcio-1.60.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:39aa848794b887120b1d35b1b994e445cc028ff602ef267f87c38122c1add50d"}, + {file = "grpcio-1.60.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:72153a0d2e425f45b884540a61c6639436ddafa1829a42056aa5764b84108b8e"}, + {file = "grpcio-1.60.1-cp37-cp37m-win_amd64.whl", hash = "sha256:50d56280b482875d1f9128ce596e59031a226a8b84bec88cb2bf76c289f5d0de"}, + {file = "grpcio-1.60.1-cp38-cp38-linux_armv7l.whl", hash = "sha256:6d140bdeb26cad8b93c1455fa00573c05592793c32053d6e0016ce05ba267549"}, + {file = "grpcio-1.60.1-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:bc808924470643b82b14fe121923c30ec211d8c693e747eba8a7414bc4351a23"}, + {file = "grpcio-1.60.1-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:70c83bb530572917be20c21f3b6be92cd86b9aecb44b0c18b1d3b2cc3ae47df0"}, + {file = "grpcio-1.60.1-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9b106bc52e7f28170e624ba61cc7dc6829566e535a6ec68528f8e1afbed1c41f"}, + {file = "grpcio-1.60.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:30e980cd6db1088c144b92fe376747328d5554bc7960ce583ec7b7d81cd47287"}, + {file = "grpcio-1.60.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:0c5807e9152eff15f1d48f6b9ad3749196f79a4a050469d99eecb679be592acc"}, + {file = "grpcio-1.60.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:f1c3dc536b3ee124e8b24feb7533e5c70b9f2ef833e3b2e5513b2897fd46763a"}, + {file = "grpcio-1.60.1-cp38-cp38-win32.whl", hash = "sha256:d7404cebcdb11bb5bd40bf94131faf7e9a7c10a6c60358580fe83913f360f929"}, + {file = "grpcio-1.60.1-cp38-cp38-win_amd64.whl", hash = "sha256:c8754c75f55781515a3005063d9a05878b2cfb3cb7e41d5401ad0cf19de14872"}, + {file = "grpcio-1.60.1-cp39-cp39-linux_armv7l.whl", hash = "sha256:0250a7a70b14000fa311de04b169cc7480be6c1a769b190769d347939d3232a8"}, + {file = "grpcio-1.60.1-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:660fc6b9c2a9ea3bb2a7e64ba878c98339abaf1811edca904ac85e9e662f1d73"}, + {file = "grpcio-1.60.1-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:76eaaba891083fcbe167aa0f03363311a9f12da975b025d30e94b93ac7a765fc"}, + {file = "grpcio-1.60.1-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e5d97c65ea7e097056f3d1ead77040ebc236feaf7f71489383d20f3b4c28412a"}, + {file = "grpcio-1.60.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2bb2a2911b028f01c8c64d126f6b632fcd8a9ac975aa1b3855766c94e4107180"}, + {file = "grpcio-1.60.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:5a1ebbae7e2214f51b1f23b57bf98eeed2cf1ba84e4d523c48c36d5b2f8829ff"}, + {file = "grpcio-1.60.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:9a66f4d2a005bc78e61d805ed95dedfcb35efa84b7bba0403c6d60d13a3de2d6"}, + {file = "grpcio-1.60.1-cp39-cp39-win32.whl", hash = "sha256:8d488fbdbf04283f0d20742b64968d44825617aa6717b07c006168ed16488804"}, + {file = "grpcio-1.60.1-cp39-cp39-win_amd64.whl", hash = "sha256:61b7199cd2a55e62e45bfb629a35b71fc2c0cb88f686a047f25b1112d3810904"}, + {file = "grpcio-1.60.1.tar.gz", hash = "sha256:dd1d3a8d1d2e50ad9b59e10aa7f07c7d1be2b367f3f2d33c5fade96ed5460962"}, +] + +[package.extras] +protobuf = ["grpcio-tools (>=1.60.1)"] [[package]] name = "grpcio-status" -version = "1.60.0" +version = "1.60.1" description = "Status proto mapping for gRPC" optional = true python-versions = ">=3.6" files = [ - {file = "grpcio-status-1.60.0.tar.gz", hash = "sha256:f10e0b6db3adc0fdc244b71962814ee982996ef06186446b5695b9fa635aa1ab"}, - {file = "grpcio_status-1.60.0-py3-none-any.whl", hash = "sha256:7d383fa36e59c1e61d380d91350badd4d12ac56e4de2c2b831b050362c3c572e"}, + {file = "grpcio-status-1.60.1.tar.gz", hash = "sha256:61b5aab8989498e8aa142c20b88829ea5d90d18c18c853b9f9e6d407d37bf8b4"}, + {file = "grpcio_status-1.60.1-py3-none-any.whl", hash = "sha256:3034fdb239185b6e0f3169d08c268c4507481e4b8a434c21311a03d9eb5889a0"}, ] [package.dependencies] googleapis-common-protos = ">=1.5.5" -grpcio = ">=1.60.0" +grpcio = ">=1.60.1" protobuf = ">=4.21.6" [[package]] @@ -2408,13 +2399,13 @@ tests = ["freezegun", "pytest", "pytest-cov"] [[package]] name = "hypothesis" -version = "6.93.0" +version = "6.97.4" description = "A library for property-based testing" optional = false python-versions = ">=3.8" files = [ - {file = "hypothesis-6.93.0-py3-none-any.whl", hash = "sha256:bfe6173e36c8cf0779a79de757a8a7151568b2703cb14dcbc186517c7a79144b"}, - {file = "hypothesis-6.93.0.tar.gz", hash = "sha256:e9ceaa5bbd244471fa1c28272fb2b0c68bb6ee014473394d63519ed02bd2d4de"}, + {file = "hypothesis-6.97.4-py3-none-any.whl", hash = "sha256:9069fe3fb18d9b7dd218bd69ab50bbc66426819dfac7cc7168ba85034d98a4df"}, + {file = "hypothesis-6.97.4.tar.gz", hash = "sha256:28ff724fa81ccc55f64f0f1eb06e4a75db6a195fe0857e9b3184cf4ff613a103"}, ] [package.dependencies] @@ -2576,13 +2567,13 @@ ipywidgets = "*" [[package]] name = "ipykernel" -version = "6.28.0" +version = "6.29.0" description = "IPython Kernel for Jupyter" optional = false python-versions = ">=3.8" files = [ - {file = "ipykernel-6.28.0-py3-none-any.whl", hash = "sha256:c6e9a9c63a7f4095c0a22a79f765f079f9ec7be4f2430a898ddea889e8665661"}, - {file = "ipykernel-6.28.0.tar.gz", hash = "sha256:69c11403d26de69df02225916f916b37ea4b9af417da0a8c827f84328d88e5f3"}, + {file = "ipykernel-6.29.0-py3-none-any.whl", hash = "sha256:076663ca68492576f051e4af7720d33f34383e655f2be0d544c8b1c9de915b2f"}, + {file = "ipykernel-6.29.0.tar.gz", hash = "sha256:b5dd3013cab7b330df712891c96cd1ab868c27a7159e606f762015e9bf8ceb3f"}, ] [package.dependencies] @@ -2605,17 +2596,17 @@ cov = ["coverage[toml]", "curio", "matplotlib", "pytest-cov", "trio"] docs = ["myst-parser", "pydata-sphinx-theme", "sphinx", "sphinx-autodoc-typehints", "sphinxcontrib-github-alt", "sphinxcontrib-spelling", "trio"] pyqt5 = ["pyqt5"] pyside6 = ["pyside6"] -test = ["flaky", "ipyparallel", "pre-commit", "pytest (>=7.0)", "pytest-asyncio", "pytest-cov", "pytest-timeout"] +test = ["flaky", "ipyparallel", "pre-commit", "pytest (>=7.0)", "pytest-asyncio (==0.23.2)", "pytest-cov", "pytest-timeout"] [[package]] name = "ipyleaflet" -version = "0.18.1" +version = "0.18.2" description = "A Jupyter widget for dynamic Leaflet maps" optional = false python-versions = ">=3.7" files = [ - {file = "ipyleaflet-0.18.1-py3-none-any.whl", hash = "sha256:c941429945248fb0fb8a7b30cc4f248d3194e4a409066a068495a633c97eb6c6"}, - {file = "ipyleaflet-0.18.1.tar.gz", hash = "sha256:f35d70ad0e0bb2c0c160b499ab8c788333fc54576596e33b974f0dfeee941d12"}, + {file = "ipyleaflet-0.18.2-py3-none-any.whl", hash = "sha256:dc5bed1bad3ba3244fe97aac9d4ed8f8096ae3d5e6ac0c5fdfbe7f1d2a01d3f8"}, + {file = "ipyleaflet-0.18.2.tar.gz", hash = "sha256:8f166529ec7784de08822b253b8cc593fa81af8a8f967d70cbc53e45a6d3755f"}, ] [package.dependencies] @@ -2778,13 +2769,13 @@ files = [ [[package]] name = "jsonschema" -version = "4.20.0" +version = "4.21.1" description = "An implementation of JSON Schema validation for Python" optional = false python-versions = ">=3.8" files = [ - {file = "jsonschema-4.20.0-py3-none-any.whl", hash = "sha256:ed6231f0429ecf966f5bc8dfef245998220549cbbcf140f913b7464c52c3b6b3"}, - {file = "jsonschema-4.20.0.tar.gz", hash = "sha256:4f614fd46d8d61258610998997743ec5492a648b33cf478c1ddc23ed4598a5fa"}, + {file = "jsonschema-4.21.1-py3-none-any.whl", hash = "sha256:7996507afae316306f9e2290407761157c6f78002dcf7419acb99822143d1c6f"}, + {file = "jsonschema-4.21.1.tar.gz", hash = "sha256:85727c00279f5fa6bedbe6238d2aa6403bedd8b4864ab11207d07df3cc1b2ee5"}, ] [package.dependencies] @@ -3151,71 +3142,71 @@ testing = ["coverage", "pytest", "pytest-cov", "pytest-regressions"] [[package]] name = "markupsafe" -version = "2.1.3" +version = "2.1.4" description = "Safely add untrusted strings to HTML/XML markup." optional = false python-versions = ">=3.7" files = [ - {file = "MarkupSafe-2.1.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cd0f502fe016460680cd20aaa5a76d241d6f35a1c3350c474bac1273803893fa"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e09031c87a1e51556fdcb46e5bd4f59dfb743061cf93c4d6831bf894f125eb57"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:68e78619a61ecf91e76aa3e6e8e33fc4894a2bebe93410754bd28fce0a8a4f9f"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:65c1a9bcdadc6c28eecee2c119465aebff8f7a584dd719facdd9e825ec61ab52"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:525808b8019e36eb524b8c68acdd63a37e75714eac50e988180b169d64480a00"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:962f82a3086483f5e5f64dbad880d31038b698494799b097bc59c2edf392fce6"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:aa7bd130efab1c280bed0f45501b7c8795f9fdbeb02e965371bbef3523627779"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:c9c804664ebe8f83a211cace637506669e7890fec1b4195b505c214e50dd4eb7"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-win32.whl", hash = "sha256:10bbfe99883db80bdbaff2dcf681dfc6533a614f700da1287707e8a5d78a8431"}, - {file = "MarkupSafe-2.1.3-cp310-cp310-win_amd64.whl", hash = "sha256:1577735524cdad32f9f694208aa75e422adba74f1baee7551620e43a3141f559"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:ad9e82fb8f09ade1c3e1b996a6337afac2b8b9e365f926f5a61aacc71adc5b3c"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3c0fae6c3be832a0a0473ac912810b2877c8cb9d76ca48de1ed31e1c68386575"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b076b6226fb84157e3f7c971a47ff3a679d837cf338547532ab866c57930dbee"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bfce63a9e7834b12b87c64d6b155fdd9b3b96191b6bd334bf37db7ff1fe457f2"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:338ae27d6b8745585f87218a3f23f1512dbf52c26c28e322dbe54bcede54ccb9"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:e4dd52d80b8c83fdce44e12478ad2e85c64ea965e75d66dbeafb0a3e77308fcc"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:df0be2b576a7abbf737b1575f048c23fb1d769f267ec4358296f31c2479db8f9"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:5bbe06f8eeafd38e5d0a4894ffec89378b6c6a625ff57e3028921f8ff59318ac"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-win32.whl", hash = "sha256:dd15ff04ffd7e05ffcb7fe79f1b98041b8ea30ae9234aed2a9168b5797c3effb"}, - {file = "MarkupSafe-2.1.3-cp311-cp311-win_amd64.whl", hash = "sha256:134da1eca9ec0ae528110ccc9e48041e0828d79f24121a1a146161103c76e686"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:f698de3fd0c4e6972b92290a45bd9b1536bffe8c6759c62471efaa8acb4c37bc"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:aa57bd9cf8ae831a362185ee444e15a93ecb2e344c8e52e4d721ea3ab6ef1823"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ffcc3f7c66b5f5b7931a5aa68fc9cecc51e685ef90282f4a82f0f5e9b704ad11"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:47d4f1c5f80fc62fdd7777d0d40a2e9dda0a05883ab11374334f6c4de38adffd"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1f67c7038d560d92149c060157d623c542173016c4babc0c1913cca0564b9939"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:9aad3c1755095ce347e26488214ef77e0485a3c34a50c5a5e2471dff60b9dd9c"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:14ff806850827afd6b07a5f32bd917fb7f45b046ba40c57abdb636674a8b559c"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8f9293864fe09b8149f0cc42ce56e3f0e54de883a9de90cd427f191c346eb2e1"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-win32.whl", hash = "sha256:715d3562f79d540f251b99ebd6d8baa547118974341db04f5ad06d5ea3eb8007"}, - {file = "MarkupSafe-2.1.3-cp312-cp312-win_amd64.whl", hash = "sha256:1b8dd8c3fd14349433c79fa8abeb573a55fc0fdd769133baac1f5e07abf54aeb"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:8e254ae696c88d98da6555f5ace2279cf7cd5b3f52be2b5cf97feafe883b58d2"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cb0932dc158471523c9637e807d9bfb93e06a95cbf010f1a38b98623b929ef2b"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9402b03f1a1b4dc4c19845e5c749e3ab82d5078d16a2a4c2cd2df62d57bb0707"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca379055a47383d02a5400cb0d110cef0a776fc644cda797db0c5696cfd7e18e"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:b7ff0f54cb4ff66dd38bebd335a38e2c22c41a8ee45aa608efc890ac3e3931bc"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c011a4149cfbcf9f03994ec2edffcb8b1dc2d2aede7ca243746df97a5d41ce48"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:56d9f2ecac662ca1611d183feb03a3fa4406469dafe241673d521dd5ae92a155"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-win32.whl", hash = "sha256:8758846a7e80910096950b67071243da3e5a20ed2546e6392603c096778d48e0"}, - {file = "MarkupSafe-2.1.3-cp37-cp37m-win_amd64.whl", hash = "sha256:787003c0ddb00500e49a10f2844fac87aa6ce977b90b0feaaf9de23c22508b24"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:2ef12179d3a291be237280175b542c07a36e7f60718296278d8593d21ca937d4"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:2c1b19b3aaacc6e57b7e25710ff571c24d6c3613a45e905b1fde04d691b98ee0"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8afafd99945ead6e075b973fefa56379c5b5c53fd8937dad92c662da5d8fd5ee"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8c41976a29d078bb235fea9b2ecd3da465df42a562910f9022f1a03107bd02be"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d080e0a5eb2529460b30190fcfcc4199bd7f827663f858a226a81bc27beaa97e"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:69c0f17e9f5a7afdf2cc9fb2d1ce6aabdb3bafb7f38017c0b77862bcec2bbad8"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:504b320cd4b7eff6f968eddf81127112db685e81f7e36e75f9f84f0df46041c3"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:42de32b22b6b804f42c5d98be4f7e5e977ecdd9ee9b660fda1a3edf03b11792d"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-win32.whl", hash = "sha256:ceb01949af7121f9fc39f7d27f91be8546f3fb112c608bc4029aef0bab86a2a5"}, - {file = "MarkupSafe-2.1.3-cp38-cp38-win_amd64.whl", hash = "sha256:1b40069d487e7edb2676d3fbdb2b0829ffa2cd63a2ec26c4938b2d34391b4ecc"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:8023faf4e01efadfa183e863fefde0046de576c6f14659e8782065bcece22198"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6b2b56950d93e41f33b4223ead100ea0fe11f8e6ee5f641eb753ce4b77a7042b"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9dcdfd0eaf283af041973bff14a2e143b8bd64e069f4c383416ecd79a81aab58"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:05fb21170423db021895e1ea1e1f3ab3adb85d1c2333cbc2310f2a26bc77272e"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:282c2cb35b5b673bbcadb33a585408104df04f14b2d9b01d4c345a3b92861c2c"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:ab4a0df41e7c16a1392727727e7998a467472d0ad65f3ad5e6e765015df08636"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:7ef3cb2ebbf91e330e3bb937efada0edd9003683db6b57bb108c4001f37a02ea"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:0a4e4a1aff6c7ac4cd55792abf96c915634c2b97e3cc1c7129578aa68ebd754e"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-win32.whl", hash = "sha256:fec21693218efe39aa7f8599346e90c705afa52c5b31ae019b2e57e8f6542bb2"}, - {file = "MarkupSafe-2.1.3-cp39-cp39-win_amd64.whl", hash = "sha256:3fd4abcb888d15a94f32b75d8fd18ee162ca0c064f35b11134be77050296d6ba"}, - {file = "MarkupSafe-2.1.3.tar.gz", hash = "sha256:af598ed32d6ae86f1b747b82783958b1a4ab8f617b06fe68795c7f026abbdcad"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:de8153a7aae3835484ac168a9a9bdaa0c5eee4e0bc595503c95d53b942879c84"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e888ff76ceb39601c59e219f281466c6d7e66bd375b4ec1ce83bcdc68306796b"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a0b838c37ba596fcbfca71651a104a611543077156cb0a26fe0c475e1f152ee8"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dac1ebf6983148b45b5fa48593950f90ed6d1d26300604f321c74a9ca1609f8e"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0fbad3d346df8f9d72622ac71b69565e621ada2ce6572f37c2eae8dacd60385d"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:d5291d98cd3ad9a562883468c690a2a238c4a6388ab3bd155b0c75dd55ece858"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:a7cc49ef48a3c7a0005a949f3c04f8baa5409d3f663a1b36f0eba9bfe2a0396e"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:b83041cda633871572f0d3c41dddd5582ad7d22f65a72eacd8d3d6d00291df26"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-win32.whl", hash = "sha256:0c26f67b3fe27302d3a412b85ef696792c4a2386293c53ba683a89562f9399b0"}, + {file = "MarkupSafe-2.1.4-cp310-cp310-win_amd64.whl", hash = "sha256:a76055d5cb1c23485d7ddae533229039b850db711c554a12ea64a0fd8a0129e2"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9e9e3c4020aa2dc62d5dd6743a69e399ce3de58320522948af6140ac959ab863"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:0042d6a9880b38e1dd9ff83146cc3c9c18a059b9360ceae207805567aacccc69"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:55d03fea4c4e9fd0ad75dc2e7e2b6757b80c152c032ea1d1de487461d8140efc"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3ab3a886a237f6e9c9f4f7d272067e712cdb4efa774bef494dccad08f39d8ae6"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:abf5ebbec056817057bfafc0445916bb688a255a5146f900445d081db08cbabb"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:e1a0d1924a5013d4f294087e00024ad25668234569289650929ab871231668e7"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:e7902211afd0af05fbadcc9a312e4cf10f27b779cf1323e78d52377ae4b72bea"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:c669391319973e49a7c6230c218a1e3044710bc1ce4c8e6eb71f7e6d43a2c131"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-win32.whl", hash = "sha256:31f57d64c336b8ccb1966d156932f3daa4fee74176b0fdc48ef580be774aae74"}, + {file = "MarkupSafe-2.1.4-cp311-cp311-win_amd64.whl", hash = "sha256:54a7e1380dfece8847c71bf7e33da5d084e9b889c75eca19100ef98027bd9f56"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:a76cd37d229fc385738bd1ce4cba2a121cf26b53864c1772694ad0ad348e509e"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:987d13fe1d23e12a66ca2073b8d2e2a75cec2ecb8eab43ff5624ba0ad42764bc"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5244324676254697fe5c181fc762284e2c5fceeb1c4e3e7f6aca2b6f107e60dc"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:78bc995e004681246e85e28e068111a4c3f35f34e6c62da1471e844ee1446250"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a4d176cfdfde84f732c4a53109b293d05883e952bbba68b857ae446fa3119b4f"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:f9917691f410a2e0897d1ef99619fd3f7dd503647c8ff2475bf90c3cf222ad74"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:f06e5a9e99b7df44640767842f414ed5d7bedaaa78cd817ce04bbd6fd86e2dd6"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:396549cea79e8ca4ba65525470d534e8a41070e6b3500ce2414921099cb73e8d"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-win32.whl", hash = "sha256:f6be2d708a9d0e9b0054856f07ac7070fbe1754be40ca8525d5adccdbda8f475"}, + {file = "MarkupSafe-2.1.4-cp312-cp312-win_amd64.whl", hash = "sha256:5045e892cfdaecc5b4c01822f353cf2c8feb88a6ec1c0adef2a2e705eef0f656"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:7a07f40ef8f0fbc5ef1000d0c78771f4d5ca03b4953fc162749772916b298fc4"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d18b66fe626ac412d96c2ab536306c736c66cf2a31c243a45025156cc190dc8a"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:698e84142f3f884114ea8cf83e7a67ca8f4ace8454e78fe960646c6c91c63bfa"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:49a3b78a5af63ec10d8604180380c13dcd870aba7928c1fe04e881d5c792dc4e"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:15866d7f2dc60cfdde12ebb4e75e41be862348b4728300c36cdf405e258415ec"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:6aa5e2e7fc9bc042ae82d8b79d795b9a62bd8f15ba1e7594e3db243f158b5565"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:54635102ba3cf5da26eb6f96c4b8c53af8a9c0d97b64bdcb592596a6255d8518"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-win32.whl", hash = "sha256:3583a3a3ab7958e354dc1d25be74aee6228938312ee875a22330c4dc2e41beb0"}, + {file = "MarkupSafe-2.1.4-cp37-cp37m-win_amd64.whl", hash = "sha256:d6e427c7378c7f1b2bef6a344c925b8b63623d3321c09a237b7cc0e77dd98ceb"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:bf1196dcc239e608605b716e7b166eb5faf4bc192f8a44b81e85251e62584bd2"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:4df98d4a9cd6a88d6a585852f56f2155c9cdb6aec78361a19f938810aa020954"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b835aba863195269ea358cecc21b400276747cc977492319fd7682b8cd2c253d"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:23984d1bdae01bee794267424af55eef4dfc038dc5d1272860669b2aa025c9e3"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1c98c33ffe20e9a489145d97070a435ea0679fddaabcafe19982fe9c971987d5"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:9896fca4a8eb246defc8b2a7ac77ef7553b638e04fbf170bff78a40fa8a91474"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:b0fe73bac2fed83839dbdbe6da84ae2a31c11cfc1c777a40dbd8ac8a6ed1560f"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:c7556bafeaa0a50e2fe7dc86e0382dea349ebcad8f010d5a7dc6ba568eaaa789"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-win32.whl", hash = "sha256:fc1a75aa8f11b87910ffd98de62b29d6520b6d6e8a3de69a70ca34dea85d2a8a"}, + {file = "MarkupSafe-2.1.4-cp38-cp38-win_amd64.whl", hash = "sha256:3a66c36a3864df95e4f62f9167c734b3b1192cb0851b43d7cc08040c074c6279"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:765f036a3d00395a326df2835d8f86b637dbaf9832f90f5d196c3b8a7a5080cb"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:21e7af8091007bf4bebf4521184f4880a6acab8df0df52ef9e513d8e5db23411"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d5c31fe855c77cad679b302aabc42d724ed87c043b1432d457f4976add1c2c3e"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7653fa39578957bc42e5ebc15cf4361d9e0ee4b702d7d5ec96cdac860953c5b4"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:47bb5f0142b8b64ed1399b6b60f700a580335c8e1c57f2f15587bd072012decc"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:fe8512ed897d5daf089e5bd010c3dc03bb1bdae00b35588c49b98268d4a01e00"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:36d7626a8cca4d34216875aee5a1d3d654bb3dac201c1c003d182283e3205949"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:b6f14a9cd50c3cb100eb94b3273131c80d102e19bb20253ac7bd7336118a673a"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-win32.whl", hash = "sha256:c8f253a84dbd2c63c19590fa86a032ef3d8cc18923b8049d91bcdeeb2581fbf6"}, + {file = "MarkupSafe-2.1.4-cp39-cp39-win_amd64.whl", hash = "sha256:8b570a1537367b52396e53325769608f2a687ec9a4363647af1cded8928af959"}, + {file = "MarkupSafe-2.1.4.tar.gz", hash = "sha256:3aae9af4cac263007fd6309c64c6ab4506dd2b79382d9d19a1994f9240b8db4f"}, ] [[package]] @@ -3395,85 +3386,101 @@ files = [ [[package]] name = "multidict" -version = "6.0.4" +version = "6.0.5" description = "multidict implementation" optional = true python-versions = ">=3.7" files = [ - {file = "multidict-6.0.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:0b1a97283e0c85772d613878028fec909f003993e1007eafa715b24b377cb9b8"}, - {file = "multidict-6.0.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:eeb6dcc05e911516ae3d1f207d4b0520d07f54484c49dfc294d6e7d63b734171"}, - {file = "multidict-6.0.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:d6d635d5209b82a3492508cf5b365f3446afb65ae7ebd755e70e18f287b0adf7"}, - {file = "multidict-6.0.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c048099e4c9e9d615545e2001d3d8a4380bd403e1a0578734e0d31703d1b0c0b"}, - {file = "multidict-6.0.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ea20853c6dbbb53ed34cb4d080382169b6f4554d394015f1bef35e881bf83547"}, - {file = "multidict-6.0.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:16d232d4e5396c2efbbf4f6d4df89bfa905eb0d4dc5b3549d872ab898451f569"}, - {file = "multidict-6.0.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:36c63aaa167f6c6b04ef2c85704e93af16c11d20de1d133e39de6a0e84582a93"}, - {file = "multidict-6.0.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:64bdf1086b6043bf519869678f5f2757f473dee970d7abf6da91ec00acb9cb98"}, - {file = "multidict-6.0.4-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:43644e38f42e3af682690876cff722d301ac585c5b9e1eacc013b7a3f7b696a0"}, - {file = "multidict-6.0.4-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:7582a1d1030e15422262de9f58711774e02fa80df0d1578995c76214f6954988"}, - {file = "multidict-6.0.4-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:ddff9c4e225a63a5afab9dd15590432c22e8057e1a9a13d28ed128ecf047bbdc"}, - {file = "multidict-6.0.4-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:ee2a1ece51b9b9e7752e742cfb661d2a29e7bcdba2d27e66e28a99f1890e4fa0"}, - {file = "multidict-6.0.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:a2e4369eb3d47d2034032a26c7a80fcb21a2cb22e1173d761a162f11e562caa5"}, - {file = "multidict-6.0.4-cp310-cp310-win32.whl", hash = "sha256:574b7eae1ab267e5f8285f0fe881f17efe4b98c39a40858247720935b893bba8"}, - {file = "multidict-6.0.4-cp310-cp310-win_amd64.whl", hash = "sha256:4dcbb0906e38440fa3e325df2359ac6cb043df8e58c965bb45f4e406ecb162cc"}, - {file = "multidict-6.0.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:0dfad7a5a1e39c53ed00d2dd0c2e36aed4650936dc18fd9a1826a5ae1cad6f03"}, - {file = "multidict-6.0.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:64da238a09d6039e3bd39bb3aee9c21a5e34f28bfa5aa22518581f910ff94af3"}, - {file = "multidict-6.0.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ff959bee35038c4624250473988b24f846cbeb2c6639de3602c073f10410ceba"}, - {file = "multidict-6.0.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:01a3a55bd90018c9c080fbb0b9f4891db37d148a0a18722b42f94694f8b6d4c9"}, - {file = "multidict-6.0.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c5cb09abb18c1ea940fb99360ea0396f34d46566f157122c92dfa069d3e0e982"}, - {file = "multidict-6.0.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:666daae833559deb2d609afa4490b85830ab0dfca811a98b70a205621a6109fe"}, - {file = "multidict-6.0.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:11bdf3f5e1518b24530b8241529d2050014c884cf18b6fc69c0c2b30ca248710"}, - {file = "multidict-6.0.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7d18748f2d30f94f498e852c67d61261c643b349b9d2a581131725595c45ec6c"}, - {file = "multidict-6.0.4-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:458f37be2d9e4c95e2d8866a851663cbc76e865b78395090786f6cd9b3bbf4f4"}, - {file = "multidict-6.0.4-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:b1a2eeedcead3a41694130495593a559a668f382eee0727352b9a41e1c45759a"}, - {file = "multidict-6.0.4-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:7d6ae9d593ef8641544d6263c7fa6408cc90370c8cb2bbb65f8d43e5b0351d9c"}, - {file = "multidict-6.0.4-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:5979b5632c3e3534e42ca6ff856bb24b2e3071b37861c2c727ce220d80eee9ed"}, - {file = "multidict-6.0.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:dcfe792765fab89c365123c81046ad4103fcabbc4f56d1c1997e6715e8015461"}, - {file = "multidict-6.0.4-cp311-cp311-win32.whl", hash = "sha256:3601a3cece3819534b11d4efc1eb76047488fddd0c85a3948099d5da4d504636"}, - {file = "multidict-6.0.4-cp311-cp311-win_amd64.whl", hash = "sha256:81a4f0b34bd92df3da93315c6a59034df95866014ac08535fc819f043bfd51f0"}, - {file = "multidict-6.0.4-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:67040058f37a2a51ed8ea8f6b0e6ee5bd78ca67f169ce6122f3e2ec80dfe9b78"}, - {file = "multidict-6.0.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:853888594621e6604c978ce2a0444a1e6e70c8d253ab65ba11657659dcc9100f"}, - {file = "multidict-6.0.4-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:39ff62e7d0f26c248b15e364517a72932a611a9b75f35b45be078d81bdb86603"}, - {file = "multidict-6.0.4-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:af048912e045a2dc732847d33821a9d84ba553f5c5f028adbd364dd4765092ac"}, - {file = "multidict-6.0.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b1e8b901e607795ec06c9e42530788c45ac21ef3aaa11dbd0c69de543bfb79a9"}, - {file = "multidict-6.0.4-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:62501642008a8b9871ddfccbf83e4222cf8ac0d5aeedf73da36153ef2ec222d2"}, - {file = "multidict-6.0.4-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:99b76c052e9f1bc0721f7541e5e8c05db3941eb9ebe7b8553c625ef88d6eefde"}, - {file = "multidict-6.0.4-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:509eac6cf09c794aa27bcacfd4d62c885cce62bef7b2c3e8b2e49d365b5003fe"}, - {file = "multidict-6.0.4-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:21a12c4eb6ddc9952c415f24eef97e3e55ba3af61f67c7bc388dcdec1404a067"}, - {file = "multidict-6.0.4-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:5cad9430ab3e2e4fa4a2ef4450f548768400a2ac635841bc2a56a2052cdbeb87"}, - {file = "multidict-6.0.4-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:ab55edc2e84460694295f401215f4a58597f8f7c9466faec545093045476327d"}, - {file = "multidict-6.0.4-cp37-cp37m-win32.whl", hash = "sha256:5a4dcf02b908c3b8b17a45fb0f15b695bf117a67b76b7ad18b73cf8e92608775"}, - {file = "multidict-6.0.4-cp37-cp37m-win_amd64.whl", hash = "sha256:6ed5f161328b7df384d71b07317f4d8656434e34591f20552c7bcef27b0ab88e"}, - {file = "multidict-6.0.4-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:5fc1b16f586f049820c5c5b17bb4ee7583092fa0d1c4e28b5239181ff9532e0c"}, - {file = "multidict-6.0.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1502e24330eb681bdaa3eb70d6358e818e8e8f908a22a1851dfd4e15bc2f8161"}, - {file = "multidict-6.0.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:b692f419760c0e65d060959df05f2a531945af31fda0c8a3b3195d4efd06de11"}, - {file = "multidict-6.0.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:45e1ecb0379bfaab5eef059f50115b54571acfbe422a14f668fc8c27ba410e7e"}, - {file = "multidict-6.0.4-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ddd3915998d93fbcd2566ddf9cf62cdb35c9e093075f862935573d265cf8f65d"}, - {file = "multidict-6.0.4-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:59d43b61c59d82f2effb39a93c48b845efe23a3852d201ed2d24ba830d0b4cf2"}, - {file = "multidict-6.0.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cc8e1d0c705233c5dd0c5e6460fbad7827d5d36f310a0fadfd45cc3029762258"}, - {file = "multidict-6.0.4-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d6aa0418fcc838522256761b3415822626f866758ee0bc6632c9486b179d0b52"}, - {file = "multidict-6.0.4-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:6748717bb10339c4760c1e63da040f5f29f5ed6e59d76daee30305894069a660"}, - {file = "multidict-6.0.4-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:4d1a3d7ef5e96b1c9e92f973e43aa5e5b96c659c9bc3124acbbd81b0b9c8a951"}, - {file = "multidict-6.0.4-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:4372381634485bec7e46718edc71528024fcdc6f835baefe517b34a33c731d60"}, - {file = "multidict-6.0.4-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:fc35cb4676846ef752816d5be2193a1e8367b4c1397b74a565a9d0389c433a1d"}, - {file = "multidict-6.0.4-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:4b9d9e4e2b37daddb5c23ea33a3417901fa7c7b3dee2d855f63ee67a0b21e5b1"}, - {file = "multidict-6.0.4-cp38-cp38-win32.whl", hash = "sha256:e41b7e2b59679edfa309e8db64fdf22399eec4b0b24694e1b2104fb789207779"}, - {file = "multidict-6.0.4-cp38-cp38-win_amd64.whl", hash = "sha256:d6c254ba6e45d8e72739281ebc46ea5eb5f101234f3ce171f0e9f5cc86991480"}, - {file = "multidict-6.0.4-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:16ab77bbeb596e14212e7bab8429f24c1579234a3a462105cda4a66904998664"}, - {file = "multidict-6.0.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:bc779e9e6f7fda81b3f9aa58e3a6091d49ad528b11ed19f6621408806204ad35"}, - {file = "multidict-6.0.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:4ceef517eca3e03c1cceb22030a3e39cb399ac86bff4e426d4fc6ae49052cc60"}, - {file = "multidict-6.0.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:281af09f488903fde97923c7744bb001a9b23b039a909460d0f14edc7bf59706"}, - {file = "multidict-6.0.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:52f2dffc8acaba9a2f27174c41c9e57f60b907bb9f096b36b1a1f3be71c6284d"}, - {file = "multidict-6.0.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b41156839806aecb3641f3208c0dafd3ac7775b9c4c422d82ee2a45c34ba81ca"}, - {file = "multidict-6.0.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d5e3fc56f88cc98ef8139255cf8cd63eb2c586531e43310ff859d6bb3a6b51f1"}, - {file = "multidict-6.0.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8316a77808c501004802f9beebde51c9f857054a0c871bd6da8280e718444449"}, - {file = "multidict-6.0.4-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:f70b98cd94886b49d91170ef23ec5c0e8ebb6f242d734ed7ed677b24d50c82cf"}, - {file = "multidict-6.0.4-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:bf6774e60d67a9efe02b3616fee22441d86fab4c6d335f9d2051d19d90a40063"}, - {file = "multidict-6.0.4-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:e69924bfcdda39b722ef4d9aa762b2dd38e4632b3641b1d9a57ca9cd18f2f83a"}, - {file = "multidict-6.0.4-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:6b181d8c23da913d4ff585afd1155a0e1194c0b50c54fcfe286f70cdaf2b7176"}, - {file = "multidict-6.0.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:52509b5be062d9eafc8170e53026fbc54cf3b32759a23d07fd935fb04fc22d95"}, - {file = "multidict-6.0.4-cp39-cp39-win32.whl", hash = "sha256:27c523fbfbdfd19c6867af7346332b62b586eed663887392cff78d614f9ec313"}, - {file = "multidict-6.0.4-cp39-cp39-win_amd64.whl", hash = "sha256:33029f5734336aa0d4c0384525da0387ef89148dc7191aae00ca5fb23d7aafc2"}, - {file = "multidict-6.0.4.tar.gz", hash = "sha256:3666906492efb76453c0e7b97f2cf459b0682e7402c0489a95484965dbc1da49"}, + {file = "multidict-6.0.5-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:228b644ae063c10e7f324ab1ab6b548bdf6f8b47f3ec234fef1093bc2735e5f9"}, + {file = "multidict-6.0.5-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:896ebdcf62683551312c30e20614305f53125750803b614e9e6ce74a96232604"}, + {file = "multidict-6.0.5-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:411bf8515f3be9813d06004cac41ccf7d1cd46dfe233705933dd163b60e37600"}, + {file = "multidict-6.0.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d147090048129ce3c453f0292e7697d333db95e52616b3793922945804a433c"}, + {file = "multidict-6.0.5-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:215ed703caf15f578dca76ee6f6b21b7603791ae090fbf1ef9d865571039ade5"}, + {file = "multidict-6.0.5-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7c6390cf87ff6234643428991b7359b5f59cc15155695deb4eda5c777d2b880f"}, + {file = "multidict-6.0.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21fd81c4ebdb4f214161be351eb5bcf385426bf023041da2fd9e60681f3cebae"}, + {file = "multidict-6.0.5-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3cc2ad10255f903656017363cd59436f2111443a76f996584d1077e43ee51182"}, + {file = "multidict-6.0.5-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:6939c95381e003f54cd4c5516740faba40cf5ad3eeff460c3ad1d3e0ea2549bf"}, + {file = "multidict-6.0.5-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:220dd781e3f7af2c2c1053da9fa96d9cf3072ca58f057f4c5adaaa1cab8fc442"}, + {file = "multidict-6.0.5-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:766c8f7511df26d9f11cd3a8be623e59cca73d44643abab3f8c8c07620524e4a"}, + {file = "multidict-6.0.5-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:fe5d7785250541f7f5019ab9cba2c71169dc7d74d0f45253f8313f436458a4ef"}, + {file = "multidict-6.0.5-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:c1c1496e73051918fcd4f58ff2e0f2f3066d1c76a0c6aeffd9b45d53243702cc"}, + {file = "multidict-6.0.5-cp310-cp310-win32.whl", hash = "sha256:7afcdd1fc07befad18ec4523a782cde4e93e0a2bf71239894b8d61ee578c1319"}, + {file = "multidict-6.0.5-cp310-cp310-win_amd64.whl", hash = "sha256:99f60d34c048c5c2fabc766108c103612344c46e35d4ed9ae0673d33c8fb26e8"}, + {file = "multidict-6.0.5-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:f285e862d2f153a70586579c15c44656f888806ed0e5b56b64489afe4a2dbfba"}, + {file = "multidict-6.0.5-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:53689bb4e102200a4fafa9de9c7c3c212ab40a7ab2c8e474491914d2305f187e"}, + {file = "multidict-6.0.5-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:612d1156111ae11d14afaf3a0669ebf6c170dbb735e510a7438ffe2369a847fd"}, + {file = "multidict-6.0.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7be7047bd08accdb7487737631d25735c9a04327911de89ff1b26b81745bd4e3"}, + {file = "multidict-6.0.5-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de170c7b4fe6859beb8926e84f7d7d6c693dfe8e27372ce3b76f01c46e489fcf"}, + {file = "multidict-6.0.5-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:04bde7a7b3de05732a4eb39c94574db1ec99abb56162d6c520ad26f83267de29"}, + {file = "multidict-6.0.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:85f67aed7bb647f93e7520633d8f51d3cbc6ab96957c71272b286b2f30dc70ed"}, + {file = "multidict-6.0.5-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:425bf820055005bfc8aa9a0b99ccb52cc2f4070153e34b701acc98d201693733"}, + {file = "multidict-6.0.5-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:d3eb1ceec286eba8220c26f3b0096cf189aea7057b6e7b7a2e60ed36b373b77f"}, + {file = "multidict-6.0.5-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:7901c05ead4b3fb75113fb1dd33eb1253c6d3ee37ce93305acd9d38e0b5f21a4"}, + {file = "multidict-6.0.5-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:e0e79d91e71b9867c73323a3444724d496c037e578a0e1755ae159ba14f4f3d1"}, + {file = "multidict-6.0.5-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:29bfeb0dff5cb5fdab2023a7a9947b3b4af63e9c47cae2a10ad58394b517fddc"}, + {file = "multidict-6.0.5-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:e030047e85cbcedbfc073f71836d62dd5dadfbe7531cae27789ff66bc551bd5e"}, + {file = "multidict-6.0.5-cp311-cp311-win32.whl", hash = "sha256:2f4848aa3baa109e6ab81fe2006c77ed4d3cd1e0ac2c1fbddb7b1277c168788c"}, + {file = "multidict-6.0.5-cp311-cp311-win_amd64.whl", hash = "sha256:2faa5ae9376faba05f630d7e5e6be05be22913782b927b19d12b8145968a85ea"}, + {file = "multidict-6.0.5-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:51d035609b86722963404f711db441cf7134f1889107fb171a970c9701f92e1e"}, + {file = "multidict-6.0.5-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:cbebcd5bcaf1eaf302617c114aa67569dd3f090dd0ce8ba9e35e9985b41ac35b"}, + {file = "multidict-6.0.5-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:2ffc42c922dbfddb4a4c3b438eb056828719f07608af27d163191cb3e3aa6cc5"}, + {file = "multidict-6.0.5-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ceb3b7e6a0135e092de86110c5a74e46bda4bd4fbfeeb3a3bcec79c0f861e450"}, + {file = "multidict-6.0.5-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:79660376075cfd4b2c80f295528aa6beb2058fd289f4c9252f986751a4cd0496"}, + {file = "multidict-6.0.5-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e4428b29611e989719874670fd152b6625500ad6c686d464e99f5aaeeaca175a"}, + {file = "multidict-6.0.5-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d84a5c3a5f7ce6db1f999fb9438f686bc2e09d38143f2d93d8406ed2dd6b9226"}, + {file = "multidict-6.0.5-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:76c0de87358b192de7ea9649beb392f107dcad9ad27276324c24c91774ca5271"}, + {file = "multidict-6.0.5-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:79a6d2ba910adb2cbafc95dad936f8b9386e77c84c35bc0add315b856d7c3abb"}, + {file = "multidict-6.0.5-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:92d16a3e275e38293623ebf639c471d3e03bb20b8ebb845237e0d3664914caef"}, + {file = "multidict-6.0.5-cp312-cp312-musllinux_1_1_ppc64le.whl", hash = "sha256:fb616be3538599e797a2017cccca78e354c767165e8858ab5116813146041a24"}, + {file = "multidict-6.0.5-cp312-cp312-musllinux_1_1_s390x.whl", hash = "sha256:14c2976aa9038c2629efa2c148022ed5eb4cb939e15ec7aace7ca932f48f9ba6"}, + {file = "multidict-6.0.5-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:435a0984199d81ca178b9ae2c26ec3d49692d20ee29bc4c11a2a8d4514c67eda"}, + {file = "multidict-6.0.5-cp312-cp312-win32.whl", hash = "sha256:9fe7b0653ba3d9d65cbe7698cca585bf0f8c83dbbcc710db9c90f478e175f2d5"}, + {file = "multidict-6.0.5-cp312-cp312-win_amd64.whl", hash = "sha256:01265f5e40f5a17f8241d52656ed27192be03bfa8764d88e8220141d1e4b3556"}, + {file = "multidict-6.0.5-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:19fe01cea168585ba0f678cad6f58133db2aa14eccaf22f88e4a6dccadfad8b3"}, + {file = "multidict-6.0.5-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6bf7a982604375a8d49b6cc1b781c1747f243d91b81035a9b43a2126c04766f5"}, + {file = "multidict-6.0.5-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:107c0cdefe028703fb5dafe640a409cb146d44a6ae201e55b35a4af8e95457dd"}, + {file = "multidict-6.0.5-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:403c0911cd5d5791605808b942c88a8155c2592e05332d2bf78f18697a5fa15e"}, + {file = "multidict-6.0.5-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:aeaf541ddbad8311a87dd695ed9642401131ea39ad7bc8cf3ef3967fd093b626"}, + {file = "multidict-6.0.5-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e4972624066095e52b569e02b5ca97dbd7a7ddd4294bf4e7247d52635630dd83"}, + {file = "multidict-6.0.5-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:d946b0a9eb8aaa590df1fe082cee553ceab173e6cb5b03239716338629c50c7a"}, + {file = "multidict-6.0.5-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:b55358304d7a73d7bdf5de62494aaf70bd33015831ffd98bc498b433dfe5b10c"}, + {file = "multidict-6.0.5-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:a3145cb08d8625b2d3fee1b2d596a8766352979c9bffe5d7833e0503d0f0b5e5"}, + {file = "multidict-6.0.5-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:d65f25da8e248202bd47445cec78e0025c0fe7582b23ec69c3b27a640dd7a8e3"}, + {file = "multidict-6.0.5-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:c9bf56195c6bbd293340ea82eafd0071cb3d450c703d2c93afb89f93b8386ccc"}, + {file = "multidict-6.0.5-cp37-cp37m-win32.whl", hash = "sha256:69db76c09796b313331bb7048229e3bee7928eb62bab5e071e9f7fcc4879caee"}, + {file = "multidict-6.0.5-cp37-cp37m-win_amd64.whl", hash = "sha256:fce28b3c8a81b6b36dfac9feb1de115bab619b3c13905b419ec71d03a3fc1423"}, + {file = "multidict-6.0.5-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:76f067f5121dcecf0d63a67f29080b26c43c71a98b10c701b0677e4a065fbd54"}, + {file = "multidict-6.0.5-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:b82cc8ace10ab5bd93235dfaab2021c70637005e1ac787031f4d1da63d493c1d"}, + {file = "multidict-6.0.5-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:5cb241881eefd96b46f89b1a056187ea8e9ba14ab88ba632e68d7a2ecb7aadf7"}, + {file = "multidict-6.0.5-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e8e94e6912639a02ce173341ff62cc1201232ab86b8a8fcc05572741a5dc7d93"}, + {file = "multidict-6.0.5-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:09a892e4a9fb47331da06948690ae38eaa2426de97b4ccbfafbdcbe5c8f37ff8"}, + {file = "multidict-6.0.5-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:55205d03e8a598cfc688c71ca8ea5f66447164efff8869517f175ea632c7cb7b"}, + {file = "multidict-6.0.5-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:37b15024f864916b4951adb95d3a80c9431299080341ab9544ed148091b53f50"}, + {file = "multidict-6.0.5-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f2a1dee728b52b33eebff5072817176c172050d44d67befd681609b4746e1c2e"}, + {file = "multidict-6.0.5-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:edd08e6f2f1a390bf137080507e44ccc086353c8e98c657e666c017718561b89"}, + {file = "multidict-6.0.5-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:60d698e8179a42ec85172d12f50b1668254628425a6bd611aba022257cac1386"}, + {file = "multidict-6.0.5-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:3d25f19500588cbc47dc19081d78131c32637c25804df8414463ec908631e453"}, + {file = "multidict-6.0.5-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:4cc0ef8b962ac7a5e62b9e826bd0cd5040e7d401bc45a6835910ed699037a461"}, + {file = "multidict-6.0.5-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:eca2e9d0cc5a889850e9bbd68e98314ada174ff6ccd1129500103df7a94a7a44"}, + {file = "multidict-6.0.5-cp38-cp38-win32.whl", hash = "sha256:4a6a4f196f08c58c59e0b8ef8ec441d12aee4125a7d4f4fef000ccb22f8d7241"}, + {file = "multidict-6.0.5-cp38-cp38-win_amd64.whl", hash = "sha256:0275e35209c27a3f7951e1ce7aaf93ce0d163b28948444bec61dd7badc6d3f8c"}, + {file = "multidict-6.0.5-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:e7be68734bd8c9a513f2b0cfd508802d6609da068f40dc57d4e3494cefc92929"}, + {file = "multidict-6.0.5-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:1d9ea7a7e779d7a3561aade7d596649fbecfa5c08a7674b11b423783217933f9"}, + {file = "multidict-6.0.5-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:ea1456df2a27c73ce51120fa2f519f1bea2f4a03a917f4a43c8707cf4cbbae1a"}, + {file = "multidict-6.0.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cf590b134eb70629e350691ecca88eac3e3b8b3c86992042fb82e3cb1830d5e1"}, + {file = "multidict-6.0.5-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5c0631926c4f58e9a5ccce555ad7747d9a9f8b10619621f22f9635f069f6233e"}, + {file = "multidict-6.0.5-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dce1c6912ab9ff5f179eaf6efe7365c1f425ed690b03341911bf4939ef2f3046"}, + {file = "multidict-6.0.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0868d64af83169e4d4152ec612637a543f7a336e4a307b119e98042e852ad9c"}, + {file = "multidict-6.0.5-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:141b43360bfd3bdd75f15ed811850763555a251e38b2405967f8e25fb43f7d40"}, + {file = "multidict-6.0.5-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:7df704ca8cf4a073334e0427ae2345323613e4df18cc224f647f251e5e75a527"}, + {file = "multidict-6.0.5-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:6214c5a5571802c33f80e6c84713b2c79e024995b9c5897f794b43e714daeec9"}, + {file = "multidict-6.0.5-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:cd6c8fca38178e12c00418de737aef1261576bd1b6e8c6134d3e729a4e858b38"}, + {file = "multidict-6.0.5-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:e02021f87a5b6932fa6ce916ca004c4d441509d33bbdbeca70d05dff5e9d2479"}, + {file = "multidict-6.0.5-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:ebd8d160f91a764652d3e51ce0d2956b38efe37c9231cd82cfc0bed2e40b581c"}, + {file = "multidict-6.0.5-cp39-cp39-win32.whl", hash = "sha256:04da1bb8c8dbadf2a18a452639771951c662c5ad03aefe4884775454be322c9b"}, + {file = "multidict-6.0.5-cp39-cp39-win_amd64.whl", hash = "sha256:d6f6d4f185481c9669b9447bf9d9cf3b95a0e9df9d169bbc17e363b7d5487755"}, + {file = "multidict-6.0.5-py3-none-any.whl", hash = "sha256:0d63c74e3d7ab26de115c49bffc92cc77ed23395303d496eae515d4204a625e7"}, + {file = "multidict-6.0.5.tar.gz", hash = "sha256:f7e301075edaf50500f0b341543c41194d8df3ae5caf4702f2095f3ca73dd8da"}, ] [[package]] @@ -3543,13 +3550,13 @@ test = ["pep440", "pre-commit", "pytest", "testpath"] [[package]] name = "nest-asyncio" -version = "1.5.8" +version = "1.6.0" description = "Patch asyncio to allow nested event loops" optional = false python-versions = ">=3.5" files = [ - {file = "nest_asyncio-1.5.8-py3-none-any.whl", hash = "sha256:accda7a339a70599cb08f9dd09a67e0c2ef8d8d6f4c07f96ab203f2ae254e48d"}, - {file = "nest_asyncio-1.5.8.tar.gz", hash = "sha256:25aa2ca0d2a5b5531956b9e273b45cf664cae2b145101d73b86b199978d48fdb"}, + {file = "nest_asyncio-1.6.0-py3-none-any.whl", hash = "sha256:87af6efd6b5e897c81050477ef65c62e2b2f35d51703cae01aff2905b1852e1c"}, + {file = "nest_asyncio-1.6.0.tar.gz", hash = "sha256:6f172d5449aca15afd6c646851f4e31e02c598d553a667e38cafa997cfec55fe"}, ] [[package]] @@ -3694,71 +3701,70 @@ files = [ [[package]] name = "pandas" -version = "2.1.4" +version = "2.0.3" description = "Powerful data structures for data analysis, time series, and statistics" optional = false -python-versions = ">=3.9" +python-versions = ">=3.8" files = [ - {file = "pandas-2.1.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bdec823dc6ec53f7a6339a0e34c68b144a7a1fd28d80c260534c39c62c5bf8c9"}, - {file = "pandas-2.1.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:294d96cfaf28d688f30c918a765ea2ae2e0e71d3536754f4b6de0ea4a496d034"}, - {file = "pandas-2.1.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6b728fb8deba8905b319f96447a27033969f3ea1fea09d07d296c9030ab2ed1d"}, - {file = "pandas-2.1.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:00028e6737c594feac3c2df15636d73ace46b8314d236100b57ed7e4b9ebe8d9"}, - {file = "pandas-2.1.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:426dc0f1b187523c4db06f96fb5c8d1a845e259c99bda74f7de97bd8a3bb3139"}, - {file = "pandas-2.1.4-cp310-cp310-win_amd64.whl", hash = "sha256:f237e6ca6421265643608813ce9793610ad09b40154a3344a088159590469e46"}, - {file = "pandas-2.1.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b7d852d16c270e4331f6f59b3e9aa23f935f5c4b0ed2d0bc77637a8890a5d092"}, - {file = "pandas-2.1.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:bd7d5f2f54f78164b3d7a40f33bf79a74cdee72c31affec86bfcabe7e0789821"}, - {file = "pandas-2.1.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0aa6e92e639da0d6e2017d9ccff563222f4eb31e4b2c3cf32a2a392fc3103c0d"}, - {file = "pandas-2.1.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d797591b6846b9db79e65dc2d0d48e61f7db8d10b2a9480b4e3faaddc421a171"}, - {file = "pandas-2.1.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d2d3e7b00f703aea3945995ee63375c61b2e6aa5aa7871c5d622870e5e137623"}, - {file = "pandas-2.1.4-cp311-cp311-win_amd64.whl", hash = "sha256:dc9bf7ade01143cddc0074aa6995edd05323974e6e40d9dbde081021ded8510e"}, - {file = "pandas-2.1.4-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:482d5076e1791777e1571f2e2d789e940dedd927325cc3cb6d0800c6304082f6"}, - {file = "pandas-2.1.4-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8a706cfe7955c4ca59af8c7a0517370eafbd98593155b48f10f9811da440248b"}, - {file = "pandas-2.1.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b0513a132a15977b4a5b89aabd304647919bc2169eac4c8536afb29c07c23540"}, - {file = "pandas-2.1.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e9f17f2b6fc076b2a0078862547595d66244db0f41bf79fc5f64a5c4d635bead"}, - {file = "pandas-2.1.4-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:45d63d2a9b1b37fa6c84a68ba2422dc9ed018bdaa668c7f47566a01188ceeec1"}, - {file = "pandas-2.1.4-cp312-cp312-win_amd64.whl", hash = "sha256:f69b0c9bb174a2342818d3e2778584e18c740d56857fc5cdb944ec8bbe4082cf"}, - {file = "pandas-2.1.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3f06bda01a143020bad20f7a85dd5f4a1600112145f126bc9e3e42077c24ef34"}, - {file = "pandas-2.1.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:ab5796839eb1fd62a39eec2916d3e979ec3130509930fea17fe6f81e18108f6a"}, - {file = "pandas-2.1.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:edbaf9e8d3a63a9276d707b4d25930a262341bca9874fcb22eff5e3da5394732"}, - {file = "pandas-2.1.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1ebfd771110b50055712b3b711b51bee5d50135429364d0498e1213a7adc2be8"}, - {file = "pandas-2.1.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8ea107e0be2aba1da619cc6ba3f999b2bfc9669a83554b1904ce3dd9507f0860"}, - {file = "pandas-2.1.4-cp39-cp39-win_amd64.whl", hash = "sha256:d65148b14788b3758daf57bf42725caa536575da2b64df9964c563b015230984"}, - {file = "pandas-2.1.4.tar.gz", hash = "sha256:fcb68203c833cc735321512e13861358079a96c174a61f5116a1de89c58c0ef7"}, + {file = "pandas-2.0.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e4c7c9f27a4185304c7caf96dc7d91bc60bc162221152de697c98eb0b2648dd8"}, + {file = "pandas-2.0.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f167beed68918d62bffb6ec64f2e1d8a7d297a038f86d4aed056b9493fca407f"}, + {file = "pandas-2.0.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ce0c6f76a0f1ba361551f3e6dceaff06bde7514a374aa43e33b588ec10420183"}, + {file = "pandas-2.0.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba619e410a21d8c387a1ea6e8a0e49bb42216474436245718d7f2e88a2f8d7c0"}, + {file = "pandas-2.0.3-cp310-cp310-win32.whl", hash = "sha256:3ef285093b4fe5058eefd756100a367f27029913760773c8bf1d2d8bebe5d210"}, + {file = "pandas-2.0.3-cp310-cp310-win_amd64.whl", hash = "sha256:9ee1a69328d5c36c98d8e74db06f4ad518a1840e8ccb94a4ba86920986bb617e"}, + {file = "pandas-2.0.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b084b91d8d66ab19f5bb3256cbd5ea661848338301940e17f4492b2ce0801fe8"}, + {file = "pandas-2.0.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:37673e3bdf1551b95bf5d4ce372b37770f9529743d2498032439371fc7b7eb26"}, + {file = "pandas-2.0.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b9cb1e14fdb546396b7e1b923ffaeeac24e4cedd14266c3497216dd4448e4f2d"}, + {file = "pandas-2.0.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d9cd88488cceb7635aebb84809d087468eb33551097d600c6dad13602029c2df"}, + {file = "pandas-2.0.3-cp311-cp311-win32.whl", hash = "sha256:694888a81198786f0e164ee3a581df7d505024fbb1f15202fc7db88a71d84ebd"}, + {file = "pandas-2.0.3-cp311-cp311-win_amd64.whl", hash = "sha256:6a21ab5c89dcbd57f78d0ae16630b090eec626360085a4148693def5452d8a6b"}, + {file = "pandas-2.0.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:9e4da0d45e7f34c069fe4d522359df7d23badf83abc1d1cef398895822d11061"}, + {file = "pandas-2.0.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:32fca2ee1b0d93dd71d979726b12b61faa06aeb93cf77468776287f41ff8fdc5"}, + {file = "pandas-2.0.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:258d3624b3ae734490e4d63c430256e716f488c4fcb7c8e9bde2d3aa46c29089"}, + {file = "pandas-2.0.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9eae3dc34fa1aa7772dd3fc60270d13ced7346fcbcfee017d3132ec625e23bb0"}, + {file = "pandas-2.0.3-cp38-cp38-win32.whl", hash = "sha256:f3421a7afb1a43f7e38e82e844e2bca9a6d793d66c1a7f9f0ff39a795bbc5e02"}, + {file = "pandas-2.0.3-cp38-cp38-win_amd64.whl", hash = "sha256:69d7f3884c95da3a31ef82b7618af5710dba95bb885ffab339aad925c3e8ce78"}, + {file = "pandas-2.0.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:5247fb1ba347c1261cbbf0fcfba4a3121fbb4029d95d9ef4dc45406620b25c8b"}, + {file = "pandas-2.0.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:81af086f4543c9d8bb128328b5d32e9986e0c84d3ee673a2ac6fb57fd14f755e"}, + {file = "pandas-2.0.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1994c789bf12a7c5098277fb43836ce090f1073858c10f9220998ac74f37c69b"}, + {file = "pandas-2.0.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5ec591c48e29226bcbb316e0c1e9423622bc7a4eaf1ef7c3c9fa1a3981f89641"}, + {file = "pandas-2.0.3-cp39-cp39-win32.whl", hash = "sha256:04dbdbaf2e4d46ca8da896e1805bc04eb85caa9a82e259e8eed00254d5e0c682"}, + {file = "pandas-2.0.3-cp39-cp39-win_amd64.whl", hash = "sha256:1168574b036cd8b93abc746171c9b4f1b83467438a5e45909fed645cf8692dbc"}, + {file = "pandas-2.0.3.tar.gz", hash = "sha256:c02f372a88e0d17f36d3093a644c73cfc1788e876a7c4bcb4020a77512e2043c"}, ] [package.dependencies] numpy = [ - {version = ">=1.23.2,<2", markers = "python_version == \"3.11\""}, - {version = ">=1.26.0,<2", markers = "python_version >= \"3.12\""}, - {version = ">=1.22.4,<2", markers = "python_version < \"3.11\""}, + {version = ">=1.23.2", markers = "python_version >= \"3.11\""}, + {version = ">=1.20.3", markers = "python_version < \"3.10\""}, + {version = ">=1.21.0", markers = "python_version >= \"3.10\" and python_version < \"3.11\""}, ] python-dateutil = ">=2.8.2" pytz = ">=2020.1" tzdata = ">=2022.1" [package.extras] -all = ["PyQt5 (>=5.15.6)", "SQLAlchemy (>=1.4.36)", "beautifulsoup4 (>=4.11.1)", "bottleneck (>=1.3.4)", "dataframe-api-compat (>=0.1.7)", "fastparquet (>=0.8.1)", "fsspec (>=2022.05.0)", "gcsfs (>=2022.05.0)", "html5lib (>=1.1)", "hypothesis (>=6.46.1)", "jinja2 (>=3.1.2)", "lxml (>=4.8.0)", "matplotlib (>=3.6.1)", "numba (>=0.55.2)", "numexpr (>=2.8.0)", "odfpy (>=1.4.1)", "openpyxl (>=3.0.10)", "pandas-gbq (>=0.17.5)", "psycopg2 (>=2.9.3)", "pyarrow (>=7.0.0)", "pymysql (>=1.0.2)", "pyreadstat (>=1.1.5)", "pytest (>=7.3.2)", "pytest-xdist (>=2.2.0)", "pyxlsb (>=1.0.9)", "qtpy (>=2.2.0)", "s3fs (>=2022.05.0)", "scipy (>=1.8.1)", "tables (>=3.7.0)", "tabulate (>=0.8.10)", "xarray (>=2022.03.0)", "xlrd (>=2.0.1)", "xlsxwriter (>=3.0.3)", "zstandard (>=0.17.0)"] -aws = ["s3fs (>=2022.05.0)"] -clipboard = ["PyQt5 (>=5.15.6)", "qtpy (>=2.2.0)"] -compression = ["zstandard (>=0.17.0)"] -computation = ["scipy (>=1.8.1)", "xarray (>=2022.03.0)"] -consortium-standard = ["dataframe-api-compat (>=0.1.7)"] -excel = ["odfpy (>=1.4.1)", "openpyxl (>=3.0.10)", "pyxlsb (>=1.0.9)", "xlrd (>=2.0.1)", "xlsxwriter (>=3.0.3)"] +all = ["PyQt5 (>=5.15.1)", "SQLAlchemy (>=1.4.16)", "beautifulsoup4 (>=4.9.3)", "bottleneck (>=1.3.2)", "brotlipy (>=0.7.0)", "fastparquet (>=0.6.3)", "fsspec (>=2021.07.0)", "gcsfs (>=2021.07.0)", "html5lib (>=1.1)", "hypothesis (>=6.34.2)", "jinja2 (>=3.0.0)", "lxml (>=4.6.3)", "matplotlib (>=3.6.1)", "numba (>=0.53.1)", "numexpr (>=2.7.3)", "odfpy (>=1.4.1)", "openpyxl (>=3.0.7)", "pandas-gbq (>=0.15.0)", "psycopg2 (>=2.8.6)", "pyarrow (>=7.0.0)", "pymysql (>=1.0.2)", "pyreadstat (>=1.1.2)", "pytest (>=7.3.2)", "pytest-asyncio (>=0.17.0)", "pytest-xdist (>=2.2.0)", "python-snappy (>=0.6.0)", "pyxlsb (>=1.0.8)", "qtpy (>=2.2.0)", "s3fs (>=2021.08.0)", "scipy (>=1.7.1)", "tables (>=3.6.1)", "tabulate (>=0.8.9)", "xarray (>=0.21.0)", "xlrd (>=2.0.1)", "xlsxwriter (>=1.4.3)", "zstandard (>=0.15.2)"] +aws = ["s3fs (>=2021.08.0)"] +clipboard = ["PyQt5 (>=5.15.1)", "qtpy (>=2.2.0)"] +compression = ["brotlipy (>=0.7.0)", "python-snappy (>=0.6.0)", "zstandard (>=0.15.2)"] +computation = ["scipy (>=1.7.1)", "xarray (>=0.21.0)"] +excel = ["odfpy (>=1.4.1)", "openpyxl (>=3.0.7)", "pyxlsb (>=1.0.8)", "xlrd (>=2.0.1)", "xlsxwriter (>=1.4.3)"] feather = ["pyarrow (>=7.0.0)"] -fss = ["fsspec (>=2022.05.0)"] -gcp = ["gcsfs (>=2022.05.0)", "pandas-gbq (>=0.17.5)"] -hdf5 = ["tables (>=3.7.0)"] -html = ["beautifulsoup4 (>=4.11.1)", "html5lib (>=1.1)", "lxml (>=4.8.0)"] -mysql = ["SQLAlchemy (>=1.4.36)", "pymysql (>=1.0.2)"] -output-formatting = ["jinja2 (>=3.1.2)", "tabulate (>=0.8.10)"] +fss = ["fsspec (>=2021.07.0)"] +gcp = ["gcsfs (>=2021.07.0)", "pandas-gbq (>=0.15.0)"] +hdf5 = ["tables (>=3.6.1)"] +html = ["beautifulsoup4 (>=4.9.3)", "html5lib (>=1.1)", "lxml (>=4.6.3)"] +mysql = ["SQLAlchemy (>=1.4.16)", "pymysql (>=1.0.2)"] +output-formatting = ["jinja2 (>=3.0.0)", "tabulate (>=0.8.9)"] parquet = ["pyarrow (>=7.0.0)"] -performance = ["bottleneck (>=1.3.4)", "numba (>=0.55.2)", "numexpr (>=2.8.0)"] +performance = ["bottleneck (>=1.3.2)", "numba (>=0.53.1)", "numexpr (>=2.7.1)"] plot = ["matplotlib (>=3.6.1)"] -postgresql = ["SQLAlchemy (>=1.4.36)", "psycopg2 (>=2.9.3)"] -spss = ["pyreadstat (>=1.1.5)"] -sql-other = ["SQLAlchemy (>=1.4.36)"] -test = ["hypothesis (>=6.46.1)", "pytest (>=7.3.2)", "pytest-xdist (>=2.2.0)"] -xml = ["lxml (>=4.8.0)"] +postgresql = ["SQLAlchemy (>=1.4.16)", "psycopg2 (>=2.8.6)"] +spss = ["pyreadstat (>=1.1.2)"] +sql-other = ["SQLAlchemy (>=1.4.16)"] +test = ["hypothesis (>=6.34.2)", "pytest (>=7.3.2)", "pytest-asyncio (>=0.17.0)", "pytest-xdist (>=2.2.0)"] +xml = ["lxml (>=4.6.3)"] [[package]] name = "parso" @@ -4040,13 +4046,13 @@ typing = ["ipython", "pandas-stubs", "pyright"] [[package]] name = "pluggy" -version = "1.3.0" +version = "1.4.0" description = "plugin and hook calling mechanisms for python" optional = false python-versions = ">=3.8" files = [ - {file = "pluggy-1.3.0-py3-none-any.whl", hash = "sha256:d89c696a773f8bd377d18e5ecda92b7a3793cbe66c87060a6fb58c7b6e1061f7"}, - {file = "pluggy-1.3.0.tar.gz", hash = "sha256:cf61ae8f126ac6f7c451172cf30e3e43d3ca77615509771b3a984a0730651e12"}, + {file = "pluggy-1.4.0-py3-none-any.whl", hash = "sha256:7db9f7b503d67d1c5b95f59773ebb58a8c1c288129a88665838012cfb07b8981"}, + {file = "pluggy-1.4.0.tar.gz", hash = "sha256:8c85c2876142a764e5b7548e7d9a0e0ddb46f5185161049a79b7e974454223be"}, ] [package.extras] @@ -4055,21 +4061,22 @@ testing = ["pytest", "pytest-benchmark"] [[package]] name = "plum-dispatch" -version = "2.2.2" +version = "2.3.2" description = "Multiple dispatch in Python" optional = false python-versions = ">=3.8" files = [ - {file = "plum_dispatch-2.2.2-py3-none-any.whl", hash = "sha256:d7ee415bd166ffa90eaa4b24d7c9dc7ca6f8875750586001e7c9baff706223bd"}, - {file = "plum_dispatch-2.2.2.tar.gz", hash = "sha256:d5d180225c9fbf0277375bb558b649d97d0b651a91037bb7155cedbe9f52764b"}, + {file = "plum_dispatch-2.3.2-py3-none-any.whl", hash = "sha256:96f519d416accf9a009117682f689114eb23e867bb6f977eed74ef85ef7fef9d"}, + {file = "plum_dispatch-2.3.2.tar.gz", hash = "sha256:f49f00dfdf7ab0f16c9b85cc27cc5241ffb59aee02218bac671ec7c1ac65e139"}, ] [package.dependencies] beartype = ">=0.16.2" +rich = ">=10.0" typing-extensions = {version = "*", markers = "python_version <= \"3.10\""} [package.extras] -dev = ["black (==23.9.0)", "build", "coveralls", "ghp-import", "ipython", "jupyter-book", "mypy", "numpy", "pre-commit", "pyright", "pytest (>=6)", "pytest-cov", "tox", "wheel"] +dev = ["black (==23.9.0)", "build", "coveralls", "ghp-import", "ipython", "jupyter-book", "mypy", "numpy", "pre-commit", "pyright (>=1.1.331)", "pytest (>=6)", "pytest-cov", "ruff (==0.1.0)", "tox", "wheel"] [[package]] name = "poetry" @@ -4155,17 +4162,17 @@ poetry-core = ">=1.7.0,<2.0.0" [[package]] name = "polars" -version = "0.20.4" +version = "0.20.6" description = "Blazingly fast DataFrame library" optional = true python-versions = ">=3.8" files = [ - {file = "polars-0.20.4-cp38-abi3-macosx_10_12_x86_64.whl", hash = "sha256:5fa84f74fc2274e3926d083ccd084c81b3e04debdc66fd917cafe7026d1df19c"}, - {file = "polars-0.20.4-cp38-abi3-macosx_11_0_arm64.whl", hash = "sha256:4c5a9f981708f3d090dd5513d83806bcb8a1725653d80bcf63bb738a097b1162"}, - {file = "polars-0.20.4-cp38-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bfdc2672423c9c73e34161b7b4833c40d042b9d36e899866dc858e8a221b0849"}, - {file = "polars-0.20.4-cp38-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:76f270fb17955c97958c2e301c5c2fa10015ccf3048697964ad9c2198e4c6fe6"}, - {file = "polars-0.20.4-cp38-abi3-win_amd64.whl", hash = "sha256:96a067be35745942d3fe6cd3ad1513f9ab7f4249d2b2502484ee64b30d221f96"}, - {file = "polars-0.20.4.tar.gz", hash = "sha256:21a90aa0c7401c80fc814b4db371dced780df6bd5ac81a329307e796b5821190"}, + {file = "polars-0.20.6-cp38-abi3-macosx_10_12_x86_64.whl", hash = "sha256:59845bae0b614b3291baa889cfc2a251e1024129696bb655596f2b5556e9f9a1"}, + {file = "polars-0.20.6-cp38-abi3-macosx_11_0_arm64.whl", hash = "sha256:9e86736f68440bf97a9100fa0a79ae7ce616d1af6fd4669fff1345f03aab14c0"}, + {file = "polars-0.20.6-cp38-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e4f4e3335fdcc863f6aac0616510b1baa5e13d5e818ebbfcb980ad534bd6edc2"}, + {file = "polars-0.20.6-cp38-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:7c7b494beea914a54bcae8868dee3988a88ecb48525df948e07aacf2fb83e711"}, + {file = "polars-0.20.6-cp38-abi3-win_amd64.whl", hash = "sha256:a96b157d68697c8d6ef2f7c2cc1734d498c3c6cc0c9c18d4fff7283ccfabdd1d"}, + {file = "polars-0.20.6.tar.gz", hash = "sha256:b53553308bc7e2b4f841b18f1949b61ed7f2cf155c5c64712298efa5af67a997"}, ] [package.extras] @@ -4272,27 +4279,27 @@ files = [ [[package]] name = "psutil" -version = "5.9.7" +version = "5.9.8" description = "Cross-platform lib for process and system monitoring in Python." optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*" files = [ - {file = "psutil-5.9.7-cp27-cp27m-macosx_10_9_x86_64.whl", hash = "sha256:0bd41bf2d1463dfa535942b2a8f0e958acf6607ac0be52265ab31f7923bcd5e6"}, - {file = "psutil-5.9.7-cp27-cp27m-manylinux2010_i686.whl", hash = "sha256:5794944462509e49d4d458f4dbfb92c47539e7d8d15c796f141f474010084056"}, - {file = "psutil-5.9.7-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:fe361f743cb3389b8efda21980d93eb55c1f1e3898269bc9a2a1d0bb7b1f6508"}, - {file = "psutil-5.9.7-cp27-cp27mu-manylinux2010_i686.whl", hash = "sha256:e469990e28f1ad738f65a42dcfc17adaed9d0f325d55047593cb9033a0ab63df"}, - {file = "psutil-5.9.7-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:3c4747a3e2ead1589e647e64aad601981f01b68f9398ddf94d01e3dc0d1e57c7"}, - {file = "psutil-5.9.7-cp27-none-win32.whl", hash = "sha256:1d4bc4a0148fdd7fd8f38e0498639ae128e64538faa507df25a20f8f7fb2341c"}, - {file = "psutil-5.9.7-cp27-none-win_amd64.whl", hash = "sha256:4c03362e280d06bbbfcd52f29acd79c733e0af33d707c54255d21029b8b32ba6"}, - {file = "psutil-5.9.7-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:ea36cc62e69a13ec52b2f625c27527f6e4479bca2b340b7a452af55b34fcbe2e"}, - {file = "psutil-5.9.7-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1132704b876e58d277168cd729d64750633d5ff0183acf5b3c986b8466cd0284"}, - {file = "psutil-5.9.7-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fe8b7f07948f1304497ce4f4684881250cd859b16d06a1dc4d7941eeb6233bfe"}, - {file = "psutil-5.9.7-cp36-cp36m-win32.whl", hash = "sha256:b27f8fdb190c8c03914f908a4555159327d7481dac2f01008d483137ef3311a9"}, - {file = "psutil-5.9.7-cp36-cp36m-win_amd64.whl", hash = "sha256:44969859757f4d8f2a9bd5b76eba8c3099a2c8cf3992ff62144061e39ba8568e"}, - {file = "psutil-5.9.7-cp37-abi3-win32.whl", hash = "sha256:c727ca5a9b2dd5193b8644b9f0c883d54f1248310023b5ad3e92036c5e2ada68"}, - {file = "psutil-5.9.7-cp37-abi3-win_amd64.whl", hash = "sha256:f37f87e4d73b79e6c5e749440c3113b81d1ee7d26f21c19c47371ddea834f414"}, - {file = "psutil-5.9.7-cp38-abi3-macosx_11_0_arm64.whl", hash = "sha256:032f4f2c909818c86cea4fe2cc407f1c0f0cde8e6c6d702b28b8ce0c0d143340"}, - {file = "psutil-5.9.7.tar.gz", hash = "sha256:3f02134e82cfb5d089fddf20bb2e03fd5cd52395321d1c8458a9e58500ff417c"}, + {file = "psutil-5.9.8-cp27-cp27m-macosx_10_9_x86_64.whl", hash = "sha256:26bd09967ae00920df88e0352a91cff1a78f8d69b3ecabbfe733610c0af486c8"}, + {file = "psutil-5.9.8-cp27-cp27m-manylinux2010_i686.whl", hash = "sha256:05806de88103b25903dff19bb6692bd2e714ccf9e668d050d144012055cbca73"}, + {file = "psutil-5.9.8-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:611052c4bc70432ec770d5d54f64206aa7203a101ec273a0cd82418c86503bb7"}, + {file = "psutil-5.9.8-cp27-cp27mu-manylinux2010_i686.whl", hash = "sha256:50187900d73c1381ba1454cf40308c2bf6f34268518b3f36a9b663ca87e65e36"}, + {file = "psutil-5.9.8-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:02615ed8c5ea222323408ceba16c60e99c3f91639b07da6373fb7e6539abc56d"}, + {file = "psutil-5.9.8-cp27-none-win32.whl", hash = "sha256:36f435891adb138ed3c9e58c6af3e2e6ca9ac2f365efe1f9cfef2794e6c93b4e"}, + {file = "psutil-5.9.8-cp27-none-win_amd64.whl", hash = "sha256:bd1184ceb3f87651a67b2708d4c3338e9b10c5df903f2e3776b62303b26cb631"}, + {file = "psutil-5.9.8-cp36-abi3-macosx_10_9_x86_64.whl", hash = "sha256:aee678c8720623dc456fa20659af736241f575d79429a0e5e9cf88ae0605cc81"}, + {file = "psutil-5.9.8-cp36-abi3-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8cb6403ce6d8e047495a701dc7c5bd788add903f8986d523e3e20b98b733e421"}, + {file = "psutil-5.9.8-cp36-abi3-manylinux_2_12_x86_64.manylinux2010_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d06016f7f8625a1825ba3732081d77c94589dca78b7a3fc072194851e88461a4"}, + {file = "psutil-5.9.8-cp36-cp36m-win32.whl", hash = "sha256:7d79560ad97af658a0f6adfef8b834b53f64746d45b403f225b85c5c2c140eee"}, + {file = "psutil-5.9.8-cp36-cp36m-win_amd64.whl", hash = "sha256:27cc40c3493bb10de1be4b3f07cae4c010ce715290a5be22b98493509c6299e2"}, + {file = "psutil-5.9.8-cp37-abi3-win32.whl", hash = "sha256:bc56c2a1b0d15aa3eaa5a60c9f3f8e3e565303b465dbf57a1b730e7a2b9844e0"}, + {file = "psutil-5.9.8-cp37-abi3-win_amd64.whl", hash = "sha256:8db4c1b57507eef143a15a6884ca10f7c73876cdf5d51e713151c1236a0e68cf"}, + {file = "psutil-5.9.8-cp38-abi3-macosx_11_0_arm64.whl", hash = "sha256:d16bbddf0693323b8c6123dd804100241da461e41d6e332fb0ba6058f630f8c8"}, + {file = "psutil-5.9.8.tar.gz", hash = "sha256:6be126e3225486dff286a8fb9a06246a5253f4c7c53b475ea5f5ac934e64194c"}, ] [package.extras] @@ -4309,8 +4316,6 @@ files = [ {file = "psycopg2-2.9.9-cp310-cp310-win_amd64.whl", hash = "sha256:426f9f29bde126913a20a96ff8ce7d73fd8a216cfb323b1f04da402d452853c3"}, {file = "psycopg2-2.9.9-cp311-cp311-win32.whl", hash = "sha256:ade01303ccf7ae12c356a5e10911c9e1c51136003a9a1d92f7aa9d010fb98372"}, {file = "psycopg2-2.9.9-cp311-cp311-win_amd64.whl", hash = "sha256:121081ea2e76729acfb0673ff33755e8703d45e926e416cb59bae3a86c6a4981"}, - {file = "psycopg2-2.9.9-cp312-cp312-win32.whl", hash = "sha256:d735786acc7dd25815e89cc4ad529a43af779db2e25aa7c626de864127e5a024"}, - {file = "psycopg2-2.9.9-cp312-cp312-win_amd64.whl", hash = "sha256:a7653d00b732afb6fc597e29c50ad28087dcb4fbfb28e86092277a559ae4e693"}, {file = "psycopg2-2.9.9-cp37-cp37m-win32.whl", hash = "sha256:5e0d98cade4f0e0304d7d6f25bbfbc5bd186e07b38eac65379309c4ca3193efa"}, {file = "psycopg2-2.9.9-cp37-cp37m-win_amd64.whl", hash = "sha256:7e2dacf8b009a1c1e843b5213a87f7c544b2b042476ed7755be813eaf4e8347a"}, {file = "psycopg2-2.9.9-cp38-cp38-win32.whl", hash = "sha256:ff432630e510709564c01dafdbe996cb552e0b9f3f065eb89bdce5bd31fabf4c"}, @@ -4430,51 +4435,51 @@ files = [ [[package]] name = "pyarrow" -version = "14.0.2" +version = "15.0.0" description = "Python library for Apache Arrow" optional = false python-versions = ">=3.8" files = [ - {file = "pyarrow-14.0.2-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:ba9fe808596c5dbd08b3aeffe901e5f81095baaa28e7d5118e01354c64f22807"}, - {file = "pyarrow-14.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:22a768987a16bb46220cef490c56c671993fbee8fd0475febac0b3e16b00a10e"}, - {file = "pyarrow-14.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2dbba05e98f247f17e64303eb876f4a80fcd32f73c7e9ad975a83834d81f3fda"}, - {file = "pyarrow-14.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a898d134d00b1eca04998e9d286e19653f9d0fcb99587310cd10270907452a6b"}, - {file = "pyarrow-14.0.2-cp310-cp310-manylinux_2_28_aarch64.whl", hash = "sha256:87e879323f256cb04267bb365add7208f302df942eb943c93a9dfeb8f44840b1"}, - {file = "pyarrow-14.0.2-cp310-cp310-manylinux_2_28_x86_64.whl", hash = "sha256:76fc257559404ea5f1306ea9a3ff0541bf996ff3f7b9209fc517b5e83811fa8e"}, - {file = "pyarrow-14.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:b0c4a18e00f3a32398a7f31da47fefcd7a927545b396e1f15d0c85c2f2c778cd"}, - {file = "pyarrow-14.0.2-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:87482af32e5a0c0cce2d12eb3c039dd1d853bd905b04f3f953f147c7a196915b"}, - {file = "pyarrow-14.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:059bd8f12a70519e46cd64e1ba40e97eae55e0cbe1695edd95384653d7626b23"}, - {file = "pyarrow-14.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3f16111f9ab27e60b391c5f6d197510e3ad6654e73857b4e394861fc79c37200"}, - {file = "pyarrow-14.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:06ff1264fe4448e8d02073f5ce45a9f934c0f3db0a04460d0b01ff28befc3696"}, - {file = "pyarrow-14.0.2-cp311-cp311-manylinux_2_28_aarch64.whl", hash = "sha256:6dd4f4b472ccf4042f1eab77e6c8bce574543f54d2135c7e396f413046397d5a"}, - {file = "pyarrow-14.0.2-cp311-cp311-manylinux_2_28_x86_64.whl", hash = "sha256:32356bfb58b36059773f49e4e214996888eeea3a08893e7dbde44753799b2a02"}, - {file = "pyarrow-14.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:52809ee69d4dbf2241c0e4366d949ba035cbcf48409bf404f071f624ed313a2b"}, - {file = "pyarrow-14.0.2-cp312-cp312-macosx_10_14_x86_64.whl", hash = "sha256:c87824a5ac52be210d32906c715f4ed7053d0180c1060ae3ff9b7e560f53f944"}, - {file = "pyarrow-14.0.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:a25eb2421a58e861f6ca91f43339d215476f4fe159eca603c55950c14f378cc5"}, - {file = "pyarrow-14.0.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c1da70d668af5620b8ba0a23f229030a4cd6c5f24a616a146f30d2386fec422"}, - {file = "pyarrow-14.0.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2cc61593c8e66194c7cdfae594503e91b926a228fba40b5cf25cc593563bcd07"}, - {file = "pyarrow-14.0.2-cp312-cp312-manylinux_2_28_aarch64.whl", hash = "sha256:78ea56f62fb7c0ae8ecb9afdd7893e3a7dbeb0b04106f5c08dbb23f9c0157591"}, - {file = "pyarrow-14.0.2-cp312-cp312-manylinux_2_28_x86_64.whl", hash = "sha256:37c233ddbce0c67a76c0985612fef27c0c92aef9413cf5aa56952f359fcb7379"}, - {file = "pyarrow-14.0.2-cp312-cp312-win_amd64.whl", hash = "sha256:e4b123ad0f6add92de898214d404e488167b87b5dd86e9a434126bc2b7a5578d"}, - {file = "pyarrow-14.0.2-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:e354fba8490de258be7687f341bc04aba181fc8aa1f71e4584f9890d9cb2dec2"}, - {file = "pyarrow-14.0.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:20e003a23a13da963f43e2b432483fdd8c38dc8882cd145f09f21792e1cf22a1"}, - {file = "pyarrow-14.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc0de7575e841f1595ac07e5bc631084fd06ca8b03c0f2ecece733d23cd5102a"}, - {file = "pyarrow-14.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:66e986dc859712acb0bd45601229021f3ffcdfc49044b64c6d071aaf4fa49e98"}, - {file = "pyarrow-14.0.2-cp38-cp38-manylinux_2_28_aarch64.whl", hash = "sha256:f7d029f20ef56673a9730766023459ece397a05001f4e4d13805111d7c2108c0"}, - {file = "pyarrow-14.0.2-cp38-cp38-manylinux_2_28_x86_64.whl", hash = "sha256:209bac546942b0d8edc8debda248364f7f668e4aad4741bae58e67d40e5fcf75"}, - {file = "pyarrow-14.0.2-cp38-cp38-win_amd64.whl", hash = "sha256:1e6987c5274fb87d66bb36816afb6f65707546b3c45c44c28e3c4133c010a881"}, - {file = "pyarrow-14.0.2-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:a01d0052d2a294a5f56cc1862933014e696aa08cc7b620e8c0cce5a5d362e976"}, - {file = "pyarrow-14.0.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:a51fee3a7db4d37f8cda3ea96f32530620d43b0489d169b285d774da48ca9785"}, - {file = "pyarrow-14.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:64df2bf1ef2ef14cee531e2dfe03dd924017650ffaa6f9513d7a1bb291e59c15"}, - {file = "pyarrow-14.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3c0fa3bfdb0305ffe09810f9d3e2e50a2787e3a07063001dcd7adae0cee3601a"}, - {file = "pyarrow-14.0.2-cp39-cp39-manylinux_2_28_aarch64.whl", hash = "sha256:c65bf4fd06584f058420238bc47a316e80dda01ec0dfb3044594128a6c2db794"}, - {file = "pyarrow-14.0.2-cp39-cp39-manylinux_2_28_x86_64.whl", hash = "sha256:63ac901baec9369d6aae1cbe6cca11178fb018a8d45068aaf5bb54f94804a866"}, - {file = "pyarrow-14.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:75ee0efe7a87a687ae303d63037d08a48ef9ea0127064df18267252cfe2e9541"}, - {file = "pyarrow-14.0.2.tar.gz", hash = "sha256:36cef6ba12b499d864d1def3e990f97949e0b79400d08b7cf74504ffbd3eb025"}, -] - -[package.dependencies] -numpy = ">=1.16.6" + {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" [[package]] name = "pyarrow-hotfix" @@ -4525,18 +4530,18 @@ files = [ [[package]] name = "pydantic" -version = "2.5.3" +version = "2.6.0" description = "Data validation using Python type hints" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "pydantic-2.5.3-py3-none-any.whl", hash = "sha256:d0caf5954bee831b6bfe7e338c32b9e30c85dfe080c843680783ac2b631673b4"}, - {file = "pydantic-2.5.3.tar.gz", hash = "sha256:b3ef57c62535b0941697cce638c08900d87fcb67e29cfa99e8a68f747f393f7a"}, + {file = "pydantic-2.6.0-py3-none-any.whl", hash = "sha256:1440966574e1b5b99cf75a13bec7b20e3512e8a61b894ae252f56275e2c465ae"}, + {file = "pydantic-2.6.0.tar.gz", hash = "sha256:ae887bd94eb404b09d86e4d12f93893bdca79d766e738528c6fa1c849f3c6bcf"}, ] [package.dependencies] annotated-types = ">=0.4.0" -pydantic-core = "2.14.6" +pydantic-core = "2.16.1" typing-extensions = ">=4.6.1" [package.extras] @@ -4544,116 +4549,90 @@ email = ["email-validator (>=2.0.0)"] [[package]] name = "pydantic-core" -version = "2.14.6" +version = "2.16.1" description = "" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "pydantic_core-2.14.6-cp310-cp310-macosx_10_7_x86_64.whl", hash = "sha256:72f9a942d739f09cd42fffe5dc759928217649f070056f03c70df14f5770acf9"}, - {file = "pydantic_core-2.14.6-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6a31d98c0d69776c2576dda4b77b8e0c69ad08e8b539c25c7d0ca0dc19a50d6c"}, - {file = "pydantic_core-2.14.6-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5aa90562bc079c6c290f0512b21768967f9968e4cfea84ea4ff5af5d917016e4"}, - {file = "pydantic_core-2.14.6-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:370ffecb5316ed23b667d99ce4debe53ea664b99cc37bfa2af47bc769056d534"}, - {file = "pydantic_core-2.14.6-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f85f3843bdb1fe80e8c206fe6eed7a1caeae897e496542cee499c374a85c6e08"}, - {file = "pydantic_core-2.14.6-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9862bf828112e19685b76ca499b379338fd4c5c269d897e218b2ae8fcb80139d"}, - {file = "pydantic_core-2.14.6-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:036137b5ad0cb0004c75b579445a1efccd072387a36c7f217bb8efd1afbe5245"}, - {file = "pydantic_core-2.14.6-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:92879bce89f91f4b2416eba4429c7b5ca22c45ef4a499c39f0c5c69257522c7c"}, - {file = "pydantic_core-2.14.6-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:0c08de15d50fa190d577e8591f0329a643eeaed696d7771760295998aca6bc66"}, - {file = "pydantic_core-2.14.6-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:36099c69f6b14fc2c49d7996cbf4f87ec4f0e66d1c74aa05228583225a07b590"}, - {file = "pydantic_core-2.14.6-cp310-none-win32.whl", hash = "sha256:7be719e4d2ae6c314f72844ba9d69e38dff342bc360379f7c8537c48e23034b7"}, - {file = "pydantic_core-2.14.6-cp310-none-win_amd64.whl", hash = "sha256:36fa402dcdc8ea7f1b0ddcf0df4254cc6b2e08f8cd80e7010d4c4ae6e86b2a87"}, - {file = "pydantic_core-2.14.6-cp311-cp311-macosx_10_7_x86_64.whl", hash = "sha256:dea7fcd62915fb150cdc373212141a30037e11b761fbced340e9db3379b892d4"}, - {file = "pydantic_core-2.14.6-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ffff855100bc066ff2cd3aa4a60bc9534661816b110f0243e59503ec2df38421"}, - {file = "pydantic_core-2.14.6-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1b027c86c66b8627eb90e57aee1f526df77dc6d8b354ec498be9a757d513b92b"}, - {file = "pydantic_core-2.14.6-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:00b1087dabcee0b0ffd104f9f53d7d3eaddfaa314cdd6726143af6bc713aa27e"}, - {file = "pydantic_core-2.14.6-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:75ec284328b60a4e91010c1acade0c30584f28a1f345bc8f72fe8b9e46ec6a96"}, - {file = "pydantic_core-2.14.6-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7e1f4744eea1501404b20b0ac059ff7e3f96a97d3e3f48ce27a139e053bb370b"}, - {file = "pydantic_core-2.14.6-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b2602177668f89b38b9f84b7b3435d0a72511ddef45dc14446811759b82235a1"}, - {file = "pydantic_core-2.14.6-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:6c8edaea3089bf908dd27da8f5d9e395c5b4dc092dbcce9b65e7156099b4b937"}, - {file = "pydantic_core-2.14.6-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:478e9e7b360dfec451daafe286998d4a1eeaecf6d69c427b834ae771cad4b622"}, - {file = "pydantic_core-2.14.6-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:b6ca36c12a5120bad343eef193cc0122928c5c7466121da7c20f41160ba00ba2"}, - {file = "pydantic_core-2.14.6-cp311-none-win32.whl", hash = "sha256:2b8719037e570639e6b665a4050add43134d80b687288ba3ade18b22bbb29dd2"}, - {file = "pydantic_core-2.14.6-cp311-none-win_amd64.whl", hash = "sha256:78ee52ecc088c61cce32b2d30a826f929e1708f7b9247dc3b921aec367dc1b23"}, - {file = "pydantic_core-2.14.6-cp311-none-win_arm64.whl", hash = "sha256:a19b794f8fe6569472ff77602437ec4430f9b2b9ec7a1105cfd2232f9ba355e6"}, - {file = "pydantic_core-2.14.6-cp312-cp312-macosx_10_7_x86_64.whl", hash = "sha256:667aa2eac9cd0700af1ddb38b7b1ef246d8cf94c85637cbb03d7757ca4c3fdec"}, - {file = "pydantic_core-2.14.6-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:cdee837710ef6b56ebd20245b83799fce40b265b3b406e51e8ccc5b85b9099b7"}, - {file = "pydantic_core-2.14.6-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2c5bcf3414367e29f83fd66f7de64509a8fd2368b1edf4351e862910727d3e51"}, - {file = "pydantic_core-2.14.6-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:26a92ae76f75d1915806b77cf459811e772d8f71fd1e4339c99750f0e7f6324f"}, - {file = "pydantic_core-2.14.6-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a983cca5ed1dd9a35e9e42ebf9f278d344603bfcb174ff99a5815f953925140a"}, - {file = "pydantic_core-2.14.6-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:cb92f9061657287eded380d7dc455bbf115430b3aa4741bdc662d02977e7d0af"}, - {file = "pydantic_core-2.14.6-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e4ace1e220b078c8e48e82c081e35002038657e4b37d403ce940fa679e57113b"}, - {file = "pydantic_core-2.14.6-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:ef633add81832f4b56d3b4c9408b43d530dfca29e68fb1b797dcb861a2c734cd"}, - {file = "pydantic_core-2.14.6-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:7e90d6cc4aad2cc1f5e16ed56e46cebf4877c62403a311af20459c15da76fd91"}, - {file = "pydantic_core-2.14.6-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:e8a5ac97ea521d7bde7621d86c30e86b798cdecd985723c4ed737a2aa9e77d0c"}, - {file = "pydantic_core-2.14.6-cp312-none-win32.whl", hash = "sha256:f27207e8ca3e5e021e2402ba942e5b4c629718e665c81b8b306f3c8b1ddbb786"}, - {file = "pydantic_core-2.14.6-cp312-none-win_amd64.whl", hash = "sha256:b3e5fe4538001bb82e2295b8d2a39356a84694c97cb73a566dc36328b9f83b40"}, - {file = "pydantic_core-2.14.6-cp312-none-win_arm64.whl", hash = "sha256:64634ccf9d671c6be242a664a33c4acf12882670b09b3f163cd00a24cffbd74e"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-macosx_10_7_x86_64.whl", hash = "sha256:24368e31be2c88bd69340fbfe741b405302993242ccb476c5c3ff48aeee1afe0"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-macosx_11_0_arm64.whl", hash = "sha256:e33b0834f1cf779aa839975f9d8755a7c2420510c0fa1e9fa0497de77cd35d2c"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6af4b3f52cc65f8a0bc8b1cd9676f8c21ef3e9132f21fed250f6958bd7223bed"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:d15687d7d7f40333bd8266f3814c591c2e2cd263fa2116e314f60d82086e353a"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:095b707bb287bfd534044166ab767bec70a9bba3175dcdc3371782175c14e43c"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:94fc0e6621e07d1e91c44e016cc0b189b48db053061cc22d6298a611de8071bb"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1ce830e480f6774608dedfd4a90c42aac4a7af0a711f1b52f807130c2e434c06"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:a306cdd2ad3a7d795d8e617a58c3a2ed0f76c8496fb7621b6cd514eb1532cae8"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:2f5fa187bde8524b1e37ba894db13aadd64faa884657473b03a019f625cee9a8"}, - {file = "pydantic_core-2.14.6-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:438027a975cc213a47c5d70672e0d29776082155cfae540c4e225716586be75e"}, - {file = "pydantic_core-2.14.6-cp37-none-win32.whl", hash = "sha256:f96ae96a060a8072ceff4cfde89d261837b4294a4f28b84a28765470d502ccc6"}, - {file = "pydantic_core-2.14.6-cp37-none-win_amd64.whl", hash = "sha256:e646c0e282e960345314f42f2cea5e0b5f56938c093541ea6dbf11aec2862391"}, - {file = "pydantic_core-2.14.6-cp38-cp38-macosx_10_7_x86_64.whl", hash = "sha256:db453f2da3f59a348f514cfbfeb042393b68720787bbef2b4c6068ea362c8149"}, - {file = "pydantic_core-2.14.6-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:3860c62057acd95cc84044e758e47b18dcd8871a328ebc8ccdefd18b0d26a21b"}, - {file = "pydantic_core-2.14.6-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:36026d8f99c58d7044413e1b819a67ca0e0b8ebe0f25e775e6c3d1fabb3c38fb"}, - {file = "pydantic_core-2.14.6-cp38-cp38-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8ed1af8692bd8d2a29d702f1a2e6065416d76897d726e45a1775b1444f5928a7"}, - {file = "pydantic_core-2.14.6-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:314ccc4264ce7d854941231cf71b592e30d8d368a71e50197c905874feacc8a8"}, - {file = "pydantic_core-2.14.6-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:982487f8931067a32e72d40ab6b47b1628a9c5d344be7f1a4e668fb462d2da42"}, - {file = "pydantic_core-2.14.6-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2dbe357bc4ddda078f79d2a36fc1dd0494a7f2fad83a0a684465b6f24b46fe80"}, - {file = "pydantic_core-2.14.6-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:2f6ffc6701a0eb28648c845f4945a194dc7ab3c651f535b81793251e1185ac3d"}, - {file = "pydantic_core-2.14.6-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:7f5025db12fc6de7bc1104d826d5aee1d172f9ba6ca936bf6474c2148ac336c1"}, - {file = "pydantic_core-2.14.6-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:dab03ed811ed1c71d700ed08bde8431cf429bbe59e423394f0f4055f1ca0ea60"}, - {file = "pydantic_core-2.14.6-cp38-none-win32.whl", hash = "sha256:dfcbebdb3c4b6f739a91769aea5ed615023f3c88cb70df812849aef634c25fbe"}, - {file = "pydantic_core-2.14.6-cp38-none-win_amd64.whl", hash = "sha256:99b14dbea2fdb563d8b5a57c9badfcd72083f6006caf8e126b491519c7d64ca8"}, - {file = "pydantic_core-2.14.6-cp39-cp39-macosx_10_7_x86_64.whl", hash = "sha256:4ce8299b481bcb68e5c82002b96e411796b844d72b3e92a3fbedfe8e19813eab"}, - {file = "pydantic_core-2.14.6-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:b9a9d92f10772d2a181b5ca339dee066ab7d1c9a34ae2421b2a52556e719756f"}, - {file = "pydantic_core-2.14.6-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fd9e98b408384989ea4ab60206b8e100d8687da18b5c813c11e92fd8212a98e0"}, - {file = "pydantic_core-2.14.6-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:4f86f1f318e56f5cbb282fe61eb84767aee743ebe32c7c0834690ebea50c0a6b"}, - {file = "pydantic_core-2.14.6-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:86ce5fcfc3accf3a07a729779d0b86c5d0309a4764c897d86c11089be61da160"}, - {file = "pydantic_core-2.14.6-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3dcf1978be02153c6a31692d4fbcc2a3f1db9da36039ead23173bc256ee3b91b"}, - {file = "pydantic_core-2.14.6-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:eedf97be7bc3dbc8addcef4142f4b4164066df0c6f36397ae4aaed3eb187d8ab"}, - {file = "pydantic_core-2.14.6-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:d5f916acf8afbcab6bacbb376ba7dc61f845367901ecd5e328fc4d4aef2fcab0"}, - {file = "pydantic_core-2.14.6-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:8a14c192c1d724c3acbfb3f10a958c55a2638391319ce8078cb36c02283959b9"}, - {file = "pydantic_core-2.14.6-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:0348b1dc6b76041516e8a854ff95b21c55f5a411c3297d2ca52f5528e49d8411"}, - {file = "pydantic_core-2.14.6-cp39-none-win32.whl", hash = "sha256:de2a0645a923ba57c5527497daf8ec5df69c6eadf869e9cd46e86349146e5975"}, - {file = "pydantic_core-2.14.6-cp39-none-win_amd64.whl", hash = "sha256:aca48506a9c20f68ee61c87f2008f81f8ee99f8d7f0104bff3c47e2d148f89d9"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-macosx_10_7_x86_64.whl", hash = "sha256:d5c28525c19f5bb1e09511669bb57353d22b94cf8b65f3a8d141c389a55dec95"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:78d0768ee59baa3de0f4adac9e3748b4b1fffc52143caebddfd5ea2961595277"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8b93785eadaef932e4fe9c6e12ba67beb1b3f1e5495631419c784ab87e975670"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a874f21f87c485310944b2b2734cd6d318765bcbb7515eead33af9641816506e"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:b89f4477d915ea43b4ceea6756f63f0288941b6443a2b28c69004fe07fde0d0d"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:172de779e2a153d36ee690dbc49c6db568d7b33b18dc56b69a7514aecbcf380d"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:dfcebb950aa7e667ec226a442722134539e77c575f6cfaa423f24371bb8d2e94"}, - {file = "pydantic_core-2.14.6-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:55a23dcd98c858c0db44fc5c04fc7ed81c4b4d33c653a7c45ddaebf6563a2f66"}, - {file = "pydantic_core-2.14.6-pp37-pypy37_pp73-macosx_10_7_x86_64.whl", hash = "sha256:4241204e4b36ab5ae466ecec5c4c16527a054c69f99bba20f6f75232a6a534e2"}, - {file = "pydantic_core-2.14.6-pp37-pypy37_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e574de99d735b3fc8364cba9912c2bec2da78775eba95cbb225ef7dda6acea24"}, - {file = "pydantic_core-2.14.6-pp37-pypy37_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1302a54f87b5cd8528e4d6d1bf2133b6aa7c6122ff8e9dc5220fbc1e07bffebd"}, - {file = "pydantic_core-2.14.6-pp37-pypy37_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:f8e81e4b55930e5ffab4a68db1af431629cf2e4066dbdbfef65348b8ab804ea8"}, - {file = "pydantic_core-2.14.6-pp37-pypy37_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:c99462ffc538717b3e60151dfaf91125f637e801f5ab008f81c402f1dff0cd0f"}, - {file = "pydantic_core-2.14.6-pp37-pypy37_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:e4cf2d5829f6963a5483ec01578ee76d329eb5caf330ecd05b3edd697e7d768a"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-macosx_10_7_x86_64.whl", hash = "sha256:cf10b7d58ae4a1f07fccbf4a0a956d705356fea05fb4c70608bb6fa81d103cda"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-macosx_11_0_arm64.whl", hash = "sha256:399ac0891c284fa8eb998bcfa323f2234858f5d2efca3950ae58c8f88830f145"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9c6a5c79b28003543db3ba67d1df336f253a87d3112dac3a51b94f7d48e4c0e1"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:599c87d79cab2a6a2a9df4aefe0455e61e7d2aeede2f8577c1b7c0aec643ee8e"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:43e166ad47ba900f2542a80d83f9fc65fe99eb63ceec4debec160ae729824052"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:3a0b5db001b98e1c649dd55afa928e75aa4087e587b9524a4992316fa23c9fba"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:747265448cb57a9f37572a488a57d873fd96bf51e5bb7edb52cfb37124516da4"}, - {file = "pydantic_core-2.14.6-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:7ebe3416785f65c28f4f9441e916bfc8a54179c8dea73c23023f7086fa601c5d"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-macosx_10_7_x86_64.whl", hash = "sha256:86c963186ca5e50d5c8287b1d1c9d3f8f024cbe343d048c5bd282aec2d8641f2"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:e0641b506486f0b4cd1500a2a65740243e8670a2549bb02bc4556a83af84ae03"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:71d72ca5eaaa8d38c8df16b7deb1a2da4f650c41b58bb142f3fb75d5ad4a611f"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:27e524624eace5c59af499cd97dc18bb201dc6a7a2da24bfc66ef151c69a5f2a"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:a3dde6cac75e0b0902778978d3b1646ca9f438654395a362cb21d9ad34b24acf"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:00646784f6cd993b1e1c0e7b0fdcbccc375d539db95555477771c27555e3c556"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:23598acb8ccaa3d1d875ef3b35cb6376535095e9405d91a3d57a8c7db5d29341"}, - {file = "pydantic_core-2.14.6-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:7f41533d7e3cf9520065f610b41ac1c76bc2161415955fbcead4981b22c7611e"}, - {file = "pydantic_core-2.14.6.tar.gz", hash = "sha256:1fd0c1d395372843fba13a51c28e3bb9d59bd7aebfeb17358ffaaa1e4dbbe948"}, + {file = "pydantic_core-2.16.1-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:300616102fb71241ff477a2cbbc847321dbec49428434a2f17f37528721c4948"}, + {file = "pydantic_core-2.16.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:5511f962dd1b9b553e9534c3b9c6a4b0c9ded3d8c2be96e61d56f933feef9e1f"}, + {file = "pydantic_core-2.16.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:98f0edee7ee9cc7f9221af2e1b95bd02810e1c7a6d115cfd82698803d385b28f"}, + {file = "pydantic_core-2.16.1-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9795f56aa6b2296f05ac79d8a424e94056730c0b860a62b0fdcfe6340b658cc8"}, + {file = "pydantic_core-2.16.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c45f62e4107ebd05166717ac58f6feb44471ed450d07fecd90e5f69d9bf03c48"}, + {file = "pydantic_core-2.16.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:462d599299c5971f03c676e2b63aa80fec5ebc572d89ce766cd11ca8bcb56f3f"}, + {file = "pydantic_core-2.16.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21ebaa4bf6386a3b22eec518da7d679c8363fb7fb70cf6972161e5542f470798"}, + {file = "pydantic_core-2.16.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:99f9a50b56713a598d33bc23a9912224fc5d7f9f292444e6664236ae471ddf17"}, + {file = "pydantic_core-2.16.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:8ec364e280db4235389b5e1e6ee924723c693cbc98e9d28dc1767041ff9bc388"}, + {file = "pydantic_core-2.16.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:653a5dfd00f601a0ed6654a8b877b18d65ac32c9d9997456e0ab240807be6cf7"}, + {file = "pydantic_core-2.16.1-cp310-none-win32.whl", hash = "sha256:1661c668c1bb67b7cec96914329d9ab66755911d093bb9063c4c8914188af6d4"}, + {file = "pydantic_core-2.16.1-cp310-none-win_amd64.whl", hash = "sha256:561be4e3e952c2f9056fba5267b99be4ec2afadc27261505d4992c50b33c513c"}, + {file = "pydantic_core-2.16.1-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:102569d371fadc40d8f8598a59379c37ec60164315884467052830b28cc4e9da"}, + {file = "pydantic_core-2.16.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:735dceec50fa907a3c314b84ed609dec54b76a814aa14eb90da31d1d36873a5e"}, + {file = "pydantic_core-2.16.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e83ebbf020be727d6e0991c1b192a5c2e7113eb66e3def0cd0c62f9f266247e4"}, + {file = "pydantic_core-2.16.1-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:30a8259569fbeec49cfac7fda3ec8123486ef1b729225222f0d41d5f840b476f"}, + {file = "pydantic_core-2.16.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:920c4897e55e2881db6a6da151198e5001552c3777cd42b8a4c2f72eedc2ee91"}, + {file = "pydantic_core-2.16.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f5247a3d74355f8b1d780d0f3b32a23dd9f6d3ff43ef2037c6dcd249f35ecf4c"}, + {file = "pydantic_core-2.16.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2d5bea8012df5bb6dda1e67d0563ac50b7f64a5d5858348b5c8cb5043811c19d"}, + {file = "pydantic_core-2.16.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:ed3025a8a7e5a59817b7494686d449ebfbe301f3e757b852c8d0d1961d6be864"}, + {file = "pydantic_core-2.16.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:06f0d5a1d9e1b7932477c172cc720b3b23c18762ed7a8efa8398298a59d177c7"}, + {file = "pydantic_core-2.16.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:150ba5c86f502c040b822777e2e519b5625b47813bd05f9273a8ed169c97d9ae"}, + {file = "pydantic_core-2.16.1-cp311-none-win32.whl", hash = "sha256:d6cbdf12ef967a6aa401cf5cdf47850559e59eedad10e781471c960583f25aa1"}, + {file = "pydantic_core-2.16.1-cp311-none-win_amd64.whl", hash = "sha256:afa01d25769af33a8dac0d905d5c7bb2d73c7c3d5161b2dd6f8b5b5eea6a3c4c"}, + {file = "pydantic_core-2.16.1-cp311-none-win_arm64.whl", hash = "sha256:1a2fe7b00a49b51047334d84aafd7e39f80b7675cad0083678c58983662da89b"}, + {file = "pydantic_core-2.16.1-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:0f478ec204772a5c8218e30eb813ca43e34005dff2eafa03931b3d8caef87d51"}, + {file = "pydantic_core-2.16.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:f1936ef138bed2165dd8573aa65e3095ef7c2b6247faccd0e15186aabdda7f66"}, + {file = "pydantic_core-2.16.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:99d3a433ef5dc3021c9534a58a3686c88363c591974c16c54a01af7efd741f13"}, + {file = "pydantic_core-2.16.1-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:bd88f40f2294440d3f3c6308e50d96a0d3d0973d6f1a5732875d10f569acef49"}, + {file = "pydantic_core-2.16.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3fac641bbfa43d5a1bed99d28aa1fded1984d31c670a95aac1bf1d36ac6ce137"}, + {file = "pydantic_core-2.16.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:72bf9308a82b75039b8c8edd2be2924c352eda5da14a920551a8b65d5ee89253"}, + {file = "pydantic_core-2.16.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fb4363e6c9fc87365c2bc777a1f585a22f2f56642501885ffc7942138499bf54"}, + {file = "pydantic_core-2.16.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:20f724a023042588d0f4396bbbcf4cffd0ddd0ad3ed4f0d8e6d4ac4264bae81e"}, + {file = "pydantic_core-2.16.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:fb4370b15111905bf8b5ba2129b926af9470f014cb0493a67d23e9d7a48348e8"}, + {file = "pydantic_core-2.16.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:23632132f1fd608034f1a56cc3e484be00854db845b3a4a508834be5a6435a6f"}, + {file = "pydantic_core-2.16.1-cp312-none-win32.whl", hash = "sha256:b9f3e0bffad6e238f7acc20c393c1ed8fab4371e3b3bc311020dfa6020d99212"}, + {file = "pydantic_core-2.16.1-cp312-none-win_amd64.whl", hash = "sha256:a0b4cfe408cd84c53bab7d83e4209458de676a6ec5e9c623ae914ce1cb79b96f"}, + {file = "pydantic_core-2.16.1-cp312-none-win_arm64.whl", hash = "sha256:d195add190abccefc70ad0f9a0141ad7da53e16183048380e688b466702195dd"}, + {file = "pydantic_core-2.16.1-cp38-cp38-macosx_10_12_x86_64.whl", hash = "sha256:502c062a18d84452858f8aea1e520e12a4d5228fc3621ea5061409d666ea1706"}, + {file = "pydantic_core-2.16.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:d8c032ccee90b37b44e05948b449a2d6baed7e614df3d3f47fe432c952c21b60"}, + {file = "pydantic_core-2.16.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:920f4633bee43d7a2818e1a1a788906df5a17b7ab6fe411220ed92b42940f818"}, + {file = "pydantic_core-2.16.1-cp38-cp38-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9f5d37ff01edcbace53a402e80793640c25798fb7208f105d87a25e6fcc9ea06"}, + {file = "pydantic_core-2.16.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:399166f24c33a0c5759ecc4801f040dbc87d412c1a6d6292b2349b4c505effc9"}, + {file = "pydantic_core-2.16.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ac89ccc39cd1d556cc72d6752f252dc869dde41c7c936e86beac5eb555041b66"}, + {file = "pydantic_core-2.16.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:73802194f10c394c2bedce7a135ba1d8ba6cff23adf4217612bfc5cf060de34c"}, + {file = "pydantic_core-2.16.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:8fa00fa24ffd8c31fac081bf7be7eb495be6d248db127f8776575a746fa55c95"}, + {file = "pydantic_core-2.16.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:601d3e42452cd4f2891c13fa8c70366d71851c1593ed42f57bf37f40f7dca3c8"}, + {file = "pydantic_core-2.16.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:07982b82d121ed3fc1c51faf6e8f57ff09b1325d2efccaa257dd8c0dd937acca"}, + {file = "pydantic_core-2.16.1-cp38-none-win32.whl", hash = "sha256:d0bf6f93a55d3fa7a079d811b29100b019784e2ee6bc06b0bb839538272a5610"}, + {file = "pydantic_core-2.16.1-cp38-none-win_amd64.whl", hash = "sha256:fbec2af0ebafa57eb82c18c304b37c86a8abddf7022955d1742b3d5471a6339e"}, + {file = "pydantic_core-2.16.1-cp39-cp39-macosx_10_12_x86_64.whl", hash = "sha256:a497be217818c318d93f07e14502ef93d44e6a20c72b04c530611e45e54c2196"}, + {file = "pydantic_core-2.16.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:694a5e9f1f2c124a17ff2d0be613fd53ba0c26de588eb4bdab8bca855e550d95"}, + {file = "pydantic_core-2.16.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8d4dfc66abea3ec6d9f83e837a8f8a7d9d3a76d25c9911735c76d6745950e62c"}, + {file = "pydantic_core-2.16.1-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8655f55fe68c4685673265a650ef71beb2d31871c049c8b80262026f23605ee3"}, + {file = "pydantic_core-2.16.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:21e3298486c4ea4e4d5cc6fb69e06fb02a4e22089304308817035ac006a7f506"}, + {file = "pydantic_core-2.16.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:71b4a48a7427f14679f0015b13c712863d28bb1ab700bd11776a5368135c7d60"}, + {file = "pydantic_core-2.16.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:10dca874e35bb60ce4f9f6665bfbfad050dd7573596608aeb9e098621ac331dc"}, + {file = "pydantic_core-2.16.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:fa496cd45cda0165d597e9d6f01e36c33c9508f75cf03c0a650018c5048f578e"}, + {file = "pydantic_core-2.16.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:5317c04349472e683803da262c781c42c5628a9be73f4750ac7d13040efb5d2d"}, + {file = "pydantic_core-2.16.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:42c29d54ed4501a30cd71015bf982fa95e4a60117b44e1a200290ce687d3e640"}, + {file = "pydantic_core-2.16.1-cp39-none-win32.whl", hash = "sha256:ba07646f35e4e49376c9831130039d1b478fbfa1215ae62ad62d2ee63cf9c18f"}, + {file = "pydantic_core-2.16.1-cp39-none-win_amd64.whl", hash = "sha256:2133b0e412a47868a358713287ff9f9a328879da547dc88be67481cdac529118"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:d25ef0c33f22649b7a088035fd65ac1ce6464fa2876578df1adad9472f918a76"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:99c095457eea8550c9fa9a7a992e842aeae1429dab6b6b378710f62bfb70b394"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b49c604ace7a7aa8af31196abbf8f2193be605db6739ed905ecaf62af31ccae0"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c56da23034fe66221f2208c813d8aa509eea34d97328ce2add56e219c3a9f41c"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:cebf8d56fee3b08ad40d332a807ecccd4153d3f1ba8231e111d9759f02edfd05"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:1ae8048cba95f382dba56766525abca438328455e35c283bb202964f41a780b0"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:780daad9e35b18d10d7219d24bfb30148ca2afc309928e1d4d53de86822593dc"}, + {file = "pydantic_core-2.16.1-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:c94b5537bf6ce66e4d7830c6993152940a188600f6ae044435287753044a8fe2"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:adf28099d061a25fbcc6531febb7a091e027605385de9fe14dd6a97319d614cf"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:644904600c15816a1f9a1bafa6aab0d21db2788abcdf4e2a77951280473f33e1"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:87bce04f09f0552b66fca0c4e10da78d17cb0e71c205864bab4e9595122cb9d9"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:877045a7969ace04d59516d5d6a7dee13106822f99a5d8df5e6822941f7bedc8"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:9c46e556ee266ed3fb7b7a882b53df3c76b45e872fdab8d9cf49ae5e91147fd7"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:4eebbd049008eb800f519578e944b8dc8e0f7d59a5abb5924cc2d4ed3a1834ff"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:c0be58529d43d38ae849a91932391eb93275a06b93b79a8ab828b012e916a206"}, + {file = "pydantic_core-2.16.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:b1fc07896fc1851558f532dffc8987e526b682ec73140886c831d773cef44b76"}, + {file = "pydantic_core-2.16.1.tar.gz", hash = "sha256:daff04257b49ab7f4b3f73f98283d3dbb1a65bf3500d55c7beac3c66c310fe34"}, ] [package.dependencies] @@ -4677,13 +4656,13 @@ setuptools = "*" [[package]] name = "pydeps" -version = "1.12.17" +version = "1.12.18" description = "Display module dependencies" optional = false python-versions = "*" files = [ - {file = "pydeps-1.12.17-py3-none-any.whl", hash = "sha256:4fb2e86071c78c1b85a1c63745a267d100e91daf6bab2f14331b3c77433b58b4"}, - {file = "pydeps-1.12.17.tar.gz", hash = "sha256:c308e8355a1e77ff0af899d6f9f1665d4eb07019692dba9fb1dc1cab05df36a4"}, + {file = "pydeps-1.12.18-py3-none-any.whl", hash = "sha256:fc57f56a6eaf92ea6b9b503dc43d55f098661e253a868bbb52fccfbbcc8e79de"}, + {file = "pydeps-1.12.18.tar.gz", hash = "sha256:15c5d023b5053308e19a69591da06d9f3ff038e7a47111c40c9986b6a2929a4b"}, ] [package.dependencies] @@ -4750,71 +4729,71 @@ windows-terminal = ["colorama (>=0.4.6)"] [[package]] name = "pyinstrument" -version = "4.6.1" +version = "4.6.2" description = "Call stack profiler for Python. Shows you why your code is slow!" optional = false python-versions = ">=3.7" files = [ - {file = "pyinstrument-4.6.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:73476e4bc6e467ac1b2c3c0dd1f0b71c9061d4de14626676adfdfbb14aa342b4"}, - {file = "pyinstrument-4.6.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:4d1da8efd974cf9df52ee03edaee2d3875105ddd00de35aa542760f7c612bdf7"}, - {file = "pyinstrument-4.6.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:507be1ee2f2b0c9fba74d622a272640dd6d1b0c9ec3388b2cdeb97ad1e77125f"}, - {file = "pyinstrument-4.6.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:95cee6de08eb45754ef4f602ce52b640d1c535d934a6a8733a974daa095def37"}, - {file = "pyinstrument-4.6.1-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c7873e8cec92321251fdf894a72b3c78f4c5c20afdd1fef0baf9042ec843bb04"}, - {file = "pyinstrument-4.6.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:a242f6cac40bc83e1f3002b6b53681846dfba007f366971db0bf21e02dbb1903"}, - {file = "pyinstrument-4.6.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:97c9660cdb4bd2a43cf4f3ab52cffd22f3ac9a748d913b750178fb34e5e39e64"}, - {file = "pyinstrument-4.6.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:e304cd0723e2b18ada5e63c187abf6d777949454c734f5974d64a0865859f0f4"}, - {file = "pyinstrument-4.6.1-cp310-cp310-win32.whl", hash = "sha256:cee21a2d78187dd8a80f72f5d0f1ddb767b2d9800f8bb4d94b6d11f217c22cdb"}, - {file = "pyinstrument-4.6.1-cp310-cp310-win_amd64.whl", hash = "sha256:2000712f71d693fed2f8a1c1638d37b7919124f367b37976d07128d49f1445eb"}, - {file = "pyinstrument-4.6.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:a366c6f3dfb11f1739bdc1dee75a01c1563ad0bf4047071e5e77598087df457f"}, - {file = "pyinstrument-4.6.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c6be327be65d934796558aa9cb0f75ce62ebd207d49ad1854610c97b0579ad47"}, - {file = "pyinstrument-4.6.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9e160d9c5d20d3e4ef82269e4e8b246ff09bdf37af5fb8cb8ccca97936d95ad6"}, - {file = "pyinstrument-4.6.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6ffbf56605ef21c2fcb60de2fa74ff81f417d8be0c5002a407e414d6ef6dee43"}, - {file = "pyinstrument-4.6.1-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c92cc4924596d6e8f30a16182bbe90893b1572d847ae12652f72b34a9a17c24a"}, - {file = "pyinstrument-4.6.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:f4b48a94d938cae981f6948d9ec603bab2087b178d2095d042d5a48aabaecaab"}, - {file = "pyinstrument-4.6.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:e7a386392275bdef4a1849712dc5b74f0023483fca14ef93d0ca27d453548982"}, - {file = "pyinstrument-4.6.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:871b131b83e9b1122f2325061c68ed1e861eebcb568c934d2fb193652f077f77"}, - {file = "pyinstrument-4.6.1-cp311-cp311-win32.whl", hash = "sha256:8d8515156dd91f5652d13b5fcc87e634f8fe1c07b68d1d0840348cdd50bf5ace"}, - {file = "pyinstrument-4.6.1-cp311-cp311-win_amd64.whl", hash = "sha256:fb868fbe089036e9f32525a249f4c78b8dc46967612393f204b8234f439c9cc4"}, - {file = "pyinstrument-4.6.1-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:a18cd234cce4f230f1733807f17a134e64a1f1acabf74a14d27f583cf2b183df"}, - {file = "pyinstrument-4.6.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:574cfca69150be4ce4461fb224712fbc0722a49b0dc02fa204d02807adf6b5a0"}, - {file = "pyinstrument-4.6.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2e02cf505e932eb8ccf561b7527550a67ec14fcae1fe0e25319b09c9c166e914"}, - {file = "pyinstrument-4.6.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:832fb2acef9d53701c1ab546564c45fb70a8770c816374f8dd11420d399103c9"}, - {file = "pyinstrument-4.6.1-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:13cb57e9607545623ebe462345b3d0c4caee0125d2d02267043ece8aca8f4ea0"}, - {file = "pyinstrument-4.6.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:9be89e7419bcfe8dd6abb0d959d6d9c439c613a4a873514c43d16b48dae697c9"}, - {file = "pyinstrument-4.6.1-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:476785cfbc44e8e1b1ad447398aa3deae81a8df4d37eb2d8bbb0c404eff979cd"}, - {file = "pyinstrument-4.6.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:e9cebd90128a3d2fee36d3ccb665c1b9dce75261061b2046203e45c4a8012d54"}, - {file = "pyinstrument-4.6.1-cp312-cp312-win32.whl", hash = "sha256:1d0b76683df2ad5c40eff73607dc5c13828c92fbca36aff1ddf869a3c5a55fa6"}, - {file = "pyinstrument-4.6.1-cp312-cp312-win_amd64.whl", hash = "sha256:c4b7af1d9d6a523cfbfedebcb69202242d5bd0cb89c4e094cc73d5d6e38279bd"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:79ae152f8c6a680a188fb3be5e0f360ac05db5bbf410169a6c40851dfaebcce9"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:07cad2745964c174c65aa75f1bf68a4394d1b4d28f33894837cfd315d1e836f0"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cb81f66f7f94045d723069cf317453d42375de9ff3c69089cf6466b078ac1db4"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0ab30ae75969da99e9a529e21ff497c18fdf958e822753db4ae7ed1e67094040"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:f36cb5b644762fb3c86289324bbef17e95f91cd710603ac19444a47f638e8e96"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:8b45075d9dbbc977dbc7007fb22bb0054c6990fbe91bf48dd80c0b96c6307ba7"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:475ac31477f6302e092463896d6a2055f3e6abcd293bad16ff94fc9185308a88"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-win32.whl", hash = "sha256:29172ab3d8609fdf821c3f2562dc61e14f1a8ff5306607c32ca743582d3a760e"}, - {file = "pyinstrument-4.6.1-cp37-cp37m-win_amd64.whl", hash = "sha256:bd176f297c99035127b264369d2bb97a65255f65f8d4e843836baf55ebb3cee4"}, - {file = "pyinstrument-4.6.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:23e9b4526978432e9999021da9a545992cf2ac3df5ee82db7beb6908fc4c978c"}, - {file = "pyinstrument-4.6.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:2dbcaccc9f456ef95557ec501caeb292119c24446d768cb4fb43578b0f3d572c"}, - {file = "pyinstrument-4.6.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2097f63c66c2bc9678c826b9ff0c25acde3ed455590d9dcac21220673fe74fbf"}, - {file = "pyinstrument-4.6.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:205ac2e76bd65d61b9611a9ce03d5f6393e34ec5b41dd38808f25d54e6b3e067"}, - {file = "pyinstrument-4.6.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3f414ddf1161976a40fc0a333000e6a4ad612719eac0b8c9bb73f47153187148"}, - {file = "pyinstrument-4.6.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:65e62ebfa2cd8fb57eda90006f4505ac4c70da00fc2f05b6d8337d776ea76d41"}, - {file = "pyinstrument-4.6.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:d96309df4df10be7b4885797c5f69bb3a89414680ebaec0722d8156fde5268c3"}, - {file = "pyinstrument-4.6.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:f3d1ad3bc8ebb4db925afa706aa865c4bfb40d52509f143491ac0df2440ee5d2"}, - {file = "pyinstrument-4.6.1-cp38-cp38-win32.whl", hash = "sha256:dc37cb988c8854eb42bda2e438aaf553536566657d157c4473cc8aad5692a779"}, - {file = "pyinstrument-4.6.1-cp38-cp38-win_amd64.whl", hash = "sha256:2cd4ce750c34a0318fc2d6c727cc255e9658d12a5cf3f2d0473f1c27157bdaeb"}, - {file = "pyinstrument-4.6.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:6ca95b21f022e995e062b371d1f42d901452bcbedd2c02f036de677119503355"}, - {file = "pyinstrument-4.6.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:ac1e1d7e1f1b64054c4eb04eb4869a7a5eef2261440e73943cc1b1bc3c828c18"}, - {file = "pyinstrument-4.6.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0711845e953fce6ab781221aacffa2a66dbc3289f8343e5babd7b2ea34da6c90"}, - {file = "pyinstrument-4.6.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5b7d28582017de35cb64eb4e4fa603e753095108ca03745f5d17295970ee631f"}, - {file = "pyinstrument-4.6.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7be57db08bd366a37db3aa3a6187941ee21196e8b14975db337ddc7d1490649d"}, - {file = "pyinstrument-4.6.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:9a0ac0f56860398d2628ce389826ce83fb3a557d0c9a2351e8a2eac6eb869983"}, - {file = "pyinstrument-4.6.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:a9045186ff13bc826fef16be53736a85029aae3c6adfe52e666cad00d7ca623b"}, - {file = "pyinstrument-4.6.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:6c4c56b6eab9004e92ad8a48bb54913fdd71fc8a748ae42a27b9e26041646f8b"}, - {file = "pyinstrument-4.6.1-cp39-cp39-win32.whl", hash = "sha256:37e989c44b51839d0c97466fa2b623638b9470d56d79e329f359f0e8fa6d83db"}, - {file = "pyinstrument-4.6.1-cp39-cp39-win_amd64.whl", hash = "sha256:5494c5a84fee4309d7d973366ca6b8b9f8ba1d6b254e93b7c506264ef74f2cef"}, - {file = "pyinstrument-4.6.1.tar.gz", hash = "sha256:f4731b27121350f5a983d358d2272fe3df2f538aed058f57217eef7801a89288"}, + {file = "pyinstrument-4.6.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7a1b1cd768ea7ea9ab6f5490f7e74431321bcc463e9441dbc2f769617252d9e2"}, + {file = "pyinstrument-4.6.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:8a386b9d09d167451fb2111eaf86aabf6e094fed42c15f62ec51d6980bce7d96"}, + {file = "pyinstrument-4.6.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:23c3e3ca8553b9aac09bd978c73d21b9032c707ac6d803bae6a20ecc048df4a8"}, + {file = "pyinstrument-4.6.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5f329f5534ca069420246f5ce57270d975229bcb92a3a3fd6b2ca086527d9764"}, + {file = "pyinstrument-4.6.2-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d4dcdcc7ba224a0c5edfbd00b0f530f5aed2b26da5aaa2f9af5519d4aa8c7e41"}, + {file = "pyinstrument-4.6.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:73db0c2c99119c65b075feee76e903b4ed82e59440fe8b5724acf5c7cb24721f"}, + {file = "pyinstrument-4.6.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:da58f265326f3cf3975366ccb8b39014f1e69ff8327958a089858d71c633d654"}, + {file = "pyinstrument-4.6.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:feebcf860f955401df30d029ec8de7a0c5515d24ea809736430fd1219686fe14"}, + {file = "pyinstrument-4.6.2-cp310-cp310-win32.whl", hash = "sha256:b2b66ff0b16c8ecf1ec22de001cfff46872b2c163c62429055105564eef50b2e"}, + {file = "pyinstrument-4.6.2-cp310-cp310-win_amd64.whl", hash = "sha256:8d104b7a7899d5fa4c5bf1ceb0c1a070615a72c5dc17bc321b612467ad5c5d88"}, + {file = "pyinstrument-4.6.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:62f6014d2b928b181a52483e7c7b82f2c27e22c577417d1681153e5518f03317"}, + {file = "pyinstrument-4.6.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:dcb5c8d763c5df55131670ba2a01a8aebd0d490a789904a55eb6a8b8d497f110"}, + {file = "pyinstrument-4.6.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6ed4e8c6c84e0e6429ba7008a66e435ede2d8cb027794c20923c55669d9c5633"}, + {file = "pyinstrument-4.6.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6c0f0e1d8f8c70faa90ff57f78ac0dda774b52ea0bfb2d9f0f41ce6f3e7c869e"}, + {file = "pyinstrument-4.6.2-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8b3c44cb037ad0d6e9d9a48c14d856254ada641fbd0ae9de40da045fc2226a2a"}, + {file = "pyinstrument-4.6.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:be9901f17ac2f527c352f2fdca3d717c1d7f2ce8a70bad5a490fc8cc5d2a6007"}, + {file = "pyinstrument-4.6.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:8a9791bf8916c1cf439c202fded32de93354b0f57328f303d71950b0027c7811"}, + {file = "pyinstrument-4.6.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d6162615e783c59e36f2d7caf903a7e3ecb6b32d4a4ae8907f2760b2ef395bf6"}, + {file = "pyinstrument-4.6.2-cp311-cp311-win32.whl", hash = "sha256:28af084aa84bbfd3620ebe71d5f9a0deca4451267f363738ca824f733de55056"}, + {file = "pyinstrument-4.6.2-cp311-cp311-win_amd64.whl", hash = "sha256:dd6007d3c2e318e09e582435dd8d111cccf30d342af66886b783208813caf3d7"}, + {file = "pyinstrument-4.6.2-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:e3813c8ecfab9d7d855c5f0f71f11793cf1507f40401aa33575c7fd613577c23"}, + {file = "pyinstrument-4.6.2-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:6c761372945e60fc1396b7a49f30592e8474e70a558f1a87346d27c8c4ce50f7"}, + {file = "pyinstrument-4.6.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4fba3244e94c117bf4d9b30b8852bbdcd510e7329fdd5c7c8b3799e00a9215a8"}, + {file = "pyinstrument-4.6.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:803ac64e526473d64283f504df3b0d5c2c203ea9603cab428641538ffdc753a7"}, + {file = "pyinstrument-4.6.2-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2e554b1bb0df78f5ce8a92df75b664912ca93aa94208386102af454ec31b647"}, + {file = "pyinstrument-4.6.2-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:7c671057fad22ee3ded897a6a361204ea2538e44c1233cad0e8e30f6d27f33db"}, + {file = "pyinstrument-4.6.2-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:d02f31fa13a9e8dc702a113878419deba859563a32474c9f68e04619d43d6f01"}, + {file = "pyinstrument-4.6.2-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:b55983a884f083f93f0fc6d12ff8df0acd1e2fb0580d2f4c7bfe6def33a84b58"}, + {file = "pyinstrument-4.6.2-cp312-cp312-win32.whl", hash = "sha256:fdc0a53b27e5d8e47147489c7dab596ddd1756b1e053217ef5bc6718567099ff"}, + {file = "pyinstrument-4.6.2-cp312-cp312-win_amd64.whl", hash = "sha256:dd5c53a0159126b5ce7cbc4994433c9c671e057c85297ff32645166a06ad2c50"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:b082df0bbf71251a7f4880a12ed28421dba84ea7110bb376e0533067a4eaff40"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:90350533396071cb2543affe01e40bf534c35cb0d4b8fa9fdb0f052f9ca2cfe3"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:67268bb0d579330cff40fd1c90b8510363ca1a0e7204225840614068658dab77"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:20e15b4e1d29ba0b7fc81aac50351e0dc0d7e911e93771ebc3f408e864a2c93b"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:2e625fc6ffcd4fd420493edd8276179c3f784df207bef4c2192725c1b310534c"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:113d2fc534c9ca7b6b5661d6ada05515bf318f6eb34e8d05860fe49eb7cfe17e"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:3098cd72b71a322a72dafeb4ba5c566465e193d2030adad4c09566bd2f89bf4f"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-win32.whl", hash = "sha256:08fdc7f88c989316fa47805234c37a40fafe7b614afd8ae863f0afa9d1707b37"}, + {file = "pyinstrument-4.6.2-cp37-cp37m-win_amd64.whl", hash = "sha256:5ebeba952c0056dcc9b9355328c78c4b5c2a33b4b4276a9157a3ab589f3d1bac"}, + {file = "pyinstrument-4.6.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:34e59e91c88ec9ad5630c0964eca823949005e97736bfa838beb4789e94912a2"}, + {file = "pyinstrument-4.6.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:cd0320c39e99e3c0a3129d1ed010ac41e5a7eb96fb79900d270080a97962e995"}, + {file = "pyinstrument-4.6.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:46992e855d630575ec635eeca0068a8ddf423d4fd32ea0875a94e9f8688f0b95"}, + {file = "pyinstrument-4.6.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e474c56da636253dfdca7cd1998b240d6b39f7ed34777362db69224fcf053b1"}, + {file = "pyinstrument-4.6.2-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d4b559322f30509ad8f082561792352d0805b3edfa508e492a36041fdc009259"}, + {file = "pyinstrument-4.6.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:06a8578b2943eb1dbbf281e1e59e44246acfefd79e1b06d4950f01b693de12af"}, + {file = "pyinstrument-4.6.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:7bd3da31c46f1c1cb7ae89031725f6a1d1015c2041d9c753fe23980f5f9fd86c"}, + {file = "pyinstrument-4.6.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:e63f4916001aa9c625976a50779282e0a5b5e9b17c52a50ef4c651e468ed5b88"}, + {file = "pyinstrument-4.6.2-cp38-cp38-win32.whl", hash = "sha256:32ec8db6896b94af790a530e1e0edad4d0f941a0ab8dd9073e5993e7ea46af7d"}, + {file = "pyinstrument-4.6.2-cp38-cp38-win_amd64.whl", hash = "sha256:a59fc4f7db738a094823afe6422509fa5816a7bf74e768ce5a7a2ddd91af40ac"}, + {file = "pyinstrument-4.6.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:3a165e0d2deb212d4cf439383982a831682009e1b08733c568cac88c89784e62"}, + {file = "pyinstrument-4.6.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:7ba858b3d6f6e5597c641edcc0e7e464f85aba86d71bc3b3592cb89897bf43f6"}, + {file = "pyinstrument-4.6.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2fd8e547cf3df5f0ec6e4dffbe2e857f6b28eda51b71c3c0b5a2fc0646527835"}, + {file = "pyinstrument-4.6.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0de2c1714a37a820033b19cf134ead43299a02662f1379140974a9ab733c5f3a"}, + {file = "pyinstrument-4.6.2-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:01fc45dedceec3df81668d702bca6d400d956c8b8494abc206638c167c78dfd9"}, + {file = "pyinstrument-4.6.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:5b6e161ef268d43ee6bbfae7fd2cdd0a52c099ddd21001c126ca1805dc906539"}, + {file = "pyinstrument-4.6.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:6ba8e368d0421f15ba6366dfd60ec131c1b46505d021477e0f865d26cf35a605"}, + {file = "pyinstrument-4.6.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:edca46f04a573ac2fb11a84b937844e6a109f38f80f4b422222fb5be8ecad8cb"}, + {file = "pyinstrument-4.6.2-cp39-cp39-win32.whl", hash = "sha256:baf375953b02fe94d00e716f060e60211ede73f49512b96687335f7071adb153"}, + {file = "pyinstrument-4.6.2-cp39-cp39-win_amd64.whl", hash = "sha256:af1a953bce9fd530040895d01ff3de485e25e1576dccb014f76ba9131376fcad"}, + {file = "pyinstrument-4.6.2.tar.gz", hash = "sha256:0002ee517ed8502bbda6eb2bb1ba8f95a55492fcdf03811ba13d4806e50dd7f6"}, ] [package.extras] @@ -5070,13 +5049,13 @@ docs = ["Sphinx (>=6.2,<7.0)", "boto3 (>=1.26,<2.0)", "cartopy (>=0.21,<1.0)", " [[package]] name = "pytest" -version = "7.4.4" +version = "8.0.0" description = "pytest: simple powerful testing with Python" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "pytest-7.4.4-py3-none-any.whl", hash = "sha256:b090cdf5ed60bf4c45261be03239c2c1c22df034fbffe691abe93cd80cea01d8"}, - {file = "pytest-7.4.4.tar.gz", hash = "sha256:2cf0005922c6ace4a3e2ec8b4080eb0d9753fdc93107415332f50ce9e7994280"}, + {file = "pytest-8.0.0-py3-none-any.whl", hash = "sha256:50fb9cbe836c3f20f0dfa99c565201fb75dc54c8d76373cd1bde06b06657bdb6"}, + {file = "pytest-8.0.0.tar.gz", hash = "sha256:249b1b0864530ba251b7438274c4d251c58d868edaaec8762893ad4a0d71c36c"}, ] [package.dependencies] @@ -5084,7 +5063,7 @@ colorama = {version = "*", markers = "sys_platform == \"win32\""} exceptiongroup = {version = ">=1.0.0rc8", markers = "python_version < \"3.11\""} iniconfig = "*" packaging = "*" -pluggy = ">=0.12,<2.0" +pluggy = ">=1.3.0,<2.0" tomli = {version = ">=1.0.0", markers = "python_version < \"3.11\""} [package.extras] @@ -5285,13 +5264,13 @@ six = ">=1.5" [[package]] name = "pytz" -version = "2023.3.post1" +version = "2023.4" description = "World timezone definitions, modern and historical" optional = false python-versions = "*" files = [ - {file = "pytz-2023.3.post1-py2.py3-none-any.whl", hash = "sha256:ce42d816b81b68506614c11e8937d3aa9e41007ceb50bfdcb0749b921bf646c7"}, - {file = "pytz-2023.3.post1.tar.gz", hash = "sha256:7b4fddbeb94a1eba4b557da24f19fdf9db575192544270a9101d8509f9f43d7b"}, + {file = "pytz-2023.4-py2.py3-none-any.whl", hash = "sha256:f90ef520d95e7c46951105338d918664ebfd6f1d995bd7d153127ce90efafa6a"}, + {file = "pytz-2023.4.tar.gz", hash = "sha256:31d4583c4ed539cd037956140d695e42c033a19e984bfce9964a3f7d59bc2b40"}, ] [[package]] @@ -5623,13 +5602,13 @@ full = ["numpy"] [[package]] name = "referencing" -version = "0.32.1" +version = "0.33.0" description = "JSON Referencing + Python" optional = false python-versions = ">=3.8" files = [ - {file = "referencing-0.32.1-py3-none-any.whl", hash = "sha256:7e4dc12271d8e15612bfe35792f5ea1c40970dadf8624602e33db2758f7ee554"}, - {file = "referencing-0.32.1.tar.gz", hash = "sha256:3c57da0513e9563eb7e203ebe9bb3a1b509b042016433bd1e45a2853466c3dd3"}, + {file = "referencing-0.33.0-py3-none-any.whl", hash = "sha256:39240f2ecc770258f28b642dd47fd74bc8b02484de54e1882b74b35ebd779bd5"}, + {file = "referencing-0.33.0.tar.gz", hash = "sha256:c775fedf74bc0f9189c2a3be1c12fd03e8c23f4d371dce795df44e06c5b412f7"}, ] [package.dependencies] @@ -5934,118 +5913,133 @@ pyasn1 = ">=0.1.3" [[package]] name = "ruff" -version = "0.1.13" +version = "0.1.15" description = "An extremely fast Python linter and code formatter, written in Rust." optional = false python-versions = ">=3.7" files = [ - {file = "ruff-0.1.13-py3-none-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:e3fd36e0d48aeac672aa850045e784673449ce619afc12823ea7868fcc41d8ba"}, - {file = "ruff-0.1.13-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:9fb6b3b86450d4ec6a6732f9f60c4406061b6851c4b29f944f8c9d91c3611c7a"}, - {file = "ruff-0.1.13-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b13ba5d7156daaf3fd08b6b993360a96060500aca7e307d95ecbc5bb47a69296"}, - {file = "ruff-0.1.13-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9ebb40442f7b531e136d334ef0851412410061e65d61ca8ce90d894a094feb22"}, - {file = "ruff-0.1.13-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:226b517f42d59a543d6383cfe03cccf0091e3e0ed1b856c6824be03d2a75d3b6"}, - {file = "ruff-0.1.13-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:5f0312ba1061e9b8c724e9a702d3c8621e3c6e6c2c9bd862550ab2951ac75c16"}, - {file = "ruff-0.1.13-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2f59bcf5217c661254bd6bc42d65a6fd1a8b80c48763cb5c2293295babd945dd"}, - {file = "ruff-0.1.13-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e6894b00495e00c27b6ba61af1fc666f17de6140345e5ef27dd6e08fb987259d"}, - {file = "ruff-0.1.13-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9a1600942485c6e66119da294c6294856b5c86fd6df591ce293e4a4cc8e72989"}, - {file = "ruff-0.1.13-py3-none-musllinux_1_2_aarch64.whl", hash = "sha256:ee3febce7863e231a467f90e681d3d89210b900d49ce88723ce052c8761be8c7"}, - {file = "ruff-0.1.13-py3-none-musllinux_1_2_armv7l.whl", hash = "sha256:dcaab50e278ff497ee4d1fe69b29ca0a9a47cd954bb17963628fa417933c6eb1"}, - {file = "ruff-0.1.13-py3-none-musllinux_1_2_i686.whl", hash = "sha256:f57de973de4edef3ad3044d6a50c02ad9fc2dff0d88587f25f1a48e3f72edf5e"}, - {file = "ruff-0.1.13-py3-none-musllinux_1_2_x86_64.whl", hash = "sha256:7a36fa90eb12208272a858475ec43ac811ac37e91ef868759770b71bdabe27b6"}, - {file = "ruff-0.1.13-py3-none-win32.whl", hash = "sha256:a623349a505ff768dad6bd57087e2461be8db58305ebd5577bd0e98631f9ae69"}, - {file = "ruff-0.1.13-py3-none-win_amd64.whl", hash = "sha256:f988746e3c3982bea7f824c8fa318ce7f538c4dfefec99cd09c8770bd33e6539"}, - {file = "ruff-0.1.13-py3-none-win_arm64.whl", hash = "sha256:6bbbc3042075871ec17f28864808540a26f0f79a4478c357d3e3d2284e832998"}, - {file = "ruff-0.1.13.tar.gz", hash = "sha256:e261f1baed6291f434ffb1d5c6bd8051d1c2a26958072d38dfbec39b3dda7352"}, + {file = "ruff-0.1.15-py3-none-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:5fe8d54df166ecc24106db7dd6a68d44852d14eb0729ea4672bb4d96c320b7df"}, + {file = "ruff-0.1.15-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:6f0bfbb53c4b4de117ac4d6ddfd33aa5fc31beeaa21d23c45c6dd249faf9126f"}, + {file = "ruff-0.1.15-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e0d432aec35bfc0d800d4f70eba26e23a352386be3a6cf157083d18f6f5881c8"}, + {file = "ruff-0.1.15-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9405fa9ac0e97f35aaddf185a1be194a589424b8713e3b97b762336ec79ff807"}, + {file = "ruff-0.1.15-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c66ec24fe36841636e814b8f90f572a8c0cb0e54d8b5c2d0e300d28a0d7bffec"}, + {file = "ruff-0.1.15-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:6f8ad828f01e8dd32cc58bc28375150171d198491fc901f6f98d2a39ba8e3ff5"}, + {file = "ruff-0.1.15-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:86811954eec63e9ea162af0ffa9f8d09088bab51b7438e8b6488b9401863c25e"}, + {file = "ruff-0.1.15-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fd4025ac5e87d9b80e1f300207eb2fd099ff8200fa2320d7dc066a3f4622dc6b"}, + {file = "ruff-0.1.15-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b17b93c02cdb6aeb696effecea1095ac93f3884a49a554a9afa76bb125c114c1"}, + {file = "ruff-0.1.15-py3-none-musllinux_1_2_aarch64.whl", hash = "sha256:ddb87643be40f034e97e97f5bc2ef7ce39de20e34608f3f829db727a93fb82c5"}, + {file = "ruff-0.1.15-py3-none-musllinux_1_2_armv7l.whl", hash = "sha256:abf4822129ed3a5ce54383d5f0e964e7fef74a41e48eb1dfad404151efc130a2"}, + {file = "ruff-0.1.15-py3-none-musllinux_1_2_i686.whl", hash = "sha256:6c629cf64bacfd136c07c78ac10a54578ec9d1bd2a9d395efbee0935868bf852"}, + {file = "ruff-0.1.15-py3-none-musllinux_1_2_x86_64.whl", hash = "sha256:1bab866aafb53da39c2cadfb8e1c4550ac5340bb40300083eb8967ba25481447"}, + {file = "ruff-0.1.15-py3-none-win32.whl", hash = "sha256:2417e1cb6e2068389b07e6fa74c306b2810fe3ee3476d5b8a96616633f40d14f"}, + {file = "ruff-0.1.15-py3-none-win_amd64.whl", hash = "sha256:3837ac73d869efc4182d9036b1405ef4c73d9b1f88da2413875e34e0d6919587"}, + {file = "ruff-0.1.15-py3-none-win_arm64.whl", hash = "sha256:9a933dfb1c14ec7a33cceb1e49ec4a16b51ce3c20fd42663198746efc0427360"}, + {file = "ruff-0.1.15.tar.gz", hash = "sha256:f6dfa8c1b21c913c326919056c390966648b680966febcb796cc9d1aaab8564e"}, ] [[package]] name = "scikit-learn" -version = "1.3.2" +version = "1.4.0" description = "A set of python modules for machine learning and data mining" optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" files = [ - {file = "scikit-learn-1.3.2.tar.gz", hash = "sha256:a2f54c76accc15a34bfb9066e6c7a56c1e7235dda5762b990792330b52ccfb05"}, - {file = "scikit_learn-1.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e326c0eb5cf4d6ba40f93776a20e9a7a69524c4db0757e7ce24ba222471ee8a1"}, - {file = "scikit_learn-1.3.2-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:535805c2a01ccb40ca4ab7d081d771aea67e535153e35a1fd99418fcedd1648a"}, - {file = "scikit_learn-1.3.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1215e5e58e9880b554b01187b8c9390bf4dc4692eedeaf542d3273f4785e342c"}, - {file = "scikit_learn-1.3.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0ee107923a623b9f517754ea2f69ea3b62fc898a3641766cb7deb2f2ce450161"}, - {file = "scikit_learn-1.3.2-cp310-cp310-win_amd64.whl", hash = "sha256:35a22e8015048c628ad099da9df5ab3004cdbf81edc75b396fd0cff8699ac58c"}, - {file = "scikit_learn-1.3.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:6fb6bc98f234fda43163ddbe36df8bcde1d13ee176c6dc9b92bb7d3fc842eb66"}, - {file = "scikit_learn-1.3.2-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:18424efee518a1cde7b0b53a422cde2f6625197de6af36da0b57ec502f126157"}, - {file = "scikit_learn-1.3.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3271552a5eb16f208a6f7f617b8cc6d1f137b52c8a1ef8edf547db0259b2c9fb"}, - {file = "scikit_learn-1.3.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fc4144a5004a676d5022b798d9e573b05139e77f271253a4703eed295bde0433"}, - {file = "scikit_learn-1.3.2-cp311-cp311-win_amd64.whl", hash = "sha256:67f37d708f042a9b8d59551cf94d30431e01374e00dc2645fa186059c6c5d78b"}, - {file = "scikit_learn-1.3.2-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:8db94cd8a2e038b37a80a04df8783e09caac77cbe052146432e67800e430c028"}, - {file = "scikit_learn-1.3.2-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:61a6efd384258789aa89415a410dcdb39a50e19d3d8410bd29be365bcdd512d5"}, - {file = "scikit_learn-1.3.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cb06f8dce3f5ddc5dee1715a9b9f19f20d295bed8e3cd4fa51e1d050347de525"}, - {file = "scikit_learn-1.3.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5b2de18d86f630d68fe1f87af690d451388bb186480afc719e5f770590c2ef6c"}, - {file = "scikit_learn-1.3.2-cp312-cp312-win_amd64.whl", hash = "sha256:0402638c9a7c219ee52c94cbebc8fcb5eb9fe9c773717965c1f4185588ad3107"}, - {file = "scikit_learn-1.3.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:a19f90f95ba93c1a7f7924906d0576a84da7f3b2282ac3bfb7a08a32801add93"}, - {file = "scikit_learn-1.3.2-cp38-cp38-macosx_12_0_arm64.whl", hash = "sha256:b8692e395a03a60cd927125eef3a8e3424d86dde9b2370d544f0ea35f78a8073"}, - {file = "scikit_learn-1.3.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:15e1e94cc23d04d39da797ee34236ce2375ddea158b10bee3c343647d615581d"}, - {file = "scikit_learn-1.3.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:785a2213086b7b1abf037aeadbbd6d67159feb3e30263434139c98425e3dcfcf"}, - {file = "scikit_learn-1.3.2-cp38-cp38-win_amd64.whl", hash = "sha256:64381066f8aa63c2710e6b56edc9f0894cc7bf59bd71b8ce5613a4559b6145e0"}, - {file = "scikit_learn-1.3.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6c43290337f7a4b969d207e620658372ba3c1ffb611f8bc2b6f031dc5c6d1d03"}, - {file = "scikit_learn-1.3.2-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:dc9002fc200bed597d5d34e90c752b74df516d592db162f756cc52836b38fe0e"}, - {file = "scikit_learn-1.3.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d08ada33e955c54355d909b9c06a4789a729977f165b8bae6f225ff0a60ec4a"}, - {file = "scikit_learn-1.3.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:763f0ae4b79b0ff9cca0bf3716bcc9915bdacff3cebea15ec79652d1cc4fa5c9"}, - {file = "scikit_learn-1.3.2-cp39-cp39-win_amd64.whl", hash = "sha256:ed932ea780517b00dae7431e031faae6b49b20eb6950918eb83bd043237950e0"}, -] - -[package.dependencies] -joblib = ">=1.1.1" -numpy = ">=1.17.3,<2.0" -scipy = ">=1.5.0" + {file = "scikit-learn-1.4.0.tar.gz", hash = "sha256:d4373c984eba20e393216edd51a3e3eede56cbe93d4247516d205643c3b93121"}, + {file = "scikit_learn-1.4.0-1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:fce93a7473e2f4ee4cc280210968288d6a7d7ad8dc6fa7bb7892145e407085f9"}, + {file = "scikit_learn-1.4.0-1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:d77df3d1e15fc37a9329999979fa7868ba8655dbab21fe97fc7ddabac9e08cc7"}, + {file = "scikit_learn-1.4.0-1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2404659fedec40eeafa310cd14d613e564d13dbf8f3c752d31c095195ec05de6"}, + {file = "scikit_learn-1.4.0-1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e98632da8f6410e6fb6bf66937712c949b4010600ccd3f22a5388a83e610cc3c"}, + {file = "scikit_learn-1.4.0-1-cp310-cp310-win_amd64.whl", hash = "sha256:11b3b140f70fbc9f6a08884631ae8dd60a4bb2d7d6d1de92738ea42b740d8992"}, + {file = "scikit_learn-1.4.0-1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a8341eabdc754d5ab91641a7763243845e96b6d68e03e472531e88a4f1b09f21"}, + {file = "scikit_learn-1.4.0-1-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:d1f6bce875ac2bb6b52514f67c185c564ccd299a05b65b7bab091a4c13dde12d"}, + {file = "scikit_learn-1.4.0-1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c408b46b2fd61952d519ea1af2f8f0a7a703e1433923ab1704c4131520b2083b"}, + {file = "scikit_learn-1.4.0-1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2b465dd1dcd237b7b1dcd1a9048ccbf70a98c659474324fa708464c3a2533fad"}, + {file = "scikit_learn-1.4.0-1-cp311-cp311-win_amd64.whl", hash = "sha256:0db8e22c42f7980fe5eb22069b1f84c48966f3e0d23a01afde5999e3987a2501"}, + {file = "scikit_learn-1.4.0-1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:e7eef6ea2ed289af40e88c0be9f7704ca8b5de18508a06897c3fe21e0905efdf"}, + {file = "scikit_learn-1.4.0-1-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:349669b01435bc4dbf25c6410b0892073befdaec52637d1a1d1ff53865dc8db3"}, + {file = "scikit_learn-1.4.0-1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d439c584e58434d0350701bd33f6c10b309e851fccaf41c121aed55f6851d8cf"}, + {file = "scikit_learn-1.4.0-1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a0e2427d9ef46477625ab9b55c1882844fe6fc500f418c3f8e650200182457bc"}, + {file = "scikit_learn-1.4.0-1-cp312-cp312-win_amd64.whl", hash = "sha256:d3d75343940e7bf9b85c830c93d34039fa015eeb341c5c0b4cd7a90dadfe00d4"}, + {file = "scikit_learn-1.4.0-1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:76986d22e884ab062b1beecdd92379656e9d3789ecc1f9870923c178de55f9fe"}, + {file = "scikit_learn-1.4.0-1-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:e22446ad89f1cb7657f0d849dcdc345b48e2d10afa3daf2925fdb740f85b714c"}, + {file = "scikit_learn-1.4.0-1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:74812c9eabb265be69d738a8ea8d4884917a59637fcbf88a5f0e9020498bc6b3"}, + {file = "scikit_learn-1.4.0-1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:aad2a63e0dd386b92da3270887a29b308af4d7c750d8c4995dfd9a4798691bcc"}, + {file = "scikit_learn-1.4.0-1-cp39-cp39-win_amd64.whl", hash = "sha256:53b9e29177897c37e2ff9d4ba6ca12fdb156e22523e463db05def303f5c72b5c"}, + {file = "scikit_learn-1.4.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:cb8f044a8f5962613ce1feb4351d66f8d784bd072d36393582f351859b065f7d"}, + {file = "scikit_learn-1.4.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:a6372c90bbf302387792108379f1ec77719c1618d88496d0df30cb8e370b4661"}, + {file = "scikit_learn-1.4.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:785ce3c352bf697adfda357c3922c94517a9376002971bc5ea50896144bc8916"}, + {file = "scikit_learn-1.4.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0aba2a20d89936d6e72d95d05e3bf1db55bca5c5920926ad7b92c34f5e7d3bbe"}, + {file = "scikit_learn-1.4.0-cp310-cp310-win_amd64.whl", hash = "sha256:2bac5d56b992f8f06816f2cd321eb86071c6f6d44bb4b1cb3d626525820d754b"}, + {file = "scikit_learn-1.4.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:27ae4b0f1b2c77107c096a7e05b33458354107b47775428d1f11b23e30a73e8a"}, + {file = "scikit_learn-1.4.0-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:5c5c62ffb52c3ffb755eb21fa74cc2cbf2c521bd53f5c04eaa10011dbecf5f80"}, + {file = "scikit_learn-1.4.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7f0d2018ac6fa055dab65fe8a485967990d33c672d55bc254c56c35287b02fab"}, + {file = "scikit_learn-1.4.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:91a8918c415c4b4bf1d60c38d32958849a9191c2428ab35d30b78354085c7c7a"}, + {file = "scikit_learn-1.4.0-cp311-cp311-win_amd64.whl", hash = "sha256:80a21de63275f8bcd7877b3e781679d2ff1eddfed515a599f95b2502a3283d42"}, + {file = "scikit_learn-1.4.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:0f33bbafb310c26b81c4d41ecaebdbc1f63498a3f13461d50ed9a2e8f24d28e4"}, + {file = "scikit_learn-1.4.0-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:8b6ac1442ec714b4911e5aef8afd82c691b5c88b525ea58299d455acc4e8dcec"}, + {file = "scikit_learn-1.4.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:05fc5915b716c6cc60a438c250108e9a9445b522975ed37e416d5ea4f9a63381"}, + {file = "scikit_learn-1.4.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:842b7d6989f3c574685e18da6f91223eb32301d0f93903dd399894250835a6f7"}, + {file = "scikit_learn-1.4.0-cp312-cp312-win_amd64.whl", hash = "sha256:88bcb586fdff865372df1bc6be88bb7e6f9e0aa080dab9f54f5cac7eca8e2b6b"}, + {file = "scikit_learn-1.4.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:f77674647dd31f56cb12ed13ed25b6ed43a056fffef051715022d2ebffd7a7d1"}, + {file = "scikit_learn-1.4.0-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:833999872e2920ce00f3a50839946bdac7539454e200eb6db54898a41f4bfd43"}, + {file = "scikit_learn-1.4.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:970ec697accaef10fb4f51763f3a7b1250f9f0553cf05514d0e94905322a0172"}, + {file = "scikit_learn-1.4.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:923d778f378ebacca2c672ab1740e5a413e437fb45ab45ab02578f8b689e5d43"}, + {file = "scikit_learn-1.4.0-cp39-cp39-win_amd64.whl", hash = "sha256:1d041bc95006b545b59e458399e3175ab11ca7a03dc9a74a573ac891f5df1489"}, +] + +[package.dependencies] +joblib = ">=1.2.0" +numpy = ">=1.19.5" +scipy = ">=1.6.0" threadpoolctl = ">=2.0.0" [package.extras] -benchmark = ["matplotlib (>=3.1.3)", "memory-profiler (>=0.57.0)", "pandas (>=1.0.5)"] -docs = ["Pillow (>=7.1.2)", "matplotlib (>=3.1.3)", "memory-profiler (>=0.57.0)", "numpydoc (>=1.2.0)", "pandas (>=1.0.5)", "plotly (>=5.14.0)", "pooch (>=1.6.0)", "scikit-image (>=0.16.2)", "seaborn (>=0.9.0)", "sphinx (>=6.0.0)", "sphinx-copybutton (>=0.5.2)", "sphinx-gallery (>=0.10.1)", "sphinx-prompt (>=1.3.0)", "sphinxext-opengraph (>=0.4.2)"] -examples = ["matplotlib (>=3.1.3)", "pandas (>=1.0.5)", "plotly (>=5.14.0)", "pooch (>=1.6.0)", "scikit-image (>=0.16.2)", "seaborn (>=0.9.0)"] -tests = ["black (>=23.3.0)", "matplotlib (>=3.1.3)", "mypy (>=1.3)", "numpydoc (>=1.2.0)", "pandas (>=1.0.5)", "pooch (>=1.6.0)", "pyamg (>=4.0.0)", "pytest (>=7.1.2)", "pytest-cov (>=2.9.0)", "ruff (>=0.0.272)", "scikit-image (>=0.16.2)"] +benchmark = ["matplotlib (>=3.3.4)", "memory-profiler (>=0.57.0)", "pandas (>=1.1.5)"] +docs = ["Pillow (>=7.1.2)", "matplotlib (>=3.3.4)", "memory-profiler (>=0.57.0)", "numpydoc (>=1.2.0)", "pandas (>=1.1.5)", "plotly (>=5.14.0)", "pooch (>=1.6.0)", "scikit-image (>=0.17.2)", "seaborn (>=0.9.0)", "sphinx (>=6.0.0)", "sphinx-copybutton (>=0.5.2)", "sphinx-gallery (>=0.15.0)", "sphinx-prompt (>=1.3.0)", "sphinxext-opengraph (>=0.4.2)"] +examples = ["matplotlib (>=3.3.4)", "pandas (>=1.1.5)", "plotly (>=5.14.0)", "pooch (>=1.6.0)", "scikit-image (>=0.17.2)", "seaborn (>=0.9.0)"] +tests = ["black (>=23.3.0)", "matplotlib (>=3.3.4)", "mypy (>=1.3)", "numpydoc (>=1.2.0)", "pandas (>=1.1.5)", "polars (>=0.19.12)", "pooch (>=1.6.0)", "pyamg (>=4.0.0)", "pyarrow (>=12.0.0)", "pytest (>=7.1.2)", "pytest-cov (>=2.9.0)", "ruff (>=0.0.272)", "scikit-image (>=0.17.2)"] [[package]] name = "scipy" -version = "1.11.4" +version = "1.12.0" description = "Fundamental algorithms for scientific computing in Python" optional = false python-versions = ">=3.9" files = [ - {file = "scipy-1.11.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bc9a714581f561af0848e6b69947fda0614915f072dfd14142ed1bfe1b806710"}, - {file = "scipy-1.11.4-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:cf00bd2b1b0211888d4dc75656c0412213a8b25e80d73898083f402b50f47e41"}, - {file = "scipy-1.11.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b9999c008ccf00e8fbcce1236f85ade5c569d13144f77a1946bef8863e8f6eb4"}, - {file = "scipy-1.11.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:933baf588daa8dc9a92c20a0be32f56d43faf3d1a60ab11b3f08c356430f6e56"}, - {file = "scipy-1.11.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:8fce70f39076a5aa62e92e69a7f62349f9574d8405c0a5de6ed3ef72de07f446"}, - {file = "scipy-1.11.4-cp310-cp310-win_amd64.whl", hash = "sha256:6550466fbeec7453d7465e74d4f4b19f905642c89a7525571ee91dd7adabb5a3"}, - {file = "scipy-1.11.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f313b39a7e94f296025e3cffc2c567618174c0b1dde173960cf23808f9fae4be"}, - {file = "scipy-1.11.4-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:1b7c3dca977f30a739e0409fb001056484661cb2541a01aba0bb0029f7b68db8"}, - {file = "scipy-1.11.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:00150c5eae7b610c32589dda259eacc7c4f1665aedf25d921907f4d08a951b1c"}, - {file = "scipy-1.11.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:530f9ad26440e85766509dbf78edcfe13ffd0ab7fec2560ee5c36ff74d6269ff"}, - {file = "scipy-1.11.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:5e347b14fe01003d3b78e196e84bd3f48ffe4c8a7b8a1afbcb8f5505cb710993"}, - {file = "scipy-1.11.4-cp311-cp311-win_amd64.whl", hash = "sha256:acf8ed278cc03f5aff035e69cb511741e0418681d25fbbb86ca65429c4f4d9cd"}, - {file = "scipy-1.11.4-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:028eccd22e654b3ea01ee63705681ee79933652b2d8f873e7949898dda6d11b6"}, - {file = "scipy-1.11.4-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:2c6ff6ef9cc27f9b3db93a6f8b38f97387e6e0591600369a297a50a8e96e835d"}, - {file = "scipy-1.11.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b030c6674b9230d37c5c60ab456e2cf12f6784596d15ce8da9365e70896effc4"}, - {file = "scipy-1.11.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ad669df80528aeca5f557712102538f4f37e503f0c5b9541655016dd0932ca79"}, - {file = "scipy-1.11.4-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:ce7fff2e23ab2cc81ff452a9444c215c28e6305f396b2ba88343a567feec9660"}, - {file = "scipy-1.11.4-cp312-cp312-win_amd64.whl", hash = "sha256:36750b7733d960d7994888f0d148d31ea3017ac15eef664194b4ef68d36a4a97"}, - {file = "scipy-1.11.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6e619aba2df228a9b34718efb023966da781e89dd3d21637b27f2e54db0410d7"}, - {file = "scipy-1.11.4-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:f3cd9e7b3c2c1ec26364856f9fbe78695fe631150f94cd1c22228456404cf1ec"}, - {file = "scipy-1.11.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d10e45a6c50211fe256da61a11c34927c68f277e03138777bdebedd933712fea"}, - {file = "scipy-1.11.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:91af76a68eeae0064887a48e25c4e616fa519fa0d38602eda7e0f97d65d57937"}, - {file = "scipy-1.11.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:6df1468153a31cf55ed5ed39647279beb9cfb5d3f84369453b49e4b8502394fd"}, - {file = "scipy-1.11.4-cp39-cp39-win_amd64.whl", hash = "sha256:ee410e6de8f88fd5cf6eadd73c135020bfbbbdfcd0f6162c36a7638a1ea8cc65"}, - {file = "scipy-1.11.4.tar.gz", hash = "sha256:90a2b78e7f5733b9de748f589f09225013685f9b218275257f8a8168ededaeaa"}, -] - -[package.dependencies] -numpy = ">=1.21.6,<1.28.0" + {file = "scipy-1.12.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:78e4402e140879387187f7f25d91cc592b3501a2e51dfb320f48dfb73565f10b"}, + {file = "scipy-1.12.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:f5f00ebaf8de24d14b8449981a2842d404152774c1a1d880c901bf454cb8e2a1"}, + {file = "scipy-1.12.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e53958531a7c695ff66c2e7bb7b79560ffdc562e2051644c5576c39ff8efb563"}, + {file = "scipy-1.12.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5e32847e08da8d895ce09d108a494d9eb78974cf6de23063f93306a3e419960c"}, + {file = "scipy-1.12.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:4c1020cad92772bf44b8e4cdabc1df5d87376cb219742549ef69fc9fd86282dd"}, + {file = "scipy-1.12.0-cp310-cp310-win_amd64.whl", hash = "sha256:75ea2a144096b5e39402e2ff53a36fecfd3b960d786b7efd3c180e29c39e53f2"}, + {file = "scipy-1.12.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:408c68423f9de16cb9e602528be4ce0d6312b05001f3de61fe9ec8b1263cad08"}, + {file = "scipy-1.12.0-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:5adfad5dbf0163397beb4aca679187d24aec085343755fcdbdeb32b3679f254c"}, + {file = "scipy-1.12.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c3003652496f6e7c387b1cf63f4bb720951cfa18907e998ea551e6de51a04467"}, + {file = "scipy-1.12.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8b8066bce124ee5531d12a74b617d9ac0ea59245246410e19bca549656d9a40a"}, + {file = "scipy-1.12.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:8bee4993817e204d761dba10dbab0774ba5a8612e57e81319ea04d84945375ba"}, + {file = "scipy-1.12.0-cp311-cp311-win_amd64.whl", hash = "sha256:a24024d45ce9a675c1fb8494e8e5244efea1c7a09c60beb1eeb80373d0fecc70"}, + {file = "scipy-1.12.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:e7e76cc48638228212c747ada851ef355c2bb5e7f939e10952bc504c11f4e372"}, + {file = "scipy-1.12.0-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:f7ce148dffcd64ade37b2df9315541f9adad6efcaa86866ee7dd5db0c8f041c3"}, + {file = "scipy-1.12.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9c39f92041f490422924dfdb782527a4abddf4707616e07b021de33467f917bc"}, + {file = "scipy-1.12.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a7ebda398f86e56178c2fa94cad15bf457a218a54a35c2a7b4490b9f9cb2676c"}, + {file = "scipy-1.12.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:95e5c750d55cf518c398a8240571b0e0782c2d5a703250872f36eaf737751338"}, + {file = "scipy-1.12.0-cp312-cp312-win_amd64.whl", hash = "sha256:e646d8571804a304e1da01040d21577685ce8e2db08ac58e543eaca063453e1c"}, + {file = "scipy-1.12.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:913d6e7956c3a671de3b05ccb66b11bc293f56bfdef040583a7221d9e22a2e35"}, + {file = "scipy-1.12.0-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:bba1b0c7256ad75401c73e4b3cf09d1f176e9bd4248f0d3112170fb2ec4db067"}, + {file = "scipy-1.12.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:730badef9b827b368f351eacae2e82da414e13cf8bd5051b4bdfd720271a5371"}, + {file = "scipy-1.12.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6546dc2c11a9df6926afcbdd8a3edec28566e4e785b915e849348c6dd9f3f490"}, + {file = "scipy-1.12.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:196ebad3a4882081f62a5bf4aeb7326aa34b110e533aab23e4374fcccb0890dc"}, + {file = "scipy-1.12.0-cp39-cp39-win_amd64.whl", hash = "sha256:b360f1b6b2f742781299514e99ff560d1fe9bd1bff2712894b52abe528d1fd1e"}, + {file = "scipy-1.12.0.tar.gz", hash = "sha256:4bf5abab8a36d20193c698b0f1fc282c1d083c94723902c447e5d2f1780936a3"}, +] + +[package.dependencies] +numpy = ">=1.22.4,<1.29.0" [package.extras] dev = ["click", "cython-lint (>=0.12.2)", "doit (>=0.36.0)", "mypy", "pycodestyle", "pydevtool", "rich-click", "ruff", "types-psutil", "typing_extensions"] doc = ["jupytext", "matplotlib (>2)", "myst-nb", "numpydoc", "pooch", "pydata-sphinx-theme (==0.9.0)", "sphinx (!=4.1.0)", "sphinx-design (>=0.2.0)"] -test = ["asv", "gmpy2", "mpmath", "pooch", "pytest", "pytest-cov", "pytest-timeout", "pytest-xdist", "scikit-umfpack", "threadpoolctl"] +test = ["asv", "gmpy2", "hypothesis", "mpmath", "pooch", "pytest", "pytest-cov", "pytest-timeout", "pytest-xdist", "scikit-umfpack", "threadpoolctl"] [[package]] name = "scooby" @@ -6063,13 +6057,13 @@ cpu = ["mkl", "psutil"] [[package]] name = "seaborn" -version = "0.13.1" +version = "0.13.2" description = "Statistical data visualization" optional = false python-versions = ">=3.8" files = [ - {file = "seaborn-0.13.1-py3-none-any.whl", hash = "sha256:6baa69b6d1169ae59037971491c450c0b73332b42bd4b23570b62a546bc61cb8"}, - {file = "seaborn-0.13.1.tar.gz", hash = "sha256:bfad65e9c5989e5e1897e61bdbd2f22e62455940ca76fd49eca3ed69345b9179"}, + {file = "seaborn-0.13.2-py3-none-any.whl", hash = "sha256:636f8336facf092165e27924f223d3c62ca560b1f2bb5dff7ab7fad265361987"}, + {file = "seaborn-0.13.2.tar.gz", hash = "sha256:93e60a40988f4d65e9f4885df477e2fdaff6b73a9ded434c1ab356dd57eefff7"}, ] [package.dependencies] @@ -6194,32 +6188,37 @@ files = [ [[package]] name = "snowflake-connector-python" -version = "3.6.0" +version = "3.7.0" description = "Snowflake Connector for Python" optional = true python-versions = ">=3.8" files = [ - {file = "snowflake-connector-python-3.6.0.tar.gz", hash = "sha256:15667a918780d79da755e6a60bbf6918051854951e8f56ccdf5692283e9a8479"}, - {file = "snowflake_connector_python-3.6.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:4093b38cf9abf95c38119f0b23b07e23dc7a8689b956cd5d34975e1875741f20"}, - {file = "snowflake_connector_python-3.6.0-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:cf5a964fe01b177063f8c44d14df3a72715580bcd195788ec2822090f37330a5"}, - {file = "snowflake_connector_python-3.6.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:55a6418cec585b050e6f05404f25e62b075a3bbea587dc1f903de15640565c58"}, - {file = "snowflake_connector_python-3.6.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7c76aea92b87f6ecd604e9c934aac8a779f2e20f3be1d990d53bb5b6d87b009"}, - {file = "snowflake_connector_python-3.6.0-cp310-cp310-win_amd64.whl", hash = "sha256:9dfcf178271e892e64e4092b9e011239a066ce5de848afd2efe3f13197a9f8b3"}, - {file = "snowflake_connector_python-3.6.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:4916f9b4a0efd7c96d1fa50a157e05907b6935f91492cca7f200b43cc178a25e"}, - {file = "snowflake_connector_python-3.6.0-cp311-cp311-macosx_11_0_x86_64.whl", hash = "sha256:f15024c66db5e87d359216ec733a2974d7562aa38f3f18c8b6e65489839e00d7"}, - {file = "snowflake_connector_python-3.6.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bcbd3102f807ebbbae52b1b5683d45cd7b3dcb0eaec131233ba6b156e8d70fa4"}, - {file = "snowflake_connector_python-3.6.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7662e2de25b885abe08ab866cf7c7b026ad1af9faa39c25e2c25015ef807abe3"}, - {file = "snowflake_connector_python-3.6.0-cp311-cp311-win_amd64.whl", hash = "sha256:d1fa102f55ee166cc766aeee3f9333b17b4bede6fb088eee1e1f022df15b6d81"}, - {file = "snowflake_connector_python-3.6.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:fde1e0727e2f23c2a07b49b30e1bc0f49977f965d08ddfda10015b24a2beeb76"}, - {file = "snowflake_connector_python-3.6.0-cp38-cp38-macosx_11_0_x86_64.whl", hash = "sha256:1b51fe000c8cf6372d30b73c7136275e52788e6af47010cd1984c9fb03378e86"}, - {file = "snowflake_connector_python-3.6.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d7a11699689a19916e65794ce58dca72b8a40fe6a7eea06764931ede10b47bcc"}, - {file = "snowflake_connector_python-3.6.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d810be5b180c6f47ce9b6f989fe64b9984383e4b77e30b284a83e33f229a3a82"}, - {file = "snowflake_connector_python-3.6.0-cp38-cp38-win_amd64.whl", hash = "sha256:b5db47d4164d6b7a07c413a46f9edc4a1d687e3df44fd9d5fa89a89aecb94a8e"}, - {file = "snowflake_connector_python-3.6.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:bf8c1ad5aab5304fefa2a4178061a24c96da45e3e3db9d901621e9953e005402"}, - {file = "snowflake_connector_python-3.6.0-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:1058ab5c98cc62fde8b3f021f0a5076cb7865b5cdab8a9bccde0df88b9e91334"}, - {file = "snowflake_connector_python-3.6.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2b93f55989f80d69278e0f40a7a1c0e737806b7c0ddb0351513a752b837243e8"}, - {file = "snowflake_connector_python-3.6.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:50dd954ea5918d3242ded69225b72f701963cd9c043ee7d9ab35dc22211611c8"}, - {file = "snowflake_connector_python-3.6.0-cp39-cp39-win_amd64.whl", hash = "sha256:4ad42613b87f31441d07a8ea242f4c28ed5eb7b6e05986f9e94a7e44b96d3d1e"}, + {file = "snowflake-connector-python-3.7.0.tar.gz", hash = "sha256:b2bfaec64059307b08caadad40214d488fefb4a23fcd7553ac75f5ea758a9169"}, + {file = "snowflake_connector_python-3.7.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:f38070af24d15cd103d565b63b08c5eac3bdf72ad06ad27cd98c46359cb4bee2"}, + {file = "snowflake_connector_python-3.7.0-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:f8f3912699030291fd82d75321cda44205c9f8fb27841ffbaaf6d3dc4065b798"}, + {file = "snowflake_connector_python-3.7.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b7ac1190c6ca48297429f0fb6515b54e3fd3bceb1b72fce7b59097044a9e98e0"}, + {file = "snowflake_connector_python-3.7.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:57deaa28baa601b64c1ae5a5c75260ab1c6a22bd07a8d8c7ac785c8deb1c556e"}, + {file = "snowflake_connector_python-3.7.0-cp310-cp310-win_amd64.whl", hash = "sha256:715635ed5b6e5ef8de659fc336c1b89296fe72fdec180c40915c10df885c8082"}, + {file = "snowflake_connector_python-3.7.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d51f3a8912fcc5169731d2b42262087e8a6da20f7344dd001ed97fbdf6ff972c"}, + {file = "snowflake_connector_python-3.7.0-cp311-cp311-macosx_11_0_x86_64.whl", hash = "sha256:46bfa243875eff9c6dfe1afc26f2034b00ac6eb9f77010b2949a174c38a59722"}, + {file = "snowflake_connector_python-3.7.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7004ccfe3c16075d33b0440b4d5241a50156bbc5dcbf11dec61674d0ac830f46"}, + {file = "snowflake_connector_python-3.7.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9ee9e6a0a94e0ac1f15fa93c0f61f6e930240280bd043f61216d942e837beb7f"}, + {file = "snowflake_connector_python-3.7.0-cp311-cp311-win_amd64.whl", hash = "sha256:b545fd11c9bd200681e182cf46bb4cbc8250ca6acc41fbea749799a2b23f574f"}, + {file = "snowflake_connector_python-3.7.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:258541df8ba90201ce6f7c4ae9f59e3a9f585ed30fbbaafd207e0774104cf6dc"}, + {file = "snowflake_connector_python-3.7.0-cp312-cp312-macosx_11_0_x86_64.whl", hash = "sha256:e548642913f7d0ef9d5a35c69c7a8308cbab8fe255fdc3c9f7e18c71e52a0c2e"}, + {file = "snowflake_connector_python-3.7.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:639d09de40c014c7ab0308f622bd1d29a9b9dd05c0ced2d858da31323fa16bda"}, + {file = "snowflake_connector_python-3.7.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:da9cf62665ee47c7ec8c18ae554a31c72cacf1cef4b42d55cfbdbae4b5ddb3f2"}, + {file = "snowflake_connector_python-3.7.0-cp312-cp312-win_amd64.whl", hash = "sha256:ad1d0e339cadb5ba79d24783c39ba21a63e2159f0d3d9540da0168f97043904c"}, + {file = "snowflake_connector_python-3.7.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:3d8e4d0fad8b00b55bc99035ad2c54d9aa3ca8495f7dfcce736a961b5dbd1d9f"}, + {file = "snowflake_connector_python-3.7.0-cp38-cp38-macosx_11_0_x86_64.whl", hash = "sha256:fc3e95d4c99472444ffda35b9bbfe4cd4c775279c7eca579f1eee9d8d2ec1e2a"}, + {file = "snowflake_connector_python-3.7.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6f93a5861333c2f87ecd1fea34a0fae35c12c196e86fa75c2dd89741e83f2d82"}, + {file = "snowflake_connector_python-3.7.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9fdf0fe7d77e02949a8a2a7d365217b822bcaf2fc9541095a241116576458568"}, + {file = "snowflake_connector_python-3.7.0-cp38-cp38-win_amd64.whl", hash = "sha256:1ec29946b224d8089070477f60ffe58923433d8c2308b6403684500e85c37699"}, + {file = "snowflake_connector_python-3.7.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:4f945c512383a8b5f1d2404c40d20e0c915ba3f0ac01983f2e43987d6eecda02"}, + {file = "snowflake_connector_python-3.7.0-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:574cf5be3c61a6ea421ac9710ac791a80f6dfcc53986ab81e68d1085dad79dab"}, + {file = "snowflake_connector_python-3.7.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fb8168458e5d23a0ba4d4e0a276bbd477ddd26d35c554f2c3c64cfe29622499a"}, + {file = "snowflake_connector_python-3.7.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ecf8f520692653775f51307140d326b53a51e338d67dc522b1d376b51b12d14e"}, + {file = "snowflake_connector_python-3.7.0-cp39-cp39-win_amd64.whl", hash = "sha256:5ed928055ed40da22b2d6bdde62eee5068c352f66415e1c9aee7c45eb67d39cb"}, ] [package.dependencies] @@ -6393,17 +6392,17 @@ sqlalchemy = ">=1.0.0" [[package]] name = "sqlglot" -version = "20.8.0" +version = "20.11.0" description = "An easily customizable SQL parser and transpiler" optional = false python-versions = ">=3.7" files = [ - {file = "sqlglot-20.8.0-py3-none-any.whl", hash = "sha256:cb73b81a26da462c34b12b98cf193d679d4b5693703d309db236d9784cef60bb"}, - {file = "sqlglot-20.8.0.tar.gz", hash = "sha256:5636e97fab9efdb4a8690c0e32bbd2d657fe91eb650f10e913a56b4bd979faef"}, + {file = "sqlglot-20.11.0-py3-none-any.whl", hash = "sha256:658509272da15e90dd1c59d9ca5281d7bff2e87121f87e6f9e6541067a057c9c"}, + {file = "sqlglot-20.11.0.tar.gz", hash = "sha256:79a1510ffad1f1e4c5915751f0ed978c099e7e83cd4010ecbd471c00331b6902"}, ] [package.extras] -dev = ["autoflake", "black", "duckdb (>=0.6)", "isort", "maturin (>=1.4,<2.0)", "mypy (>=0.990)", "pandas", "pdoc", "pre-commit", "pyspark", "python-dateutil", "types-python-dateutil"] +dev = ["autoflake", "black", "duckdb (>=0.6)", "isort", "maturin (>=1.4,<2.0)", "mypy (>=0.990)", "pandas", "pdoc", "pre-commit", "pyspark", "python-dateutil", "types-python-dateutil", "typing-extensions"] rs = ["sqlglotrs (==0.1.0)"] [[package]] @@ -6604,13 +6603,13 @@ files = [ [[package]] name = "toolz" -version = "0.12.0" +version = "0.12.1" description = "List processing tools and functional utilities" optional = false -python-versions = ">=3.5" +python-versions = ">=3.7" files = [ - {file = "toolz-0.12.0-py3-none-any.whl", hash = "sha256:2059bd4148deb1884bb0eb770a3cde70e7f954cfbbdc2285f1f2de01fd21eb6f"}, - {file = "toolz-0.12.0.tar.gz", hash = "sha256:88c570861c440ee3f2f6037c4654613228ff40c93a6c25e0eba70d17282c6194"}, + {file = "toolz-0.12.1-py3-none-any.whl", hash = "sha256:d22731364c07d72eea0a0ad45bafb2c2937ab6fd38a3507bf55eae8744aa7d85"}, + {file = "toolz-0.12.1.tar.gz", hash = "sha256:ecca342664893f177a13dac0e6b41cbd8ac25a358e5f215316d43e2100224f4d"}, ] [[package]] @@ -6711,13 +6710,13 @@ tests = ["black", "httpretty (<1.1)", "isort", "pre-commit", "pytest", "pytest-r [[package]] name = "trove-classifiers" -version = "2024.1.8" +version = "2024.1.31" description = "Canonical source for classifiers on PyPI (pypi.org)." optional = false python-versions = "*" files = [ - {file = "trove-classifiers-2024.1.8.tar.gz", hash = "sha256:6e36caf430ff6485c4b57a4c6b364a13f6a898d16b9417c6c37467e59c14b05a"}, - {file = "trove_classifiers-2024.1.8-py3-none-any.whl", hash = "sha256:3c1ff4deb10149c7e39ede6e5bbc107def64362ef1ee7590ec98d71fb92f1b6a"}, + {file = "trove-classifiers-2024.1.31.tar.gz", hash = "sha256:bfdfe60bbf64985c524416afb637ecc79c558e0beb4b7f52b0039e01044b0229"}, + {file = "trove_classifiers-2024.1.31-py3-none-any.whl", hash = "sha256:854aba3358f3cf10e5c0916aa533f5a39e27aadd8ade26a54cdc2a93257e39c4"}, ] [[package]] @@ -7353,4 +7352,4 @@ visualization = ["graphviz"] [metadata] lock-version = "2.0" python-versions = "^3.9" -content-hash = "39f5035cdd3ae4bcc67b642e3f6b733f3c8956e903897c946624f39362a0b2cb" +content-hash = "51b82d38f90ea4682385f0fbdafd5613f57483482a076dd4ff1799217cffb2e6" diff --git a/pyproject.toml b/pyproject.toml index cc74e67c1a0de..f90144a22823b 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -41,7 +41,7 @@ atpublic = ">=2.3,<5" bidict = ">=0.22.1,<1" multipledispatch = ">=0.6,<2" numpy = ">=1,<2" -pandas = ">=1.2.5,<3" +pandas = ">=1.2.5,<2.1" parsy = ">=2,<3" pyarrow = ">=2,<16" pyarrow-hotfix = ">=0.4,<1" @@ -191,11 +191,11 @@ mysql = ["pymysql"] oracle = ["oracledb", "packaging"] pandas = ["regex"] polars = ["polars", "packaging"] -risingwave = ["psycopg2"] postgres = ["psycopg2"] pyspark = ["pyspark", "packaging"] snowflake = ["snowflake-connector-python", "packaging"] sqlite = ["regex"] +risingwave = ["psycopg2"] trino = ["trino"] # non-backend extras visualization = ["graphviz"]