From 8b293cba11958f40351cc0986b1d99e02f6560a8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 8 Feb 2023 20:56:47 +0100 Subject: [PATCH 01/36] Python: Add positional deletes Closes #6568 --- python/pyiceberg/io/pyarrow.py | 62 ++++++++++++++++++++++++++++-- python/pyiceberg/table/__init__.py | 21 +--------- python/tests/io/test_pyarrow.py | 32 +++++++++++++-- python/tests/table/test_init.py | 24 +----------- 4 files changed, 91 insertions(+), 48 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index a064ae3bdfdf..aa6daeeac058 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -31,6 +31,7 @@ TYPE_CHECKING, Any, Callable, + Dict, Iterable, List, Optional, @@ -72,6 +73,7 @@ OutputFile, OutputStream, ) +from pyiceberg.manifest import DataFileContent from pyiceberg.schema import ( PartnerAccessor, Schema, @@ -391,7 +393,7 @@ def visit_timestamp(self, _: TimestampType) -> pa.DataType: return pa.timestamp(unit="us") def visit_timestampz(self, _: TimestamptzType) -> pa.DataType: - return pa.timestamp(unit="us", tz="+00:00") + return pa.timestamp(unit="us", tz="UTC") def visit_string(self, _: StringType) -> pa.DataType: return pa.string() @@ -470,6 +472,17 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression: return boolean_expression_visit(expr, _ConvertToArrowExpression()) +def _read_deletes(fs: FileSystem, file_path: str) -> Dict[str, pa.ChunkedArray]: + _, path = PyArrowFileIO.parse_location(file_path) + table = pq.read_table( + source=path, pre_buffer=True, buffer_size=8 * ONE_MEGABYTE, read_dictionary=["file_path"], filesystem=fs + ) + table.unify_dictionaries() + return { + file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.columns[0].chunks[0].dictionary + } + + def _file_to_table( fs: FileSystem, task: FileScanTask, @@ -477,6 +490,7 @@ def _file_to_table( projected_schema: Schema, projected_field_ids: Set[int], case_sensitive: bool, + positional_deletes: List[pa.ChunkedArray], ) -> pa.Table: _, path = PyArrowFileIO.parse_location(task.file.file_path) @@ -515,6 +529,14 @@ def _file_to_table( if pyarrow_filter is not None: arrow_table = arrow_table.filter(pyarrow_filter) + if len(positional_deletes) > 0: + # When there are positional deletes, create a filter mask + mask = [True] * len(arrow_table) + for buffer in positional_deletes: + for pos in buffer: + mask[pos.as_py()] = False + arrow_table = arrow_table.filter(mask) + return to_requested_schema(projected_schema, file_project_schema, arrow_table) @@ -546,11 +568,45 @@ def project_table( id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType)) }.union(extract_field_ids(bound_row_filter)) + tasks_data_files: List[FileScanTask] = [] + tasks_positional_deletes: List[FileScanTask] = [] + for task in tasks: + if task.file.content == DataFileContent.DATA: + tasks_data_files.append(task) + elif task.file.content == DataFileContent.POSITION_DELETES: + tasks_positional_deletes.append(task) + elif task.file.content == DataFileContent.EQUALITY_DELETES: + raise ValueError("PyIceberg does not yet support equality deletes: https://github.com/apache/iceberg/issues/6568") + else: + raise ValueError(f"Unknown file content: {task.file.content}") + with ThreadPool() as pool: + positional_deletes_per_file: Dict[str, List[pa.ChunkedArray]] = {} + if tasks_positional_deletes: + # If there are any positional deletes, get those first + for delete_files in pool.starmap( + func=_read_deletes, + iterable=[(fs, task.file.file_path) for task in tasks_positional_deletes], + ): + for file, buffer in delete_files.items(): + positional_deletes_per_file[file] = positional_deletes_per_file.get(file, []) + [buffer] + tables = pool.starmap( func=_file_to_table, - iterable=[(fs, task, bound_row_filter, projected_schema, projected_field_ids, case_sensitive) for task in tasks], - chunksize=None, # we could use this to control how to materialize the generator of tasks (we should also make the expression above lazy) + iterable=[ + ( + fs, + task, + bound_row_filter, + projected_schema, + projected_field_ids, + case_sensitive, + positional_deletes_per_file.get(task.file.file_path, []), + ) + for task in tasks_data_files + ], + chunksize=None, + # we could use this to control how to materialize the generator of tasks (we should also make the expression above lazy) ) if len(tables) > 1: diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index dc9cc9e03643..1a69f4f7e59b 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -47,12 +47,7 @@ from pyiceberg.expressions.visitors import inclusive_projection from pyiceberg.io import FileIO from pyiceberg.io.pyarrow import project_table -from pyiceberg.manifest import ( - DataFile, - ManifestContent, - ManifestFile, - files, -) +from pyiceberg.manifest import DataFile, ManifestFile, files from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema from pyiceberg.table.metadata import TableMetadata @@ -282,21 +277,10 @@ def __init__(self, data_file: DataFile, start: Optional[int] = None, length: Opt self.length = length or data_file.file_size_in_bytes -def _check_content(file: DataFile) -> DataFile: - try: - if file.content == ManifestContent.DELETES: - raise ValueError("PyIceberg does not support deletes: https://github.com/apache/iceberg/issues/6568") - return file - except AttributeError: - # If the attribute is not there, it is a V1 record - return file - - def _open_manifest(io: FileIO, manifest: ManifestFile, partition_filter: Callable[[DataFile], bool]) -> List[FileScanTask]: all_files = files(io.new_input(manifest.manifest_path)) matching_partition_files = filter(partition_filter, all_files) - matching_partition_data_files = map(_check_content, matching_partition_files) - return [FileScanTask(file) for file in matching_partition_data_files] + return [FileScanTask(file) for file in matching_partition_files] class DataScan(TableScan["DataScan"]): @@ -352,7 +336,6 @@ def plan_files(self) -> Iterator[FileScanTask]: # step 2: filter the data files in each manifest # this filter depends on the partition spec used to write the manifest file - partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) with ThreadPool() as pool: diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index f894963d9421..64f3102baad1 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -55,11 +55,12 @@ PyArrowFile, PyArrowFileIO, _ConvertToArrowSchema, + _read_deletes, expression_to_pyarrow, project_table, schema_to_pyarrow, ) -from pyiceberg.manifest import DataFile, FileFormat +from pyiceberg.manifest import DataFile, DataFileContent, FileFormat from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema, visit from pyiceberg.table import FileScanTask, Table @@ -377,7 +378,7 @@ def test_timestamp_type_to_pyarrow() -> None: def test_timestamptz_type_to_pyarrow() -> None: iceberg_type = TimestamptzType() - assert visit(iceberg_type, _ConvertToArrowSchema()) == pa.timestamp(unit="us", tz="+00:00") + assert visit(iceberg_type, _ConvertToArrowSchema()) == pa.timestamp(unit="us", tz="UTC") def test_string_type_to_pyarrow() -> None: @@ -749,7 +750,14 @@ def project( return project_table( [ FileScanTask( - DataFile(file_path=file, file_format=FileFormat.PARQUET, partition={}, record_count=3, file_size_in_bytes=3) + DataFile( + content=DataFileContent.DATA, + file_path=file, + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ) ) for file in files ], @@ -1114,3 +1122,21 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str _ = project(schema, [file_int_str], GreaterThan("unknown_field", "1"), schema_int_str) assert "Could not find field with name unknown_field, case_sensitive=True" in str(exc_info.value) + + +@pytest.fixture +def deletes_file(tmp_path: str) -> str: + path = "s3://bucket/default.db/table/data.parquet" + table = pa.table({"file_path": [path, path, path], "pos": [19, 22, 25]}) + + deletes_file_path = f"{tmp_path}/deletes.parquet" + pq.write_table(table, deletes_file_path) + + return deletes_file_path + + +def test_read_deletes(deletes_file: str) -> None: + # None filesystem will default to a local filesystem + deletes = _read_deletes(None, deletes_file) + assert set(deletes.keys()) == {"s3://bucket/default.db/table/data.parquet"} + assert list(deletes.values())[0] == pa.chunked_array([[19, 22, 25]]) diff --git a/python/tests/table/test_init.py b/python/tests/table/test_init.py index 506c6609f7f9..3c663ceeb442 100644 --- a/python/tests/table/test_init.py +++ b/python/tests/table/test_init.py @@ -26,10 +26,9 @@ In, ) from pyiceberg.io import load_file_io -from pyiceberg.manifest import DataFile, ManifestContent from pyiceberg.partitioning import PartitionField, PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import Table, _check_content +from pyiceberg.table import Table from pyiceberg.table.metadata import TableMetadataV2 from pyiceberg.table.snapshots import ( Operation, @@ -44,7 +43,6 @@ SortOrder, ) from pyiceberg.transforms import BucketTransform, IdentityTransform -from pyiceberg.typedef import Record from pyiceberg.types import LongType, NestedField @@ -245,23 +243,3 @@ def test_table_scan_projection_unknown_column(table: Table) -> None: _ = scan.select("a").projection() assert "Could not find column: 'a'" in str(exc_info.value) - - -def test_check_content_deletes() -> None: - with pytest.raises(ValueError) as exc_info: - _check_content( - DataFile( - content=ManifestContent.DELETES, - ) - ) - assert "PyIceberg does not support deletes: https://github.com/apache/iceberg/issues/6568" in str(exc_info.value) - - -def test_check_content_data() -> None: - manifest_file = DataFile(content=ManifestContent.DATA) - assert _check_content(manifest_file) == manifest_file - - -def test_check_content_missing_attr() -> None: - r = Record(*([None] * 15)) - assert _check_content(r) == r # type: ignore From b013b5a291fa4eb79e87f76c268633cce4471da2 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sat, 18 Feb 2023 00:25:18 +0100 Subject: [PATCH 02/36] Revert unrelated changes --- python/pyiceberg/io/pyarrow.py | 2 +- python/pyiceberg/table/__init__.py | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index aa6daeeac058..41cd96e492a4 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -393,7 +393,7 @@ def visit_timestamp(self, _: TimestampType) -> pa.DataType: return pa.timestamp(unit="us") def visit_timestampz(self, _: TimestamptzType) -> pa.DataType: - return pa.timestamp(unit="us", tz="UTC") + return pa.timestamp(unit="us", tz="+00:00") def visit_string(self, _: StringType) -> pa.DataType: return pa.string() diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 6e56e455c998..8164f0da97ab 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -358,6 +358,7 @@ def plan_files(self) -> Iterator[FileScanTask]: # step 2: filter the data files in each manifest # this filter depends on the partition spec used to write the manifest file + partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) with ThreadPool() as pool: From e35ecb3240b4cacb53e808abb8d9bdb51cd594b8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 20 Feb 2023 16:34:27 +0100 Subject: [PATCH 03/36] Update based on comments --- python/pyiceberg/io/pyarrow.py | 32 +++++++++++++++++++++----------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 41cd96e492a4..a2ba6aaf807a 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -32,6 +32,7 @@ Any, Callable, Dict, + Generator, Iterable, List, Optional, @@ -393,7 +394,7 @@ def visit_timestamp(self, _: TimestampType) -> pa.DataType: return pa.timestamp(unit="us") def visit_timestampz(self, _: TimestamptzType) -> pa.DataType: - return pa.timestamp(unit="us", tz="+00:00") + return pa.timestamp(unit="us", tz="UTC") def visit_string(self, _: StringType) -> pa.DataType: return pa.string() @@ -490,7 +491,7 @@ def _file_to_table( projected_schema: Schema, projected_field_ids: Set[int], case_sensitive: bool, - positional_deletes: List[pa.ChunkedArray], + positional_deletes: Optional[pa.ChunkedArray], ) -> pa.Table: _, path = PyArrowFileIO.parse_location(task.file.file_path) @@ -529,12 +530,19 @@ def _file_to_table( if pyarrow_filter is not None: arrow_table = arrow_table.filter(pyarrow_filter) - if len(positional_deletes) > 0: + if positional_deletes is not None: # When there are positional deletes, create a filter mask - mask = [True] * len(arrow_table) - for buffer in positional_deletes: - for pos in buffer: - mask[pos.as_py()] = False + def generator() -> Generator[int, None, None]: + itr = iter(positional_deletes) + next_delete = next(itr) + for pos in range(len(arrow_table)): + if pos == next_delete: + yield True + next_delete = next(itr) + else: + yield False + + mask = pa.array(generator(), type=pa.bool_()) arrow_table = arrow_table.filter(mask) return to_requested_schema(projected_schema, file_project_schema, arrow_table) @@ -571,7 +579,7 @@ def project_table( tasks_data_files: List[FileScanTask] = [] tasks_positional_deletes: List[FileScanTask] = [] for task in tasks: - if task.file.content == DataFileContent.DATA: + if task.file.content is None or task.file.content == DataFileContent.DATA: tasks_data_files.append(task) elif task.file.content == DataFileContent.POSITION_DELETES: tasks_positional_deletes.append(task) @@ -581,7 +589,7 @@ def project_table( raise ValueError(f"Unknown file content: {task.file.content}") with ThreadPool() as pool: - positional_deletes_per_file: Dict[str, List[pa.ChunkedArray]] = {} + positional_deletes_per_file: Dict[str, pa.ChunkedArray] = {} if tasks_positional_deletes: # If there are any positional deletes, get those first for delete_files in pool.starmap( @@ -589,7 +597,9 @@ def project_table( iterable=[(fs, task.file.file_path) for task in tasks_positional_deletes], ): for file, buffer in delete_files.items(): - positional_deletes_per_file[file] = positional_deletes_per_file.get(file, []) + [buffer] + if file in positional_deletes_per_file: + raise ValueError(f"Duplicate deletes found for {file}") + positional_deletes_per_file[file] = buffer tables = pool.starmap( func=_file_to_table, @@ -601,7 +611,7 @@ def project_table( projected_schema, projected_field_ids, case_sensitive, - positional_deletes_per_file.get(task.file.file_path, []), + positional_deletes_per_file.get(task.file.file_path), ) for task in tasks_data_files ], From a0e5f0b467e51a6c763c8c0bd5dc126c70c1657e Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 20 Feb 2023 18:26:08 +0100 Subject: [PATCH 04/36] Fix false annotation --- python/pyiceberg/io/pyarrow.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index a2ba6aaf807a..163b57e0e62c 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -532,8 +532,8 @@ def _file_to_table( if positional_deletes is not None: # When there are positional deletes, create a filter mask - def generator() -> Generator[int, None, None]: - itr = iter(positional_deletes) + def generator() -> Generator[bool, None, None]: + itr = iter(positional_deletes) # type: ignore next_delete = next(itr) for pos in range(len(arrow_table)): if pos == next_delete: From aeea029744af26f6b093052bd197362a53dbd2aa Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 22 Feb 2023 11:23:13 +0100 Subject: [PATCH 05/36] WIP --- .../org/apache/iceberg/DeleteFileIndex.java | 4 +- .../org/apache/iceberg/TableMetadata.java | 4 +- .../org/apache/iceberg/util/SnapshotUtil.java | 10 ++- python/pyiceberg/expressions/visitors.py | 6 +- python/pyiceberg/io/pyarrow.py | 2 +- python/pyiceberg/table/__init__.py | 62 +++++++++++++++++-- python/pyiceberg/table/snapshots.py | 59 ++++++++++++++++-- .../catalog/integration_test_dynamodb.py | 2 +- python/tests/catalog/integration_test_glue.py | 2 +- python/tests/catalog/test_dynamodb.py | 2 +- python/tests/catalog/test_glue.py | 2 +- python/tests/table/test_snapshots.py | 33 ++++++++++ 12 files changed, 161 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index eedde21397eb..5054d7bca773 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -357,7 +357,7 @@ static Builder builderFor(FileIO io, Iterable deleteManifests) { static class Builder { private final FileIO io; private final Set deleteManifests; - private long minSequenceNumber = 0L; + private long minSequenceNumber = TableMetadata.INITIAL_SEQUENCE_NUMBER; private Map specsById = null; private Expression dataFilter = Expressions.alwaysTrue(); private Expression partitionFilter = Expressions.alwaysTrue(); @@ -412,7 +412,7 @@ Builder scanMetrics(ScanMetrics newScanMetrics) { } DeleteFileIndex build() { - // read all of the matching delete manifests in parallel and accumulate the matching files in + // read all the matching delete manifests in parallel and accumulate the matching files in // a queue Queue> deleteEntries = new ConcurrentLinkedQueue<>(); Tasks.foreach(deleteManifestReaders()) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 2cb223db0fb4..f1182af8c0d4 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -48,8 +48,8 @@ /** Metadata for a table. */ public class TableMetadata implements Serializable { - static final long INITIAL_SEQUENCE_NUMBER = 0; - static final long INVALID_SEQUENCE_NUMBER = -1; + static final long INITIAL_SEQUENCE_NUMBER = 0L; + static final long INVALID_SEQUENCE_NUMBER = -1L; static final int DEFAULT_TABLE_FORMAT_VERSION = 1; static final int SUPPORTED_TABLE_FORMAT_VERSION = 2; static final int INITIAL_SPEC_ID = 0; diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index c7d36bdc0078..fee6c41881f0 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -190,12 +190,10 @@ public static Snapshot oldestAncestorAfter(Table table, long timestampMillis) { *

This method assumes that fromSnapshotId is an ancestor of toSnapshotId. */ public static List snapshotIdsBetween(Table table, long fromSnapshotId, long toSnapshotId) { - List snapshotIds = - Lists.newArrayList( - ancestorIds( - table.snapshot(toSnapshotId), - snapshotId -> snapshotId != fromSnapshotId ? table.snapshot(snapshotId) : null)); - return snapshotIds; + return Lists.newArrayList( + ancestorIds( + table.snapshot(toSnapshotId), + snapshotId -> snapshotId != fromSnapshotId ? table.snapshot(snapshotId) : null)); } public static Iterable ancestorIdsBetween( diff --git a/python/pyiceberg/expressions/visitors.py b/python/pyiceberg/expressions/visitors.py index 81e0ebc559c9..cf4e24583108 100644 --- a/python/pyiceberg/expressions/visitors.py +++ b/python/pyiceberg/expressions/visitors.py @@ -16,6 +16,7 @@ # under the License. from abc import ABC, abstractmethod from functools import singledispatch +from multiprocessing.pool import ThreadPool from typing import ( Any, Callable, @@ -24,9 +25,11 @@ Set, Tuple, TypeVar, - Union, + Union, Dict, ) +from pyarrow.filesystem import FileSystem + from pyiceberg.conversions import from_bytes from pyiceberg.expressions import ( AlwaysFalse, @@ -986,3 +989,4 @@ def expression_to_plain_format( # In the form of expr1 ∨ expr2 ∨ ... ∨ exprN visitor = ExpressionToPlainFormat(cast_int_to_datetime) return [visit(expression, visitor) for expression in expressions] + diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 163b57e0e62c..65fe828141a8 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -74,7 +74,7 @@ OutputFile, OutputStream, ) -from pyiceberg.manifest import DataFileContent +from pyiceberg.manifest import DataFileContent, ManifestFile from pyiceberg.schema import ( PartnerAccessor, Schema, diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 8164f0da97ab..a81d426f1985 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -31,7 +31,7 @@ Optional, Tuple, TypeVar, - Union, + Union, Set, ) from pydantic import Field @@ -45,7 +45,7 @@ ) from pyiceberg.expressions.visitors import inclusive_projection from pyiceberg.io import FileIO, load_file_io -from pyiceberg.manifest import DataFile, ManifestFile, files +from pyiceberg.manifest import DataFile, ManifestFile, files, ManifestContent from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema from pyiceberg.table.metadata import TableMetadata @@ -299,10 +299,11 @@ def __init__(self, data_file: DataFile, start: Optional[int] = None, length: Opt self.length = length or data_file.file_size_in_bytes -def _open_manifest(io: FileIO, manifest: ManifestFile, partition_filter: Callable[[DataFile], bool]) -> List[FileScanTask]: - all_files = files(io.new_input(manifest.manifest_path)) - matching_partition_files = filter(partition_filter, all_files) - return [FileScanTask(file) for file in matching_partition_files] +def _open_manifest(io: FileIO, manifest: ManifestFile, partition_filter: Optional[Callable[[DataFile], bool]] = None) -> List[FileScanTask]: + result_manifests = files(io.new_input(manifest.manifest_path)) + if partition_filter is not None: + result_manifests = filter(partition_filter, result_manifests) + return [FileScanTask(file) for file in result_manifests] class DataScan(TableScan): @@ -361,6 +362,19 @@ def plan_files(self) -> Iterator[FileScanTask]: partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) + data_files = [] + delete_files = [] + + for manifest in manifests: + if manifest.content is None or manifest.content == ManifestContent.DATA: + data_files.append(manifest) + else: + delete_files.append(manifest) + + # if delete_files: + # # Start building the index + + with ThreadPool() as pool: return chain( *pool.starmap( @@ -386,3 +400,39 @@ def to_duckdb(self, table_name: str, connection: Optional[DuckDBPyConnection] = con.register(table_name, self.to_arrow()) return con + +# def _read_deletes(io: FileIO, file_path: str) -> Dict[str, pa.ChunkedArray]: +# _, path = PyArrowFileIO.parse_location(file_path) +# table = pq.read_table( +# source=path, pre_buffer=True, buffer_size=8 * ONE_MEGABYTE, read_dictionary=["file_path"], filesystem=fs +# ) +# table.unify_dictionaries() +# return { +# file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.columns[0].chunks[0].dictionary +# } + +def validate_history(table: Table, starting_snapshot_id: int, matching_operations: Set[str], content: ManifestContent, parent: Snapshot): + pass + +class DeleteFileIndex: + io: FileIO + manifests: Set[ManifestFile] + filter: BooleanExpression + case_sensitive: bool + + def __init__(self, + io: FileIO, + manifests: Set[ManifestFile], + filter: BooleanExpression, + case_sensitive: bool + ) -> None: + self.io = io + self.manifests = manifests + self.filter = filter + self.case_sensitive = case_sensitive + + with ThreadPool() as pool: + deletes = pool.starmap( + func=_open_manifest, + iterable=[(io, manifest) for manifest in manifests], + ) diff --git a/python/pyiceberg/table/snapshots.py b/python/pyiceberg/table/snapshots.py index 83dd3f66a7cf..77f58fb8ae1c 100644 --- a/python/pyiceberg/table/snapshots.py +++ b/python/pyiceberg/table/snapshots.py @@ -20,7 +20,7 @@ Dict, List, Optional, - Union, + Union, Iterable, Callable, Generator, ) from pydantic import Field, PrivateAttr, root_validator @@ -61,7 +61,8 @@ class Summary(IcebergBaseModel): _additional_properties: Dict[str, str] = PrivateAttr() @root_validator - def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> Dict[str, Dict[str, Union[str, Operation]]]: + def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> Dict[ + str, Dict[str, Union[str, Operation]]]: if operation := values["__root__"].get(OPERATION): if isinstance(operation, str): values["__root__"][OPERATION] = Operation(operation.lower()) @@ -70,11 +71,13 @@ def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> return values def __init__( - self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, **data: Any + self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, + **data: Any ) -> None: super().__init__(__root__={"operation": operation, **data} if not __root__ else __root__) self._additional_properties = { - k: v for k, v in self.__root__.items() if k != OPERATION # type: ignore # We know that they are all string, and we don't want to check + k: v for k, v in self.__root__.items() if k != OPERATION + # type: ignore # We know that they are all string, and we don't want to check } @property @@ -100,7 +103,8 @@ class Snapshot(IcebergBaseModel): parent_snapshot_id: Optional[int] = Field(alias="parent-snapshot-id") sequence_number: Optional[int] = Field(alias="sequence-number", default=None) timestamp_ms: int = Field(alias="timestamp-ms") - manifest_list: Optional[str] = Field(alias="manifest-list", description="Location of the snapshot's manifest list file") + manifest_list: Optional[str] = Field(alias="manifest-list", + description="Location of the snapshot's manifest list file") summary: Optional[Summary] = Field() schema_id: Optional[int] = Field(alias="schema-id", default=None) @@ -126,3 +130,48 @@ class MetadataLogEntry(IcebergBaseModel): class SnapshotLogEntry(IcebergBaseModel): snapshot_id: str = Field(alias="snapshot-id") timestamp_ms: int = Field(alias="timestamp-ms") + + +def ancestors_of( + latest_snapshot: Union[int, Snapshot], + lookup: Callable[[int], Optional[Snapshot]] +) -> Iterable[Snapshot]: + if isinstance(latest_snapshot, int): + start = lookup(latest_snapshot) + if start is None: + raise ValueError(f"Cannot find snapshot: {latest_snapshot}") + else: + start = latest_snapshot + + if start is not None: + def snapshot_generator() -> Generator[Snapshot, None, None]: + next = start + while next is not None: + yield next + next = lookup(next.parent_snapshot_id) + + return snapshot_generator() + else: + return [] + + + +def ancestors_between( + latest_snapshot_id: int, + oldest_snapshot_id: Optional[int], + lookup: Dict[int, Snapshot] +) -> Iterable[Snapshot]: + if oldest_snapshot_id is not None: + if latest_snapshot_id == oldest_snapshot_id: + return [] + + def lookup(snapshot_id: int) -> Optional[Snapshot]: + return lookup.get(snapshot_id) if oldest_snapshot_id != snapshot_id else None + else: + def lookup(snapshot_id: int) -> Optional[Snapshot]: + return lookup.get(snapshot_id) + + return ancestor_of( + latest_snapshot_id, + lookup + ) diff --git a/python/tests/catalog/integration_test_dynamodb.py b/python/tests/catalog/integration_test_dynamodb.py index a430175c5a9c..fa8aad2e05fb 100644 --- a/python/tests/catalog/integration_test_dynamodb.py +++ b/python/tests/catalog/integration_test_dynamodb.py @@ -20,6 +20,7 @@ import boto3 import pytest from botocore.exceptions import ClientError +from tests.conftest import clean_up, get_bucket_name, get_s3_path from pyiceberg.catalog import Catalog from pyiceberg.catalog.dynamodb import DynamoDbCatalog @@ -31,7 +32,6 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema -from tests.conftest import clean_up, get_bucket_name, get_s3_path # The number of tables/databases used in list_table/namespace test LIST_TEST_NUMBER = 2 diff --git a/python/tests/catalog/integration_test_glue.py b/python/tests/catalog/integration_test_glue.py index 62870390b3ee..db58ca7895e7 100644 --- a/python/tests/catalog/integration_test_glue.py +++ b/python/tests/catalog/integration_test_glue.py @@ -20,6 +20,7 @@ import boto3 import pytest from botocore.exceptions import ClientError +from tests.conftest import clean_up, get_bucket_name, get_s3_path from pyiceberg.catalog import Catalog from pyiceberg.catalog.glue import GlueCatalog @@ -31,7 +32,6 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema -from tests.conftest import clean_up, get_bucket_name, get_s3_path # The number of tables/databases used in list_table/namespace test LIST_TEST_NUMBER = 2 diff --git a/python/tests/catalog/test_dynamodb.py b/python/tests/catalog/test_dynamodb.py index 582cb034e8aa..73282436bf49 100644 --- a/python/tests/catalog/test_dynamodb.py +++ b/python/tests/catalog/test_dynamodb.py @@ -18,6 +18,7 @@ import pytest from moto import mock_dynamodb +from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX from pyiceberg.catalog import METADATA_LOCATION, TABLE_TYPE from pyiceberg.catalog.dynamodb import ( @@ -38,7 +39,6 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema -from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX @mock_dynamodb diff --git a/python/tests/catalog/test_glue.py b/python/tests/catalog/test_glue.py index 0fef1fad5fdd..9bd84ce74abd 100644 --- a/python/tests/catalog/test_glue.py +++ b/python/tests/catalog/test_glue.py @@ -18,6 +18,7 @@ import pytest from moto import mock_glue +from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX from pyiceberg.catalog.glue import GlueCatalog from pyiceberg.exceptions import ( @@ -30,7 +31,6 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema -from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX @mock_glue diff --git a/python/tests/table/test_snapshots.py b/python/tests/table/test_snapshots.py index b119ae9945e3..a727413dd603 100644 --- a/python/tests/table/test_snapshots.py +++ b/python/tests/table/test_snapshots.py @@ -17,6 +17,10 @@ # pylint:disable=redefined-outer-name,eval-used import pytest +from pyiceberg.partitioning import PartitionSpec +from pyiceberg.schema import Schema +from pyiceberg.table import Table, TableMetadata, SortOrder +from pyiceberg.table.metadata import TableMetadataV2 from pyiceberg.table.snapshots import Operation, Snapshot, Summary @@ -45,6 +49,32 @@ def snapshot_with_properties() -> Snapshot: schema_id=3, ) +@pytest.fixture +def table_with_snapshots(table_schema_simple: Schema) -> Table: + return TableMetadataV2( + location="s3://bucket/warehouse/table/", + table_uuid="9c12d441-03fe-4693-9a96-a0705ddf69c1", + last_updated_ms=1602638573590, + last_column_id=3, + schemas=[ + table_schema_simple + ], + current_schema_id=1, + last_partition_id=1000, + properties={"owner": "javaberg"}, + partition_specs=[PartitionSpec()], + default_spec_id=0, + current_snapshot_id=None, + snapshots=[], + snapshot_log=[], + metadata_log=[], + sort_orders=[SortOrder(order_id=0)], + default_sort_order_id=0, + refs={}, + format_version=2, + last_sequence_number=0, + ) + def test_serialize_summary() -> None: assert Summary(Operation.APPEND).json() == """{"operation": "append"}""" @@ -119,3 +149,6 @@ def test_snapshot_with_properties_repr(snapshot_with_properties: Snapshot) -> No == """Snapshot(snapshot_id=25, parent_snapshot_id=19, sequence_number=200, timestamp_ms=1602638573590, manifest_list='s3:/a/b/c.avro', summary=Summary(Operation.APPEND, **{'foo': 'bar'}), schema_id=3)""" ) assert snapshot_with_properties == eval(repr(snapshot_with_properties)) + +def test_ancestors_of(table_with_snapshots: TableMetadata) -> None: + pass \ No newline at end of file From a7927648bbdf70cc8e5e2b5784ec9363950e1838 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 24 Feb 2023 17:33:02 +0100 Subject: [PATCH 06/36] WIP --- python/pyiceberg/expressions/visitors.py | 4 +- python/pyiceberg/table/__init__.py | 51 ++- python/pyiceberg/table/snapshots.py | 380 +++++++++++++++++++++-- python/tests/table/test_snapshots.py | 330 ++++++++++++++++++-- 4 files changed, 691 insertions(+), 74 deletions(-) diff --git a/python/pyiceberg/expressions/visitors.py b/python/pyiceberg/expressions/visitors.py index cf4e24583108..a3be1f18977e 100644 --- a/python/pyiceberg/expressions/visitors.py +++ b/python/pyiceberg/expressions/visitors.py @@ -25,7 +25,8 @@ Set, Tuple, TypeVar, - Union, Dict, + Union, + Dict, ) from pyarrow.filesystem import FileSystem @@ -989,4 +990,3 @@ def expression_to_plain_format( # In the form of expr1 ∨ expr2 ∨ ... ∨ exprN visitor = ExpressionToPlainFormat(cast_int_to_datetime) return [visit(expression, visitor) for expression in expressions] - diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index a81d426f1985..35af0bbc1b95 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -31,7 +31,8 @@ Optional, Tuple, TypeVar, - Union, Set, + Union, + Set, ) from pydantic import Field @@ -106,7 +107,14 @@ def scan( def schema(self) -> Schema: """Return the schema for this table""" - return next(schema for schema in self.metadata.schemas if schema.schema_id == self.metadata.current_schema_id) + return self.schema_by_id(self.metadata.current_schema_id) + + def schema_by_id(self, schema_id: int) -> Schema: + """Return the schema for this table by ID""" + try: + return next(schema for schema in self.metadata.schemas if schema.schema_id == schema_id) + except StopIteration: + raise ValueError(f"Schema id not found in table: {schema_id}") def schemas(self) -> Dict[int, Schema]: """Return a dict of the schema of this table""" @@ -140,12 +148,22 @@ def current_snapshot(self) -> Optional[Snapshot]: return self.snapshot_by_id(snapshot_id) return None - def snapshot_by_id(self, snapshot_id: int) -> Optional[Snapshot]: - """Get the snapshot of this table with the given id, or None if there is no matching snapshot.""" + def snapshot_by_id(self, snapshot_id: int) -> Snapshot: + """Get the snapshot of this table with the given id. + + Args: + snapshot_id: The id of the snapshot to lookup in the table + + Returns: + The snapshot that corresponds to snapshot_id + + Raises: + ValueError: If the snapshot cannot be found + """ try: return next(snapshot for snapshot in self.metadata.snapshots if snapshot.snapshot_id == snapshot_id) except StopIteration: - return None + raise ValueError(f"Snapshot id not found in table: {snapshot_id}") def snapshot_by_name(self, name: str) -> Optional[Snapshot]: """Returns the snapshot referenced by the given name or null if no such reference exists.""" @@ -299,7 +317,9 @@ def __init__(self, data_file: DataFile, start: Optional[int] = None, length: Opt self.length = length or data_file.file_size_in_bytes -def _open_manifest(io: FileIO, manifest: ManifestFile, partition_filter: Optional[Callable[[DataFile], bool]] = None) -> List[FileScanTask]: +def _open_manifest( + io: FileIO, manifest: ManifestFile, partition_filter: Optional[Callable[[DataFile], bool]] = None +) -> List[FileScanTask]: result_manifests = files(io.new_input(manifest.manifest_path)) if partition_filter is not None: result_manifests = filter(partition_filter, result_manifests) @@ -374,7 +394,6 @@ def plan_files(self) -> Iterator[FileScanTask]: # if delete_files: # # Start building the index - with ThreadPool() as pool: return chain( *pool.starmap( @@ -401,6 +420,7 @@ def to_duckdb(self, table_name: str, connection: Optional[DuckDBPyConnection] = return con + # def _read_deletes(io: FileIO, file_path: str) -> Dict[str, pa.ChunkedArray]: # _, path = PyArrowFileIO.parse_location(file_path) # table = pq.read_table( @@ -411,21 +431,20 @@ def to_duckdb(self, table_name: str, connection: Optional[DuckDBPyConnection] = # file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.columns[0].chunks[0].dictionary # } -def validate_history(table: Table, starting_snapshot_id: int, matching_operations: Set[str], content: ManifestContent, parent: Snapshot): + +def validate_history( + table: Table, starting_snapshot_id: int, matching_operations: Set[str], content: ManifestContent, parent: Snapshot +): pass + class DeleteFileIndex: io: FileIO manifests: Set[ManifestFile] filter: BooleanExpression case_sensitive: bool - def __init__(self, - io: FileIO, - manifests: Set[ManifestFile], - filter: BooleanExpression, - case_sensitive: bool - ) -> None: + def __init__(self, io: FileIO, manifests: Set[ManifestFile], filter: BooleanExpression, case_sensitive: bool) -> None: self.io = io self.manifests = manifests self.filter = filter @@ -433,6 +452,6 @@ def __init__(self, with ThreadPool() as pool: deletes = pool.starmap( - func=_open_manifest, - iterable=[(io, manifest) for manifest in manifests], + func=_open_manifest, + iterable=[(io, manifest) for manifest in manifests], ) diff --git a/python/pyiceberg/table/snapshots.py b/python/pyiceberg/table/snapshots.py index 77f58fb8ae1c..d757c9e3a400 100644 --- a/python/pyiceberg/table/snapshots.py +++ b/python/pyiceberg/table/snapshots.py @@ -20,15 +20,23 @@ Dict, List, Optional, - Union, Iterable, Callable, Generator, + Union, + Iterable, + Callable, + Generator, + TYPE_CHECKING, ) from pydantic import Field, PrivateAttr, root_validator from pyiceberg.io import FileIO -from pyiceberg.manifest import ManifestFile, read_manifest_list +from pyiceberg.manifest import ManifestFile, read_manifest_list, DataFile +from pyiceberg.schema import Schema from pyiceberg.typedef import IcebergBaseModel +if TYPE_CHECKING: + from pyiceberg.table import Table + OPERATION = "operation" @@ -61,8 +69,7 @@ class Summary(IcebergBaseModel): _additional_properties: Dict[str, str] = PrivateAttr() @root_validator - def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> Dict[ - str, Dict[str, Union[str, Operation]]]: + def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> Dict[str, Dict[str, Union[str, Operation]]]: if operation := values["__root__"].get(OPERATION): if isinstance(operation, str): values["__root__"][OPERATION] = Operation(operation.lower()) @@ -71,12 +78,13 @@ def check_operation(cls, values: Dict[str, Dict[str, Union[str, Operation]]]) -> return values def __init__( - self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, - **data: Any + self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, **data: Any ) -> None: super().__init__(__root__={"operation": operation, **data} if not __root__ else __root__) self._additional_properties = { - k: v for k, v in self.__root__.items() if k != OPERATION + k: v + for k, v in self.__root__.items() + if k != OPERATION # type: ignore # We know that they are all string, and we don't want to check } @@ -103,8 +111,7 @@ class Snapshot(IcebergBaseModel): parent_snapshot_id: Optional[int] = Field(alias="parent-snapshot-id") sequence_number: Optional[int] = Field(alias="sequence-number", default=None) timestamp_ms: int = Field(alias="timestamp-ms") - manifest_list: Optional[str] = Field(alias="manifest-list", - description="Location of the snapshot's manifest list file") + manifest_list: Optional[str] = Field(alias="manifest-list", description="Location of the snapshot's manifest list file") summary: Optional[Summary] = Field() schema_id: Optional[int] = Field(alias="schema-id", default=None) @@ -121,6 +128,10 @@ def manifests(self, io: FileIO) -> List[ManifestFile]: return list(read_manifest_list(file)) return [] + def added_data_files(self, io: FileIO) -> List[DataFile]: + for manifest in self.manifests(io): + yield from [entry.data_file for entry in manifest.fetch_manifest_entry(io)] + class MetadataLogEntry(IcebergBaseModel): metadata_file: str = Field(alias="metadata-file") @@ -128,14 +139,198 @@ class MetadataLogEntry(IcebergBaseModel): class SnapshotLogEntry(IcebergBaseModel): - snapshot_id: str = Field(alias="snapshot-id") + snapshot_id: int = Field(alias="snapshot-id") timestamp_ms: int = Field(alias="timestamp-ms") -def ancestors_of( - latest_snapshot: Union[int, Snapshot], - lookup: Callable[[int], Optional[Snapshot]] -) -> Iterable[Snapshot]: +def is_ancestor_of(table: "Table", snapshot_id: int, ancestor_snapshot_id: int) -> bool: + """ + Returns whether ancestor_snapshot_id is an ancestor of snapshot_id using the given lookup function. + + Args: + table: The table + snapshot_id: The snapshot id of the snapshot + ancestor_snapshot_id: The snapshot id of the possible ancestor + + Returns: + True if it is an ancestor or not + """ + snapshots = ancestors_of(snapshot_id, table.snapshot_by_id) + for snapshot in snapshots: + if snapshot.snapshot_id == ancestor_snapshot_id: + return True + return False + + +def is_parent_ancestor_of(table: "Table", snapshot_id: int, ancestor_parent_snapshot_id: int) -> bool: + """ + Returns whether some ancestor of snapshot_id has parent_id matches ancestor_parent_snapshot_id + + Args: + table: The table + snapshot_id: The snapshot id of the snapshot + ancestor_parent_snapshot_id: The snapshot id of the possible parent ancestor + + Returns: + True if there is an ancestor with a parent + """ + snapshots = ancestors_of(snapshot_id, table.snapshot_by_id) + for snapshot in snapshots: + if snapshot.parent_snapshot_id == ancestor_parent_snapshot_id: + return True + return False + + +def current_ancestors(table: "Table") -> Iterable[Snapshot]: + """ + Returns an iterable that traverses the table's snapshots from the current to the last known ancestor. + + Args: + table: The table + + Returns: + An iterable of all the ancestors + """ + return ancestors_of(table.current_snapshot(), table.snapshot_by_id) + + +def current_ancestor_ids(table: "Table") -> Iterable[int]: + """ + Return the snapshot IDs for the ancestors of the current table state. + + Ancestor IDs are ordered by commit time, descending. The first ID is + the current snapshot, followed by its parent, and so on. + + Args: + table: The table + + Returns: + An iterable of all the snapshot IDs + """ + return ancestor_ids(table.current_snapshot(), table.snapshot_by_id) + + +def oldest_ancestor(table: "Table") -> Optional[Snapshot]: + """ + Traverses the history of the table's current snapshot and finds the oldest Snapshot. + + Args: + table: The table + + Returns: + None if there is no current snapshot in the table, else the oldest Snapshot. + """ + oldest_snapshot: Optional[Snapshot] = None + + for snapshot in current_ancestors(table): + oldest_snapshot = snapshot + + return oldest_snapshot + + +def oldest_ancestor_of(table: "Table", snapshot_id: int) -> Optional[Snapshot]: + """ + Traverses the history and finds the oldest ancestor of the specified snapshot. + + Oldest ancestor is defined as the ancestor snapshot whose parent is null or has been + expired. If the specified snapshot has no parent or parent has been expired, the specified + snapshot itself is returned. + + Args: + table: The table + snapshot_id: the ID of the snapshot to find the oldest ancestor + + Returns: + None if there is no current snapshot in the table, else the oldest Snapshot. + """ + oldest_snapshot: Optional[Snapshot] = None + + for snapshot in ancestors_of(snapshot_id, table.snapshot_by_id): + oldest_snapshot = snapshot + + return oldest_snapshot + + +def oldest_ancestor_after(table: "Table", timestamp_ms: int) -> Snapshot: + """ + Looks up the snapshot after a given point in time + + Args: + table: The table + timestamp_ms: The timestamp in millis since the Unix epoch + + Returns: + The snapshot after the given point in time + + Raises: + ValueError: When there is no snapshot older than the given time + """ + last_snapshot = table.current_snapshot() + + for snapshot in current_ancestors(table): + if snapshot.timestamp_ms < timestamp_ms: + return last_snapshot + elif snapshot.timestamp_ms == timestamp_ms: + return snapshot + + last_snapshot = snapshot + + if last_snapshot is not None and last_snapshot.parent_snapshot_id is None: + return last_snapshot + + raise ValueError(f"Cannot find snapshot older than: {timestamp_ms}") + + +def snapshots_ids_between(table: "Table", from_snapshot_id: int, to_snapshot_id: int) -> Iterable[int]: + """ + Returns list of snapshot ids in the range - (fromSnapshotId, toSnapshotId] + + This method assumes that fromSnapshotId is an ancestor of toSnapshotId. + + Args: + table: The table + from_snapshot_id: The starting snapshot ID + to_snapshot_id: The ending snapshot ID + + Returns: + The list of snapshot IDs that are between the given snapshot IDs + """ + + def lookup(snapshot_id: int) -> Optional[Snapshot]: + return table.snapshot_by_id(snapshot_id) if snapshot_id != from_snapshot_id else None + + return ancestor_ids(table.snapshot_by_id(snapshot_id=to_snapshot_id), lookup) + + +def ancestor_ids(latest_snapshot: Union[int, Snapshot], lookup: Callable[[int], Optional[Snapshot]]) -> Iterable[int]: + """ + Returns list of the snapshot IDs of the ancestors + + Args: + latest_snapshot: The snapshot where to start from + lookup: Lookup function to get the snapshot for the snapshot ID + + Returns: + The list of snapshot IDs that are ancestor of the given snapshot + """ + + def get_id(snapshot: Snapshot) -> int: + return snapshot.snapshot_id + + return map(get_id, ancestors_of(latest_snapshot, lookup)) + + +def ancestors_of(latest_snapshot: Union[int, Snapshot], lookup: Callable[[int], Optional[Snapshot]]) -> Iterable[Snapshot]: + """ + Returns list of snapshot that are ancestor of the given snapshot + + Args: + latest_snapshot: The snapshot where to start from + lookup: Lookup function to get the snapshot for the snapshot ID + + Returns: + The list of snapshots that are ancestor of the given snapshot + """ if isinstance(latest_snapshot, int): start = lookup(latest_snapshot) if start is None: @@ -144,34 +339,159 @@ def ancestors_of( start = latest_snapshot if start is not None: + def snapshot_generator() -> Generator[Snapshot, None, None]: - next = start + next = start.snapshot_id + # https://github.com/apache/iceberg/issues/6930 + # next = start.parent_snapshot_id while next is not None: - yield next - next = lookup(next.parent_snapshot_id) + if snap := lookup(next): + yield snap + next = snap.parent_snapshot_id + else: + break return snapshot_generator() else: return [] - def ancestors_between( - latest_snapshot_id: int, - oldest_snapshot_id: Optional[int], - lookup: Dict[int, Snapshot] + latest_snapshot_id: int, oldest_snapshot_id: Optional[int], lookup: Callable[[int], Snapshot] ) -> Iterable[Snapshot]: + """ + Returns list of snapshot that are ancestor between two IDs + + Args: + latest_snapshot_id: The latest snapshot + oldest_snapshot_id: The oldest snapshot + lookup: Lookup function to get the snapshot for the snapshot ID + + Returns: + The list of snapshots that are ancestor between the two IDs + """ if oldest_snapshot_id is not None: if latest_snapshot_id == oldest_snapshot_id: return [] - def lookup(snapshot_id: int) -> Optional[Snapshot]: - return lookup.get(snapshot_id) if oldest_snapshot_id != snapshot_id else None - else: - def lookup(snapshot_id: int) -> Optional[Snapshot]: - return lookup.get(snapshot_id) + def lookup_callable(snapshot_id: int) -> Optional[Snapshot]: + return lookup(snapshot_id) if oldest_snapshot_id != snapshot_id else None + + return ancestors_of(latest_snapshot_id, lookup_callable) + return ancestors_of(latest_snapshot_id, lookup) + + +def new_files( + base_snapshot_id: int, latest_snapshot_id: int, lookup: Callable[[int], Snapshot], io: FileIO +) -> Iterable[DataFile]: + """ + Returns list of DataFiles that are added along the way + + Args: + base_snapshot_id: The latest snapshot + latest_snapshot_id: The oldest snapshot + lookup: Lookup function to get the snapshot for the snapshot ID + io: FileIO to fetch files + + Returns: + List of DataFiles that are added along the way + """ + added_files = [] + + last_snapshot: Optional[Snapshot] = None + + for snapshot in ancestors_of(latest_snapshot_id, lookup): + last_snapshot = snapshot + + if snapshot.snapshot_id == base_snapshot_id: + return added_files - return ancestor_of( - latest_snapshot_id, - lookup - ) + added_files += list(snapshot.added_data_files(io)) + + if last_snapshot.snapshot_id != base_snapshot_id: + raise ValueError( + f"Cannot determine history between read snapshot {base_snapshot_id} and the last known ancestor {last_snapshot.snapshot_id}" + ) + + return added_files + + +def snapshot_after(table: "Table", snapshot_id: int) -> Snapshot: + """Traverses the history of the table's current snapshot + and finds the snapshot with the given snapshot id as its parent. + + Args: + table: The table + snapshot_id: The snapshot ID + + Returns: + the snapshot for which the given snapshot is the parent + + Raises: + ValueError: When the snapshot isn't found + """ + for current in current_ancestors(table): + if current.parent_snapshot_id == snapshot_id: + return current + + raise ValueError(f"Cannot find snapshot after {snapshot_id}: not an ancestor of table's current snapshot") + + +def snapshot_id_as_of_time(table: "Table", timestamp_ms: int) -> int: + """ + Returns the ID of the most recent snapshot for the table as of the timestamp. + + Args: + table: The table + timestamp_ms: the timestamp in millis since the Unix epoch + + Returns: + The snapshot ID + + Raises: + ValueError: When the snapshot id cannot be found or there are no snapshots older + than the given timestamp. + """ + snapshot_id = None + + for snapshot in current_ancestors(table): + if snapshot.timestamp_ms <= timestamp_ms: + snapshot_id = snapshot.snapshot_id + + if snapshot_id is None: + raise ValueError(f"Cannot find a snapshot older than: {timestamp_ms}") + + return snapshot_id + + +def schema_for(table: "Table", snapshot_id: int) -> Schema: + """Returns the schema of the table for the specified snapshot. + + Args: + table: The table + snapshot_id: The snapshot ID that will match with the schema + + Returns: + The schema of the snapshot, if available, otherwise the table schema + """ + snapshot = table.snapshot_by_id(snapshot_id) + schema_id = snapshot.schema_id + + if schema_id is not None: + return table.schema_by_id(schema_id) + + return table.schema() + + +def schema_as_of_time(table: "Table", timestamp_ms: int) -> Schema: + """Returns the schema of the table for the specified snapshot. + + Args: + table: The table + timestamp_ms: The timestamp in millis since the Unix epoch + + Returns: + The schema of the snapshot, if available, otherwise the table schema + """ + snapshot_id = snapshot_id_as_of_time(table, timestamp_ms) + return schema_for(table, snapshot_id) diff --git a/python/tests/table/test_snapshots.py b/python/tests/table/test_snapshots.py index a727413dd603..4176cacf1ece 100644 --- a/python/tests/table/test_snapshots.py +++ b/python/tests/table/test_snapshots.py @@ -17,11 +17,30 @@ # pylint:disable=redefined-outer-name,eval-used import pytest +from pyiceberg.io import load_file_io +from pyiceberg.manifest import DataFile from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import Table, TableMetadata, SortOrder +from pyiceberg.table import Table, SortOrder from pyiceberg.table.metadata import TableMetadataV2 -from pyiceberg.table.snapshots import Operation, Snapshot, Summary +from pyiceberg.table.refs import SnapshotRefType, SnapshotRef +from pyiceberg.table.snapshots import ( + Operation, + Snapshot, + Summary, + is_parent_ancestor_of, + is_ancestor_of, + current_ancestors, + current_ancestor_ids, + oldest_ancestor, + oldest_ancestor_of, + oldest_ancestor_after, + snapshots_ids_between, + ancestor_ids, + ancestors_between, + new_files, snapshot_after, snapshot_id_as_of_time, schema_for, schema_as_of_time, +) +from pyiceberg.typedef import Record @pytest.fixture @@ -49,30 +68,76 @@ def snapshot_with_properties() -> Snapshot: schema_id=3, ) + +SNAPSHOT_A_ID = 0xA +SNAPSHOT_A_TIME = 1677222221000 + +SNAPSHOT_B_ID = 0xB +SNAPSHOT_B_TIME = 1677222222000 + +SNAPSHOT_C_ID = 0xC +SNAPSHOT_C_TIME = 1677222223000 + +SNAPSHOT_D_ID = 0xD +SNAPSHOT_D_TIME = 1677222224000 + + @pytest.fixture def table_with_snapshots(table_schema_simple: Schema) -> Table: - return TableMetadataV2( - location="s3://bucket/warehouse/table/", - table_uuid="9c12d441-03fe-4693-9a96-a0705ddf69c1", - last_updated_ms=1602638573590, - last_column_id=3, - schemas=[ - table_schema_simple - ], - current_schema_id=1, - last_partition_id=1000, - properties={"owner": "javaberg"}, - partition_specs=[PartitionSpec()], - default_spec_id=0, - current_snapshot_id=None, - snapshots=[], - snapshot_log=[], - metadata_log=[], - sort_orders=[SortOrder(order_id=0)], - default_sort_order_id=0, - refs={}, - format_version=2, - last_sequence_number=0, + return Table( + identifier=("database", "table"), + metadata=TableMetadataV2( + location="s3://bucket/warehouse/table/", + table_uuid="9c12d441-03fe-4693-9a96-a0705ddf69c1", + last_updated_ms=1602638573590, + last_column_id=3, + schemas=[table_schema_simple], + current_schema_id=1, + last_partition_id=1000, + properties={"owner": "javaberg"}, + partition_specs=[PartitionSpec()], + default_spec_id=0, + current_snapshot_id=SNAPSHOT_D_ID, + snapshots=[ + Snapshot( + snapshot_id=SNAPSHOT_A_ID, + parent_snapshot_id=None, + sequence_number=1, + timestamp_ms=SNAPSHOT_A_TIME, + manifest_list="s3://bucket/table/snapshot-a.avro", + ), + Snapshot( + snapshot_id=SNAPSHOT_B_ID, + parent_snapshot_id=SNAPSHOT_A_ID, + sequence_number=2, + timestamp_ms=SNAPSHOT_B_TIME, + manifest_list="s3://bucket/table/snapshot-b.avro", + ), + Snapshot( + snapshot_id=SNAPSHOT_D_ID, + parent_snapshot_id=SNAPSHOT_B_ID, + sequence_number=3, + timestamp_ms=SNAPSHOT_D_TIME, + manifest_list="s3://bucket/table/snapshot-c.avro", + ), + Snapshot( + snapshot_id=SNAPSHOT_C_ID, + parent_snapshot_id=SNAPSHOT_A_ID, + sequence_number=4, + timestamp_ms=SNAPSHOT_C_TIME, + manifest_list="s3://bucket/table/snapshot1.avro", + ), + ], + snapshot_log=[], + metadata_log=[], + sort_orders=[SortOrder(order_id=0)], + default_sort_order_id=0, + refs={"b1": SnapshotRef(snapshot_id=SNAPSHOT_C_ID, snapshot_ref_type=SnapshotRefType.BRANCH)}, + format_version=2, + last_sequence_number=0, + ), + metadata_location="s3://bucket/table/metadata.json", + io=None, ) @@ -150,5 +215,218 @@ def test_snapshot_with_properties_repr(snapshot_with_properties: Snapshot) -> No ) assert snapshot_with_properties == eval(repr(snapshot_with_properties)) -def test_ancestors_of(table_with_snapshots: TableMetadata) -> None: - pass \ No newline at end of file + +def test_is_ancestor_of(table_with_snapshots: Table): + assert is_ancestor_of(table_with_snapshots, SNAPSHOT_B_ID, SNAPSHOT_A_ID) + assert not is_ancestor_of(table_with_snapshots, SNAPSHOT_C_ID, SNAPSHOT_B_ID) + + +def test_is_parent_ancestor_of(table_with_snapshots: Table): + assert is_parent_ancestor_of(table_with_snapshots, SNAPSHOT_B_ID, SNAPSHOT_A_ID) + assert not is_parent_ancestor_of(table_with_snapshots, SNAPSHOT_C_ID, SNAPSHOT_B_ID) + + +def test_current_ancestors(table_with_snapshots: Table): + assert list(current_ancestors(table_with_snapshots)) == [ + Snapshot( + snapshot_id=13, + parent_snapshot_id=11, + sequence_number=3, + timestamp_ms=1677222224000, + manifest_list="s3://bucket/table/snapshot-c.avro", + summary=None, + schema_id=None, + ), + Snapshot( + snapshot_id=11, + parent_snapshot_id=10, + sequence_number=2, + timestamp_ms=1677222222000, + manifest_list="s3://bucket/table/snapshot-b.avro", + summary=None, + schema_id=None, + ), + Snapshot( + snapshot_id=10, + parent_snapshot_id=None, + sequence_number=1, + timestamp_ms=1677222221000, + manifest_list="s3://bucket/table/snapshot-a.avro", + summary=None, + schema_id=None, + ), + ] + + +def test_current_ancestors_ids(table_with_snapshots: Table): + assert list(current_ancestor_ids(table_with_snapshots)) == [13, 11, 10] + + +def test_oldest_ancestor(table_with_snapshots: Table): + assert oldest_ancestor(table_with_snapshots) == Snapshot( + snapshot_id=10, + parent_snapshot_id=None, + sequence_number=1, + timestamp_ms=1677222221000, + manifest_list="s3://bucket/table/snapshot-a.avro", + summary=None, + schema_id=None, + ) + + +def test_oldest_ancestor_of(table_with_snapshots: Table): + assert oldest_ancestor_of(table_with_snapshots, SNAPSHOT_B_ID) == Snapshot( + snapshot_id=10, + parent_snapshot_id=None, + sequence_number=1, + timestamp_ms=1677222221000, + manifest_list="s3://bucket/table/snapshot-a.avro", + summary=None, + schema_id=None, + ) + + +def test_oldest_ancestor_after(table_with_snapshots: Table): + assert oldest_ancestor_after(table_with_snapshots, SNAPSHOT_B_TIME) == Snapshot( + snapshot_id=11, + parent_snapshot_id=10, + sequence_number=2, + timestamp_ms=1677222222000, + manifest_list="s3://bucket/table/snapshot-b.avro", + summary=None, + schema_id=None, + ) + + +def test_snapshots_ids_between(table_with_snapshots: Table): + assert list(snapshots_ids_between(table_with_snapshots, SNAPSHOT_A_ID, SNAPSHOT_D_ID)) == [SNAPSHOT_D_ID, SNAPSHOT_B_ID] + + +def test_ancestor_ids(table_with_snapshots: Table): + assert list(ancestor_ids(SNAPSHOT_B_ID, table_with_snapshots.snapshot_by_id)) == [SNAPSHOT_B_ID, SNAPSHOT_A_ID] + + +def test_ancestors_between(table_with_snapshots: Table): + assert list(ancestors_between(SNAPSHOT_D_ID, SNAPSHOT_A_ID, table_with_snapshots.snapshot_by_id)) == [ + Snapshot(snapshot_id=13, parent_snapshot_id=11, sequence_number=3, timestamp_ms=1677222224000, + manifest_list='s3://bucket/table/snapshot-c.avro', summary=None, schema_id=None) + , + Snapshot( + snapshot_id=11, + parent_snapshot_id=10, + sequence_number=2, + timestamp_ms=1677222222000, + manifest_list="s3://bucket/table/snapshot-b.avro", + summary=None, + schema_id=None, + ), + ] + + assert list(ancestors_between(SNAPSHOT_D_ID, None, table_with_snapshots.snapshot_by_id)) == [ + Snapshot( + snapshot_id=13, + parent_snapshot_id=11, + sequence_number=3, + timestamp_ms=1677222224000, + manifest_list="s3://bucket/table/snapshot-c.avro", + summary=None, + schema_id=None, + ), + Snapshot( + snapshot_id=11, + parent_snapshot_id=10, + sequence_number=2, + timestamp_ms=1677222222000, + manifest_list="s3://bucket/table/snapshot-b.avro", + summary=None, + schema_id=None, + ), + Snapshot( + snapshot_id=10, + parent_snapshot_id=None, + sequence_number=1, + timestamp_ms=1677222221000, + manifest_list="s3://bucket/table/snapshot-a.avro", + summary=None, + schema_id=None, + ), + ] + + +def test_new_files(table_schema_simple: Schema, generated_manifest_file_file: str): + io = load_file_io({}) + table = Table( + identifier=("database", "table"), + metadata=TableMetadataV2( + location="s3://bucket/warehouse/table/", + table_uuid="9c12d441-03fe-4693-9a96-a0705ddf69c1", + last_updated_ms=1602638573590, + last_column_id=3, + schemas=[table_schema_simple], + current_schema_id=1, + last_partition_id=1000, + properties={"owner": "javaberg"}, + partition_specs=[PartitionSpec()], + default_spec_id=0, + current_snapshot_id=SNAPSHOT_D_ID, + snapshots=[ + Snapshot( + snapshot_id=SNAPSHOT_A_ID, + parent_snapshot_id=None, + sequence_number=1, + timestamp_ms=SNAPSHOT_A_TIME, + manifest_list=generated_manifest_file_file, + ), + Snapshot( + snapshot_id=SNAPSHOT_B_ID, + parent_snapshot_id=SNAPSHOT_A_ID, + sequence_number=2, + timestamp_ms=SNAPSHOT_B_TIME, + manifest_list=generated_manifest_file_file, + ), + Snapshot( + snapshot_id=SNAPSHOT_D_ID, + parent_snapshot_id=SNAPSHOT_B_ID, + sequence_number=3, + timestamp_ms=SNAPSHOT_D_TIME, + manifest_list=generated_manifest_file_file, + ), + Snapshot( + snapshot_id=SNAPSHOT_C_ID, + parent_snapshot_id=SNAPSHOT_A_ID, + sequence_number=4, + timestamp_ms=SNAPSHOT_C_TIME, + manifest_list=generated_manifest_file_file, + ), + ], + snapshot_log=[], + metadata_log=[], + sort_orders=[SortOrder(order_id=0)], + default_sort_order_id=0, + refs={"b1": SnapshotRef(snapshot_id=SNAPSHOT_C_ID, snapshot_ref_type=SnapshotRefType.BRANCH)}, + format_version=2, + last_sequence_number=0, + ), + metadata_location="s3://bucket/table/metadata.json", + io=io, + ) + + files = list(new_files(SNAPSHOT_A_ID, SNAPSHOT_D_ID, table.snapshot_by_id, io)) + + assert len(files) == 4 + assert files[0].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" + assert files[1].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" + assert files[2].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" + assert files[3].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" + +def test_snapshot_after(table_with_snapshots: Table): + assert snapshot_after(table_with_snapshots, SNAPSHOT_B_ID).snapshot_id == SNAPSHOT_D_ID + +def test_snapshot_id_as_of_time(table_with_snapshots: Table): + assert snapshot_id_as_of_time(table_with_snapshots, SNAPSHOT_B_TIME) == SNAPSHOT_A_ID + +def test_schema_for(table_with_snapshots: Table, table_schema_simple: Schema): + assert schema_for(table_with_snapshots, SNAPSHOT_A_ID) == table_schema_simple + +def test_schema_as_of_time(table_with_snapshots: Table, table_schema_simple: Schema): + assert schema_as_of_time(table_with_snapshots, SNAPSHOT_B_TIME) == table_schema_simple From 3fd53c9059426f6978db2067840dd8d9ff8c2c6f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Feb 2023 12:16:03 +0100 Subject: [PATCH 07/36] Cleanup --- .../org/apache/iceberg/DeleteFileIndex.java | 4 +- .../org/apache/iceberg/TableMetadata.java | 4 +- .../org/apache/iceberg/util/SnapshotUtil.java | 10 +++-- python/pyiceberg/expressions/visitors.py | 4 -- python/pyiceberg/manifest.py | 6 +++ python/pyiceberg/table/__init__.py | 43 +------------------ python/tests/table/test_snapshots.py | 42 ++++++++++++++---- 7 files changed, 52 insertions(+), 61 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index 5054d7bca773..eedde21397eb 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -357,7 +357,7 @@ static Builder builderFor(FileIO io, Iterable deleteManifests) { static class Builder { private final FileIO io; private final Set deleteManifests; - private long minSequenceNumber = TableMetadata.INITIAL_SEQUENCE_NUMBER; + private long minSequenceNumber = 0L; private Map specsById = null; private Expression dataFilter = Expressions.alwaysTrue(); private Expression partitionFilter = Expressions.alwaysTrue(); @@ -412,7 +412,7 @@ Builder scanMetrics(ScanMetrics newScanMetrics) { } DeleteFileIndex build() { - // read all the matching delete manifests in parallel and accumulate the matching files in + // read all of the matching delete manifests in parallel and accumulate the matching files in // a queue Queue> deleteEntries = new ConcurrentLinkedQueue<>(); Tasks.foreach(deleteManifestReaders()) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index f1182af8c0d4..2cb223db0fb4 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -48,8 +48,8 @@ /** Metadata for a table. */ public class TableMetadata implements Serializable { - static final long INITIAL_SEQUENCE_NUMBER = 0L; - static final long INVALID_SEQUENCE_NUMBER = -1L; + static final long INITIAL_SEQUENCE_NUMBER = 0; + static final long INVALID_SEQUENCE_NUMBER = -1; static final int DEFAULT_TABLE_FORMAT_VERSION = 1; static final int SUPPORTED_TABLE_FORMAT_VERSION = 2; static final int INITIAL_SPEC_ID = 0; diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index fee6c41881f0..969d8ed1a9d2 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -190,10 +190,12 @@ public static Snapshot oldestAncestorAfter(Table table, long timestampMillis) { *

This method assumes that fromSnapshotId is an ancestor of toSnapshotId. */ public static List snapshotIdsBetween(Table table, long fromSnapshotId, long toSnapshotId) { - return Lists.newArrayList( - ancestorIds( - table.snapshot(toSnapshotId), - snapshotId -> snapshotId != fromSnapshotId ? table.snapshot(snapshotId) : null)); + List snapshotIds = + Lists.newArrayList( + ancestorIds( + table.snapshot(toSnapshotId), + snapshotId -> snapshotId != fromSnapshotId ? table.snapshot(snapshotId) : null)); + return snapshotIds; } public static Iterable ancestorIdsBetween( diff --git a/python/pyiceberg/expressions/visitors.py b/python/pyiceberg/expressions/visitors.py index a3be1f18977e..81e0ebc559c9 100644 --- a/python/pyiceberg/expressions/visitors.py +++ b/python/pyiceberg/expressions/visitors.py @@ -16,7 +16,6 @@ # under the License. from abc import ABC, abstractmethod from functools import singledispatch -from multiprocessing.pool import ThreadPool from typing import ( Any, Callable, @@ -26,11 +25,8 @@ Tuple, TypeVar, Union, - Dict, ) -from pyarrow.filesystem import FileSystem - from pyiceberg.conversions import from_bytes from pyiceberg.expressions import ( AlwaysFalse, diff --git a/python/pyiceberg/manifest.py b/python/pyiceberg/manifest.py index 757f3bd016da..ffd4ea9577c0 100644 --- a/python/pyiceberg/manifest.py +++ b/python/pyiceberg/manifest.py @@ -258,6 +258,12 @@ class ManifestFile(Record): def __init__(self, *data: Any, **named_data: Any) -> None: super().__init__(*data, **{"struct": MANIFEST_FILE_SCHEMA.as_struct(), **named_data}) + def has_added_files(self) -> bool: + return self.added_files_count is None or self.added_files_count > 0 + + def has_existing_files(self) -> bool: + return self.existing_files_count is None or self.existing_files_count > 0 + def fetch_manifest_entry(self, io: FileIO) -> List[ManifestEntry]: file = io.new_input(self.manifest_path) return list(read_manifest_entry(file)) diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 35af0bbc1b95..6913c605fe2f 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -46,7 +46,7 @@ ) from pyiceberg.expressions.visitors import inclusive_projection from pyiceberg.io import FileIO, load_file_io -from pyiceberg.manifest import DataFile, ManifestFile, files, ManifestContent +from pyiceberg.manifest import DataFile, ManifestFile, files, ManifestContent, ManifestEntry from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema from pyiceberg.table.metadata import TableMetadata @@ -388,12 +388,9 @@ def plan_files(self) -> Iterator[FileScanTask]: for manifest in manifests: if manifest.content is None or manifest.content == ManifestContent.DATA: data_files.append(manifest) - else: + elif manifest.content == ManifestContent.DELETES and (manifest.has_added_files() or manifest.has_existing_files()): delete_files.append(manifest) - # if delete_files: - # # Start building the index - with ThreadPool() as pool: return chain( *pool.starmap( @@ -419,39 +416,3 @@ def to_duckdb(self, table_name: str, connection: Optional[DuckDBPyConnection] = con.register(table_name, self.to_arrow()) return con - - -# def _read_deletes(io: FileIO, file_path: str) -> Dict[str, pa.ChunkedArray]: -# _, path = PyArrowFileIO.parse_location(file_path) -# table = pq.read_table( -# source=path, pre_buffer=True, buffer_size=8 * ONE_MEGABYTE, read_dictionary=["file_path"], filesystem=fs -# ) -# table.unify_dictionaries() -# return { -# file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.columns[0].chunks[0].dictionary -# } - - -def validate_history( - table: Table, starting_snapshot_id: int, matching_operations: Set[str], content: ManifestContent, parent: Snapshot -): - pass - - -class DeleteFileIndex: - io: FileIO - manifests: Set[ManifestFile] - filter: BooleanExpression - case_sensitive: bool - - def __init__(self, io: FileIO, manifests: Set[ManifestFile], filter: BooleanExpression, case_sensitive: bool) -> None: - self.io = io - self.manifests = manifests - self.filter = filter - self.case_sensitive = case_sensitive - - with ThreadPool() as pool: - deletes = pool.starmap( - func=_open_manifest, - iterable=[(io, manifest) for manifest in manifests], - ) diff --git a/python/tests/table/test_snapshots.py b/python/tests/table/test_snapshots.py index 4176cacf1ece..886951a7c799 100644 --- a/python/tests/table/test_snapshots.py +++ b/python/tests/table/test_snapshots.py @@ -38,7 +38,11 @@ snapshots_ids_between, ancestor_ids, ancestors_between, - new_files, snapshot_after, snapshot_id_as_of_time, schema_for, schema_as_of_time, + new_files, + snapshot_after, + snapshot_id_as_of_time, + schema_for, + schema_as_of_time, ) from pyiceberg.typedef import Record @@ -308,9 +312,15 @@ def test_ancestor_ids(table_with_snapshots: Table): def test_ancestors_between(table_with_snapshots: Table): assert list(ancestors_between(SNAPSHOT_D_ID, SNAPSHOT_A_ID, table_with_snapshots.snapshot_by_id)) == [ - Snapshot(snapshot_id=13, parent_snapshot_id=11, sequence_number=3, timestamp_ms=1677222224000, - manifest_list='s3://bucket/table/snapshot-c.avro', summary=None, schema_id=None) - , + Snapshot( + snapshot_id=13, + parent_snapshot_id=11, + sequence_number=3, + timestamp_ms=1677222224000, + manifest_list="s3://bucket/table/snapshot-c.avro", + summary=None, + schema_id=None, + ), Snapshot( snapshot_id=11, parent_snapshot_id=10, @@ -414,19 +424,35 @@ def test_new_files(table_schema_simple: Schema, generated_manifest_file_file: st files = list(new_files(SNAPSHOT_A_ID, SNAPSHOT_D_ID, table.snapshot_by_id, io)) assert len(files) == 4 - assert files[0].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" - assert files[1].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" - assert files[2].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" - assert files[3].file_path == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" + assert ( + files[0].file_path + == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" + ) + assert ( + files[1].file_path + == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" + ) + assert ( + files[2].file_path + == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" + ) + assert ( + files[3].file_path + == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" + ) + def test_snapshot_after(table_with_snapshots: Table): assert snapshot_after(table_with_snapshots, SNAPSHOT_B_ID).snapshot_id == SNAPSHOT_D_ID + def test_snapshot_id_as_of_time(table_with_snapshots: Table): assert snapshot_id_as_of_time(table_with_snapshots, SNAPSHOT_B_TIME) == SNAPSHOT_A_ID + def test_schema_for(table_with_snapshots: Table, table_schema_simple: Schema): assert schema_for(table_with_snapshots, SNAPSHOT_A_ID) == table_schema_simple + def test_schema_as_of_time(table_with_snapshots: Table, table_schema_simple: Schema): assert schema_as_of_time(table_with_snapshots, SNAPSHOT_B_TIME) == table_schema_simple From fcead795bd703f16902eeaa0d0612b8dbd4de63d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Feb 2023 13:36:49 +0100 Subject: [PATCH 08/36] Cleanup --- python/pyiceberg/io/pyarrow.py | 21 +- python/pyiceberg/table/__init__.py | 8 +- python/pyiceberg/table/snapshots.py | 375 +----------------- .../catalog/integration_test_dynamodb.py | 2 +- python/tests/catalog/integration_test_glue.py | 2 +- python/tests/catalog/test_dynamodb.py | 2 +- python/tests/catalog/test_glue.py | 2 +- python/tests/table/test_snapshots.py | 326 +-------------- python/tests/test_fokko.py | 18 + 9 files changed, 39 insertions(+), 717 deletions(-) create mode 100644 python/tests/test_fokko.py diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 65fe828141a8..f97c7986a177 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -74,7 +74,7 @@ OutputFile, OutputStream, ) -from pyiceberg.manifest import DataFileContent, ManifestFile +from pyiceberg.manifest import DataFileContent from pyiceberg.schema import ( PartnerAccessor, Schema, @@ -491,7 +491,7 @@ def _file_to_table( projected_schema: Schema, projected_field_ids: Set[int], case_sensitive: bool, - positional_deletes: Optional[pa.ChunkedArray], + positional_deletes: Optional[Set[int]], ) -> pa.Table: _, path = PyArrowFileIO.parse_location(task.file.file_path) @@ -533,14 +533,8 @@ def _file_to_table( if positional_deletes is not None: # When there are positional deletes, create a filter mask def generator() -> Generator[bool, None, None]: - itr = iter(positional_deletes) # type: ignore - next_delete = next(itr) for pos in range(len(arrow_table)): - if pos == next_delete: - yield True - next_delete = next(itr) - else: - yield False + yield pos in positional_deletes # type: ignore mask = pa.array(generator(), type=pa.bool_()) arrow_table = arrow_table.filter(mask) @@ -589,17 +583,18 @@ def project_table( raise ValueError(f"Unknown file content: {task.file.content}") with ThreadPool() as pool: - positional_deletes_per_file: Dict[str, pa.ChunkedArray] = {} + positional_deletes_per_file: Dict[str, Set[int]] = {} if tasks_positional_deletes: # If there are any positional deletes, get those first for delete_files in pool.starmap( func=_read_deletes, iterable=[(fs, task.file.file_path) for task in tasks_positional_deletes], ): - for file, buffer in delete_files.items(): + for file, array in delete_files.items(): if file in positional_deletes_per_file: - raise ValueError(f"Duplicate deletes found for {file}") - positional_deletes_per_file[file] = buffer + positional_deletes_per_file[file] |= set(array) + else: + positional_deletes_per_file[file] = set(array) tables = pool.starmap( func=_file_to_table, diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 6913c605fe2f..a02facc8952a 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -32,7 +32,6 @@ Tuple, TypeVar, Union, - Set, ) from pydantic import Field @@ -46,7 +45,12 @@ ) from pyiceberg.expressions.visitors import inclusive_projection from pyiceberg.io import FileIO, load_file_io -from pyiceberg.manifest import DataFile, ManifestFile, files, ManifestContent, ManifestEntry +from pyiceberg.manifest import ( + DataFile, + ManifestContent, + ManifestFile, + files, +) from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema from pyiceberg.table.metadata import TableMetadata diff --git a/python/pyiceberg/table/snapshots.py b/python/pyiceberg/table/snapshots.py index d757c9e3a400..0783a12f9ae7 100644 --- a/python/pyiceberg/table/snapshots.py +++ b/python/pyiceberg/table/snapshots.py @@ -21,22 +21,14 @@ List, Optional, Union, - Iterable, - Callable, - Generator, - TYPE_CHECKING, ) from pydantic import Field, PrivateAttr, root_validator from pyiceberg.io import FileIO -from pyiceberg.manifest import ManifestFile, read_manifest_list, DataFile -from pyiceberg.schema import Schema +from pyiceberg.manifest import ManifestFile, read_manifest_list from pyiceberg.typedef import IcebergBaseModel -if TYPE_CHECKING: - from pyiceberg.table import Table - OPERATION = "operation" @@ -81,12 +73,7 @@ def __init__( self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, **data: Any ) -> None: super().__init__(__root__={"operation": operation, **data} if not __root__ else __root__) - self._additional_properties = { - k: v - for k, v in self.__root__.items() - if k != OPERATION - # type: ignore # We know that they are all string, and we don't want to check - } + self._additional_properties = {k: v for k, v in self.__root__.items() if k != OPERATION} # type: ignore # We know that they are all string, and we don't want to check @property def operation(self) -> Operation: @@ -128,10 +115,6 @@ def manifests(self, io: FileIO) -> List[ManifestFile]: return list(read_manifest_list(file)) return [] - def added_data_files(self, io: FileIO) -> List[DataFile]: - for manifest in self.manifests(io): - yield from [entry.data_file for entry in manifest.fetch_manifest_entry(io)] - class MetadataLogEntry(IcebergBaseModel): metadata_file: str = Field(alias="metadata-file") @@ -141,357 +124,3 @@ class MetadataLogEntry(IcebergBaseModel): class SnapshotLogEntry(IcebergBaseModel): snapshot_id: int = Field(alias="snapshot-id") timestamp_ms: int = Field(alias="timestamp-ms") - - -def is_ancestor_of(table: "Table", snapshot_id: int, ancestor_snapshot_id: int) -> bool: - """ - Returns whether ancestor_snapshot_id is an ancestor of snapshot_id using the given lookup function. - - Args: - table: The table - snapshot_id: The snapshot id of the snapshot - ancestor_snapshot_id: The snapshot id of the possible ancestor - - Returns: - True if it is an ancestor or not - """ - snapshots = ancestors_of(snapshot_id, table.snapshot_by_id) - for snapshot in snapshots: - if snapshot.snapshot_id == ancestor_snapshot_id: - return True - return False - - -def is_parent_ancestor_of(table: "Table", snapshot_id: int, ancestor_parent_snapshot_id: int) -> bool: - """ - Returns whether some ancestor of snapshot_id has parent_id matches ancestor_parent_snapshot_id - - Args: - table: The table - snapshot_id: The snapshot id of the snapshot - ancestor_parent_snapshot_id: The snapshot id of the possible parent ancestor - - Returns: - True if there is an ancestor with a parent - """ - snapshots = ancestors_of(snapshot_id, table.snapshot_by_id) - for snapshot in snapshots: - if snapshot.parent_snapshot_id == ancestor_parent_snapshot_id: - return True - return False - - -def current_ancestors(table: "Table") -> Iterable[Snapshot]: - """ - Returns an iterable that traverses the table's snapshots from the current to the last known ancestor. - - Args: - table: The table - - Returns: - An iterable of all the ancestors - """ - return ancestors_of(table.current_snapshot(), table.snapshot_by_id) - - -def current_ancestor_ids(table: "Table") -> Iterable[int]: - """ - Return the snapshot IDs for the ancestors of the current table state. - - Ancestor IDs are ordered by commit time, descending. The first ID is - the current snapshot, followed by its parent, and so on. - - Args: - table: The table - - Returns: - An iterable of all the snapshot IDs - """ - return ancestor_ids(table.current_snapshot(), table.snapshot_by_id) - - -def oldest_ancestor(table: "Table") -> Optional[Snapshot]: - """ - Traverses the history of the table's current snapshot and finds the oldest Snapshot. - - Args: - table: The table - - Returns: - None if there is no current snapshot in the table, else the oldest Snapshot. - """ - oldest_snapshot: Optional[Snapshot] = None - - for snapshot in current_ancestors(table): - oldest_snapshot = snapshot - - return oldest_snapshot - - -def oldest_ancestor_of(table: "Table", snapshot_id: int) -> Optional[Snapshot]: - """ - Traverses the history and finds the oldest ancestor of the specified snapshot. - - Oldest ancestor is defined as the ancestor snapshot whose parent is null or has been - expired. If the specified snapshot has no parent or parent has been expired, the specified - snapshot itself is returned. - - Args: - table: The table - snapshot_id: the ID of the snapshot to find the oldest ancestor - - Returns: - None if there is no current snapshot in the table, else the oldest Snapshot. - """ - oldest_snapshot: Optional[Snapshot] = None - - for snapshot in ancestors_of(snapshot_id, table.snapshot_by_id): - oldest_snapshot = snapshot - - return oldest_snapshot - - -def oldest_ancestor_after(table: "Table", timestamp_ms: int) -> Snapshot: - """ - Looks up the snapshot after a given point in time - - Args: - table: The table - timestamp_ms: The timestamp in millis since the Unix epoch - - Returns: - The snapshot after the given point in time - - Raises: - ValueError: When there is no snapshot older than the given time - """ - last_snapshot = table.current_snapshot() - - for snapshot in current_ancestors(table): - if snapshot.timestamp_ms < timestamp_ms: - return last_snapshot - elif snapshot.timestamp_ms == timestamp_ms: - return snapshot - - last_snapshot = snapshot - - if last_snapshot is not None and last_snapshot.parent_snapshot_id is None: - return last_snapshot - - raise ValueError(f"Cannot find snapshot older than: {timestamp_ms}") - - -def snapshots_ids_between(table: "Table", from_snapshot_id: int, to_snapshot_id: int) -> Iterable[int]: - """ - Returns list of snapshot ids in the range - (fromSnapshotId, toSnapshotId] - - This method assumes that fromSnapshotId is an ancestor of toSnapshotId. - - Args: - table: The table - from_snapshot_id: The starting snapshot ID - to_snapshot_id: The ending snapshot ID - - Returns: - The list of snapshot IDs that are between the given snapshot IDs - """ - - def lookup(snapshot_id: int) -> Optional[Snapshot]: - return table.snapshot_by_id(snapshot_id) if snapshot_id != from_snapshot_id else None - - return ancestor_ids(table.snapshot_by_id(snapshot_id=to_snapshot_id), lookup) - - -def ancestor_ids(latest_snapshot: Union[int, Snapshot], lookup: Callable[[int], Optional[Snapshot]]) -> Iterable[int]: - """ - Returns list of the snapshot IDs of the ancestors - - Args: - latest_snapshot: The snapshot where to start from - lookup: Lookup function to get the snapshot for the snapshot ID - - Returns: - The list of snapshot IDs that are ancestor of the given snapshot - """ - - def get_id(snapshot: Snapshot) -> int: - return snapshot.snapshot_id - - return map(get_id, ancestors_of(latest_snapshot, lookup)) - - -def ancestors_of(latest_snapshot: Union[int, Snapshot], lookup: Callable[[int], Optional[Snapshot]]) -> Iterable[Snapshot]: - """ - Returns list of snapshot that are ancestor of the given snapshot - - Args: - latest_snapshot: The snapshot where to start from - lookup: Lookup function to get the snapshot for the snapshot ID - - Returns: - The list of snapshots that are ancestor of the given snapshot - """ - if isinstance(latest_snapshot, int): - start = lookup(latest_snapshot) - if start is None: - raise ValueError(f"Cannot find snapshot: {latest_snapshot}") - else: - start = latest_snapshot - - if start is not None: - - def snapshot_generator() -> Generator[Snapshot, None, None]: - next = start.snapshot_id - # https://github.com/apache/iceberg/issues/6930 - # next = start.parent_snapshot_id - while next is not None: - if snap := lookup(next): - yield snap - next = snap.parent_snapshot_id - else: - break - - return snapshot_generator() - else: - return [] - - -def ancestors_between( - latest_snapshot_id: int, oldest_snapshot_id: Optional[int], lookup: Callable[[int], Snapshot] -) -> Iterable[Snapshot]: - """ - Returns list of snapshot that are ancestor between two IDs - - Args: - latest_snapshot_id: The latest snapshot - oldest_snapshot_id: The oldest snapshot - lookup: Lookup function to get the snapshot for the snapshot ID - - Returns: - The list of snapshots that are ancestor between the two IDs - """ - if oldest_snapshot_id is not None: - if latest_snapshot_id == oldest_snapshot_id: - return [] - - def lookup_callable(snapshot_id: int) -> Optional[Snapshot]: - return lookup(snapshot_id) if oldest_snapshot_id != snapshot_id else None - - return ancestors_of(latest_snapshot_id, lookup_callable) - return ancestors_of(latest_snapshot_id, lookup) - - -def new_files( - base_snapshot_id: int, latest_snapshot_id: int, lookup: Callable[[int], Snapshot], io: FileIO -) -> Iterable[DataFile]: - """ - Returns list of DataFiles that are added along the way - - Args: - base_snapshot_id: The latest snapshot - latest_snapshot_id: The oldest snapshot - lookup: Lookup function to get the snapshot for the snapshot ID - io: FileIO to fetch files - - Returns: - List of DataFiles that are added along the way - """ - added_files = [] - - last_snapshot: Optional[Snapshot] = None - - for snapshot in ancestors_of(latest_snapshot_id, lookup): - last_snapshot = snapshot - - if snapshot.snapshot_id == base_snapshot_id: - return added_files - - added_files += list(snapshot.added_data_files(io)) - - if last_snapshot.snapshot_id != base_snapshot_id: - raise ValueError( - f"Cannot determine history between read snapshot {base_snapshot_id} and the last known ancestor {last_snapshot.snapshot_id}" - ) - - return added_files - - -def snapshot_after(table: "Table", snapshot_id: int) -> Snapshot: - """Traverses the history of the table's current snapshot - and finds the snapshot with the given snapshot id as its parent. - - Args: - table: The table - snapshot_id: The snapshot ID - - Returns: - the snapshot for which the given snapshot is the parent - - Raises: - ValueError: When the snapshot isn't found - """ - for current in current_ancestors(table): - if current.parent_snapshot_id == snapshot_id: - return current - - raise ValueError(f"Cannot find snapshot after {snapshot_id}: not an ancestor of table's current snapshot") - - -def snapshot_id_as_of_time(table: "Table", timestamp_ms: int) -> int: - """ - Returns the ID of the most recent snapshot for the table as of the timestamp. - - Args: - table: The table - timestamp_ms: the timestamp in millis since the Unix epoch - - Returns: - The snapshot ID - - Raises: - ValueError: When the snapshot id cannot be found or there are no snapshots older - than the given timestamp. - """ - snapshot_id = None - - for snapshot in current_ancestors(table): - if snapshot.timestamp_ms <= timestamp_ms: - snapshot_id = snapshot.snapshot_id - - if snapshot_id is None: - raise ValueError(f"Cannot find a snapshot older than: {timestamp_ms}") - - return snapshot_id - - -def schema_for(table: "Table", snapshot_id: int) -> Schema: - """Returns the schema of the table for the specified snapshot. - - Args: - table: The table - snapshot_id: The snapshot ID that will match with the schema - - Returns: - The schema of the snapshot, if available, otherwise the table schema - """ - snapshot = table.snapshot_by_id(snapshot_id) - schema_id = snapshot.schema_id - - if schema_id is not None: - return table.schema_by_id(schema_id) - - return table.schema() - - -def schema_as_of_time(table: "Table", timestamp_ms: int) -> Schema: - """Returns the schema of the table for the specified snapshot. - - Args: - table: The table - timestamp_ms: The timestamp in millis since the Unix epoch - - Returns: - The schema of the snapshot, if available, otherwise the table schema - """ - snapshot_id = snapshot_id_as_of_time(table, timestamp_ms) - return schema_for(table, snapshot_id) diff --git a/python/tests/catalog/integration_test_dynamodb.py b/python/tests/catalog/integration_test_dynamodb.py index fa8aad2e05fb..a430175c5a9c 100644 --- a/python/tests/catalog/integration_test_dynamodb.py +++ b/python/tests/catalog/integration_test_dynamodb.py @@ -20,7 +20,6 @@ import boto3 import pytest from botocore.exceptions import ClientError -from tests.conftest import clean_up, get_bucket_name, get_s3_path from pyiceberg.catalog import Catalog from pyiceberg.catalog.dynamodb import DynamoDbCatalog @@ -32,6 +31,7 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema +from tests.conftest import clean_up, get_bucket_name, get_s3_path # The number of tables/databases used in list_table/namespace test LIST_TEST_NUMBER = 2 diff --git a/python/tests/catalog/integration_test_glue.py b/python/tests/catalog/integration_test_glue.py index db58ca7895e7..62870390b3ee 100644 --- a/python/tests/catalog/integration_test_glue.py +++ b/python/tests/catalog/integration_test_glue.py @@ -20,7 +20,6 @@ import boto3 import pytest from botocore.exceptions import ClientError -from tests.conftest import clean_up, get_bucket_name, get_s3_path from pyiceberg.catalog import Catalog from pyiceberg.catalog.glue import GlueCatalog @@ -32,6 +31,7 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema +from tests.conftest import clean_up, get_bucket_name, get_s3_path # The number of tables/databases used in list_table/namespace test LIST_TEST_NUMBER = 2 diff --git a/python/tests/catalog/test_dynamodb.py b/python/tests/catalog/test_dynamodb.py index 73282436bf49..582cb034e8aa 100644 --- a/python/tests/catalog/test_dynamodb.py +++ b/python/tests/catalog/test_dynamodb.py @@ -18,7 +18,6 @@ import pytest from moto import mock_dynamodb -from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX from pyiceberg.catalog import METADATA_LOCATION, TABLE_TYPE from pyiceberg.catalog.dynamodb import ( @@ -39,6 +38,7 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema +from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX @mock_dynamodb diff --git a/python/tests/catalog/test_glue.py b/python/tests/catalog/test_glue.py index 9bd84ce74abd..0fef1fad5fdd 100644 --- a/python/tests/catalog/test_glue.py +++ b/python/tests/catalog/test_glue.py @@ -18,7 +18,6 @@ import pytest from moto import mock_glue -from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX from pyiceberg.catalog.glue import GlueCatalog from pyiceberg.exceptions import ( @@ -31,6 +30,7 @@ TableAlreadyExistsError, ) from pyiceberg.schema import Schema +from tests.conftest import BUCKET_NAME, TABLE_METADATA_LOCATION_REGEX @mock_glue diff --git a/python/tests/table/test_snapshots.py b/python/tests/table/test_snapshots.py index 886951a7c799..7ff7ed549231 100644 --- a/python/tests/table/test_snapshots.py +++ b/python/tests/table/test_snapshots.py @@ -17,34 +17,7 @@ # pylint:disable=redefined-outer-name,eval-used import pytest -from pyiceberg.io import load_file_io -from pyiceberg.manifest import DataFile -from pyiceberg.partitioning import PartitionSpec -from pyiceberg.schema import Schema -from pyiceberg.table import Table, SortOrder -from pyiceberg.table.metadata import TableMetadataV2 -from pyiceberg.table.refs import SnapshotRefType, SnapshotRef -from pyiceberg.table.snapshots import ( - Operation, - Snapshot, - Summary, - is_parent_ancestor_of, - is_ancestor_of, - current_ancestors, - current_ancestor_ids, - oldest_ancestor, - oldest_ancestor_of, - oldest_ancestor_after, - snapshots_ids_between, - ancestor_ids, - ancestors_between, - new_files, - snapshot_after, - snapshot_id_as_of_time, - schema_for, - schema_as_of_time, -) -from pyiceberg.typedef import Record +from pyiceberg.table.snapshots import Operation, Snapshot, Summary @pytest.fixture @@ -86,65 +59,6 @@ def snapshot_with_properties() -> Snapshot: SNAPSHOT_D_TIME = 1677222224000 -@pytest.fixture -def table_with_snapshots(table_schema_simple: Schema) -> Table: - return Table( - identifier=("database", "table"), - metadata=TableMetadataV2( - location="s3://bucket/warehouse/table/", - table_uuid="9c12d441-03fe-4693-9a96-a0705ddf69c1", - last_updated_ms=1602638573590, - last_column_id=3, - schemas=[table_schema_simple], - current_schema_id=1, - last_partition_id=1000, - properties={"owner": "javaberg"}, - partition_specs=[PartitionSpec()], - default_spec_id=0, - current_snapshot_id=SNAPSHOT_D_ID, - snapshots=[ - Snapshot( - snapshot_id=SNAPSHOT_A_ID, - parent_snapshot_id=None, - sequence_number=1, - timestamp_ms=SNAPSHOT_A_TIME, - manifest_list="s3://bucket/table/snapshot-a.avro", - ), - Snapshot( - snapshot_id=SNAPSHOT_B_ID, - parent_snapshot_id=SNAPSHOT_A_ID, - sequence_number=2, - timestamp_ms=SNAPSHOT_B_TIME, - manifest_list="s3://bucket/table/snapshot-b.avro", - ), - Snapshot( - snapshot_id=SNAPSHOT_D_ID, - parent_snapshot_id=SNAPSHOT_B_ID, - sequence_number=3, - timestamp_ms=SNAPSHOT_D_TIME, - manifest_list="s3://bucket/table/snapshot-c.avro", - ), - Snapshot( - snapshot_id=SNAPSHOT_C_ID, - parent_snapshot_id=SNAPSHOT_A_ID, - sequence_number=4, - timestamp_ms=SNAPSHOT_C_TIME, - manifest_list="s3://bucket/table/snapshot1.avro", - ), - ], - snapshot_log=[], - metadata_log=[], - sort_orders=[SortOrder(order_id=0)], - default_sort_order_id=0, - refs={"b1": SnapshotRef(snapshot_id=SNAPSHOT_C_ID, snapshot_ref_type=SnapshotRefType.BRANCH)}, - format_version=2, - last_sequence_number=0, - ), - metadata_location="s3://bucket/table/metadata.json", - io=None, - ) - - def test_serialize_summary() -> None: assert Summary(Operation.APPEND).json() == """{"operation": "append"}""" @@ -218,241 +132,3 @@ def test_snapshot_with_properties_repr(snapshot_with_properties: Snapshot) -> No == """Snapshot(snapshot_id=25, parent_snapshot_id=19, sequence_number=200, timestamp_ms=1602638573590, manifest_list='s3:/a/b/c.avro', summary=Summary(Operation.APPEND, **{'foo': 'bar'}), schema_id=3)""" ) assert snapshot_with_properties == eval(repr(snapshot_with_properties)) - - -def test_is_ancestor_of(table_with_snapshots: Table): - assert is_ancestor_of(table_with_snapshots, SNAPSHOT_B_ID, SNAPSHOT_A_ID) - assert not is_ancestor_of(table_with_snapshots, SNAPSHOT_C_ID, SNAPSHOT_B_ID) - - -def test_is_parent_ancestor_of(table_with_snapshots: Table): - assert is_parent_ancestor_of(table_with_snapshots, SNAPSHOT_B_ID, SNAPSHOT_A_ID) - assert not is_parent_ancestor_of(table_with_snapshots, SNAPSHOT_C_ID, SNAPSHOT_B_ID) - - -def test_current_ancestors(table_with_snapshots: Table): - assert list(current_ancestors(table_with_snapshots)) == [ - Snapshot( - snapshot_id=13, - parent_snapshot_id=11, - sequence_number=3, - timestamp_ms=1677222224000, - manifest_list="s3://bucket/table/snapshot-c.avro", - summary=None, - schema_id=None, - ), - Snapshot( - snapshot_id=11, - parent_snapshot_id=10, - sequence_number=2, - timestamp_ms=1677222222000, - manifest_list="s3://bucket/table/snapshot-b.avro", - summary=None, - schema_id=None, - ), - Snapshot( - snapshot_id=10, - parent_snapshot_id=None, - sequence_number=1, - timestamp_ms=1677222221000, - manifest_list="s3://bucket/table/snapshot-a.avro", - summary=None, - schema_id=None, - ), - ] - - -def test_current_ancestors_ids(table_with_snapshots: Table): - assert list(current_ancestor_ids(table_with_snapshots)) == [13, 11, 10] - - -def test_oldest_ancestor(table_with_snapshots: Table): - assert oldest_ancestor(table_with_snapshots) == Snapshot( - snapshot_id=10, - parent_snapshot_id=None, - sequence_number=1, - timestamp_ms=1677222221000, - manifest_list="s3://bucket/table/snapshot-a.avro", - summary=None, - schema_id=None, - ) - - -def test_oldest_ancestor_of(table_with_snapshots: Table): - assert oldest_ancestor_of(table_with_snapshots, SNAPSHOT_B_ID) == Snapshot( - snapshot_id=10, - parent_snapshot_id=None, - sequence_number=1, - timestamp_ms=1677222221000, - manifest_list="s3://bucket/table/snapshot-a.avro", - summary=None, - schema_id=None, - ) - - -def test_oldest_ancestor_after(table_with_snapshots: Table): - assert oldest_ancestor_after(table_with_snapshots, SNAPSHOT_B_TIME) == Snapshot( - snapshot_id=11, - parent_snapshot_id=10, - sequence_number=2, - timestamp_ms=1677222222000, - manifest_list="s3://bucket/table/snapshot-b.avro", - summary=None, - schema_id=None, - ) - - -def test_snapshots_ids_between(table_with_snapshots: Table): - assert list(snapshots_ids_between(table_with_snapshots, SNAPSHOT_A_ID, SNAPSHOT_D_ID)) == [SNAPSHOT_D_ID, SNAPSHOT_B_ID] - - -def test_ancestor_ids(table_with_snapshots: Table): - assert list(ancestor_ids(SNAPSHOT_B_ID, table_with_snapshots.snapshot_by_id)) == [SNAPSHOT_B_ID, SNAPSHOT_A_ID] - - -def test_ancestors_between(table_with_snapshots: Table): - assert list(ancestors_between(SNAPSHOT_D_ID, SNAPSHOT_A_ID, table_with_snapshots.snapshot_by_id)) == [ - Snapshot( - snapshot_id=13, - parent_snapshot_id=11, - sequence_number=3, - timestamp_ms=1677222224000, - manifest_list="s3://bucket/table/snapshot-c.avro", - summary=None, - schema_id=None, - ), - Snapshot( - snapshot_id=11, - parent_snapshot_id=10, - sequence_number=2, - timestamp_ms=1677222222000, - manifest_list="s3://bucket/table/snapshot-b.avro", - summary=None, - schema_id=None, - ), - ] - - assert list(ancestors_between(SNAPSHOT_D_ID, None, table_with_snapshots.snapshot_by_id)) == [ - Snapshot( - snapshot_id=13, - parent_snapshot_id=11, - sequence_number=3, - timestamp_ms=1677222224000, - manifest_list="s3://bucket/table/snapshot-c.avro", - summary=None, - schema_id=None, - ), - Snapshot( - snapshot_id=11, - parent_snapshot_id=10, - sequence_number=2, - timestamp_ms=1677222222000, - manifest_list="s3://bucket/table/snapshot-b.avro", - summary=None, - schema_id=None, - ), - Snapshot( - snapshot_id=10, - parent_snapshot_id=None, - sequence_number=1, - timestamp_ms=1677222221000, - manifest_list="s3://bucket/table/snapshot-a.avro", - summary=None, - schema_id=None, - ), - ] - - -def test_new_files(table_schema_simple: Schema, generated_manifest_file_file: str): - io = load_file_io({}) - table = Table( - identifier=("database", "table"), - metadata=TableMetadataV2( - location="s3://bucket/warehouse/table/", - table_uuid="9c12d441-03fe-4693-9a96-a0705ddf69c1", - last_updated_ms=1602638573590, - last_column_id=3, - schemas=[table_schema_simple], - current_schema_id=1, - last_partition_id=1000, - properties={"owner": "javaberg"}, - partition_specs=[PartitionSpec()], - default_spec_id=0, - current_snapshot_id=SNAPSHOT_D_ID, - snapshots=[ - Snapshot( - snapshot_id=SNAPSHOT_A_ID, - parent_snapshot_id=None, - sequence_number=1, - timestamp_ms=SNAPSHOT_A_TIME, - manifest_list=generated_manifest_file_file, - ), - Snapshot( - snapshot_id=SNAPSHOT_B_ID, - parent_snapshot_id=SNAPSHOT_A_ID, - sequence_number=2, - timestamp_ms=SNAPSHOT_B_TIME, - manifest_list=generated_manifest_file_file, - ), - Snapshot( - snapshot_id=SNAPSHOT_D_ID, - parent_snapshot_id=SNAPSHOT_B_ID, - sequence_number=3, - timestamp_ms=SNAPSHOT_D_TIME, - manifest_list=generated_manifest_file_file, - ), - Snapshot( - snapshot_id=SNAPSHOT_C_ID, - parent_snapshot_id=SNAPSHOT_A_ID, - sequence_number=4, - timestamp_ms=SNAPSHOT_C_TIME, - manifest_list=generated_manifest_file_file, - ), - ], - snapshot_log=[], - metadata_log=[], - sort_orders=[SortOrder(order_id=0)], - default_sort_order_id=0, - refs={"b1": SnapshotRef(snapshot_id=SNAPSHOT_C_ID, snapshot_ref_type=SnapshotRefType.BRANCH)}, - format_version=2, - last_sequence_number=0, - ), - metadata_location="s3://bucket/table/metadata.json", - io=io, - ) - - files = list(new_files(SNAPSHOT_A_ID, SNAPSHOT_D_ID, table.snapshot_by_id, io)) - - assert len(files) == 4 - assert ( - files[0].file_path - == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" - ) - assert ( - files[1].file_path - == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" - ) - assert ( - files[2].file_path - == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" - ) - assert ( - files[3].file_path - == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet" - ) - - -def test_snapshot_after(table_with_snapshots: Table): - assert snapshot_after(table_with_snapshots, SNAPSHOT_B_ID).snapshot_id == SNAPSHOT_D_ID - - -def test_snapshot_id_as_of_time(table_with_snapshots: Table): - assert snapshot_id_as_of_time(table_with_snapshots, SNAPSHOT_B_TIME) == SNAPSHOT_A_ID - - -def test_schema_for(table_with_snapshots: Table, table_schema_simple: Schema): - assert schema_for(table_with_snapshots, SNAPSHOT_A_ID) == table_schema_simple - - -def test_schema_as_of_time(table_with_snapshots: Table, table_schema_simple: Schema): - assert schema_as_of_time(table_with_snapshots, SNAPSHOT_B_TIME) == table_schema_simple diff --git a/python/tests/test_fokko.py b/python/tests/test_fokko.py new file mode 100644 index 000000000000..24ac8df39d93 --- /dev/null +++ b/python/tests/test_fokko.py @@ -0,0 +1,18 @@ +from pyiceberg.expressions import And, GreaterThan, LessThan + + +def test_vo(): + from pyiceberg.catalog import load_catalog + + cat = load_catalog("local") + + tbl = cat.load_table("nyc.taxis") + + df = tbl.scan( + row_filter=And( + GreaterThan("tpep_pickup_datetime", "2021-12-02T12:00:00+00:00"), + LessThan("tpep_pickup_datetime", "2021-12-20T12:00:00+00:00"), + ) + ).to_arrow() + + print(df) From 97d84bfc0e5ebf9d17caff071988e4ecfdc9467d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Feb 2023 13:38:06 +0100 Subject: [PATCH 09/36] Cleanup --- .../main/java/org/apache/iceberg/util/SnapshotUtil.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index 969d8ed1a9d2..c7d36bdc0078 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -191,10 +191,10 @@ public static Snapshot oldestAncestorAfter(Table table, long timestampMillis) { */ public static List snapshotIdsBetween(Table table, long fromSnapshotId, long toSnapshotId) { List snapshotIds = - Lists.newArrayList( - ancestorIds( - table.snapshot(toSnapshotId), - snapshotId -> snapshotId != fromSnapshotId ? table.snapshot(snapshotId) : null)); + Lists.newArrayList( + ancestorIds( + table.snapshot(toSnapshotId), + snapshotId -> snapshotId != fromSnapshotId ? table.snapshot(snapshotId) : null)); return snapshotIds; } From 0a2537ffdfb2b9d1f2d7674978395e4665627664 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Feb 2023 13:39:25 +0100 Subject: [PATCH 10/36] Cleanup --- python/pyiceberg/table/snapshots.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyiceberg/table/snapshots.py b/python/pyiceberg/table/snapshots.py index 0783a12f9ae7..83dd3f66a7cf 100644 --- a/python/pyiceberg/table/snapshots.py +++ b/python/pyiceberg/table/snapshots.py @@ -73,7 +73,9 @@ def __init__( self, operation: Optional[Operation] = None, __root__: Optional[Dict[str, Union[str, Operation]]] = None, **data: Any ) -> None: super().__init__(__root__={"operation": operation, **data} if not __root__ else __root__) - self._additional_properties = {k: v for k, v in self.__root__.items() if k != OPERATION} # type: ignore # We know that they are all string, and we don't want to check + self._additional_properties = { + k: v for k, v in self.__root__.items() if k != OPERATION # type: ignore # We know that they are all string, and we don't want to check + } @property def operation(self) -> Operation: @@ -122,5 +124,5 @@ class MetadataLogEntry(IcebergBaseModel): class SnapshotLogEntry(IcebergBaseModel): - snapshot_id: int = Field(alias="snapshot-id") + snapshot_id: str = Field(alias="snapshot-id") timestamp_ms: int = Field(alias="timestamp-ms") From 9281caa049f354e88c7632a7cd5318bd1ad99107 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Feb 2023 13:40:14 +0100 Subject: [PATCH 11/36] Cleanup --- python/pyiceberg/io/pyarrow.py | 2 +- python/tests/io/test_pyarrow.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index f97c7986a177..eef333ba0523 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -394,7 +394,7 @@ def visit_timestamp(self, _: TimestampType) -> pa.DataType: return pa.timestamp(unit="us") def visit_timestampz(self, _: TimestamptzType) -> pa.DataType: - return pa.timestamp(unit="us", tz="UTC") + return pa.timestamp(unit="us", tz="+00:00") def visit_string(self, _: StringType) -> pa.DataType: return pa.string() diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index 64f3102baad1..7470a19020f5 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -378,7 +378,7 @@ def test_timestamp_type_to_pyarrow() -> None: def test_timestamptz_type_to_pyarrow() -> None: iceberg_type = TimestamptzType() - assert visit(iceberg_type, _ConvertToArrowSchema()) == pa.timestamp(unit="us", tz="UTC") + assert visit(iceberg_type, _ConvertToArrowSchema()) == pa.timestamp(unit="us", tz="+00:00") def test_string_type_to_pyarrow() -> None: From 8f3b777e3a6c2872258f64f3a89689f705c3fca4 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Feb 2023 13:43:54 +0100 Subject: [PATCH 12/36] Cleanup --- python/pyiceberg/table/__init__.py | 26 +------------------------- python/tests/table/test_snapshots.py | 13 ------------- python/tests/test_fokko.py | 18 ------------------ 3 files changed, 1 insertion(+), 56 deletions(-) delete mode 100644 python/tests/test_fokko.py diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index a02facc8952a..57fc6464e382 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -111,14 +111,7 @@ def scan( def schema(self) -> Schema: """Return the schema for this table""" - return self.schema_by_id(self.metadata.current_schema_id) - - def schema_by_id(self, schema_id: int) -> Schema: - """Return the schema for this table by ID""" - try: - return next(schema for schema in self.metadata.schemas if schema.schema_id == schema_id) - except StopIteration: - raise ValueError(f"Schema id not found in table: {schema_id}") + return next(schema for schema in self.metadata.schemas if schema.schema_id == self.metadata.current_schema_id) def schemas(self) -> Dict[int, Schema]: """Return a dict of the schema of this table""" @@ -152,23 +145,6 @@ def current_snapshot(self) -> Optional[Snapshot]: return self.snapshot_by_id(snapshot_id) return None - def snapshot_by_id(self, snapshot_id: int) -> Snapshot: - """Get the snapshot of this table with the given id. - - Args: - snapshot_id: The id of the snapshot to lookup in the table - - Returns: - The snapshot that corresponds to snapshot_id - - Raises: - ValueError: If the snapshot cannot be found - """ - try: - return next(snapshot for snapshot in self.metadata.snapshots if snapshot.snapshot_id == snapshot_id) - except StopIteration: - raise ValueError(f"Snapshot id not found in table: {snapshot_id}") - def snapshot_by_name(self, name: str) -> Optional[Snapshot]: """Returns the snapshot referenced by the given name or null if no such reference exists.""" if ref := self.metadata.refs.get(name): diff --git a/python/tests/table/test_snapshots.py b/python/tests/table/test_snapshots.py index 7ff7ed549231..b119ae9945e3 100644 --- a/python/tests/table/test_snapshots.py +++ b/python/tests/table/test_snapshots.py @@ -46,19 +46,6 @@ def snapshot_with_properties() -> Snapshot: ) -SNAPSHOT_A_ID = 0xA -SNAPSHOT_A_TIME = 1677222221000 - -SNAPSHOT_B_ID = 0xB -SNAPSHOT_B_TIME = 1677222222000 - -SNAPSHOT_C_ID = 0xC -SNAPSHOT_C_TIME = 1677222223000 - -SNAPSHOT_D_ID = 0xD -SNAPSHOT_D_TIME = 1677222224000 - - def test_serialize_summary() -> None: assert Summary(Operation.APPEND).json() == """{"operation": "append"}""" diff --git a/python/tests/test_fokko.py b/python/tests/test_fokko.py deleted file mode 100644 index 24ac8df39d93..000000000000 --- a/python/tests/test_fokko.py +++ /dev/null @@ -1,18 +0,0 @@ -from pyiceberg.expressions import And, GreaterThan, LessThan - - -def test_vo(): - from pyiceberg.catalog import load_catalog - - cat = load_catalog("local") - - tbl = cat.load_table("nyc.taxis") - - df = tbl.scan( - row_filter=And( - GreaterThan("tpep_pickup_datetime", "2021-12-02T12:00:00+00:00"), - LessThan("tpep_pickup_datetime", "2021-12-20T12:00:00+00:00"), - ) - ).to_arrow() - - print(df) From bfb4837fb1864223b92119f3a6eb4055babed21a Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Feb 2023 13:44:59 +0100 Subject: [PATCH 13/36] Cleanup --- python/pyiceberg/table/__init__.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 57fc6464e382..ea045fd98493 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -145,6 +145,13 @@ def current_snapshot(self) -> Optional[Snapshot]: return self.snapshot_by_id(snapshot_id) return None + def snapshot_by_id(self, snapshot_id: int) -> Optional[Snapshot]: + """Get the snapshot of this table with the given id, or None if there is no matching snapshot.""" + try: + return next(snapshot for snapshot in self.metadata.snapshots if snapshot.snapshot_id == snapshot_id) + except StopIteration: + return None + def snapshot_by_name(self, name: str) -> Optional[Snapshot]: """Returns the snapshot referenced by the given name or null if no such reference exists.""" if ref := self.metadata.refs.get(name): From 01fa072b78d54efceec48d3a1df2952498f1006b Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 28 Feb 2023 18:18:14 +0100 Subject: [PATCH 14/36] WIP --- python/pyiceberg/io/pyarrow.py | 2 +- python/pyiceberg/table/__init__.py | 39 ++++++++++++++++++++---------- python/tests/test_fokko.py | 13 ++++++++++ 3 files changed, 40 insertions(+), 14 deletions(-) create mode 100644 python/tests/test_fokko.py diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index c5031aa72e84..2043b6b1918f 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -394,7 +394,7 @@ def visit_timestamp(self, _: TimestampType) -> pa.DataType: return pa.timestamp(unit="us") def visit_timestampz(self, _: TimestamptzType) -> pa.DataType: - return pa.timestamp(unit="us", tz="+00:00") + return pa.timestamp(unit="us", tz="UTC") def visit_string(self, _: StringType) -> pa.DataType: return pa.string() diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 04a8846ddedc..40421e48b675 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -53,7 +53,7 @@ ) from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table.metadata import TableMetadata +from pyiceberg.table.metadata import TableMetadata, INITIAL_SEQUENCE_NUMBER from pyiceberg.table.snapshots import Snapshot, SnapshotLogEntry from pyiceberg.table.sorting import SortOrder from pyiceberg.typedef import ( @@ -68,7 +68,6 @@ import pyarrow as pa from duckdb import DuckDBPyConnection - ALWAYS_TRUE = AlwaysTrue() @@ -316,7 +315,6 @@ def _open_manifest( return [FileScanTask(file) for file in result_manifests if metrics_evaluator(file)] - class DataScan(TableScan): def __init__( self, @@ -351,6 +349,12 @@ def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool] return lambda data_file: evaluator(data_file.partition) def plan_files(self) -> Iterator[FileScanTask]: + """Plans the relevant files by filtering on the PartitionSpecs + + Returns: + List of FileScanTasks that contain both data and delete files + """ + snapshot = self.snapshot() if not snapshot: return iter([]) @@ -359,7 +363,6 @@ def plan_files(self) -> Iterator[FileScanTask]: # step 1: filter manifests using partition summaries # the filter depends on the partition spec used to write the manifest file, so create a cache of filters for each spec id - manifest_evaluators: Dict[int, Callable[[ManifestFile], bool]] = KeyDefaultDict(self._build_manifest_evaluator) manifests = [ @@ -370,18 +373,15 @@ def plan_files(self) -> Iterator[FileScanTask]: # step 2: filter the data files in each manifest # this filter depends on the partition spec used to write the manifest file - partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) - data_files = [] - delete_files = [] + metrics_evaluator = _InclusiveMetricsEvaluator(self.table.schema(), self.row_filter, self.case_sensitive).eval - for manifest in manifests: - if manifest.content is None or manifest.content == ManifestContent.DATA: - data_files.append(manifest) - elif manifest.content == ManifestContent.DELETES and (manifest.has_added_files() or manifest.has_existing_files()): - delete_files.append(manifest) + min_sequence_number = min( + manifest.min_sequence_number or INITIAL_SEQUENCE_NUMBER + for manifest in manifests + if manifest.content == ManifestContent.DATA + ) - metrics_evaluator = _InclusiveMetricsEvaluator(self.table.schema(), self.row_filter, self.case_sensitive).eval with ThreadPool() as pool: return chain( *pool.starmap( @@ -394,6 +394,12 @@ def plan_files(self) -> Iterator[FileScanTask]: metrics_evaluator, ) for manifest in manifests + if manifest.content == ManifestContent.DATA + or ( + # Not interested in deletes that are older than the data + manifest.content == ManifestContent.DELETES + and (manifest.sequence_number or INITIAL_SEQUENCE_NUMBER) > min_sequence_number + ) ], ) ) @@ -415,3 +421,10 @@ def to_duckdb(self, table_name: str, connection: Optional[DuckDBPyConnection] = con.register(table_name, self.to_arrow()) return con + + +class DeleteFileIndex: + delete_manifests: List[ManifestFile] + + def __init__(self, delete_manifests: List[ManifestFile], after_sequence_number: int = INITIAL_SEQUENCE_NUMBER) -> None: + self.delete_manifests = delete_manifests diff --git a/python/tests/test_fokko.py b/python/tests/test_fokko.py new file mode 100644 index 000000000000..ad7bf666eed3 --- /dev/null +++ b/python/tests/test_fokko.py @@ -0,0 +1,13 @@ +from pyiceberg.expressions import GreaterThan, LessThan + +from pyiceberg.catalog import load_catalog + + +def test_vo(): + cat = load_catalog("rest") + + tbl = cat.load_table("nyc.taxis") + + df = tbl.scan(row_filter=LessThan("tpep_pickup_datetime", "2022-01-01T12:12:00+00:00")).to_arrow() + + print(df) From 5ddb3948fc67deb4348677d3d2cd7f75c0f09ff4 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 28 Feb 2023 18:19:55 +0100 Subject: [PATCH 15/36] Cleanup --- python/pyiceberg/table/__init__.py | 2 +- python/tests/test_fokko.py | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 40421e48b675..8ed879da721d 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -53,7 +53,7 @@ ) from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table.metadata import TableMetadata, INITIAL_SEQUENCE_NUMBER +from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadata from pyiceberg.table.snapshots import Snapshot, SnapshotLogEntry from pyiceberg.table.sorting import SortOrder from pyiceberg.typedef import ( diff --git a/python/tests/test_fokko.py b/python/tests/test_fokko.py index ad7bf666eed3..fa01b204a608 100644 --- a/python/tests/test_fokko.py +++ b/python/tests/test_fokko.py @@ -1,6 +1,5 @@ -from pyiceberg.expressions import GreaterThan, LessThan - from pyiceberg.catalog import load_catalog +from pyiceberg.expressions import LessThan def test_vo(): From 4769001c56648b5151bc9987e326d0d60f4de1b0 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 2 Mar 2023 11:29:08 +0100 Subject: [PATCH 16/36] Core: Allow dropping columns referenced in old sort orders --- .../org/apache/iceberg/SortOrderParser.java | 10 +++ .../apache/iceberg/TableMetadataParser.java | 2 +- .../iceberg/util/TestSortOrderUtil.java | 33 +++++++++ .../spark/extensions/TestAlterSortOrder.java | 70 +++++++++++++++++++ 4 files changed, 114 insertions(+), 1 deletion(-) create mode 100644 spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java diff --git a/core/src/main/java/org/apache/iceberg/SortOrderParser.java b/core/src/main/java/org/apache/iceberg/SortOrderParser.java index 4481f835137b..31307cf9dc7f 100644 --- a/core/src/main/java/org/apache/iceberg/SortOrderParser.java +++ b/core/src/main/java/org/apache/iceberg/SortOrderParser.java @@ -112,6 +112,16 @@ public static SortOrder fromJson(Schema schema, String json) { return fromJson(json).bind(schema); } + public static SortOrder fromJson(Schema schema, JsonNode json, int defaultSortOrderId) { + UnboundSortOrder unboundSortOrder = fromJson(json); + + if (unboundSortOrder.orderId() == defaultSortOrderId) { + return unboundSortOrder.bind(schema); + } else { + return unboundSortOrder.bindUnchecked(schema); + } + } + public static SortOrder fromJson(Schema schema, JsonNode json) { return fromJson(json).bind(schema); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index 207d9ac686ab..dc234c32ae06 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -420,7 +420,7 @@ static TableMetadata fromJson(String metadataLocation, JsonNode node) { defaultSortOrderId = JsonUtil.getInt(DEFAULT_SORT_ORDER_ID, node); ImmutableList.Builder sortOrdersBuilder = ImmutableList.builder(); for (JsonNode sortOrder : sortOrderArray) { - sortOrdersBuilder.add(SortOrderParser.fromJson(schema, sortOrder)); + sortOrdersBuilder.add(SortOrderParser.fromJson(schema, sortOrder, defaultSortOrderId)); } sortOrders = sortOrdersBuilder.build(); } else { diff --git a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java index 516279bc7f7b..63a4d0ee9502 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java @@ -24,10 +24,12 @@ import java.io.File; import java.io.IOException; +import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.apache.iceberg.TestTables; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.types.Types; import org.junit.After; @@ -303,4 +305,35 @@ public void testSortOrderClusteringWithRedundantPartitionFieldsMissing() { expected, SortOrderUtil.buildSortOrder(table.schema(), updatedSpec, order)); } + + @Test + public void testDropColumnFromDefaultSortOrder() { + TestTables.TestTable table = + TestTables.create( + tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), SortOrder.unsorted(), 1); + table.replaceSortOrder().asc("ts").commit(); + + AssertHelpers.assertThrows( + "Should complain about dropping a column from the current sort order", + ValidationException.class, + "Cannot find source column for sort field: identity(3) ASC NULLS FIRST", + () -> table.updateSchema().deleteColumn("ts").commit()); + } + + @Test + public void testDropColumnFromOldSortOrder() { + TestTables.TestTable table = + TestTables.create( + tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), SortOrder.unsorted(), 2); + table.replaceSortOrder().asc("ts").commit(); + + table.replaceSortOrder().asc("id").commit(); + + table.updateSchema().deleteColumn("ts").commit(); + + Assert.assertEquals( + "Should be on id(1)", + "[\n" + " identity(1) ASC NULLS FIRST\n" + "]", + table.sortOrder().toString()); + } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java new file mode 100644 index 000000000000..8696413a58d2 --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import java.util.Map; +import org.apache.iceberg.Table; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestAlterSortOrder extends SparkExtensionsTestBase { + public TestAlterSortOrder(String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testDeleteColumnFromOldSortOrder() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + + Assert.assertTrue("Table should start unsorted", table.sortOrder().isUnsorted()); + + sql("ALTER TABLE %s WRITE ORDERED BY ts", tableName); + + table.refresh(); + + Assert.assertEquals("Should have two sort orders", 2, table.sortOrders().size()); + Assert.assertEquals( + "Expected identity sort order on ts", + "[\n" + " identity(3) ASC NULLS FIRST\n" + "]", + table.sortOrders().get(1).toString()); + + sql("ALTER TABLE %s WRITE ORDERED BY id", tableName); + + table.refresh(); + + Assert.assertEquals("Should have three sort orders", 3, table.sortOrders().size()); + Assert.assertEquals( + "Expected identity sort order on id", + "[\n" + " identity(1) ASC NULLS FIRST\n" + "]", + table.sortOrders().get(2).toString()); + + sql("ALTER TABLE %s DROP COLUMN ts", tableName); + + table.refresh(); + } +} From 6af113b32827c48fba988276f42d288d458d965f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 2 Mar 2023 19:05:45 +0100 Subject: [PATCH 17/36] WIP --- python/pyiceberg/io/pyarrow.py | 35 ++++++++----------- python/pyiceberg/manifest.py | 4 +++ python/pyiceberg/table/__init__.py | 56 +++++++++++++++++++++++++----- python/tests/io/test_pyarrow.py | 9 ++--- python/tests/test_fokko.py | 5 ++- 5 files changed, 72 insertions(+), 37 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index e68c9eb44204..43d18cbf0395 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -31,7 +31,6 @@ TYPE_CHECKING, Any, Callable, - Dict, Generator, Iterable, List, @@ -74,7 +73,7 @@ OutputFile, OutputStream, ) -from pyiceberg.manifest import DataFileContent +from pyiceberg.manifest import DataFile from pyiceberg.schema import ( PartnerAccessor, Schema, @@ -473,15 +472,18 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression: return boolean_expression_visit(expr, _ConvertToArrowExpression()) -def _read_deletes(fs: FileSystem, file_path: str) -> Dict[str, pa.ChunkedArray]: +def _read_deletes(fs: FileSystem, file_path: str, deletes: List[DataFile]) -> pa.Array: _, path = PyArrowFileIO.parse_location(file_path) + delete_files = [PyArrowFileIO.parse_location(delete_file.file_path)[1] for delete_file in deletes] table = pq.read_table( - source=path, pre_buffer=True, buffer_size=8 * ONE_MEGABYTE, read_dictionary=["file_path"], filesystem=fs + source=delete_files, + pre_buffer=True, + buffer_size=8 * ONE_MEGABYTE, + filesystem=fs, + filters=pc.field("file_path") == file_path, + columns=["pos"], ) - table.unify_dictionaries() - return { - file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.columns[0].chunks[0].dictionary - } + return table["pos"].combine_chunks().sort() def _file_to_table( @@ -491,10 +493,13 @@ def _file_to_table( projected_schema: Schema, projected_field_ids: Set[int], case_sensitive: bool, - positional_deletes: Optional[Set[int]], ) -> Optional[pa.Table]: _, path = PyArrowFileIO.parse_location(task.file.file_path) + positional_deletes = None + if len(task.delete_files) > 0: + positional_deletes = _read_deletes(fs, task.file.file_path, task.delete_files) + # Get the schema with fs.open_input_file(path) as fout: parquet_schema = pq.read_schema(fout) @@ -571,18 +576,6 @@ def project_table( id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType)) }.union(extract_field_ids(bound_row_filter)) - tasks_data_files: List[FileScanTask] = [] - tasks_positional_deletes: List[FileScanTask] = [] - for task in tasks: - if task.file.content is None or task.file.content == DataFileContent.DATA: - tasks_data_files.append(task) - elif task.file.content == DataFileContent.POSITION_DELETES: - tasks_positional_deletes.append(task) - elif task.file.content == DataFileContent.EQUALITY_DELETES: - raise ValueError("PyIceberg does not yet support equality deletes: https://github.com/apache/iceberg/issues/6568") - else: - raise ValueError(f"Unknown file content: {task.file.content}") - with ThreadPool() as pool: tables = [ table diff --git a/python/pyiceberg/manifest.py b/python/pyiceberg/manifest.py index ffd4ea9577c0..710c907e228c 100644 --- a/python/pyiceberg/manifest.py +++ b/python/pyiceberg/manifest.py @@ -237,6 +237,10 @@ def __init__(self, *data: Any, **named_data: Any) -> None: NestedField(519, "key_metadata", BinaryType(), required=False), ) +POSITIONAL_DELETE_SCHEMA = Schema( + NestedField(2147483546, "file_path", StringType()), NestedField(2147483545, "pos", IntegerType()) +) + class ManifestFile(Record): manifest_path: str diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 8ed879da721d..e65a657f3d8d 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -26,6 +26,7 @@ Any, Callable, Dict, + Iterable, Iterator, List, Optional, @@ -40,13 +41,16 @@ AlwaysTrue, And, BooleanExpression, + EqualTo, parser, visitors, ) from pyiceberg.expressions.visitors import _InclusiveMetricsEvaluator, inclusive_projection from pyiceberg.io import FileIO, load_file_io from pyiceberg.manifest import ( + POSITIONAL_DELETE_SCHEMA, DataFile, + DataFileContent, ManifestContent, ManifestFile, files, @@ -252,7 +256,7 @@ def projection(self) -> Schema: return snapshot_schema.select(*self.selected_fields, case_sensitive=self.case_sensitive) @abstractmethod - def plan_files(self) -> Iterator[ScanTask]: + def plan_files(self) -> Iterable[ScanTask]: ... @abstractmethod @@ -294,11 +298,19 @@ class ScanTask(ABC): @dataclass(init=False) class FileScanTask(ScanTask): file: DataFile + delete_files: List[DataFile] start: int length: int - def __init__(self, data_file: DataFile, start: Optional[int] = None, length: Optional[int] = None): + def __init__( + self, + data_file: DataFile, + delete_files: Optional[List[DataFile]] = None, + start: Optional[int] = None, + length: Optional[int] = None, + ): self.file = data_file + self.delete_files = delete_files or [] self.start = start or 0 self.length = length or data_file.file_size_in_bytes @@ -308,11 +320,11 @@ def _open_manifest( manifest: ManifestFile, partition_filter: Callable[[DataFile], bool], metrics_evaluator: Callable[[DataFile], bool], -) -> List[FileScanTask]: +) -> List[DataFile]: result_manifests = files(io.new_input(manifest.manifest_path)) if partition_filter is not None: result_manifests = filter(partition_filter, result_manifests) - return [FileScanTask(file) for file in result_manifests if metrics_evaluator(file)] + return [file for file in result_manifests if metrics_evaluator(file)] class DataScan(TableScan): @@ -348,7 +360,7 @@ def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool] evaluator = visitors.expression_evaluator(partition_schema, partition_expr, self.case_sensitive) return lambda data_file: evaluator(data_file.partition) - def plan_files(self) -> Iterator[FileScanTask]: + def plan_files(self) -> Iterable[FileScanTask]: """Plans the relevant files by filtering on the PartitionSpecs Returns: @@ -379,11 +391,14 @@ def plan_files(self) -> Iterator[FileScanTask]: min_sequence_number = min( manifest.min_sequence_number or INITIAL_SEQUENCE_NUMBER for manifest in manifests - if manifest.content == ManifestContent.DATA + if manifest.content is None or manifest.content == ManifestContent.DATA ) + data_datafiles = [] + deletes_positional = [] + with ThreadPool() as pool: - return chain( + for datafile in chain( *pool.starmap( func=_open_manifest, iterable=[ @@ -394,15 +409,38 @@ def plan_files(self) -> Iterator[FileScanTask]: metrics_evaluator, ) for manifest in manifests - if manifest.content == ManifestContent.DATA + if (manifest.content is None or manifest.content == ManifestContent.DATA) or ( # Not interested in deletes that are older than the data manifest.content == ManifestContent.DELETES - and (manifest.sequence_number or INITIAL_SEQUENCE_NUMBER) > min_sequence_number + and (manifest.sequence_number or INITIAL_SEQUENCE_NUMBER) >= min_sequence_number ) ], ) + ): + if datafile.content is None or datafile.content == DataFileContent.DATA: + data_datafiles.append(datafile) + elif datafile.content == DataFileContent.POSITION_DELETES: + deletes_positional.append(datafile) + elif datafile.content == DataFileContent.EQUALITY_DELETES: + raise ValueError( + "PyIceberg does not yet support equality deletes: https://github.com/apache/iceberg/issues/6568" + ) + else: + raise ValueError(f"Unknown DataFileContent: {datafile.content}") + + return [ + FileScanTask(data_file, delete_files=self._match_deletes_to_datafile(data_file, deletes_positional)) + for data_file in data_datafiles + ] + + def _match_deletes_to_datafile(self, data_file: DataFile, positional_delete_files: List[DataFile]) -> List[DataFile]: + return list( + filter( + _InclusiveMetricsEvaluator(POSITIONAL_DELETE_SCHEMA, EqualTo("file_path", data_file.file_path)).eval, + positional_delete_files, ) + ) def to_arrow(self) -> pa.Table: from pyiceberg.io.pyarrow import project_table diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index 64f3102baad1..cf8ed582b698 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -24,7 +24,7 @@ import pyarrow as pa import pyarrow.parquet as pq import pytest -from pyarrow.fs import FileType +from pyarrow.fs import FileType, LocalFileSystem from pyiceberg.avro.resolver import ResolveError from pyiceberg.expressions import ( @@ -1137,6 +1137,7 @@ def deletes_file(tmp_path: str) -> str: def test_read_deletes(deletes_file: str) -> None: # None filesystem will default to a local filesystem - deletes = _read_deletes(None, deletes_file) - assert set(deletes.keys()) == {"s3://bucket/default.db/table/data.parquet"} - assert list(deletes.values())[0] == pa.chunked_array([[19, 22, 25]]) + deletes = _read_deletes(LocalFileSystem(), "s3://bucket/default.db/table/data.parquet", [DataFile( + file_path=deletes_file + )]) + assert list(deletes) == {19, 22, 25} diff --git a/python/tests/test_fokko.py b/python/tests/test_fokko.py index fa01b204a608..d3a367d18fe0 100644 --- a/python/tests/test_fokko.py +++ b/python/tests/test_fokko.py @@ -1,12 +1,11 @@ from pyiceberg.catalog import load_catalog -from pyiceberg.expressions import LessThan def test_vo(): - cat = load_catalog("rest") + cat = load_catalog("local") tbl = cat.load_table("nyc.taxis") - df = tbl.scan(row_filter=LessThan("tpep_pickup_datetime", "2022-01-01T12:12:00+00:00")).to_arrow() + df = tbl.scan().to_arrow() print(df) From a98e0a773028bb0e089d596c17f47efef6f8af3a Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 2 Mar 2023 22:28:17 +0100 Subject: [PATCH 18/36] WIP --- python/pyiceberg/io/pyarrow.py | 63 ++++++++++++++++++++++-------- python/pyiceberg/manifest.py | 6 +++ python/pyiceberg/table/__init__.py | 19 +++------ python/tests/io/test_pyarrow.py | 4 +- 4 files changed, 59 insertions(+), 33 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 43d18cbf0395..9f850b0d23f3 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -26,11 +26,13 @@ import os from functools import lru_cache +from itertools import chain from multiprocessing.pool import ThreadPool from typing import ( TYPE_CHECKING, Any, Callable, + Dict, Generator, Iterable, List, @@ -44,6 +46,7 @@ import pyarrow as pa import pyarrow.compute as pc import pyarrow.parquet as pq +from pyarrow import ChunkedArray from pyarrow.fs import ( FileInfo, FileSystem, @@ -73,7 +76,6 @@ OutputFile, OutputStream, ) -from pyiceberg.manifest import DataFile from pyiceberg.schema import ( PartnerAccessor, Schema, @@ -472,18 +474,15 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression: return boolean_expression_visit(expr, _ConvertToArrowExpression()) -def _read_deletes(fs: FileSystem, file_path: str, deletes: List[DataFile]) -> pa.Array: +def _read_deletes(fs: FileSystem, file_path: str) -> Dict[str, pa.ChunkedArray]: _, path = PyArrowFileIO.parse_location(file_path) - delete_files = [PyArrowFileIO.parse_location(delete_file.file_path)[1] for delete_file in deletes] table = pq.read_table( - source=delete_files, - pre_buffer=True, - buffer_size=8 * ONE_MEGABYTE, - filesystem=fs, - filters=pc.field("file_path") == file_path, - columns=["pos"], + source=path, pre_buffer=True, buffer_size=8 * ONE_MEGABYTE, read_dictionary=["file_path"], filesystem=fs ) - return table["pos"].combine_chunks().sort() + table.unify_dictionaries() + return { + file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.columns[0].chunks[0].dictionary + } def _file_to_table( @@ -492,14 +491,11 @@ def _file_to_table( bound_row_filter: BooleanExpression, projected_schema: Schema, projected_field_ids: Set[int], + positional_deletes: Optional[ChunkedArray], case_sensitive: bool, ) -> Optional[pa.Table]: _, path = PyArrowFileIO.parse_location(task.file.file_path) - positional_deletes = None - if len(task.delete_files) > 0: - positional_deletes = _read_deletes(fs, task.file.file_path, task.delete_files) - # Get the schema with fs.open_input_file(path) as fout: parquet_schema = pq.read_schema(fout) @@ -534,9 +530,19 @@ def _file_to_table( if positional_deletes is not None: # When there are positional deletes, create a filter mask + sorted_deleted = iter(positional_deletes.combine_chunks().sort()) + def generator() -> Generator[bool, None, None]: + deleted_pos = next(sorted_deleted).as_py() for pos in range(len(arrow_table)): - yield pos in positional_deletes # type: ignore + if deleted_pos == pos: + yield True + try: + deleted_pos = next(sorted_deleted).as_py() + except: + deleted_pos = -1 + else: + yield False mask = pa.array(generator(), type=pa.bool_()) arrow_table = arrow_table.filter(mask) @@ -577,12 +583,35 @@ def project_table( }.union(extract_field_ids(bound_row_filter)) with ThreadPool() as pool: + deletes_per_file: Dict[str, ChunkedArray] = {} + unique_deletes = set(chain.from_iterable([{delete_file.file_path for delete_file in task.delete_files} for task in tasks])) + deletes_per_files: List[Dict[str, ChunkedArray]] = pool.starmap( + func=_read_deletes, iterable=[(fs, delete) for delete in unique_deletes] + ) + + for delete in deletes_per_files: + for file, arr in delete.items(): + if first_arr := deletes_per_file.get(file): + deletes_per_file[file] = pa.chunked_array([first_arr, arr], first_arr.type) + else: + deletes_per_file[file] = arr + tables = [ table for table in pool.starmap( func=_file_to_table, - iterable=[(fs, task, bound_row_filter, projected_schema, projected_field_ids, case_sensitive) for task in tasks], - chunksize=None, # we could use this to control how to materialize the generator of tasks (we should also make the expression above lazy) + iterable=[ + ( + fs, + task, + bound_row_filter, + projected_schema, + projected_field_ids, + deletes_per_file.get(task.file.file_path), + case_sensitive, + ) + for task in tasks + ], ) if table is not None ] diff --git a/python/pyiceberg/manifest.py b/python/pyiceberg/manifest.py index 710c907e228c..778a1107be97 100644 --- a/python/pyiceberg/manifest.py +++ b/python/pyiceberg/manifest.py @@ -180,6 +180,12 @@ class DataFile(Record): def __init__(self, *data: Any, **named_data: Any) -> None: super().__init__(*data, **{"struct": DATA_FILE_TYPE, **named_data}) + def __hash__(self) -> int: + return hash(self.file_path) + + def __eq__(self, other: Any) -> bool: + return self.file_path == other.file_path if isinstance(other, DataFile) else False + MANIFEST_ENTRY_SCHEMA = Schema( NestedField(0, "status", IntegerType(), required=True), diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index e65a657f3d8d..ce882926b532 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -27,9 +27,9 @@ Callable, Dict, Iterable, - Iterator, List, Optional, + Set, Tuple, TypeVar, Union, @@ -298,19 +298,19 @@ class ScanTask(ABC): @dataclass(init=False) class FileScanTask(ScanTask): file: DataFile - delete_files: List[DataFile] + delete_files: Set[DataFile] start: int length: int def __init__( self, data_file: DataFile, - delete_files: Optional[List[DataFile]] = None, + delete_files: Optional[Set[DataFile]] = None, start: Optional[int] = None, length: Optional[int] = None, ): self.file = data_file - self.delete_files = delete_files or [] + self.delete_files = delete_files or set() self.start = start or 0 self.length = length or data_file.file_size_in_bytes @@ -434,8 +434,8 @@ def plan_files(self) -> Iterable[FileScanTask]: for data_file in data_datafiles ] - def _match_deletes_to_datafile(self, data_file: DataFile, positional_delete_files: List[DataFile]) -> List[DataFile]: - return list( + def _match_deletes_to_datafile(self, data_file: DataFile, positional_delete_files: List[DataFile]) -> Set[DataFile]: + return set( filter( _InclusiveMetricsEvaluator(POSITIONAL_DELETE_SCHEMA, EqualTo("file_path", data_file.file_path)).eval, positional_delete_files, @@ -459,10 +459,3 @@ def to_duckdb(self, table_name: str, connection: Optional[DuckDBPyConnection] = con.register(table_name, self.to_arrow()) return con - - -class DeleteFileIndex: - delete_manifests: List[ManifestFile] - - def __init__(self, delete_manifests: List[ManifestFile], after_sequence_number: int = INITIAL_SEQUENCE_NUMBER) -> None: - self.delete_manifests = delete_manifests diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index cf8ed582b698..0bc3627581e9 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -1137,7 +1137,5 @@ def deletes_file(tmp_path: str) -> str: def test_read_deletes(deletes_file: str) -> None: # None filesystem will default to a local filesystem - deletes = _read_deletes(LocalFileSystem(), "s3://bucket/default.db/table/data.parquet", [DataFile( - file_path=deletes_file - )]) + deletes = _read_deletes(LocalFileSystem(), "s3://bucket/default.db/table/data.parquet", [DataFile(file_path=deletes_file)]) assert list(deletes) == {19, 22, 25} From 735d68931e43c5f3e791daa710559ee3c669f323 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 2 Mar 2023 22:29:42 +0100 Subject: [PATCH 19/36] Revert "Core: Allow dropping columns referenced in old sort orders" This reverts commit 4769001c56648b5151bc9987e326d0d60f4de1b0. --- .../org/apache/iceberg/SortOrderParser.java | 10 --- .../apache/iceberg/TableMetadataParser.java | 2 +- .../iceberg/util/TestSortOrderUtil.java | 33 --------- .../spark/extensions/TestAlterSortOrder.java | 70 ------------------- 4 files changed, 1 insertion(+), 114 deletions(-) delete mode 100644 spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java diff --git a/core/src/main/java/org/apache/iceberg/SortOrderParser.java b/core/src/main/java/org/apache/iceberg/SortOrderParser.java index 31307cf9dc7f..4481f835137b 100644 --- a/core/src/main/java/org/apache/iceberg/SortOrderParser.java +++ b/core/src/main/java/org/apache/iceberg/SortOrderParser.java @@ -112,16 +112,6 @@ public static SortOrder fromJson(Schema schema, String json) { return fromJson(json).bind(schema); } - public static SortOrder fromJson(Schema schema, JsonNode json, int defaultSortOrderId) { - UnboundSortOrder unboundSortOrder = fromJson(json); - - if (unboundSortOrder.orderId() == defaultSortOrderId) { - return unboundSortOrder.bind(schema); - } else { - return unboundSortOrder.bindUnchecked(schema); - } - } - public static SortOrder fromJson(Schema schema, JsonNode json) { return fromJson(json).bind(schema); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java index dc234c32ae06..207d9ac686ab 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadataParser.java @@ -420,7 +420,7 @@ static TableMetadata fromJson(String metadataLocation, JsonNode node) { defaultSortOrderId = JsonUtil.getInt(DEFAULT_SORT_ORDER_ID, node); ImmutableList.Builder sortOrdersBuilder = ImmutableList.builder(); for (JsonNode sortOrder : sortOrderArray) { - sortOrdersBuilder.add(SortOrderParser.fromJson(schema, sortOrder, defaultSortOrderId)); + sortOrdersBuilder.add(SortOrderParser.fromJson(schema, sortOrder)); } sortOrders = sortOrdersBuilder.build(); } else { diff --git a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java index 63a4d0ee9502..516279bc7f7b 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java @@ -24,12 +24,10 @@ import java.io.File; import java.io.IOException; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.apache.iceberg.TestTables; -import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.types.Types; import org.junit.After; @@ -305,35 +303,4 @@ public void testSortOrderClusteringWithRedundantPartitionFieldsMissing() { expected, SortOrderUtil.buildSortOrder(table.schema(), updatedSpec, order)); } - - @Test - public void testDropColumnFromDefaultSortOrder() { - TestTables.TestTable table = - TestTables.create( - tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), SortOrder.unsorted(), 1); - table.replaceSortOrder().asc("ts").commit(); - - AssertHelpers.assertThrows( - "Should complain about dropping a column from the current sort order", - ValidationException.class, - "Cannot find source column for sort field: identity(3) ASC NULLS FIRST", - () -> table.updateSchema().deleteColumn("ts").commit()); - } - - @Test - public void testDropColumnFromOldSortOrder() { - TestTables.TestTable table = - TestTables.create( - tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), SortOrder.unsorted(), 2); - table.replaceSortOrder().asc("ts").commit(); - - table.replaceSortOrder().asc("id").commit(); - - table.updateSchema().deleteColumn("ts").commit(); - - Assert.assertEquals( - "Should be on id(1)", - "[\n" + " identity(1) ASC NULLS FIRST\n" + "]", - table.sortOrder().toString()); - } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java deleted file mode 100644 index 8696413a58d2..000000000000 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterSortOrder.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.extensions; - -import java.util.Map; -import org.apache.iceberg.Table; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; - -public class TestAlterSortOrder extends SparkExtensionsTestBase { - public TestAlterSortOrder(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After - public void removeTable() { - sql("DROP TABLE IF EXISTS %s", tableName); - } - - @Test - public void testDeleteColumnFromOldSortOrder() { - sql( - "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", - tableName); - Table table = validationCatalog.loadTable(tableIdent); - - Assert.assertTrue("Table should start unsorted", table.sortOrder().isUnsorted()); - - sql("ALTER TABLE %s WRITE ORDERED BY ts", tableName); - - table.refresh(); - - Assert.assertEquals("Should have two sort orders", 2, table.sortOrders().size()); - Assert.assertEquals( - "Expected identity sort order on ts", - "[\n" + " identity(3) ASC NULLS FIRST\n" + "]", - table.sortOrders().get(1).toString()); - - sql("ALTER TABLE %s WRITE ORDERED BY id", tableName); - - table.refresh(); - - Assert.assertEquals("Should have three sort orders", 3, table.sortOrders().size()); - Assert.assertEquals( - "Expected identity sort order on id", - "[\n" + " identity(1) ASC NULLS FIRST\n" + "]", - table.sortOrders().get(2).toString()); - - sql("ALTER TABLE %s DROP COLUMN ts", tableName); - - table.refresh(); - } -} From 5cbb866ad6217c4cfd885cafb73bad950c320b30 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 3 Mar 2023 15:41:26 +0100 Subject: [PATCH 20/36] Working again --- python/pyiceberg/io/pyarrow.py | 61 +++++++++++++++++++++++---------- python/tests/io/test_pyarrow.py | 6 ++-- python/tests/test_fokko.py | 11 ------ 3 files changed, 45 insertions(+), 33 deletions(-) delete mode 100644 python/tests/test_fokko.py diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 9f850b0d23f3..81b01707fcfb 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -45,6 +45,7 @@ import pyarrow as pa import pyarrow.compute as pc +import pyarrow.dataset as ds import pyarrow.parquet as pq from pyarrow import ChunkedArray from pyarrow.fs import ( @@ -76,6 +77,7 @@ OutputFile, OutputStream, ) +from pyiceberg.manifest import DataFile, FileFormat from pyiceberg.schema import ( PartnerAccessor, Schema, @@ -474,18 +476,34 @@ def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression: return boolean_expression_visit(expr, _ConvertToArrowExpression()) -def _read_deletes(fs: FileSystem, file_path: str) -> Dict[str, pa.ChunkedArray]: - _, path = PyArrowFileIO.parse_location(file_path) - table = pq.read_table( - source=path, pre_buffer=True, buffer_size=8 * ONE_MEGABYTE, read_dictionary=["file_path"], filesystem=fs +@lru_cache +def _get_file_format(file_format: FileFormat, **kwargs: Dict[str, Any]) -> ds.FileFormat: + if file_format == FileFormat.PARQUET: + return ds.ParquetFileFormat(**kwargs) + else: + raise ValueError(f"Unsupported file format: {file_format}") + + +def _construct_fragment(fs: FileSystem, data_file: DataFile, file_format_kwargs: Dict[str, Any] = EMPTY_DICT) -> ds.Fragment: + _, path = PyArrowFileIO.parse_location(data_file.file_path) + return _get_file_format(data_file.file_format, **file_format_kwargs).make_fragment(path, fs) + + +def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedArray]: + delete_fragment = _construct_fragment( + fs, data_file, file_format_kwargs={"dictionary_columns": ("file_path",), "pre_buffer": True, "buffer_size": ONE_MEGABYTE} ) + table = ds.Scanner.from_fragment( + fragment=delete_fragment, + ).to_table() table.unify_dictionaries() return { - file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.columns[0].chunks[0].dictionary + file.as_py(): table.filter(pc.field("file_path") == file).column("pos") + for file in table.column("file_path").chunks[0].dictionary } -def _file_to_table( +def _task_to_table( fs: FileSystem, task: FileScanTask, bound_row_filter: BooleanExpression, @@ -530,6 +548,9 @@ def _file_to_table( if positional_deletes is not None: # When there are positional deletes, create a filter mask + # there is room for optimization here, maybe we can sort + # them earlier at insertion, but then you might need to do + # additional copying sorted_deleted = iter(positional_deletes.combine_chunks().sort()) def generator() -> Generator[bool, None, None]: @@ -539,7 +560,7 @@ def generator() -> Generator[bool, None, None]: yield True try: deleted_pos = next(sorted_deleted).as_py() - except: + except StopIteration: deleted_pos = -1 else: yield False @@ -583,23 +604,25 @@ def project_table( }.union(extract_field_ids(bound_row_filter)) with ThreadPool() as pool: + # Fetch the deletes deletes_per_file: Dict[str, ChunkedArray] = {} - unique_deletes = set(chain.from_iterable([{delete_file.file_path for delete_file in task.delete_files} for task in tasks])) - deletes_per_files: List[Dict[str, ChunkedArray]] = pool.starmap( - func=_read_deletes, iterable=[(fs, delete) for delete in unique_deletes] - ) - - for delete in deletes_per_files: - for file, arr in delete.items(): - if first_arr := deletes_per_file.get(file): - deletes_per_file[file] = pa.chunked_array([first_arr, arr], first_arr.type) - else: - deletes_per_file[file] = arr + unique_deletes = set(chain.from_iterable([task.delete_files for task in tasks])) + if len(unique_deletes) > 0: + deletes_per_files: List[Dict[str, ChunkedArray]] = pool.starmap( + func=_read_deletes, iterable=[(fs, delete) for delete in unique_deletes] + ) + for delete in deletes_per_files: + for file, arr in delete.items(): + if first_arr := deletes_per_file.get(file): + deletes_per_file[file] = pa.chunked_array([first_arr, arr], first_arr.type) + else: + deletes_per_file[file] = arr + # Fetch teh data tables = [ table for table in pool.starmap( - func=_file_to_table, + func=_task_to_table, iterable=[ ( fs, diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index 0bc3627581e9..6cc3a082cd12 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -1136,6 +1136,6 @@ def deletes_file(tmp_path: str) -> str: def test_read_deletes(deletes_file: str) -> None: - # None filesystem will default to a local filesystem - deletes = _read_deletes(LocalFileSystem(), "s3://bucket/default.db/table/data.parquet", [DataFile(file_path=deletes_file)]) - assert list(deletes) == {19, 22, 25} + deletes = _read_deletes(LocalFileSystem(), DataFile(file_path=deletes_file, file_format=FileFormat.PARQUET)) + assert set(deletes.keys()) == {"s3://bucket/default.db/table/data.parquet"} + assert list(deletes.values())[0] == pa.chunked_array([[19, 22, 25]]) diff --git a/python/tests/test_fokko.py b/python/tests/test_fokko.py deleted file mode 100644 index d3a367d18fe0..000000000000 --- a/python/tests/test_fokko.py +++ /dev/null @@ -1,11 +0,0 @@ -from pyiceberg.catalog import load_catalog - - -def test_vo(): - cat = load_catalog("local") - - tbl = cat.load_table("nyc.taxis") - - df = tbl.scan().to_arrow() - - print(df) From eb72088ad29e68a148e3ea84ada5fe6f29cf3fa1 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 8 Mar 2023 13:23:06 +0100 Subject: [PATCH 21/36] Optimize using an iterator that takes multiple arrays - Added `_OrderedChunkedArrayConsumer` so we don't have to reallocate and sort into a single array - Fixed a bug where we first filtered a table, and then applied the positional deletes --- python/pyiceberg/io/pyarrow.py | 78 +++++++++++++++++++++++------ python/pyiceberg/table/__init__.py | 21 +++++--- python/tests/io/test_pyarrow.py | 80 +++++++++++++++++++++++++++--- 3 files changed, 151 insertions(+), 28 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index eedd5a5211bb..5aede27c64c0 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -517,13 +517,55 @@ def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedAr } +class _OrderedChunkedArrayConsumer: + """ + A wrapper to consume multiple individually ordered chunked-arrays + simultaneously as an ordered iterator + """ + + arrays: Tuple[pa.ChunkedArray, ...] + arrays_len: Tuple[int, ...] + arrays_pos: List[int] + + def _reset(self) -> None: + self.arrays_pos = [0] * len(self.arrays) + self.arrays_len = tuple(len(array) for array in self.arrays) + + def __init__(self, *arrays: pa.ChunkedArray) -> None: + self.arrays = arrays + self._reset() + + def __iter__(self) -> _OrderedChunkedArrayConsumer: + self._reset() + return self + + def __next__(self) -> int: + next_val = None + next_pos = None + for peek_pos in range(len(self.arrays)): + array_pos = self.arrays_pos[peek_pos] + if array_pos < self.arrays_len[peek_pos]: + peek_val = self.arrays[peek_pos][array_pos].as_py() + if next_val is None or peek_val < next_val: # type: ignore + next_val = peek_val + next_pos = peek_pos + + if next_val is not None: + # Increment the consumed array with one + self.arrays_pos[next_pos] = self.arrays_pos[next_pos] + 1 # type: ignore + + return next_val + else: + raise StopIteration + + def _task_to_table( fs: FileSystem, task: FileScanTask, bound_row_filter: BooleanExpression, projected_schema: Schema, projected_field_ids: Set[int], - positional_deletes: Optional[ChunkedArray], + positional_deletes: Optional[List[ChunkedArray]], case_sensitive: bool, ) -> Optional[pa.Table]: _, path = PyArrowFileIO.parse_location(task.file.file_path) @@ -556,32 +598,38 @@ def _task_to_table( schema=parquet_schema, pre_buffer=True, buffer_size=8 * ONE_MEGABYTE, - filters=pyarrow_filter, + # We only want to filter when there are no positional deletes, + # Otherwise we'll mess up the positions + filters=pyarrow_filter if positional_deletes is None else None, columns=[col.name for col in file_project_schema.columns], ) if positional_deletes is not None: - # When there are positional deletes, create a filter mask - # there is room for optimization here, maybe we can sort - # them earlier at insertion, but then you might need to do - # additional copying - sorted_deleted = iter(positional_deletes.combine_chunks().sort()) + # It can be that there are multiple delete files on top, + # since the delete files themselves are sorted, we need to + # weave them together using the _OrderedChunkedArrayConsumer + sorted_deleted = _OrderedChunkedArrayConsumer(*positional_deletes) def generator() -> Generator[bool, None, None]: - deleted_pos = next(sorted_deleted).as_py() + deleted_pos = next(sorted_deleted) for pos in range(len(arrow_table)): if deleted_pos == pos: - yield True + yield False try: - deleted_pos = next(sorted_deleted).as_py() + deleted_pos = next(sorted_deleted) except StopIteration: deleted_pos = -1 else: - yield False + yield True + # Filter on the positions mask = pa.array(generator(), type=pa.bool_()) arrow_table = arrow_table.filter(mask) + # Apply the user filter + if pyarrow_filter is not None: + arrow_table.filter(pyarrow_filter) + # If there is no data, we don't have to go through the schema if len(arrow_table) > 0: return to_requested_schema(projected_schema, file_project_schema, arrow_table) @@ -628,7 +676,7 @@ def project_table( with ThreadPool() as pool: # Fetch the deletes - deletes_per_file: Dict[str, ChunkedArray] = {} + deletes_per_file: Dict[str, List[ChunkedArray]] = {} unique_deletes = set(chain.from_iterable([task.delete_files for task in tasks])) if len(unique_deletes) > 0: deletes_per_files: List[Dict[str, ChunkedArray]] = pool.starmap( @@ -636,10 +684,10 @@ def project_table( ) for delete in deletes_per_files: for file, arr in delete.items(): - if first_arr := deletes_per_file.get(file): - deletes_per_file[file] = pa.chunked_array([first_arr, arr], first_arr.type) + if file in deletes_per_file: + deletes_per_file[file].append(arr) else: - deletes_per_file[file] = arr + deletes_per_file[file] = [arr] # Fetch teh data tables = [ diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index ce882926b532..d9b7052b25c5 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -322,11 +322,22 @@ def _open_manifest( metrics_evaluator: Callable[[DataFile], bool], ) -> List[DataFile]: result_manifests = files(io.new_input(manifest.manifest_path)) - if partition_filter is not None: - result_manifests = filter(partition_filter, result_manifests) + result_manifests = filter(partition_filter, result_manifests) return [file for file in result_manifests if metrics_evaluator(file)] +def _min_sequence_number(manifests: List[ManifestFile]) -> int: + try: + return min( + manifest.min_sequence_number or INITIAL_SEQUENCE_NUMBER + for manifest in manifests + if manifest.content is None or manifest.content == ManifestContent.DATA + ) + except ValueError: + # In case of an empty iterator + return INITIAL_SEQUENCE_NUMBER + + class DataScan(TableScan): def __init__( self, @@ -388,11 +399,7 @@ def plan_files(self) -> Iterable[FileScanTask]: partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) metrics_evaluator = _InclusiveMetricsEvaluator(self.table.schema(), self.row_filter, self.case_sensitive).eval - min_sequence_number = min( - manifest.min_sequence_number or INITIAL_SEQUENCE_NUMBER - for manifest in manifests - if manifest.content is None or manifest.content == ManifestContent.DATA - ) + min_sequence_number = _min_sequence_number(manifests) data_datafiles = [] deletes_positional = [] diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index 83bcc1c0a3d8..3e79345754f7 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -57,6 +57,7 @@ PyArrowFile, PyArrowFileIO, _ConvertToArrowSchema, + _OrderedChunkedArrayConsumer, _read_deletes, expression_to_pyarrow, project_table, @@ -1141,9 +1142,9 @@ def test_projection_filter_on_unknown_field(schema_int_str: Schema, file_int_str @pytest.fixture -def deletes_file(tmp_path: str) -> str: - path = "s3://bucket/default.db/table/data.parquet" - table = pa.table({"file_path": [path, path, path], "pos": [19, 22, 25]}) +def deletes_file(tmp_path: str, example_task: FileScanTask) -> str: + path = example_task.file.file_path + table = pa.table({"file_path": [path, path, path], "pos": [1, 3, 5]}) deletes_file_path = f"{tmp_path}/deletes.parquet" pq.write_table(table, deletes_file_path) @@ -1151,10 +1152,51 @@ def deletes_file(tmp_path: str) -> str: return deletes_file_path -def test_read_deletes(deletes_file: str) -> None: +def test_read_deletes(deletes_file: str, example_task: FileScanTask) -> None: deletes = _read_deletes(LocalFileSystem(), DataFile(file_path=deletes_file, file_format=FileFormat.PARQUET)) - assert set(deletes.keys()) == {"s3://bucket/default.db/table/data.parquet"} - assert list(deletes.values())[0] == pa.chunked_array([[19, 22, 25]]) + assert set(deletes.keys()) == {example_task.file.file_path} + assert list(deletes.values())[0] == pa.chunked_array([[1, 3, 5]]) + + +def test_delete(deletes_file: str, example_task: FileScanTask, table_schema_simple: Schema) -> None: + metadata_location = "file://a/b/c.json" + example_task_with_delete = FileScanTask( + data_file=example_task.file, + delete_files={DataFile(content=DataFileContent.POSITION_DELETES, file_path=deletes_file, file_format=FileFormat.PARQUET)}, + ) + + with_deletes = project_table( + tasks=[example_task_with_delete], + table=Table( + ("namespace", "table"), + metadata=TableMetadataV2( + location=metadata_location, + last_column_id=1, + format_version=2, + current_schema_id=1, + schemas=[table_schema_simple], + partition_specs=[PartitionSpec()], + ), + metadata_location=metadata_location, + io=load_file_io(), + ), + row_filter=AlwaysTrue(), + projected_schema=table_schema_simple, + case_sensitive=True, + ) + + assert ( + str(with_deletes) + == """pyarrow.Table +foo: string +bar: int64 not null +baz: bool +---- +foo: [["a","c"]] +bar: [[1,3]] +baz: [[true,null]]""" + ) + def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Schema) -> None: metadata_location = "file://a/b/c.json" @@ -1189,3 +1231,29 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc bar: [[1,2,3]] baz: [[true,false,null]]""" ) + + +def test_ordered_chunked_array_consumer() -> None: + con = _OrderedChunkedArrayConsumer(pa.chunked_array([[1, 4, 6]]), pa.chunked_array([[3, 5, 8]])) + + assert list(con) == [1, 3, 4, 5, 6, 8] + + +def test_ordered_chunked_array_consumer_single_array() -> None: + con = _OrderedChunkedArrayConsumer( + pa.chunked_array([[1, 4, 6]]), + ) + + assert list(con) == [1, 4, 6] + + +def test_ordered_chunked_array_consumer_empty_array() -> None: + con = _OrderedChunkedArrayConsumer() + + assert list(con) == [] + + +def test_ordered_chunked_array_consumer_one_empty_array() -> None: + con = _OrderedChunkedArrayConsumer(pa.chunked_array([[1, 4, 6]]), pa.chunked_array([[]])) + + assert list(con) == [1, 4, 6] From 1e27f709a618013040534b4feac41e2edb41c915 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 11 Apr 2023 00:39:13 +0200 Subject: [PATCH 22/36] Make the linters happy --- python/pyiceberg/io/pyarrow.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 3e53b1e687e5..0f41bb1948f6 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -48,7 +48,6 @@ import pyarrow as pa import pyarrow.compute as pc import pyarrow.dataset as ds -import pyarrow.parquet as pq from pyarrow import ChunkedArray from pyarrow.fs import ( FileInfo, @@ -723,7 +722,7 @@ def project_table( deletes_per_file.get(task.file.file_path), case_sensitive, rows_counter, - limit + limit, ) for task in tasks ], From 9e826ff89bce554e3e93a32353ceefddedeaeba9 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 24 Apr 2023 13:50:21 +0200 Subject: [PATCH 23/36] Fix the PR in combination with the limit --- python/pyiceberg/io/pyarrow.py | 90 +++++++++++++++++++++------------- 1 file changed, 56 insertions(+), 34 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 0f41bb1948f6..d2fc8dd892ef 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -14,7 +14,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# pylint: disable=redefined-outer-name,arguments-renamed +# pylint: disable=redefined-outer-name,arguments-renamed,fixme """FileIO implementation for reading and writing table files that uses pyarrow.fs This file contains a FileIO implementation that relies on the filesystem interface provided @@ -508,9 +508,7 @@ def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedAr delete_fragment = _construct_fragment( fs, data_file, file_format_kwargs={"dictionary_columns": ("file_path",), "pre_buffer": True, "buffer_size": ONE_MEGABYTE} ) - table = ds.Scanner.from_fragment( - fragment=delete_fragment, - ).to_table() + table = ds.Scanner.from_fragment(fragment=delete_fragment).to_table() table.unify_dictionaries() return { file.as_py(): table.filter(pc.field("file_path") == file).column("pos") @@ -560,6 +558,33 @@ def __next__(self) -> int: raise StopIteration +def _create_positional_deletes( + positional_deletes: Optional[List[pa.ChunkedArray]], fn_rows: Callable[[], int] +) -> Optional[pa.Array]: + if positional_deletes is not None and len(positional_deletes) > 0: + # It can be that there are multiple delete files on top, + # since the delete files themselves are sorted, we need to + # weave them together using the _OrderedChunkedArrayConsumer + sorted_deleted = _OrderedChunkedArrayConsumer(*positional_deletes) + + def generator() -> Generator[bool, None, None]: + deleted_pos = next(sorted_deleted) + for pos in range(fn_rows()): + if deleted_pos == pos: + yield False + try: + deleted_pos = next(sorted_deleted) + except StopIteration: + deleted_pos = -1 + else: + yield True + + # Filter on the positions + return pa.array(generator(), type=pa.bool_()) + else: + return None + + def _task_to_table( fs: FileSystem, task: FileScanTask, @@ -599,47 +624,44 @@ def _task_to_table( if file_schema is None: raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}") + mask = _create_positional_deletes(positional_deletes, fragment.count_rows) + fragment_scanner = ds.Scanner.from_fragment( fragment=fragment, schema=physical_schema, - filter=pyarrow_filter, + # This will push down the query to Arrow. + # But in case there are positional deletes, we have to apply them first + # TODO: Allow to pass in a mask here, so we can push down the positional deletes + # https://github.com/apache/arrow/issues/35301 + filter=pyarrow_filter if mask is None else None, columns=[col.name for col in file_project_schema.columns], ) + if mask is not None: + # In the case of a mask, it is a bit awkward because we first + # need to go to a table to apply the bitwise mask, and then + # the table is warped into a dataset to apply the expression + arrow_table = fragment_scanner.to_table().filter(mask) + + # Apply the user filter + if pyarrow_filter is not None: + arrow_table = arrow_table.filter(pyarrow_filter) + + if limit: + arrow_table = arrow_table.slice(0, limit) + else: + # If there are no deletes, we can just take the head + # and the user-filter is already applied + if limit: + arrow_table = fragment_scanner.head(limit) + else: + arrow_table = fragment_scanner.to_table() + if limit: - arrow_table = fragment_scanner.head(limit) with rows_counter.get_lock(): if rows_counter.value >= limit: return None rows_counter.value += len(arrow_table) - else: - arrow_table = fragment_scanner.to_table() - - if positional_deletes is not None: - # It can be that there are multiple delete files on top, - # since the delete files themselves are sorted, we need to - # weave them together using the _OrderedChunkedArrayConsumer - sorted_deleted = _OrderedChunkedArrayConsumer(*positional_deletes) - - def generator() -> Generator[bool, None, None]: - deleted_pos = next(sorted_deleted) - for pos in range(len(arrow_table)): - if deleted_pos == pos: - yield False - try: - deleted_pos = next(sorted_deleted) - except StopIteration: - deleted_pos = -1 - else: - yield True - - # Filter on the positions - mask = pa.array(generator(), type=pa.bool_()) - arrow_table = arrow_table.filter(mask) - - # Apply the user filter - if pyarrow_filter is not None: - arrow_table.filter(pyarrow_filter) # If there is no data, we don't have to go through the schema if len(arrow_table) > 0: From ac24f889e58dcc44184e5dc555873dd1f52c9190 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 2 May 2023 11:32:52 +0200 Subject: [PATCH 24/36] Use `take()` instead --- python/pyiceberg/io/pyarrow.py | 54 +++++++++++++++------------------- 1 file changed, 23 insertions(+), 31 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 2d301c992e67..41277c9b4e9d 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -570,31 +570,27 @@ def __next__(self) -> int: raise StopIteration -def _create_positional_deletes( - positional_deletes: Optional[List[pa.ChunkedArray]], fn_rows: Callable[[], int] +def _create_positional_deletes_indices( + positional_deletes: List[pa.ChunkedArray], fn_rows: Callable[[], int] ) -> Optional[pa.Array]: - if positional_deletes is not None and len(positional_deletes) > 0: - # It can be that there are multiple delete files on top, - # since the delete files themselves are sorted, we need to - # weave them together using the _OrderedChunkedArrayConsumer - sorted_deleted = _OrderedChunkedArrayConsumer(*positional_deletes) - - def generator() -> Generator[bool, None, None]: - deleted_pos = next(sorted_deleted) - for pos in range(fn_rows()): - if deleted_pos == pos: - yield False - try: - deleted_pos = next(sorted_deleted) - except StopIteration: - deleted_pos = -1 - else: - yield True - - # Filter on the positions - return pa.array(generator(), type=pa.bool_()) - else: - return None + # It can be that there are multiple delete files on top, + # since the delete files themselves are sorted, we need to + # weave them together using the _OrderedChunkedArrayConsumer + sorted_deleted = _OrderedChunkedArrayConsumer(*positional_deletes) + + def generator() -> Generator[int, None, None]: + deleted_pos = next(sorted_deleted) + for pos in range(fn_rows()): + if deleted_pos == pos: + try: + deleted_pos = next(sorted_deleted) + except StopIteration: + deleted_pos = -1 + else: + yield pos + + # Filter on the positions + return pa.array(generator(), type=pa.int64()) def pyarrow_to_schema(schema: pa.Schema) -> Schema: @@ -818,24 +814,20 @@ def _task_to_table( if file_schema is None: raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}") - mask = _create_positional_deletes(positional_deletes, fragment.count_rows) - fragment_scanner = ds.Scanner.from_fragment( fragment=fragment, schema=physical_schema, # This will push down the query to Arrow. # But in case there are positional deletes, we have to apply them first - # TODO: Allow to pass in a mask here, so we can push down the positional deletes - # https://github.com/apache/arrow/issues/35301 - filter=pyarrow_filter if mask is None else None, + filter=pyarrow_filter if positional_deletes else None, columns=[col.name for col in file_project_schema.columns], ) - if mask is not None: + if positional_deletes: # In the case of a mask, it is a bit awkward because we first # need to go to a table to apply the bitwise mask, and then # the table is warped into a dataset to apply the expression - arrow_table = fragment_scanner.to_table().filter(mask) + arrow_table = fragment_scanner.take(_create_positional_deletes_indices(positional_deletes, fragment.count_rows)) # Apply the user filter if pyarrow_filter is not None: From e9ce6453b7173206f6fd802f2368b5e5c3999de3 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 3 May 2023 16:35:46 +0200 Subject: [PATCH 25/36] Replace `_OrderedChunkedArrayConsumer` with `heapq.merge` --- python/pyiceberg/io/pyarrow.py | 52 +++------------------------------ python/tests/io/test_pyarrow.py | 27 ----------------- 2 files changed, 4 insertions(+), 75 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 41277c9b4e9d..63943794b398 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -28,6 +28,7 @@ import os from abc import ABC, abstractmethod from functools import lru_cache, singledispatch +from heapq import merge from itertools import chain from multiprocessing.pool import ThreadPool from multiprocessing.sharedctypes import Synchronized @@ -528,62 +529,17 @@ def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedAr } -class _OrderedChunkedArrayConsumer: - """ - A wrapper to consume multiple individually ordered chunked-arrays - simultaneously as an ordered iterator - """ - - arrays: Tuple[pa.ChunkedArray, ...] - arrays_len: Tuple[int, ...] - arrays_pos: List[int] - - def _reset(self) -> None: - self.arrays_pos = [0] * len(self.arrays) - self.arrays_len = tuple(len(array) for array in self.arrays) - - def __init__(self, *arrays: pa.ChunkedArray) -> None: - self.arrays = arrays - self._reset() - - def __iter__(self) -> _OrderedChunkedArrayConsumer: - self._reset() - return self - - def __next__(self) -> int: - next_val = None - next_pos = None - for peek_pos in range(len(self.arrays)): - array_pos = self.arrays_pos[peek_pos] - if array_pos < self.arrays_len[peek_pos]: - peek_val = self.arrays[peek_pos][array_pos].as_py() - if next_val is None or peek_val < next_val: # type: ignore - next_val = peek_val - next_pos = peek_pos - - if next_val is not None: - # Increment the consumed array with one - self.arrays_pos[next_pos] = self.arrays_pos[next_pos] + 1 # type: ignore - - return next_val - else: - raise StopIteration - - def _create_positional_deletes_indices( positional_deletes: List[pa.ChunkedArray], fn_rows: Callable[[], int] ) -> Optional[pa.Array]: - # It can be that there are multiple delete files on top, - # since the delete files themselves are sorted, we need to - # weave them together using the _OrderedChunkedArrayConsumer - sorted_deleted = _OrderedChunkedArrayConsumer(*positional_deletes) + sorted_deleted = merge(*positional_deletes) def generator() -> Generator[int, None, None]: - deleted_pos = next(sorted_deleted) + deleted_pos = next(sorted_deleted) # type: ignore for pos in range(fn_rows()): if deleted_pos == pos: try: - deleted_pos = next(sorted_deleted) + deleted_pos = next(sorted_deleted) # type: ignore except StopIteration: deleted_pos = -1 else: diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index 9ffc1f3abbe0..b9df3fab3e47 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -57,7 +57,6 @@ PyArrowFile, PyArrowFileIO, _ConvertToArrowSchema, - _OrderedChunkedArrayConsumer, _read_deletes, expression_to_pyarrow, project_table, @@ -1285,29 +1284,3 @@ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Sc bar: [[1,2,3]] baz: [[true,false,null]]""" ) - - -def test_ordered_chunked_array_consumer() -> None: - con = _OrderedChunkedArrayConsumer(pa.chunked_array([[1, 4, 6]]), pa.chunked_array([[3, 5, 8]])) - - assert list(con) == [1, 3, 4, 5, 6, 8] - - -def test_ordered_chunked_array_consumer_single_array() -> None: - con = _OrderedChunkedArrayConsumer( - pa.chunked_array([[1, 4, 6]]), - ) - - assert list(con) == [1, 4, 6] - - -def test_ordered_chunked_array_consumer_empty_array() -> None: - con = _OrderedChunkedArrayConsumer() - - assert list(con) == [] - - -def test_ordered_chunked_array_consumer_one_empty_array() -> None: - con = _OrderedChunkedArrayConsumer(pa.chunked_array([[1, 4, 6]]), pa.chunked_array([[]])) - - assert list(con) == [1, 4, 6] From 31c4309fe914b0c900ef801285f72c33ec4ad834 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 3 May 2023 17:03:38 +0200 Subject: [PATCH 26/36] Oops, luckily we have tests --- python/pyiceberg/io/pyarrow.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 63943794b398..5a2eaeefcfd5 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -535,11 +535,11 @@ def _create_positional_deletes_indices( sorted_deleted = merge(*positional_deletes) def generator() -> Generator[int, None, None]: - deleted_pos = next(sorted_deleted) # type: ignore + deleted_pos = next(sorted_deleted).as_py() # type: ignore for pos in range(fn_rows()): if deleted_pos == pos: try: - deleted_pos = next(sorted_deleted) # type: ignore + deleted_pos = next(sorted_deleted).as_py() # type: ignore except StopIteration: deleted_pos = -1 else: @@ -775,7 +775,7 @@ def _task_to_table( schema=physical_schema, # This will push down the query to Arrow. # But in case there are positional deletes, we have to apply them first - filter=pyarrow_filter if positional_deletes else None, + filter=pyarrow_filter if not positional_deletes else None, columns=[col.name for col in file_project_schema.columns], ) @@ -783,7 +783,8 @@ def _task_to_table( # In the case of a mask, it is a bit awkward because we first # need to go to a table to apply the bitwise mask, and then # the table is warped into a dataset to apply the expression - arrow_table = fragment_scanner.take(_create_positional_deletes_indices(positional_deletes, fragment.count_rows)) + indices = _create_positional_deletes_indices(positional_deletes, fragment.count_rows) + arrow_table = fragment_scanner.take(indices) # Apply the user filter if pyarrow_filter is not None: From 3ea8099ffb16a525852b068627083884bdd94fd7 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 10 May 2023 22:37:50 +0200 Subject: [PATCH 27/36] Moar optimizations --- python/pyiceberg/io/pyarrow.py | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 5a2eaeefcfd5..b12a4dcaf408 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -529,9 +529,7 @@ def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedAr } -def _create_positional_deletes_indices( - positional_deletes: List[pa.ChunkedArray], fn_rows: Callable[[], int] -) -> Optional[pa.Array]: +def _create_positional_deletes_indices(positional_deletes: List[pa.ChunkedArray], fn_rows: Callable[[], int]) -> pa.Array: sorted_deleted = merge(*positional_deletes) def generator() -> Generator[int, None, None]: @@ -784,14 +782,23 @@ def _task_to_table( # need to go to a table to apply the bitwise mask, and then # the table is warped into a dataset to apply the expression indices = _create_positional_deletes_indices(positional_deletes, fragment.count_rows) - arrow_table = fragment_scanner.take(indices) - - # Apply the user filter - if pyarrow_filter is not None: - arrow_table = arrow_table.filter(pyarrow_filter) if limit: - arrow_table = arrow_table.slice(0, limit) + if pyarrow_filter is not None: + # In case of the filter, we don't exactly know how many rows + # we need to fetch upfront, can be optimized in the future: + # https://github.com/apache/arrow/issues/35301 + arrow_table = fragment_scanner.take(indices) + arrow_table = arrow_table.filter(pyarrow_filter) + arrow_table = arrow_table.slice(0, limit) + else: + arrow_table = fragment_scanner.take(indices[0:limit]) + else: + arrow_table = fragment_scanner.take(indices) + # Apply the user filter + if pyarrow_filter is not None: + arrow_table = arrow_table.filter(pyarrow_filter) + else: # If there are no deletes, we can just take the head # and the user-filter is already applied From 43aa76da67d96220a3ac8a48eb98863da0e4f142 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 12 May 2023 10:30:18 +0200 Subject: [PATCH 28/36] Update python/pyiceberg/io/pyarrow.py Co-authored-by: Joris Van den Bossche --- python/pyiceberg/io/pyarrow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index b12a4dcaf408..8687d897214c 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -522,7 +522,7 @@ def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedAr fs, data_file, file_format_kwargs={"dictionary_columns": ("file_path",), "pre_buffer": True, "buffer_size": ONE_MEGABYTE} ) table = ds.Scanner.from_fragment(fragment=delete_fragment).to_table() - table.unify_dictionaries() + table = table.unify_dictionaries() return { file.as_py(): table.filter(pc.field("file_path") == file).column("pos") for file in table.column("file_path").chunks[0].dictionary From 2a653422504ea43e20fc07f66a9f158fd9e95db8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 21 May 2023 19:46:30 -0500 Subject: [PATCH 29/36] Thanks Ryan! --- python/dev/provision.py | 83 +++- python/poetry.lock | 594 ++++++++++++++++------------- python/pyiceberg/io/pyarrow.py | 45 ++- python/pyiceberg/table/__init__.py | 104 +++-- python/pyproject.toml | 5 + python/tests/io/test_pyarrow.py | 43 ++- python/tests/test_integration.py | 73 +++- 7 files changed, 614 insertions(+), 333 deletions(-) diff --git a/python/dev/provision.py b/python/dev/provision.py index 81bd094c5826..8f11a253f6c4 100644 --- a/python/dev/provision.py +++ b/python/dev/provision.py @@ -112,28 +112,95 @@ spark.sql( """ - CREATE TABLE test_deletes + CREATE TABLE test_positional_mor_deletes ( + number integer, + letter string + ) USING iceberg TBLPROPERTIES ( 'write.delete.mode'='merge-on-read', 'write.update.mode'='merge-on-read', 'write.merge.mode'='merge-on-read' + ); +""" +) + +spark.sql( + """ + INSERT INTO test_positional_mor_deletes + VALUES ( + (1, 'a'), + (2, 'b'), + (3, 'c'), + (4, 'd'), + (5, 'e'), + (6, 'f'), + (7, 'g'), + (8, 'h'), + (9, 'i'), + (10, 'j'), + (11, 'k'), + (12, 'l'), + ) +""" +) + +spark.sql( + """ + DELETE FROM test_positional_mor_deletes WHERE number % 2 = 0 +""" +) + + +spark.sql( + """ + CREATE TABLE test_positional_mor_double_deletes ( + number integer, + letter string ) - AS SELECT - 1 AS idx, - True AS deleted -UNION ALL SELECT - 2 AS idx, - False AS deleted; + USING iceberg + TBLPROPERTIES ( + 'write.delete.mode'='merge-on-read', + 'write.update.mode'='merge-on-read', + 'write.merge.mode'='merge-on-read' + ); +""" +) + +spark.sql( + """ + INSERT INTO test_positional_mor_double_deletes + VALUES ( + (1, 'a'), + (2, 'b'), + (3, 'c'), + (4, 'd'), + (5, 'e'), + (6, 'f'), + (7, 'g'), + (8, 'h'), + (9, 'i'), + (10, 'j'), + (11, 'k'), + (12, 'l'), + ) """ ) spark.sql( """ - DELETE FROM test_deletes WHERE deleted = True; + DELETE FROM test_positional_mor_double_deletes WHERE number % 2 = 1 """ ) + +spark.sql( + """ + DELETE FROM test_positional_mor_double_deletes WHERE letter == 'f' +""" +) + + all_types_dataframe = ( spark.range(0, 5, 1, 5) .withColumnRenamed("id", "longCol") diff --git a/python/poetry.lock b/python/poetry.lock index 97bcebb695e7..4b2055352fcf 100644 --- a/python/poetry.lock +++ b/python/poetry.lock @@ -1,33 +1,15 @@ -# This file is automatically @generated by Poetry and should not be changed by hand. - -[[package]] -name = "adal" -version = "1.2.7" -description = "Note: This library is already replaced by MSAL Python, available here: https://pypi.org/project/msal/ .ADAL Python remains available here as a legacy. The ADAL for Python library makes it easy for python application to authenticate to Azure Active Directory (AAD) in order to access AAD protected web resources." -category = "main" -optional = true -python-versions = "*" -files = [ - {file = "adal-1.2.7-py2.py3-none-any.whl", hash = "sha256:2a7451ed7441ddbc57703042204a3e30ef747478eea022c70f789fc7f084bc3d"}, - {file = "adal-1.2.7.tar.gz", hash = "sha256:d74f45b81317454d96e982fd1c50e6fb5c99ac2223728aea8764433a39f566f1"}, -] - -[package.dependencies] -cryptography = ">=1.1.0" -PyJWT = ">=1.0.0,<3" -python-dateutil = ">=2.1.0,<3" -requests = ">=2.0.0,<3" +# This file is automatically @generated by Poetry 1.4.0 and should not be changed by hand. [[package]] name = "adlfs" -version = "2023.1.0" +version = "2023.4.0" description = "Access Azure Datalake Gen1 with fsspec and dask" category = "main" optional = true python-versions = ">=3.8" files = [ - {file = "adlfs-2023.1.0-py3-none-any.whl", hash = "sha256:ccbdd6d33d5b7bce99a4a07c884c82fb135745d39b2d9b7b672f8e9d4d04407f"}, - {file = "adlfs-2023.1.0.tar.gz", hash = "sha256:eca53f53d88fc8e2e7a2f1d8f5a40b8a1c56aa3b541e4aa2e7eaf55a6a789262"}, + {file = "adlfs-2023.4.0-py3-none-any.whl", hash = "sha256:43f91a7478a7bb8e1521f4b9369ca6685cdae5392b2b382744f58a40c6d7c877"}, + {file = "adlfs-2023.4.0.tar.gz", hash = "sha256:84bf8875c57d6cc7e8b3f38034b117b4f43be1c7010aef9947bb5044c7b2fa37"}, ] [package.dependencies] @@ -255,37 +237,37 @@ aio = ["aiohttp (>=3.0)"] [[package]] name = "azure-datalake-store" -version = "0.0.52" +version = "0.0.53" description = "Azure Data Lake Store Filesystem Client Library for Python" category = "main" optional = true python-versions = "*" files = [ - {file = "azure-datalake-store-0.0.52.tar.gz", hash = "sha256:4198ddb32614d16d4502b43d5c9739f81432b7e0e4d75d30e05149fe6007fea2"}, - {file = "azure_datalake_store-0.0.52-py2.py3-none-any.whl", hash = "sha256:aaed72b9c856824aeab554f4dbe0ef2c6d0ff36700bdd8b93d8298793117c48e"}, + {file = "azure-datalake-store-0.0.53.tar.gz", hash = "sha256:05b6de62ee3f2a0a6e6941e6933b792b800c3e7f6ffce2fc324bc19875757393"}, + {file = "azure_datalake_store-0.0.53-py2.py3-none-any.whl", hash = "sha256:a30c902a6e360aa47d7f69f086b426729784e71c536f330b691647a51dc42b2b"}, ] [package.dependencies] -adal = ">=0.4.2" cffi = "*" +msal = ">=1.16.0,<2" requests = ">=2.20.0" [[package]] name = "azure-identity" -version = "1.12.0" +version = "1.13.0" description = "Microsoft Azure Identity Library for Python" category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "azure-identity-1.12.0.zip", hash = "sha256:7f9b1ae7d97ea7af3f38dd09305e19ab81a1e16ab66ea186b6579d85c1ca2347"}, - {file = "azure_identity-1.12.0-py3-none-any.whl", hash = "sha256:2a58ce4a209a013e37eaccfd5937570ab99e9118b3e1acf875eed3a85d541b92"}, + {file = "azure-identity-1.13.0.zip", hash = "sha256:c931c27301ffa86b07b4dcf574e29da73e3deba9ab5d1fe4f445bb6a3117e260"}, + {file = "azure_identity-1.13.0-py3-none-any.whl", hash = "sha256:bd700cebb80cd9862098587c29d8677e819beca33c62568ced6d5a8e5e332b82"}, ] [package.dependencies] azure-core = ">=1.11.0,<2.0.0" cryptography = ">=2.5" -msal = ">=1.12.0,<2.0.0" +msal = ">=1.20.0,<2.0.0" msal-extensions = ">=0.3.0,<2.0.0" six = ">=1.12.0" @@ -376,14 +358,14 @@ virtualenv = ["virtualenv (>=20.0.35)"] [[package]] name = "certifi" -version = "2022.12.7" +version = "2023.5.7" description = "Python package for providing Mozilla's CA Bundle." category = "main" optional = false python-versions = ">=3.6" files = [ - {file = "certifi-2022.12.7-py3-none-any.whl", hash = "sha256:4ad3232f5e926d6718ec31cfc1fcadfde020920e278684144551c91769c7bc18"}, - {file = "certifi-2022.12.7.tar.gz", hash = "sha256:35824b4c3a97115964b408844d64aa14db1cc518f6562e8d7261699d1350a9e3"}, + {file = "certifi-2023.5.7-py3-none-any.whl", hash = "sha256:c6c2e98f5c7869efca1f8916fed228dd91539f9f1b444c314c06eef02980c716"}, + {file = "certifi-2023.5.7.tar.gz", hash = "sha256:0f0d56dc5a6ad56fd4ba36484d6cc34451e1c6548c61daad8c320169f91eddc7"}, ] [[package]] @@ -589,63 +571,63 @@ files = [ [[package]] name = "coverage" -version = "7.2.3" +version = "7.2.5" description = "Code coverage measurement for Python" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "coverage-7.2.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e58c0d41d336569d63d1b113bd573db8363bc4146f39444125b7f8060e4e04f5"}, - {file = "coverage-7.2.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:344e714bd0fe921fc72d97404ebbdbf9127bac0ca1ff66d7b79efc143cf7c0c4"}, - {file = "coverage-7.2.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:974bc90d6f6c1e59ceb1516ab00cf1cdfbb2e555795d49fa9571d611f449bcb2"}, - {file = "coverage-7.2.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0743b0035d4b0e32bc1df5de70fba3059662ace5b9a2a86a9f894cfe66569013"}, - {file = "coverage-7.2.3-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5d0391fb4cfc171ce40437f67eb050a340fdbd0f9f49d6353a387f1b7f9dd4fa"}, - {file = "coverage-7.2.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:4a42e1eff0ca9a7cb7dc9ecda41dfc7cbc17cb1d02117214be0561bd1134772b"}, - {file = "coverage-7.2.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:be19931a8dcbe6ab464f3339966856996b12a00f9fe53f346ab3be872d03e257"}, - {file = "coverage-7.2.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:72fcae5bcac3333a4cf3b8f34eec99cea1187acd55af723bcbd559adfdcb5535"}, - {file = "coverage-7.2.3-cp310-cp310-win32.whl", hash = "sha256:aeae2aa38395b18106e552833f2a50c27ea0000122bde421c31d11ed7e6f9c91"}, - {file = "coverage-7.2.3-cp310-cp310-win_amd64.whl", hash = "sha256:83957d349838a636e768251c7e9979e899a569794b44c3728eaebd11d848e58e"}, - {file = "coverage-7.2.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:dfd393094cd82ceb9b40df4c77976015a314b267d498268a076e940fe7be6b79"}, - {file = "coverage-7.2.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:182eb9ac3f2b4874a1f41b78b87db20b66da6b9cdc32737fbbf4fea0c35b23fc"}, - {file = "coverage-7.2.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1bb1e77a9a311346294621be905ea8a2c30d3ad371fc15bb72e98bfcfae532df"}, - {file = "coverage-7.2.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca0f34363e2634deffd390a0fef1aa99168ae9ed2af01af4a1f5865e362f8623"}, - {file = "coverage-7.2.3-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:55416d7385774285b6e2a5feca0af9652f7f444a4fa3d29d8ab052fafef9d00d"}, - {file = "coverage-7.2.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:06ddd9c0249a0546997fdda5a30fbcb40f23926df0a874a60a8a185bc3a87d93"}, - {file = "coverage-7.2.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:fff5aaa6becf2c6a1699ae6a39e2e6fb0672c2d42eca8eb0cafa91cf2e9bd312"}, - {file = "coverage-7.2.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:ea53151d87c52e98133eb8ac78f1206498c015849662ca8dc246255265d9c3c4"}, - {file = "coverage-7.2.3-cp311-cp311-win32.whl", hash = "sha256:8f6c930fd70d91ddee53194e93029e3ef2aabe26725aa3c2753df057e296b925"}, - {file = "coverage-7.2.3-cp311-cp311-win_amd64.whl", hash = "sha256:fa546d66639d69aa967bf08156eb8c9d0cd6f6de84be9e8c9819f52ad499c910"}, - {file = "coverage-7.2.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:b2317d5ed777bf5a033e83d4f1389fd4ef045763141d8f10eb09a7035cee774c"}, - {file = "coverage-7.2.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:be9824c1c874b73b96288c6d3de793bf7f3a597770205068c6163ea1f326e8b9"}, - {file = "coverage-7.2.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2c3b2803e730dc2797a017335827e9da6da0e84c745ce0f552e66400abdfb9a1"}, - {file = "coverage-7.2.3-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8f69770f5ca1994cb32c38965e95f57504d3aea96b6c024624fdd5bb1aa494a1"}, - {file = "coverage-7.2.3-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:1127b16220f7bfb3f1049ed4a62d26d81970a723544e8252db0efde853268e21"}, - {file = "coverage-7.2.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:aa784405f0c640940595fa0f14064d8e84aff0b0f762fa18393e2760a2cf5841"}, - {file = "coverage-7.2.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:3146b8e16fa60427e03884301bf8209221f5761ac754ee6b267642a2fd354c48"}, - {file = "coverage-7.2.3-cp37-cp37m-win32.whl", hash = "sha256:1fd78b911aea9cec3b7e1e2622c8018d51c0d2bbcf8faaf53c2497eb114911c1"}, - {file = "coverage-7.2.3-cp37-cp37m-win_amd64.whl", hash = "sha256:0f3736a5d34e091b0a611964c6262fd68ca4363df56185902528f0b75dbb9c1f"}, - {file = "coverage-7.2.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:981b4df72c93e3bc04478153df516d385317628bd9c10be699c93c26ddcca8ab"}, - {file = "coverage-7.2.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:c0045f8f23a5fb30b2eb3b8a83664d8dc4fb58faddf8155d7109166adb9f2040"}, - {file = "coverage-7.2.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f760073fcf8f3d6933178d67754f4f2d4e924e321f4bb0dcef0424ca0215eba1"}, - {file = "coverage-7.2.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c86bd45d1659b1ae3d0ba1909326b03598affbc9ed71520e0ff8c31a993ad911"}, - {file = "coverage-7.2.3-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:172db976ae6327ed4728e2507daf8a4de73c7cc89796483e0a9198fd2e47b462"}, - {file = "coverage-7.2.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:d2a3a6146fe9319926e1d477842ca2a63fe99af5ae690b1f5c11e6af074a6b5c"}, - {file = "coverage-7.2.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:f649dd53833b495c3ebd04d6eec58479454a1784987af8afb77540d6c1767abd"}, - {file = "coverage-7.2.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:7c4ed4e9f3b123aa403ab424430b426a1992e6f4c8fd3cb56ea520446e04d152"}, - {file = "coverage-7.2.3-cp38-cp38-win32.whl", hash = "sha256:eb0edc3ce9760d2f21637766c3aa04822030e7451981ce569a1b3456b7053f22"}, - {file = "coverage-7.2.3-cp38-cp38-win_amd64.whl", hash = "sha256:63cdeaac4ae85a179a8d6bc09b77b564c096250d759eed343a89d91bce8b6367"}, - {file = "coverage-7.2.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:20d1a2a76bb4eb00e4d36b9699f9b7aba93271c9c29220ad4c6a9581a0320235"}, - {file = "coverage-7.2.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:4ea748802cc0de4de92ef8244dd84ffd793bd2e7be784cd8394d557a3c751e21"}, - {file = "coverage-7.2.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:21b154aba06df42e4b96fc915512ab39595105f6c483991287021ed95776d934"}, - {file = "coverage-7.2.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fd214917cabdd6f673a29d708574e9fbdb892cb77eb426d0eae3490d95ca7859"}, - {file = "coverage-7.2.3-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2c2e58e45fe53fab81f85474e5d4d226eeab0f27b45aa062856c89389da2f0d9"}, - {file = "coverage-7.2.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:87ecc7c9a1a9f912e306997ffee020297ccb5ea388421fe62a2a02747e4d5539"}, - {file = "coverage-7.2.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:387065e420aed3c71b61af7e82c7b6bc1c592f7e3c7a66e9f78dd178699da4fe"}, - {file = "coverage-7.2.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:ea3f5bc91d7d457da7d48c7a732beaf79d0c8131df3ab278e6bba6297e23c6c4"}, - {file = "coverage-7.2.3-cp39-cp39-win32.whl", hash = "sha256:ae7863a1d8db6a014b6f2ff9c1582ab1aad55a6d25bac19710a8df68921b6e30"}, - {file = "coverage-7.2.3-cp39-cp39-win_amd64.whl", hash = "sha256:3f04becd4fcda03c0160d0da9c8f0c246bc78f2f7af0feea1ec0930e7c93fa4a"}, - {file = "coverage-7.2.3-pp37.pp38.pp39-none-any.whl", hash = "sha256:965ee3e782c7892befc25575fa171b521d33798132692df428a09efacaffe8d0"}, - {file = "coverage-7.2.3.tar.gz", hash = "sha256:d298c2815fa4891edd9abe5ad6e6cb4207104c7dd9fd13aea3fdebf6f9b91259"}, + {file = "coverage-7.2.5-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:883123d0bbe1c136f76b56276074b0c79b5817dd4238097ffa64ac67257f4b6c"}, + {file = "coverage-7.2.5-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:d2fbc2a127e857d2f8898aaabcc34c37771bf78a4d5e17d3e1f5c30cd0cbc62a"}, + {file = "coverage-7.2.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5f3671662dc4b422b15776cdca89c041a6349b4864a43aa2350b6b0b03bbcc7f"}, + {file = "coverage-7.2.5-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:780551e47d62095e088f251f5db428473c26db7829884323e56d9c0c3118791a"}, + {file = "coverage-7.2.5-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:066b44897c493e0dcbc9e6a6d9f8bbb6607ef82367cf6810d387c09f0cd4fe9a"}, + {file = "coverage-7.2.5-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:b9a4ee55174b04f6af539218f9f8083140f61a46eabcaa4234f3c2a452c4ed11"}, + {file = "coverage-7.2.5-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:706ec567267c96717ab9363904d846ec009a48d5f832140b6ad08aad3791b1f5"}, + {file = "coverage-7.2.5-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:ae453f655640157d76209f42c62c64c4d4f2c7f97256d3567e3b439bd5c9b06c"}, + {file = "coverage-7.2.5-cp310-cp310-win32.whl", hash = "sha256:f81c9b4bd8aa747d417407a7f6f0b1469a43b36a85748145e144ac4e8d303cb5"}, + {file = "coverage-7.2.5-cp310-cp310-win_amd64.whl", hash = "sha256:dc945064a8783b86fcce9a0a705abd7db2117d95e340df8a4333f00be5efb64c"}, + {file = "coverage-7.2.5-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:40cc0f91c6cde033da493227797be2826cbf8f388eaa36a0271a97a332bfd7ce"}, + {file = "coverage-7.2.5-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a66e055254a26c82aead7ff420d9fa8dc2da10c82679ea850d8feebf11074d88"}, + {file = "coverage-7.2.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c10fbc8a64aa0f3ed136b0b086b6b577bc64d67d5581acd7cc129af52654384e"}, + {file = "coverage-7.2.5-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9a22cbb5ede6fade0482111fa7f01115ff04039795d7092ed0db43522431b4f2"}, + {file = "coverage-7.2.5-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:292300f76440651529b8ceec283a9370532f4ecba9ad67d120617021bb5ef139"}, + {file = "coverage-7.2.5-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:7ff8f3fb38233035028dbc93715551d81eadc110199e14bbbfa01c5c4a43f8d8"}, + {file = "coverage-7.2.5-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:a08c7401d0b24e8c2982f4e307124b671c6736d40d1c39e09d7a8687bddf83ed"}, + {file = "coverage-7.2.5-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:ef9659d1cda9ce9ac9585c045aaa1e59223b143f2407db0eaee0b61a4f266fb6"}, + {file = "coverage-7.2.5-cp311-cp311-win32.whl", hash = "sha256:30dcaf05adfa69c2a7b9f7dfd9f60bc8e36b282d7ed25c308ef9e114de7fc23b"}, + {file = "coverage-7.2.5-cp311-cp311-win_amd64.whl", hash = "sha256:97072cc90f1009386c8a5b7de9d4fc1a9f91ba5ef2146c55c1f005e7b5c5e068"}, + {file = "coverage-7.2.5-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:bebea5f5ed41f618797ce3ffb4606c64a5de92e9c3f26d26c2e0aae292f015c1"}, + {file = "coverage-7.2.5-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:828189fcdda99aae0d6bf718ea766b2e715eabc1868670a0a07bf8404bf58c33"}, + {file = "coverage-7.2.5-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6e8a95f243d01ba572341c52f89f3acb98a3b6d1d5d830efba86033dd3687ade"}, + {file = "coverage-7.2.5-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e8834e5f17d89e05697c3c043d3e58a8b19682bf365048837383abfe39adaed5"}, + {file = "coverage-7.2.5-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:d1f25ee9de21a39b3a8516f2c5feb8de248f17da7eead089c2e04aa097936b47"}, + {file = "coverage-7.2.5-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:1637253b11a18f453e34013c665d8bf15904c9e3c44fbda34c643fbdc9d452cd"}, + {file = "coverage-7.2.5-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:8e575a59315a91ccd00c7757127f6b2488c2f914096077c745c2f1ba5b8c0969"}, + {file = "coverage-7.2.5-cp37-cp37m-win32.whl", hash = "sha256:509ecd8334c380000d259dc66feb191dd0a93b21f2453faa75f7f9cdcefc0718"}, + {file = "coverage-7.2.5-cp37-cp37m-win_amd64.whl", hash = "sha256:12580845917b1e59f8a1c2ffa6af6d0908cb39220f3019e36c110c943dc875b0"}, + {file = "coverage-7.2.5-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:b5016e331b75310610c2cf955d9f58a9749943ed5f7b8cfc0bb89c6134ab0a84"}, + {file = "coverage-7.2.5-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:373ea34dca98f2fdb3e5cb33d83b6d801007a8074f992b80311fc589d3e6b790"}, + {file = "coverage-7.2.5-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a063aad9f7b4c9f9da7b2550eae0a582ffc7623dca1c925e50c3fbde7a579771"}, + {file = "coverage-7.2.5-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:38c0a497a000d50491055805313ed83ddba069353d102ece8aef5d11b5faf045"}, + {file = "coverage-7.2.5-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a2b3b05e22a77bb0ae1a3125126a4e08535961c946b62f30985535ed40e26614"}, + {file = "coverage-7.2.5-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:0342a28617e63ad15d96dca0f7ae9479a37b7d8a295f749c14f3436ea59fdcb3"}, + {file = "coverage-7.2.5-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:cf97ed82ca986e5c637ea286ba2793c85325b30f869bf64d3009ccc1a31ae3fd"}, + {file = "coverage-7.2.5-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:c2c41c1b1866b670573657d584de413df701f482574bad7e28214a2362cb1fd1"}, + {file = "coverage-7.2.5-cp38-cp38-win32.whl", hash = "sha256:10b15394c13544fce02382360cab54e51a9e0fd1bd61ae9ce012c0d1e103c813"}, + {file = "coverage-7.2.5-cp38-cp38-win_amd64.whl", hash = "sha256:a0b273fe6dc655b110e8dc89b8ec7f1a778d78c9fd9b4bda7c384c8906072212"}, + {file = "coverage-7.2.5-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:5c587f52c81211d4530fa6857884d37f514bcf9453bdeee0ff93eaaf906a5c1b"}, + {file = "coverage-7.2.5-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:4436cc9ba5414c2c998eaedee5343f49c02ca93b21769c5fdfa4f9d799e84200"}, + {file = "coverage-7.2.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6599bf92f33ab041e36e06d25890afbdf12078aacfe1f1d08c713906e49a3fe5"}, + {file = "coverage-7.2.5-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:857abe2fa6a4973f8663e039ead8d22215d31db613ace76e4a98f52ec919068e"}, + {file = "coverage-7.2.5-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f6f5cab2d7f0c12f8187a376cc6582c477d2df91d63f75341307fcdcb5d60303"}, + {file = "coverage-7.2.5-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:aa387bd7489f3e1787ff82068b295bcaafbf6f79c3dad3cbc82ef88ce3f48ad3"}, + {file = "coverage-7.2.5-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:156192e5fd3dbbcb11cd777cc469cf010a294f4c736a2b2c891c77618cb1379a"}, + {file = "coverage-7.2.5-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:bd3b4b8175c1db502adf209d06136c000df4d245105c8839e9d0be71c94aefe1"}, + {file = "coverage-7.2.5-cp39-cp39-win32.whl", hash = "sha256:ddc5a54edb653e9e215f75de377354e2455376f416c4378e1d43b08ec50acc31"}, + {file = "coverage-7.2.5-cp39-cp39-win_amd64.whl", hash = "sha256:338aa9d9883aaaad53695cb14ccdeb36d4060485bb9388446330bef9c361c252"}, + {file = "coverage-7.2.5-pp37.pp38.pp39-none-any.whl", hash = "sha256:8877d9b437b35a85c18e3c6499b23674684bf690f5d96c1006a1ef61f9fdf0f3"}, + {file = "coverage-7.2.5.tar.gz", hash = "sha256:f99ef080288f09ffc687423b8d60978cf3a465d3f404a18d1a05474bd8575a47"}, ] [package.dependencies] @@ -710,71 +692,71 @@ files = [ [[package]] name = "docutils" -version = "0.19" +version = "0.20.1" description = "Docutils -- Python Documentation Utilities" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "docutils-0.19-py3-none-any.whl", hash = "sha256:5e1de4d849fee02c63b040a4a3fd567f4ab104defd8a5511fbbc24a8a017efbc"}, - {file = "docutils-0.19.tar.gz", hash = "sha256:33995a6753c30b7f577febfc2c50411fec6aac7f7ffeb7c4cfe5991072dcf9e6"}, + {file = "docutils-0.20.1-py3-none-any.whl", hash = "sha256:96f387a2c5562db4476f09f13bbab2192e764cac08ebbf3a34a95d9b1e4a59d6"}, + {file = "docutils-0.20.1.tar.gz", hash = "sha256:f08a4e276c3a1583a86dce3e34aba3fe04d02bba2dd51ed16106244e8a923e3b"}, ] [[package]] name = "duckdb" -version = "0.7.1" +version = "0.8.0" description = "DuckDB embedded database" category = "main" optional = true python-versions = "*" files = [ - {file = "duckdb-0.7.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:3e0170be6cc315c179169dfa3e06485ef7009ef8ce399cd2908f29105ef2c67b"}, - {file = "duckdb-0.7.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:6360d41023e726646507d5479ba60960989a09f04527b36abeef3643c61d8c48"}, - {file = "duckdb-0.7.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:578c269d7aa27184e8d45421694f89deda3f41fe6bd2a8ce48b262b9fc975326"}, - {file = "duckdb-0.7.1-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:36aae9a923c9f78da1cf3fcf75873f62d32ea017d4cef7c706d16d3eca527ca2"}, - {file = "duckdb-0.7.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:630e0122a02f19bb1fafae00786350b2c31ae8422fce97c827bd3686e7c386af"}, - {file = "duckdb-0.7.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:9b9ca2d294725e523ce207bc37f28787478ae6f7a223e2cf3a213a2d498596c3"}, - {file = "duckdb-0.7.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:0bd89f388205b6c99b62650169efe9a02933555ee1d46ddf79fbd0fb9e62652b"}, - {file = "duckdb-0.7.1-cp310-cp310-win32.whl", hash = "sha256:a9e987565a268fd8da9f65e54621d28f39c13105b8aee34c96643074babe6d9c"}, - {file = "duckdb-0.7.1-cp310-cp310-win_amd64.whl", hash = "sha256:5d986b5ad1307b069309f9707c0c5051323e29865aefa059eb6c3b22dc9751b6"}, - {file = "duckdb-0.7.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:54606dfd24d7181d3098030ca6858f6be52f3ccbf42fff05f7587f2d9cdf4343"}, - {file = "duckdb-0.7.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:bd9367ae650b6605ffe00412183cf0edb688a5fc9fbb03ed757e8310e7ec3b6c"}, - {file = "duckdb-0.7.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:aaf33aeb543c7816bd915cd10141866d54f92f698e1b5712de9d8b7076da19df"}, - {file = "duckdb-0.7.1-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2e56b0329c38c0356b40449917bab6fce6ac27d356257b9a9da613d2a0f064e0"}, - {file = "duckdb-0.7.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:604b8b476d6cc6bf91625d8c2722ef9c50c402b3d64bc518c838d6c279e6d93b"}, - {file = "duckdb-0.7.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:32a268508c6d7fdc99d5442736051de74c28a5166c4cc3dcbbf35d383299b941"}, - {file = "duckdb-0.7.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:90794406fa2111414877ee9db154fef940911f3920c312c1cf69947621737c8d"}, - {file = "duckdb-0.7.1-cp311-cp311-win32.whl", hash = "sha256:bf20c5ee62cbbf10b39ebdfd70d454ce914e70545c7cb6cb78cb5befef96328a"}, - {file = "duckdb-0.7.1-cp311-cp311-win_amd64.whl", hash = "sha256:bb2700785cab37cd1e7a76c4547a5ab0f8a7c28ad3f3e4d02a8fae52be223090"}, - {file = "duckdb-0.7.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:b09741cfa31388b8f9cdf5c5200e0995d55a5b54d2d1a75b54784e2f5c042f7f"}, - {file = "duckdb-0.7.1-cp36-cp36m-win32.whl", hash = "sha256:766e6390f7ace7f1e322085c2ca5d0ad94767bde78a38d168253d2b0b4d5cd5c"}, - {file = "duckdb-0.7.1-cp36-cp36m-win_amd64.whl", hash = "sha256:6a3f3315e2b553db3463f07324f62dfebaf3b97656a87558e59e2f1f816eaf15"}, - {file = "duckdb-0.7.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:278edb8c912d836b3b77fd1695887e1dbd736137c3912478af3608c9d7307bb0"}, - {file = "duckdb-0.7.1-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e189b558d10b58fe6ed85ce79f728e143eb4115db1e63147a44db613cd4dd0d9"}, - {file = "duckdb-0.7.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b91ec3544ee4dc9e6abbdf2669475d5adedaaea51987c67acf161673e6b7443"}, - {file = "duckdb-0.7.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:3fe3f3dbd62b76a773144eef31aa29794578c359da932e77fef04516535318ca"}, - {file = "duckdb-0.7.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:1e78c7f59325e99f0b3d9fe7c2bad4aaadf42d2c7711925cc26331d7647a91b2"}, - {file = "duckdb-0.7.1-cp37-cp37m-win32.whl", hash = "sha256:bc2a12d9f4fc8ef2fd1022d610287c9fc9972ea06b7510fc87387f1fa256a390"}, - {file = "duckdb-0.7.1-cp37-cp37m-win_amd64.whl", hash = "sha256:53e3db1bc0f445ee48b23cde47bfba08c7fa5a69976c740ec8cdf89543d2405d"}, - {file = "duckdb-0.7.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:1247cc11bac17f2585d11681329806c86295e32242f84a10a604665e697d5c81"}, - {file = "duckdb-0.7.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:5feaff16a012075b49dfa09d4cb24455938d6b0e06b08e1404ec00089119dba2"}, - {file = "duckdb-0.7.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:b411a0c361eab9b26dcd0d0c7a0d1bc0ad6b214068555de7e946fbdd2619961a"}, - {file = "duckdb-0.7.1-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c7c76d8694ecdb579241ecfeaf03c51d640b984dbbe8e1d9f919089ebf3cdea6"}, - {file = "duckdb-0.7.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:193b896eed44d8751a755ccf002a137630020af0bc3505affa21bf19fdc90df3"}, - {file = "duckdb-0.7.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:7da132ee452c80a3784b8daffd86429fa698e1b0e3ecb84660db96d36c27ad55"}, - {file = "duckdb-0.7.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:5fd08c97c3e8cb5bec3822cf78b966b489213dcaab24b25c05a99f7caf8db467"}, - {file = "duckdb-0.7.1-cp38-cp38-win32.whl", hash = "sha256:9cb956f94fa55c4782352dac7cc7572a58312bd7ce97332bb14591d6059f0ea4"}, - {file = "duckdb-0.7.1-cp38-cp38-win_amd64.whl", hash = "sha256:289a5f65213e66d320ebcd51a94787e7097b9d1c3492d01a121a2c809812bf19"}, - {file = "duckdb-0.7.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:8085ad58c9b5854ee3820804fa1797e6b3134429c1506c3faab3cb96e71b07e9"}, - {file = "duckdb-0.7.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b47c19d1f2f662a5951fc6c5f6939d0d3b96689604b529cdcffd9afdcc95bff2"}, - {file = "duckdb-0.7.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:6a611f598226fd634b7190f509cc6dd668132ffe436b0a6b43847b4b32b99e4a"}, - {file = "duckdb-0.7.1-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6730f03b5b78f3943b752c90bdf37b62ae3ac52302282a942cc675825b4a8dc9"}, - {file = "duckdb-0.7.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fe23e938d29cd8ea6953d77dc828b7f5b95a4dbc7cd7fe5bcc3531da8cec3dba"}, - {file = "duckdb-0.7.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:feffe503c2e2a99480e1e5e15176f37796b3675e4dadad446fe7c2cc672aed3c"}, - {file = "duckdb-0.7.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:72fceb06f5bf24ad6bb5974c60d397a7a7e61b3d847507a22276de076f3392e2"}, - {file = "duckdb-0.7.1-cp39-cp39-win32.whl", hash = "sha256:c4d5217437d20d05fe23317bbc161befa1f9363f3622887cd1d2f4719b407936"}, - {file = "duckdb-0.7.1-cp39-cp39-win_amd64.whl", hash = "sha256:066885e1883464ce3b7d1fd844f9431227dcffe1ee39bfd2a05cd6d53f304557"}, - {file = "duckdb-0.7.1.tar.gz", hash = "sha256:a7db6da0366b239ea1e4541fcc19556b286872f5015c9a54c2e347146e25a2ad"}, + {file = "duckdb-0.8.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6455aee00af30770c20f4a8c5e4347918cf59b578f49ee996a13807b12911871"}, + {file = "duckdb-0.8.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b8cf0622ae7f86d4ce72791f8928af4357a46824aadf1b6879c7936b3db65344"}, + {file = "duckdb-0.8.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6132e8183ca3ae08a593e43c97cb189794077dedd48546e27ce43bd6a51a9c33"}, + {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fe29e5343fa2a95f2cde4519a4f4533f4fd551a48d2d9a8ab5220d40ebf53610"}, + {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:945165987ca87c097dc0e578dcf47a100cad77e1c29f5dd8443d53ce159dc22e"}, + {file = "duckdb-0.8.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:673c60daf7ada1d9a8518286a6893ec45efabb64602954af5f3d98f42912fda6"}, + {file = "duckdb-0.8.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d5075fe1ff97ae62331ca5c61e3597e6e9f7682a6fdd418c23ba5c4873ed5cd1"}, + {file = "duckdb-0.8.0-cp310-cp310-win32.whl", hash = "sha256:001f5102f45d3d67f389fa8520046c8f55a99e2c6d43b8e68b38ea93261c5395"}, + {file = "duckdb-0.8.0-cp310-cp310-win_amd64.whl", hash = "sha256:cb00800f2e1e865584b13221e0121fce9341bb3a39a93e569d563eaed281f528"}, + {file = "duckdb-0.8.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:b2707096d6df4321044fcde2c9f04da632d11a8be60957fd09d49a42fae71a29"}, + {file = "duckdb-0.8.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b27df1b70ae74d2c88efb5ffca8490954fdc678099509a9c4404ca30acc53426"}, + {file = "duckdb-0.8.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:75a97c800271b52dd0f37696d074c50576dcb4b2750b6115932a98696a268070"}, + {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:804cac261a5e016506a6d67838a65d19b06a237f7949f1704f0e800eb708286a"}, + {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c6b9abca7fa6713e1d031c18485343b4de99742c7e1b85c10718aa2f31a4e2c6"}, + {file = "duckdb-0.8.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:51aa6d606d49072abcfeb3be209eb559ac94c1b5e70f58ac3adbb94aca9cd69f"}, + {file = "duckdb-0.8.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:7c8dc769aaf2be0a1c57995ca657e5b92c1c56fc8437edb720ca6cab571adf14"}, + {file = "duckdb-0.8.0-cp311-cp311-win32.whl", hash = "sha256:c4207d18b42387c4a035846d8878eb967070198be8ac26fd77797ce320d1a400"}, + {file = "duckdb-0.8.0-cp311-cp311-win_amd64.whl", hash = "sha256:0c392257547c20794c3072fcbca99a49ef0a49974005d755e93893e2b4875267"}, + {file = "duckdb-0.8.0-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:2832379e122020814dbe869af7b9ddf3c9f21474cf345531145b099c63ffe17e"}, + {file = "duckdb-0.8.0-cp36-cp36m-win32.whl", hash = "sha256:914896526f7caba86b170f2c4f17f11fd06540325deeb0000cb4fb24ec732966"}, + {file = "duckdb-0.8.0-cp36-cp36m-win_amd64.whl", hash = "sha256:022ebda86d0e3204cdc206e4af45aa9f0ae0668b34c2c68cf88e08355af4a372"}, + {file = "duckdb-0.8.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:96a31c0f3f4ccbf0f5b18f94319f37691205d82f80aae48c6fe04860d743eb2c"}, + {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a07c73c6e6a8cf4ce1a634625e0d1b17e5b817242a8a530d26ed84508dfbdc26"}, + {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:424acbd6e857531b06448d757d7c2557938dbddbff0632092090efbf413b4699"}, + {file = "duckdb-0.8.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c83cfd2a868f1acb0692b9c3fd5ef1d7da8faa1348c6eabf421fbf5d8c2f3eb8"}, + {file = "duckdb-0.8.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:5c6f6b2d8db56936f662c649539df81856b5a8cb769a31f9544edf18af2a11ff"}, + {file = "duckdb-0.8.0-cp37-cp37m-win32.whl", hash = "sha256:0bd6376b40a512172eaf4aa816813b1b9d68994292ca436ce626ccd5f77f8184"}, + {file = "duckdb-0.8.0-cp37-cp37m-win_amd64.whl", hash = "sha256:931221885bcf1e7dfce2400f11fd048a7beef566b775f1453bb1db89b828e810"}, + {file = "duckdb-0.8.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:42e7853d963d68e72403ea208bcf806b0f28c7b44db0aa85ce49bb124d56c133"}, + {file = "duckdb-0.8.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:fcc338399175be3d43366576600aef7d72e82114d415992a7a95aded98a0f3fd"}, + {file = "duckdb-0.8.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:03dd08a4624d6b581a59f9f9dbfd34902416398d16795ad19f92361cf21fd9b5"}, + {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0c7c24ea0c9d8563dbd5ad49ccb54b7a9a3c7b8c2833d35e5d32a08549cacea5"}, + {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cb58f6505cc0f34b4e976154302d26563d2e5d16b206758daaa04b65e55d9dd8"}, + {file = "duckdb-0.8.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ef37ac7880100c4b3f913c8483a29a13f8289313b9a07df019fadfa8e7427544"}, + {file = "duckdb-0.8.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:c2a4f5ee913ca8a6a069c78f8944b9934ffdbc71fd935f9576fdcea2a6f476f1"}, + {file = "duckdb-0.8.0-cp38-cp38-win32.whl", hash = "sha256:73831c6d7aefcb5f4072cd677b9efebecbf6c578946d21710791e10a1fc41b9a"}, + {file = "duckdb-0.8.0-cp38-cp38-win_amd64.whl", hash = "sha256:faa36d2854734364d234f37d7ef4f3d763b73cd6b0f799cbc2a0e3b7e2575450"}, + {file = "duckdb-0.8.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:50a31ec237ed619e50f9ab79eb0ec5111eb9697d4475da6e0ab22c08495ce26b"}, + {file = "duckdb-0.8.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:351abb4cc2d229d043920c4bc2a4c29ca31a79fef7d7ef8f6011cf4331f297bf"}, + {file = "duckdb-0.8.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:568550a163aca6a787bef8313e358590254de3f4019025a8d68c3a61253fedc1"}, + {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2b82617f0e7f9fc080eda217090d82b42d4fad083bc9f6d58dfda9cecb7e3b29"}, + {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d01c9be34d272532b75e8faedda0ff77fa76d1034cde60b8f5768ae85680d6d3"}, + {file = "duckdb-0.8.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:8549d6a6bf5f00c012b6916f605416226507e733a3ffc57451682afd6e674d1b"}, + {file = "duckdb-0.8.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8d145c6d51e55743c3ed1a74cffa109d9e72f82b07e203b436cfa453c925313a"}, + {file = "duckdb-0.8.0-cp39-cp39-win32.whl", hash = "sha256:f8610dfd21e90d7b04e8598b244bf3ad68599fd6ba0daad3428c03cbfd74dced"}, + {file = "duckdb-0.8.0-cp39-cp39-win_amd64.whl", hash = "sha256:d0f0f104d30418808bafbe9bccdcd238588a07bd246b3cff13842d60bfd8e8ba"}, + {file = "duckdb-0.8.0.tar.gz", hash = "sha256:c68da35bab5072a64ada2646a5b343da620ddc75a7a6e84aa4a1e0628a7ec18f"}, ] [[package]] @@ -941,14 +923,14 @@ files = [ [[package]] name = "fsspec" -version = "2023.4.0" +version = "2023.5.0" description = "File-system specification" category = "main" optional = false python-versions = ">=3.8" files = [ - {file = "fsspec-2023.4.0-py3-none-any.whl", hash = "sha256:f398de9b49b14e9d84d2c2d11b7b67121bc072fe97b930c4e5668ac3917d8307"}, - {file = "fsspec-2023.4.0.tar.gz", hash = "sha256:bf064186cd8808f0b2f6517273339ba0a0c8fb1b7048991c28bc67f58b8b67cd"}, + {file = "fsspec-2023.5.0-py3-none-any.whl", hash = "sha256:51a4ad01a5bb66fcc58036e288c0d53d3975a0df2a5dc59a93b59bade0391f2a"}, + {file = "fsspec-2023.5.0.tar.gz", hash = "sha256:b3b56e00fb93ea321bc9e5d9cf6f8522a0198b20eb24e02774d329e9c6fb84ce"}, ] [package.extras] @@ -1036,16 +1018,74 @@ six = ">=1.5.2" [package.extras] protobuf = ["grpcio-tools (>=1.49.1)"] +[[package]] +name = "grpcio" +version = "1.51.3" +description = "HTTP/2-based RPC framework" +category = "main" +optional = true +python-versions = ">=3.7" +files = [ + {file = "grpcio-1.51.3-cp310-cp310-linux_armv7l.whl", hash = "sha256:f601aaeae18dab81930fb8d4f916b0da21e89bb4b5f7367ef793f46b4a76b7b0"}, + {file = "grpcio-1.51.3-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:eef0450a4b5ed11feab639bf3eb1b6e23d0efa9b911bf7b06fb60e14f5f8a585"}, + {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:82b0ad8ac825d4bb31bff9f638557c045f4a6d824d84b21e893968286f88246b"}, + {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3667c06e37d6cd461afdd51cefe6537702f3d1dc5ff4cac07e88d8b4795dc16f"}, + {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3709048fe0aa23dda09b3e69849a12055790171dab9e399a72ea8f9dfbf9ac80"}, + {file = "grpcio-1.51.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:200d69857f9910f7458b39b9bcf83ee4a180591b40146ba9e49314e3a7419313"}, + {file = "grpcio-1.51.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:cd9a5e68e79c5f031500e67793048a90209711e0854a9ddee8a3ce51728de4e5"}, + {file = "grpcio-1.51.3-cp310-cp310-win32.whl", hash = "sha256:6604f614016127ae10969176bbf12eb0e03d2fb3d643f050b3b69e160d144fb4"}, + {file = "grpcio-1.51.3-cp310-cp310-win_amd64.whl", hash = "sha256:e95c7ccd4c5807adef1602005513bf7c7d14e5a41daebcf9d8d30d8bf51b8f81"}, + {file = "grpcio-1.51.3-cp311-cp311-linux_armv7l.whl", hash = "sha256:5e77ee138100f0bb55cbd147840f87ee6241dbd25f09ea7cd8afe7efff323449"}, + {file = "grpcio-1.51.3-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:68a7514b754e38e8de9075f7bb4dee919919515ec68628c43a894027e40ddec4"}, + {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3c1b9f8afa62ff265d86a4747a2990ec5a96e4efce5d5888f245a682d66eca47"}, + {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8de30f0b417744288cec65ec8cf84b8a57995cf7f1e84ccad2704d93f05d0aae"}, + {file = "grpcio-1.51.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:b69c7adc7ed60da1cb1b502853db61f453fc745f940cbcc25eb97c99965d8f41"}, + {file = "grpcio-1.51.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d81528ffe0e973dc840ec73a4132fd18b8203ad129d7410155d951a0a7e4f5d0"}, + {file = "grpcio-1.51.3-cp311-cp311-win32.whl", hash = "sha256:040eb421613b57c696063abde405916dd830203c184c9000fc8c3b3b3c950325"}, + {file = "grpcio-1.51.3-cp311-cp311-win_amd64.whl", hash = "sha256:2a8e17286c4240137d933b8ca506465472248b4ce0fe46f3404459e708b65b68"}, + {file = "grpcio-1.51.3-cp37-cp37m-linux_armv7l.whl", hash = "sha256:d5cd1389669a847555df54177b911d9ff6f17345b2a6f19388707b7a9f724c88"}, + {file = "grpcio-1.51.3-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:be1bf35ce82cdbcac14e39d5102d8de4079a1c1a6a06b68e41fcd9ef64f9dd28"}, + {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:5eed34994c095e2bf7194ffac7381c6068b057ef1e69f8f08db77771350a7566"}, + {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3f9a7d88082b2a17ae7bd3c2354d13bab0453899e0851733f6afa6918373f476"}, + {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:36c8abbc5f837111e7bd619612eedc223c290b0903b952ce0c7b00840ea70f14"}, + {file = "grpcio-1.51.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:165b05af77e6aecb4210ae7663e25acf234ba78a7c1c157fa5f2efeb0d6ec53c"}, + {file = "grpcio-1.51.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:54e36c2ee304ff15f2bfbdc43d2b56c63331c52d818c364e5b5214e5bc2ad9f6"}, + {file = "grpcio-1.51.3-cp37-cp37m-win32.whl", hash = "sha256:cd0daac21d9ef5e033a5100c1d3aa055bbed28bfcf070b12d8058045c4e821b1"}, + {file = "grpcio-1.51.3-cp37-cp37m-win_amd64.whl", hash = "sha256:2fdd6333ce96435408565a9dbbd446212cd5d62e4d26f6a3c0feb1e3c35f1cc8"}, + {file = "grpcio-1.51.3-cp38-cp38-linux_armv7l.whl", hash = "sha256:54b0c29bdd9a3b1e1b61443ab152f060fc719f1c083127ab08d03fac5efd51be"}, + {file = "grpcio-1.51.3-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:ffaaf7e93fcb437356b5a4b23bf36e8a3d0221399ff77fd057e4bc77776a24be"}, + {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:eafbe7501a3268d05f2e450e1ddaffb950d842a8620c13ec328b501d25d2e2c3"}, + {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:881ecb34feabf31c6b3b9bbbddd1a5b57e69f805041e5a2c6c562a28574f71c4"}, + {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e860a3222139b41d430939bbec2ec9c3f6c740938bf7a04471a9a8caaa965a2e"}, + {file = "grpcio-1.51.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:49ede0528e9dac7e8a9fe30b16c73b630ddd9a576bf4b675eb6b0c53ee5ca00f"}, + {file = "grpcio-1.51.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6972b009638b40a448d10e1bc18e2223143b8a7aa20d7def0d78dd4af4126d12"}, + {file = "grpcio-1.51.3-cp38-cp38-win32.whl", hash = "sha256:5694448256e3cdfe5bd358f1574a3f2f51afa20cc834713c4b9788d60b7cc646"}, + {file = "grpcio-1.51.3-cp38-cp38-win_amd64.whl", hash = "sha256:3ea4341efe603b049e8c9a5f13c696ca37fcdf8a23ca35f650428ad3606381d9"}, + {file = "grpcio-1.51.3-cp39-cp39-linux_armv7l.whl", hash = "sha256:6c677581ce129f5fa228b8f418cee10bd28dd449f3a544ea73c8ba590ee49d0b"}, + {file = "grpcio-1.51.3-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:30e09b5e0531685e176f49679b6a3b190762cc225f4565e55a899f5e14b3aa62"}, + {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:c831f31336e81243f85b6daff3e5e8a123302ce0ea1f2726ad752fd7a59f3aee"}, + {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2cd2e4cefb724cab1ba2df4b7535a9980531b9ec51b4dbb5f137a1f3a3754ef0"}, + {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7a0d0bf44438869d307f85a54f25a896ad6b4b0ca12370f76892ad732928d87"}, + {file = "grpcio-1.51.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:c02abd55409bfb293371554adf6a4401197ec2133dd97727c01180889014ba4d"}, + {file = "grpcio-1.51.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:2f8ff75e61e1227ba7a3f16b2eadbcc11d0a54096d52ab75a6b88cfbe56f55d1"}, + {file = "grpcio-1.51.3-cp39-cp39-win32.whl", hash = "sha256:6c99a73a6260bdf844b2e5ddad02dcd530310f80e1fa72c300fa19c1c7496962"}, + {file = "grpcio-1.51.3-cp39-cp39-win_amd64.whl", hash = "sha256:22bdfac4f7f27acdd4da359b5e7e1973dc74bf1ed406729b07d0759fde2f064b"}, + {file = "grpcio-1.51.3.tar.gz", hash = "sha256:be7b2265b7527bb12109a7727581e274170766d5b3c9258d4e466f4872522d7a"}, +] + +[package.extras] +protobuf = ["grpcio-tools (>=1.51.3)"] + [[package]] name = "identify" -version = "2.5.22" +version = "2.5.24" description = "File identification library for Python" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "identify-2.5.22-py2.py3-none-any.whl", hash = "sha256:f0faad595a4687053669c112004178149f6c326db71ee999ae4636685753ad2f"}, - {file = "identify-2.5.22.tar.gz", hash = "sha256:f7a93d6cf98e29bd07663c60728e7a4057615068d7a639d132dc883b2d54d31e"}, + {file = "identify-2.5.24-py2.py3-none-any.whl", hash = "sha256:986dbfb38b1140e763e413e6feb44cd731faf72d1909543178aa79b0e258265d"}, + {file = "identify-2.5.24.tar.gz", hash = "sha256:0aac67d5b4812498056d28a9a512a483f5085cc28640b02b258a59dac34301d4"}, ] [package.extras] @@ -1324,14 +1364,14 @@ files = [ [[package]] name = "moto" -version = "4.1.8" +version = "4.1.10" description = "" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "moto-4.1.8-py2.py3-none-any.whl", hash = "sha256:df5b52eff70bf125ee03ea72c4e01f2daff243796f984a534c3dee92a0b93522"}, - {file = "moto-4.1.8.tar.gz", hash = "sha256:c3ecc2dda1a7b3a3c46655490bc6a4660b7bb47e31eaed7bbd54adeb01f8471f"}, + {file = "moto-4.1.10-py2.py3-none-any.whl", hash = "sha256:86ee1f6b23a0c3dcf5c829fb1210fa3f293a2e8f1c95412538c0ea4a2f437021"}, + {file = "moto-4.1.10.tar.gz", hash = "sha256:65e15fc3e80a9589f50f8a5267b9a99f327224ed4d5cc44fe20edcbdb9248e06"}, ] [package.dependencies] @@ -1567,14 +1607,14 @@ files = [ [[package]] name = "nodeenv" -version = "1.7.0" +version = "1.8.0" description = "Node.js virtual environment builder" category = "dev" optional = false python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*" files = [ - {file = "nodeenv-1.7.0-py2.py3-none-any.whl", hash = "sha256:27083a7b96a25f2f5e1d8cb4b6317ee8aeda3bdd121394e5ac54e498028a042e"}, - {file = "nodeenv-1.7.0.tar.gz", hash = "sha256:e0e7f7dfb85fc5394c6fe1e8fa98131a2473e04311a45afb6508f7cf1836fa2b"}, + {file = "nodeenv-1.8.0-py2.py3-none-any.whl", hash = "sha256:df865724bb3c3adc86b3876fa209771517b0cfe596beff01a92700e0e8be4cec"}, + {file = "nodeenv-1.8.0.tar.gz", hash = "sha256:d51e0c37e64fbf47d017feac3145cdbb58836d7eee8c6f6d3b6880c5456227d2"}, ] [package.dependencies] @@ -1712,19 +1752,19 @@ files = [ [[package]] name = "platformdirs" -version = "3.2.0" +version = "3.5.1" description = "A small Python package for determining appropriate platform-specific dirs, e.g. a \"user data dir\"." category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "platformdirs-3.2.0-py3-none-any.whl", hash = "sha256:ebe11c0d7a805086e99506aa331612429a72ca7cd52a1f0d277dc4adc20cb10e"}, - {file = "platformdirs-3.2.0.tar.gz", hash = "sha256:d5b638ca397f25f979350ff789db335903d7ea010ab28903f57b27e1b16c2b08"}, + {file = "platformdirs-3.5.1-py3-none-any.whl", hash = "sha256:e2378146f1964972c03c085bb5662ae80b2b8c06226c54b2ff4aa9483e8a13a5"}, + {file = "platformdirs-3.5.1.tar.gz", hash = "sha256:412dae91f52a6f84830f39a8078cecd0e866cb72294a5c66808e74d5e88d251f"}, ] [package.extras] -docs = ["furo (>=2022.12.7)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-autodoc-typehints (>=1.22,!=1.23.4)"] -test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=7.2.2)", "pytest-cov (>=4)", "pytest-mock (>=3.10)"] +docs = ["furo (>=2023.3.27)", "proselint (>=0.13)", "sphinx (>=6.2.1)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] +test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=7.3.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)"] [[package]] name = "pluggy" @@ -1783,25 +1823,25 @@ virtualenv = ">=20.10.0" [[package]] name = "protobuf" -version = "4.22.3" +version = "4.23.1" description = "" category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "protobuf-4.22.3-cp310-abi3-win32.whl", hash = "sha256:8b54f56d13ae4a3ec140076c9d937221f887c8f64954673d46f63751209e839a"}, - {file = "protobuf-4.22.3-cp310-abi3-win_amd64.whl", hash = "sha256:7760730063329d42a9d4c4573b804289b738d4931e363ffbe684716b796bde51"}, - {file = "protobuf-4.22.3-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:d14fc1a41d1a1909998e8aff7e80d2a7ae14772c4a70e4bf7db8a36690b54425"}, - {file = "protobuf-4.22.3-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:70659847ee57a5262a65954538088a1d72dfc3e9882695cab9f0c54ffe71663b"}, - {file = "protobuf-4.22.3-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:13233ee2b9d3bd9a5f216c1fa2c321cd564b93d8f2e4f521a85b585447747997"}, - {file = "protobuf-4.22.3-cp37-cp37m-win32.whl", hash = "sha256:ecae944c6c2ce50dda6bf76ef5496196aeb1b85acb95df5843cd812615ec4b61"}, - {file = "protobuf-4.22.3-cp37-cp37m-win_amd64.whl", hash = "sha256:d4b66266965598ff4c291416be429cef7989d8fae88b55b62095a2331511b3fa"}, - {file = "protobuf-4.22.3-cp38-cp38-win32.whl", hash = "sha256:f08aa300b67f1c012100d8eb62d47129e53d1150f4469fd78a29fa3cb68c66f2"}, - {file = "protobuf-4.22.3-cp38-cp38-win_amd64.whl", hash = "sha256:f2f4710543abec186aee332d6852ef5ae7ce2e9e807a3da570f36de5a732d88e"}, - {file = "protobuf-4.22.3-cp39-cp39-win32.whl", hash = "sha256:7cf56e31907c532e460bb62010a513408e6cdf5b03fb2611e4b67ed398ad046d"}, - {file = "protobuf-4.22.3-cp39-cp39-win_amd64.whl", hash = "sha256:e0e630d8e6a79f48c557cd1835865b593d0547dce221c66ed1b827de59c66c97"}, - {file = "protobuf-4.22.3-py3-none-any.whl", hash = "sha256:52f0a78141078077cfe15fe333ac3e3a077420b9a3f5d1bf9b5fe9d286b4d881"}, - {file = "protobuf-4.22.3.tar.gz", hash = "sha256:23452f2fdea754a8251d0fc88c0317735ae47217e0d27bf330a30eec2848811a"}, + {file = "protobuf-4.23.1-cp310-abi3-win32.whl", hash = "sha256:410bcc0a5b279f634d3e16082ce221dfef7c3392fac723500e2e64d1806dd2be"}, + {file = "protobuf-4.23.1-cp310-abi3-win_amd64.whl", hash = "sha256:32e78beda26d7a101fecf15d7a4a792278a0d26a31bc327ff05564a9d68ab8ee"}, + {file = "protobuf-4.23.1-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:f9510cac91e764e86acd74e2b7f7bc5e6127a7f3fb646d7c8033cfb84fd1176a"}, + {file = "protobuf-4.23.1-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:346990f634272caac1f09efbcfbbacb23098b1f606d172534c6fa2d9758bb436"}, + {file = "protobuf-4.23.1-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:3ce113b3f3362493bddc9069c2163a38f240a9ed685ff83e7bcb756b05e1deb0"}, + {file = "protobuf-4.23.1-cp37-cp37m-win32.whl", hash = "sha256:2036a3a1e7fc27f973fa0a7888dce712393af644f4695385f117886abc792e39"}, + {file = "protobuf-4.23.1-cp37-cp37m-win_amd64.whl", hash = "sha256:3b8905eafe4439076e1f58e9d1fa327025fd2777cf90f14083092ae47f77b0aa"}, + {file = "protobuf-4.23.1-cp38-cp38-win32.whl", hash = "sha256:5b9cd6097e6acae48a68cb29b56bc79339be84eca65b486910bb1e7a30e2b7c1"}, + {file = "protobuf-4.23.1-cp38-cp38-win_amd64.whl", hash = "sha256:decf119d54e820f298ee6d89c72d6b289ea240c32c521f00433f9dc420595f38"}, + {file = "protobuf-4.23.1-cp39-cp39-win32.whl", hash = "sha256:91fac0753c3c4951fbb98a93271c43cc7cf3b93cf67747b3e600bb1e5cc14d61"}, + {file = "protobuf-4.23.1-cp39-cp39-win_amd64.whl", hash = "sha256:ac50be82491369a9ec3710565777e4da87c6d2e20404e0abb1f3a8f10ffd20f0"}, + {file = "protobuf-4.23.1-py3-none-any.whl", hash = "sha256:65f0ac96ef67d7dd09b19a46aad81a851b6f85f89725577f16de38f2d68ad477"}, + {file = "protobuf-4.23.1.tar.gz", hash = "sha256:95789b569418a3e32a53f43d7763be3d490a831e9c08042539462b6d972c2d7e"}, ] [[package]] @@ -1924,14 +1964,14 @@ plugins = ["importlib-metadata"] [[package]] name = "pyjwt" -version = "2.6.0" +version = "2.7.0" description = "JSON Web Token implementation in Python" category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "PyJWT-2.6.0-py3-none-any.whl", hash = "sha256:d83c3d892a77bbb74d3e1a2cfa90afaadb60945205d1095d9221f04466f64c14"}, - {file = "PyJWT-2.6.0.tar.gz", hash = "sha256:69285c7e31fc44f68a1feb309e948e0df53259d579295e6cfe2b1792329f05fd"}, + {file = "PyJWT-2.7.0-py3-none-any.whl", hash = "sha256:ba2b425b15ad5ef12f200dc67dd56af4e26de2331f965c5439994dad075876e1"}, + {file = "PyJWT-2.7.0.tar.gz", hash = "sha256:bd6ca4a3c4285c1a2d4349e5a035fdf8fb94e04ccd0fcbe6ba289dae9cc3e074"}, ] [package.dependencies] @@ -2286,21 +2326,21 @@ tune = ["pandas", "requests", "tabulate", "tensorboardX (>=1.9)"] [[package]] name = "requests" -version = "2.28.2" +version = "2.30.0" description = "Python HTTP for Humans." category = "main" optional = false -python-versions = ">=3.7, <4" +python-versions = ">=3.7" files = [ - {file = "requests-2.28.2-py3-none-any.whl", hash = "sha256:64299f4909223da747622c030b781c0d7811e359c37124b4bd368fb8c6518baa"}, - {file = "requests-2.28.2.tar.gz", hash = "sha256:98b1b2782e3c6c4904938b84c0eb932721069dfdb9134313beff7c83c2df24bf"}, + {file = "requests-2.30.0-py3-none-any.whl", hash = "sha256:10e94cc4f3121ee6da529d358cdaeaff2f1c409cd377dbc72b825852f2f7e294"}, + {file = "requests-2.30.0.tar.gz", hash = "sha256:239d7d4458afcb28a692cdd298d87542235f4ca8d36d03a15bfc128a6559a2f4"}, ] [package.dependencies] certifi = ">=2017.4.17" charset-normalizer = ">=2,<4" idna = ">=2.5,<4" -urllib3 = ">=1.21.1,<1.27" +urllib3 = ">=1.21.1,<3" [package.extras] socks = ["PySocks (>=1.5.6,!=1.5.7)"] @@ -2349,14 +2389,14 @@ tests = ["coverage (>=6.0.0)", "flake8", "mypy", "pytest (>=7.0.0)", "pytest-asy [[package]] name = "rich" -version = "13.3.4" +version = "13.3.5" description = "Render rich text, tables, progress bars, syntax highlighting, markdown and more to the terminal" category = "main" optional = false python-versions = ">=3.7.0" files = [ - {file = "rich-13.3.4-py3-none-any.whl", hash = "sha256:22b74cae0278fd5086ff44144d3813be1cedc9115bdfabbfefd86400cb88b20a"}, - {file = "rich-13.3.4.tar.gz", hash = "sha256:b5d573e13605423ec80bdd0cd5f8541f7844a0e71a13f74cf454ccb2f490708b"}, + {file = "rich-13.3.5-py3-none-any.whl", hash = "sha256:69cdf53799e63f38b95b9bf9c875f8c90e78dd62b2f00c13a911c7a3b9fa4704"}, + {file = "rich-13.3.5.tar.gz", hash = "sha256:2d11b9b8dd03868f09b4fffadc84a6a8cda574e40dc90821bd845720ebb8e89c"}, ] [package.dependencies] @@ -2369,20 +2409,20 @@ jupyter = ["ipywidgets (>=7.5.1,<9)"] [[package]] name = "s3fs" -version = "2023.4.0" +version = "2023.5.0" description = "Convenient Filesystem interface over S3" category = "main" optional = true python-versions = ">= 3.8" files = [ - {file = "s3fs-2023.4.0-py3-none-any.whl", hash = "sha256:be0afe8f01c67951566d6e9b8a954eaf09b674a4a28002d08eecde29c74a3310"}, - {file = "s3fs-2023.4.0.tar.gz", hash = "sha256:963ee2e070e03e2952d535dbc6fd2b341869fcd16006cf74b90bcf39fd26cded"}, + {file = "s3fs-2023.5.0-py3-none-any.whl", hash = "sha256:0d82c4fa43d1214117f56b239c3e03c9a2886f41c31000c1c967ac6030d20362"}, + {file = "s3fs-2023.5.0.tar.gz", hash = "sha256:106b5d9a1000e6af413f918156ba4b96789ac832b7e08c99d186eb08164e6981"}, ] [package.dependencies] aiobotocore = ">=2.5.0,<2.6.0" aiohttp = "<4.0.0a0 || >4.0.0a0,<4.0.0a1 || >4.0.0a1" -fsspec = "2023.4.0" +fsspec = "2023.5.0" [package.extras] awscli = ["aiobotocore[awscli] (>=2.5.0,<2.6.0)"] @@ -2390,14 +2430,14 @@ boto3 = ["aiobotocore[boto3] (>=2.5.0,<2.6.0)"] [[package]] name = "s3transfer" -version = "0.6.0" +version = "0.6.1" description = "An Amazon S3 Transfer Manager" category = "main" optional = false python-versions = ">= 3.7" files = [ - {file = "s3transfer-0.6.0-py3-none-any.whl", hash = "sha256:06176b74f3a15f61f1b4f25a1fc29a4429040b7647133a463da8fa5bd28d5ecd"}, - {file = "s3transfer-0.6.0.tar.gz", hash = "sha256:2ed07d3866f523cc561bf4a00fc5535827981b117dd7876f036b0c1aca42c947"}, + {file = "s3transfer-0.6.1-py3-none-any.whl", hash = "sha256:3c0da2d074bf35d6870ef157158641178a4204a6e689e82546083e31e0311346"}, + {file = "s3transfer-0.6.1.tar.gz", hash = "sha256:640bb492711f4c0c0905e1f62b6aaeb771881935ad27884852411f8e9cacbca9"}, ] [package.dependencies] @@ -2408,19 +2448,19 @@ crt = ["botocore[crt] (>=1.20.29,<2.0a.0)"] [[package]] name = "setuptools" -version = "67.7.2" +version = "67.8.0" description = "Easily download, build, install, upgrade, and uninstall Python packages" category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "setuptools-67.7.2-py3-none-any.whl", hash = "sha256:23aaf86b85ca52ceb801d32703f12d77517b2556af839621c641fca11287952b"}, - {file = "setuptools-67.7.2.tar.gz", hash = "sha256:f104fa03692a2602fa0fec6c6a9e63b6c8a968de13e17c026957dd1f53d80990"}, + {file = "setuptools-67.8.0-py3-none-any.whl", hash = "sha256:5df61bf30bb10c6f756eb19e7c9f3b473051f48db77fddbe06ff2ca307df9a6f"}, + {file = "setuptools-67.8.0.tar.gz", hash = "sha256:62642358adc77ffa87233bc4d2354c4b2682d214048f500964dbe760ccedf102"}, ] [package.extras] docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-hoverxref (<2)", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (==0.8.3)", "sphinx-reredirects", "sphinxcontrib-towncrier"] -testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8 (<5)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pip-run (>=8.8)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] +testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pip-run (>=8.8)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-ruff", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] testing-integration = ["build[virtualenv]", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pytest", "pytest-enabler", "pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"] [[package]] @@ -2435,6 +2475,18 @@ files = [ {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"}, ] +[[package]] +name = "sortedcontainers" +version = "2.4.0" +description = "Sorted Containers -- Sorted List, Sorted Dict, Sorted Set" +category = "main" +optional = false +python-versions = "*" +files = [ + {file = "sortedcontainers-2.4.0-py2.py3-none-any.whl", hash = "sha256:a163dcaede0f1c021485e957a39245190e74249897e2ae4b2aa38595db237ee0"}, + {file = "sortedcontainers-2.4.0.tar.gz", hash = "sha256:25caa5a06cc30b6b83d11423433f65d1f9d76c4c6a0c90e3379eaa43b9bfdb88"}, +] + [[package]] name = "thrift" version = "0.16.0" @@ -2542,21 +2594,21 @@ test = ["covdefaults (>=2.2.2)", "coverage (>=7.1)", "coverage-enable-subprocess [[package]] name = "werkzeug" -version = "2.2.3" +version = "2.3.4" description = "The comprehensive WSGI web application library." category = "dev" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "Werkzeug-2.2.3-py3-none-any.whl", hash = "sha256:56433961bc1f12533306c624f3be5e744389ac61d722175d543e1751285da612"}, - {file = "Werkzeug-2.2.3.tar.gz", hash = "sha256:2e1ccc9417d4da358b9de6f174e3ac094391ea1d4fbef2d667865d819dfd0afe"}, + {file = "Werkzeug-2.3.4-py3-none-any.whl", hash = "sha256:48e5e61472fee0ddee27ebad085614ebedb7af41e88f687aaf881afb723a162f"}, + {file = "Werkzeug-2.3.4.tar.gz", hash = "sha256:1d5a58e0377d1fe39d061a5de4469e414e78ccb1e1e59c0f5ad6fa1c36c52b76"}, ] [package.dependencies] MarkupSafe = ">=2.1.1" [package.extras] -watchdog = ["watchdog"] +watchdog = ["watchdog (>=2.3)"] [[package]] name = "wrapt" @@ -2657,86 +2709,86 @@ files = [ [[package]] name = "yarl" -version = "1.9.1" +version = "1.9.2" description = "Yet another URL library" category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "yarl-1.9.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e124b283a04cc06d22443cae536f93d86cd55108fa369f22b8fe1f2288b2fe1c"}, - {file = "yarl-1.9.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:56956b13ec275de31fe4fb991510b735c4fb3e1b01600528c952b9ac90464430"}, - {file = "yarl-1.9.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:ecaa5755a39f6f26079bf13f336c67af589c222d76b53cd3824d3b684b84d1f1"}, - {file = "yarl-1.9.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:92a101f6d5a9464e86092adc36cd40ef23d18a25bfb1eb32eaeb62edc22776bb"}, - {file = "yarl-1.9.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:92e37999e36f9f3ded78e9d839face6baa2abdf9344ea8ed2735f495736159de"}, - {file = "yarl-1.9.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ef7e2f6c47c41e234600a02e1356b799761485834fe35d4706b0094cb3a587ee"}, - {file = "yarl-1.9.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1d7a0075a55380b19aa43b9e8056e128b058460d71d75018a4f9d60ace01e78c"}, - {file = "yarl-1.9.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e2f01351b7809182822b21061d2a4728b7b9e08f4585ba90ee4c5c4d3faa0812"}, - {file = "yarl-1.9.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:6cf47fe9df9b1ededc77e492581cdb6890a975ad96b4172e1834f1b8ba0fc3ba"}, - {file = "yarl-1.9.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:098bdc06ffb4db39c73883325b8c738610199f5f12e85339afedf07e912a39af"}, - {file = "yarl-1.9.1-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:6cdb47cbbacae8e1d7941b0d504d0235d686090eef5212ca2450525905e9cf02"}, - {file = "yarl-1.9.1-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:73a4b46689f2d59c8ec6b71c9a0cdced4e7863dd6eb98a8c30ea610e191f9e1c"}, - {file = "yarl-1.9.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:65d952e464df950eed32bb5dcbc1b4443c7c2de4d7abd7265b45b1b3b27f5fa2"}, - {file = "yarl-1.9.1-cp310-cp310-win32.whl", hash = "sha256:39a7a9108e9fc633ae381562f8f0355bb4ba00355218b5fb19cf5263fcdbfa68"}, - {file = "yarl-1.9.1-cp310-cp310-win_amd64.whl", hash = "sha256:b63d41e0eecf3e3070d44f97456cf351fff7cb960e97ecb60a936b877ff0b4f6"}, - {file = "yarl-1.9.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:4295790981630c4dab9d6de7b0f555a4c8defe3ed7704a8e9e595a321e59a0f5"}, - {file = "yarl-1.9.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b2b2382d59dec0f1fdca18ea429c4c4cee280d5e0dbc841180abb82e188cf6e9"}, - {file = "yarl-1.9.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:575975d28795a61e82c85f114c02333ca54cbd325fd4e4b27598c9832aa732e7"}, - {file = "yarl-1.9.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9bb794882818fae20ff65348985fdf143ea6dfaf6413814db1848120db8be33e"}, - {file = "yarl-1.9.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:89da1fd6068553e3a333011cc17ad91c414b2100c32579ddb51517edc768b49c"}, - {file = "yarl-1.9.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4d817593d345fefda2fae877accc8a0d9f47ada57086da6125fa02a62f6d1a94"}, - {file = "yarl-1.9.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:85aa6fd779e194901386709e0eedd45710b68af2709f82a84839c44314b68c10"}, - {file = "yarl-1.9.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:eed9827033b7f67ad12cb70bd0cb59d36029144a7906694317c2dbf5c9eb5ddd"}, - {file = "yarl-1.9.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:df747104ef27ab1aa9a1145064fa9ea26ad8cf24bfcbdba7db7abf0f8b3676b9"}, - {file = "yarl-1.9.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:efec77851231410125cb5be04ec96fa4a075ca637f415a1f2d2c900b09032a8a"}, - {file = "yarl-1.9.1-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:d5c407e530cf2979ea383885516ae79cc4f3c3530623acf5e42daf521f5c2564"}, - {file = "yarl-1.9.1-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:f76edb386178a54ea7ceffa798cb830c3c22ab50ea10dfb25dc952b04848295f"}, - {file = "yarl-1.9.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:75676110bce59944dd48fd18d0449bd37eaeb311b38a0c768f7670864b5f8b68"}, - {file = "yarl-1.9.1-cp311-cp311-win32.whl", hash = "sha256:9ba5a18c4fbd408fe49dc5da85478a76bc75c1ce912d7fd7b43ed5297c4403e1"}, - {file = "yarl-1.9.1-cp311-cp311-win_amd64.whl", hash = "sha256:b20a5ddc4e243cbaa54886bfe9af6ffc4ba4ef58f17f1bb691e973eb65bba84d"}, - {file = "yarl-1.9.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:791357d537a09a194f92b834f28c98d074e7297bac0a8f1d5b458a906cafa17c"}, - {file = "yarl-1.9.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:89099c887338608da935ba8bee027564a94f852ac40e472de15d8309517ad5fe"}, - {file = "yarl-1.9.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:395ea180257a3742d09dcc5071739682a95f7874270ebe3982d6696caec75be0"}, - {file = "yarl-1.9.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:90ebaf448b5f048352ec7c76cb8d452df30c27cb6b8627dfaa9cf742a14f141a"}, - {file = "yarl-1.9.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f878a78ed2ccfbd973cab46dd0933ecd704787724db23979e5731674d76eb36f"}, - {file = "yarl-1.9.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:74390c2318d066962500045aa145f5412169bce842e734b8c3e6e3750ad5b817"}, - {file = "yarl-1.9.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:f8e73f526140c1c32f5fca4cd0bc3b511a1abcd948f45b2a38a95e4edb76ca72"}, - {file = "yarl-1.9.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:ac8e593df1fbea820da7676929f821a0c7c2cecb8477d010254ce8ed54328ea8"}, - {file = "yarl-1.9.1-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:01cf88cb80411978a14aa49980968c1aeb7c18a90ac978c778250dd234d8e0ba"}, - {file = "yarl-1.9.1-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:97d76a3128f48fa1c721ef8a50e2c2f549296b2402dc8a8cde12ff60ed922f53"}, - {file = "yarl-1.9.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:01a073c9175481dfed6b40704a1b67af5a9435fc4a58a27d35fd6b303469b0c7"}, - {file = "yarl-1.9.1-cp37-cp37m-win32.whl", hash = "sha256:ecad20c3ef57c513dce22f58256361d10550a89e8eaa81d5082f36f8af305375"}, - {file = "yarl-1.9.1-cp37-cp37m-win_amd64.whl", hash = "sha256:f5bcb80006efe9bf9f49ae89711253dd06df8053ff814622112a9219346566a7"}, - {file = "yarl-1.9.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:e7ddebeabf384099814353a2956ed3ab5dbaa6830cc7005f985fcb03b5338f05"}, - {file = "yarl-1.9.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:13a1ad1f35839b3bb5226f59816b71e243d95d623f5b392efaf8820ddb2b3cd5"}, - {file = "yarl-1.9.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:f0cd87949d619157a0482c6c14e5011f8bf2bc0b91cb5087414d9331f4ef02dd"}, - {file = "yarl-1.9.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d21887cbcf6a3cc5951662d8222bc9c04e1b1d98eebe3bb659c3a04ed49b0eec"}, - {file = "yarl-1.9.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4764114e261fe49d5df9b316b3221493d177247825c735b2aae77bc2e340d800"}, - {file = "yarl-1.9.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3abe37fd89a93ebe0010417ca671f422fa6fcffec54698f623b09f46b4d4a512"}, - {file = "yarl-1.9.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e9fe3a1c073ab80a28a06f41d2b623723046709ed29faf2c56bea41848597d86"}, - {file = "yarl-1.9.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b3b5f8da07a21f2e57551f88a6709c2d340866146cf7351e5207623cfe8aad16"}, - {file = "yarl-1.9.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:88f6413ff5edfb9609e2769e32ce87a62353e66e75d264bf0eaad26fb9daa8f2"}, - {file = "yarl-1.9.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:b5d5fb6c94b620a7066a3adb7c246c87970f453813979818e4707ac32ce4d7bd"}, - {file = "yarl-1.9.1-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:f206adb89424dca4a4d0b31981869700e44cd62742527e26d6b15a510dd410a2"}, - {file = "yarl-1.9.1-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:44fa6158e6b4b8ccfa2872c3900a226b29e8ce543ce3e48aadc99816afa8874d"}, - {file = "yarl-1.9.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:08c8599d6aa8a24425f8635f6c06fa8726afe3be01c8e53e236f519bcfa5db5b"}, - {file = "yarl-1.9.1-cp38-cp38-win32.whl", hash = "sha256:6b09cce412386ea9b4dda965d8e78d04ac5b5792b2fa9cced3258ec69c7d1c16"}, - {file = "yarl-1.9.1-cp38-cp38-win_amd64.whl", hash = "sha256:09c56a32c26e24ef98d5757c5064e252836f621f9a8b42737773aa92936b8e08"}, - {file = "yarl-1.9.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:b86e98c3021b7e2740d8719bf074301361bf2f51221ca2765b7a58afbfbd9042"}, - {file = "yarl-1.9.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:5faf3ec98747318cb980aaf9addf769da68a66431fc203a373d95d7ee9c1fbb4"}, - {file = "yarl-1.9.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:a21789bdf28549d4eb1de6910cabc762c9f6ae3eef85efc1958197c1c6ef853b"}, - {file = "yarl-1.9.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a8b8d4b478a9862447daef4cafc89d87ea4ed958672f1d11db7732b77ead49cc"}, - {file = "yarl-1.9.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:307a782736ebf994e7600dcaeea3b3113083584da567272f2075f1540919d6b3"}, - {file = "yarl-1.9.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:46c4010de941e2e1365c07fb4418ddca10fcff56305a6067f5ae857f8c98f3a7"}, - {file = "yarl-1.9.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bab67d041c78e305ff3eef5e549304d843bd9b603c8855b68484ee663374ce15"}, - {file = "yarl-1.9.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1baf8cdaaab65d9ccedbf8748d626ad648b74b0a4d033e356a2f3024709fb82f"}, - {file = "yarl-1.9.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:27efc2e324f72df02818cd72d7674b1f28b80ab49f33a94f37c6473c8166ce49"}, - {file = "yarl-1.9.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:ca14b84091700ae7c1fcd3a6000bd4ec1a3035009b8bcb94f246741ca840bb22"}, - {file = "yarl-1.9.1-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:c3ca8d71b23bdf164b36d06df2298ec8a5bd3de42b17bf3e0e8e6a7489195f2c"}, - {file = "yarl-1.9.1-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:8c72a1dc7e2ea882cd3df0417c808ad3b69e559acdc43f3b096d67f2fb801ada"}, - {file = "yarl-1.9.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:d966cd59df9a4b218480562e8daab39e87e746b78a96add51a3ab01636fc4291"}, - {file = "yarl-1.9.1-cp39-cp39-win32.whl", hash = "sha256:518a92a34c741836a315150460b5c1c71ae782d569eabd7acf53372e437709f7"}, - {file = "yarl-1.9.1-cp39-cp39-win_amd64.whl", hash = "sha256:78755ce43b6e827e65ec0c68be832f86d059fcf05d4b33562745ebcfa91b26b1"}, - {file = "yarl-1.9.1.tar.gz", hash = "sha256:5ce0bcab7ec759062c818d73837644cde567ab8aa1e0d6c45db38dfb7c284441"}, + {file = "yarl-1.9.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:8c2ad583743d16ddbdf6bb14b5cd76bf43b0d0006e918809d5d4ddf7bde8dd82"}, + {file = "yarl-1.9.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:82aa6264b36c50acfb2424ad5ca537a2060ab6de158a5bd2a72a032cc75b9eb8"}, + {file = "yarl-1.9.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c0c77533b5ed4bcc38e943178ccae29b9bcf48ffd1063f5821192f23a1bd27b9"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ee4afac41415d52d53a9833ebae7e32b344be72835bbb589018c9e938045a560"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9bf345c3a4f5ba7f766430f97f9cc1320786f19584acc7086491f45524a551ac"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2a96c19c52ff442a808c105901d0bdfd2e28575b3d5f82e2f5fd67e20dc5f4ea"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:891c0e3ec5ec881541f6c5113d8df0315ce5440e244a716b95f2525b7b9f3608"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c3a53ba34a636a256d767c086ceb111358876e1fb6b50dfc4d3f4951d40133d5"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:566185e8ebc0898b11f8026447eacd02e46226716229cea8db37496c8cdd26e0"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:2b0738fb871812722a0ac2154be1f049c6223b9f6f22eec352996b69775b36d4"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:32f1d071b3f362c80f1a7d322bfd7b2d11e33d2adf395cc1dd4df36c9c243095"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:e9fdc7ac0d42bc3ea78818557fab03af6181e076a2944f43c38684b4b6bed8e3"}, + {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:56ff08ab5df8429901ebdc5d15941b59f6253393cb5da07b4170beefcf1b2528"}, + {file = "yarl-1.9.2-cp310-cp310-win32.whl", hash = "sha256:8ea48e0a2f931064469bdabca50c2f578b565fc446f302a79ba6cc0ee7f384d3"}, + {file = "yarl-1.9.2-cp310-cp310-win_amd64.whl", hash = "sha256:50f33040f3836e912ed16d212f6cc1efb3231a8a60526a407aeb66c1c1956dde"}, + {file = "yarl-1.9.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:646d663eb2232d7909e6601f1a9107e66f9791f290a1b3dc7057818fe44fc2b6"}, + {file = "yarl-1.9.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:aff634b15beff8902d1f918012fc2a42e0dbae6f469fce134c8a0dc51ca423bb"}, + {file = "yarl-1.9.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a83503934c6273806aed765035716216cc9ab4e0364f7f066227e1aaea90b8d0"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b25322201585c69abc7b0e89e72790469f7dad90d26754717f3310bfe30331c2"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:22a94666751778629f1ec4280b08eb11815783c63f52092a5953faf73be24191"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8ec53a0ea2a80c5cd1ab397925f94bff59222aa3cf9c6da938ce05c9ec20428d"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:159d81f22d7a43e6eabc36d7194cb53f2f15f498dbbfa8edc8a3239350f59fe7"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:832b7e711027c114d79dffb92576acd1bd2decc467dec60e1cac96912602d0e6"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:95d2ecefbcf4e744ea952d073c6922e72ee650ffc79028eb1e320e732898d7e8"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:d4e2c6d555e77b37288eaf45b8f60f0737c9efa3452c6c44626a5455aeb250b9"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:783185c75c12a017cc345015ea359cc801c3b29a2966c2655cd12b233bf5a2be"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:b8cc1863402472f16c600e3e93d542b7e7542a540f95c30afd472e8e549fc3f7"}, + {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:822b30a0f22e588b32d3120f6d41e4ed021806418b4c9f0bc3048b8c8cb3f92a"}, + {file = "yarl-1.9.2-cp311-cp311-win32.whl", hash = "sha256:a60347f234c2212a9f0361955007fcf4033a75bf600a33c88a0a8e91af77c0e8"}, + {file = "yarl-1.9.2-cp311-cp311-win_amd64.whl", hash = "sha256:be6b3fdec5c62f2a67cb3f8c6dbf56bbf3f61c0f046f84645cd1ca73532ea051"}, + {file = "yarl-1.9.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:38a3928ae37558bc1b559f67410df446d1fbfa87318b124bf5032c31e3447b74"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ac9bb4c5ce3975aeac288cfcb5061ce60e0d14d92209e780c93954076c7c4367"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3da8a678ca8b96c8606bbb8bfacd99a12ad5dd288bc6f7979baddd62f71c63ef"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:13414591ff516e04fcdee8dc051c13fd3db13b673c7a4cb1350e6b2ad9639ad3"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bf74d08542c3a9ea97bb8f343d4fcbd4d8f91bba5ec9d5d7f792dbe727f88938"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6e7221580dc1db478464cfeef9b03b95c5852cc22894e418562997df0d074ccc"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:494053246b119b041960ddcd20fd76224149cfea8ed8777b687358727911dd33"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:52a25809fcbecfc63ac9ba0c0fb586f90837f5425edfd1ec9f3372b119585e45"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:e65610c5792870d45d7b68c677681376fcf9cc1c289f23e8e8b39c1485384185"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:1b1bba902cba32cdec51fca038fd53f8beee88b77efc373968d1ed021024cc04"}, + {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:662e6016409828ee910f5d9602a2729a8a57d74b163c89a837de3fea050c7582"}, + {file = "yarl-1.9.2-cp37-cp37m-win32.whl", hash = "sha256:f364d3480bffd3aa566e886587eaca7c8c04d74f6e8933f3f2c996b7f09bee1b"}, + {file = "yarl-1.9.2-cp37-cp37m-win_amd64.whl", hash = "sha256:6a5883464143ab3ae9ba68daae8e7c5c95b969462bbe42e2464d60e7e2698368"}, + {file = "yarl-1.9.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:5610f80cf43b6202e2c33ba3ec2ee0a2884f8f423c8f4f62906731d876ef4fac"}, + {file = "yarl-1.9.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:b9a4e67ad7b646cd6f0938c7ebfd60e481b7410f574c560e455e938d2da8e0f4"}, + {file = "yarl-1.9.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:83fcc480d7549ccebe9415d96d9263e2d4226798c37ebd18c930fce43dfb9574"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5fcd436ea16fee7d4207c045b1e340020e58a2597301cfbcfdbe5abd2356c2fb"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:84e0b1599334b1e1478db01b756e55937d4614f8654311eb26012091be109d59"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3458a24e4ea3fd8930e934c129b676c27452e4ebda80fbe47b56d8c6c7a63a9e"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:838162460b3a08987546e881a2bfa573960bb559dfa739e7800ceeec92e64417"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f4e2d08f07a3d7d3e12549052eb5ad3eab1c349c53ac51c209a0e5991bbada78"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:de119f56f3c5f0e2fb4dee508531a32b069a5f2c6e827b272d1e0ff5ac040333"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:149ddea5abf329752ea5051b61bd6c1d979e13fbf122d3a1f9f0c8be6cb6f63c"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:674ca19cbee4a82c9f54e0d1eee28116e63bc6fd1e96c43031d11cbab8b2afd5"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:9b3152f2f5677b997ae6c804b73da05a39daa6a9e85a512e0e6823d81cdad7cc"}, + {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:5415d5a4b080dc9612b1b63cba008db84e908b95848369aa1da3686ae27b6d2b"}, + {file = "yarl-1.9.2-cp38-cp38-win32.whl", hash = "sha256:f7a3d8146575e08c29ed1cd287068e6d02f1c7bdff8970db96683b9591b86ee7"}, + {file = "yarl-1.9.2-cp38-cp38-win_amd64.whl", hash = "sha256:63c48f6cef34e6319a74c727376e95626f84ea091f92c0250a98e53e62c77c72"}, + {file = "yarl-1.9.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:75df5ef94c3fdc393c6b19d80e6ef1ecc9ae2f4263c09cacb178d871c02a5ba9"}, + {file = "yarl-1.9.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:c027a6e96ef77d401d8d5a5c8d6bc478e8042f1e448272e8d9752cb0aff8b5c8"}, + {file = "yarl-1.9.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f3b078dbe227f79be488ffcfc7a9edb3409d018e0952cf13f15fd6512847f3f7"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:59723a029760079b7d991a401386390c4be5bfec1e7dd83e25a6a0881859e716"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b03917871bf859a81ccb180c9a2e6c1e04d2f6a51d953e6a5cdd70c93d4e5a2a"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c1012fa63eb6c032f3ce5d2171c267992ae0c00b9e164efe4d73db818465fac3"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a74dcbfe780e62f4b5a062714576f16c2f3493a0394e555ab141bf0d746bb955"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8c56986609b057b4839968ba901944af91b8e92f1725d1a2d77cbac6972b9ed1"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:2c315df3293cd521033533d242d15eab26583360b58f7ee5d9565f15fee1bef4"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:b7232f8dfbd225d57340e441d8caf8652a6acd06b389ea2d3222b8bc89cbfca6"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:53338749febd28935d55b41bf0bcc79d634881195a39f6b2f767870b72514caf"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:066c163aec9d3d073dc9ffe5dd3ad05069bcb03fcaab8d221290ba99f9f69ee3"}, + {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8288d7cd28f8119b07dd49b7230d6b4562f9b61ee9a4ab02221060d21136be80"}, + {file = "yarl-1.9.2-cp39-cp39-win32.whl", hash = "sha256:b124e2a6d223b65ba8768d5706d103280914d61f5cae3afbc50fc3dfcc016623"}, + {file = "yarl-1.9.2-cp39-cp39-win_amd64.whl", hash = "sha256:61016e7d582bc46a5378ffdd02cd0314fb8ba52f40f9cf4d9a5e7dbef88dee18"}, + {file = "yarl-1.9.2.tar.gz", hash = "sha256:04ab9d4b9f587c06d801c2abfe9317b77cdf996c65a90d5e84ecc45010823571"}, ] [package.dependencies] @@ -2826,7 +2878,7 @@ glue = ["boto3"] hive = ["thrift"] pandas = ["pandas", "pyarrow"] pyarrow = ["pyarrow"] -ray = ["ray", "pyarrow", "pandas"] +ray = ["pandas", "pyarrow", "ray"] s3fs = ["s3fs"] snappy = ["python-snappy"] zstandard = ["zstandard"] @@ -2834,4 +2886,4 @@ zstandard = ["zstandard"] [metadata] lock-version = "2.0" python-versions = "^3.8" -content-hash = "dbebfc54b732784368525d623784ca897e62db055bbf0c9dad1fad8f22ac8b88" +content-hash = "282a29b854330203b775c44c50f90b914dab2352220629d11cb4c958ee847557" diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index b12a4dcaf408..464eaffdabae 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -536,10 +536,11 @@ def generator() -> Generator[int, None, None]: deleted_pos = next(sorted_deleted).as_py() # type: ignore for pos in range(fn_rows()): if deleted_pos == pos: - try: - deleted_pos = next(sorted_deleted).as_py() # type: ignore - except StopIteration: - deleted_pos = -1 + while deleted_pos == pos: + try: + deleted_pos = next(sorted_deleted).as_py() # type: ignore + except StopIteration: + deleted_pos = -1 else: yield pos @@ -820,12 +821,29 @@ def _task_to_table( return None +def _read_all_delete_files(fs: FileSystem, pool: ThreadPool, tasks: Iterable[FileScanTask]) -> Dict[str, List[ChunkedArray]]: + deletes_per_file: Dict[str, List[ChunkedArray]] = {} + unique_deletes = set(chain.from_iterable([task.delete_files for task in tasks])) + if len(unique_deletes) > 0: + deletes_per_files: List[Dict[str, ChunkedArray]] = pool.starmap( + func=_read_deletes, iterable=[(fs, delete) for delete in unique_deletes] + ) + for delete in deletes_per_files: + for file, arr in delete.items(): + if file in deletes_per_file: + deletes_per_file[file].append(arr) + else: + deletes_per_file[file] = [arr] + + return deletes_per_file + + def project_table( tasks: Iterable[FileScanTask], table: Table, row_filter: BooleanExpression, projected_schema: Schema, - case_sensitive: bool, + case_sensitive: bool = True, limit: Optional[int] = None, ) -> pa.Table: """Resolves the right columns based on the identifier @@ -836,6 +854,7 @@ def project_table( row_filter (BooleanExpression): The expression for filtering rows projected_schema (Schema): The output schema case_sensitive (bool): Case sensitivity when looking up column names + limit (Optional[int]): Limit the number of records Raises: ResolveError: When an incompatible query is done @@ -865,21 +884,7 @@ def project_table( rows_counter = multiprocessing.Value("i", 0) with ThreadPool() as pool: - # Fetch the deletes - deletes_per_file: Dict[str, List[ChunkedArray]] = {} - unique_deletes = set(chain.from_iterable([task.delete_files for task in tasks])) - if len(unique_deletes) > 0: - deletes_per_files: List[Dict[str, ChunkedArray]] = pool.starmap( - func=_read_deletes, iterable=[(fs, delete) for delete in unique_deletes] - ) - for delete in deletes_per_files: - for file, arr in delete.items(): - if file in deletes_per_file: - deletes_per_file[file].append(arr) - else: - deletes_per_file[file] = [arr] - - # Fetch teh data + deletes_per_file = _read_all_delete_files(fs, pool, tasks) tables = [ table for table in pool.starmap( diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index b0c060aa3fc7..b44a4ec680cb 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -36,6 +36,7 @@ ) from pydantic import Field +from sortedcontainers import SortedList from pyiceberg.expressions import ( AlwaysTrue, @@ -52,8 +53,9 @@ DataFile, DataFileContent, ManifestContent, + ManifestEntry, ManifestFile, - files, + live_entries, ) from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema @@ -314,7 +316,7 @@ def __init__( delete_files: Optional[Set[DataFile]] = None, start: Optional[int] = None, length: Optional[int] = None, - ): + ) -> None: self.file = data_file self.delete_files = delete_files or set() self.start = start or 0 @@ -326,13 +328,15 @@ def _open_manifest( manifest: ManifestFile, partition_filter: Callable[[DataFile], bool], metrics_evaluator: Callable[[DataFile], bool], -) -> List[DataFile]: - result_manifests = files(io.new_input(manifest.manifest_path)) - result_manifests = filter(partition_filter, result_manifests) - return [file for file in result_manifests if metrics_evaluator(file)] +) -> List[ManifestEntry]: + return [ + manifest_entry + for manifest_entry in live_entries(io.new_input(manifest.manifest_path)) + if partition_filter(manifest_entry.data_file) and metrics_evaluator(manifest_entry.data_file) + ] -def _min_sequence_number(manifests: List[ManifestFile]) -> int: +def _min_data_file_sequence_number(manifests: List[ManifestFile]) -> int: try: return min( manifest.min_sequence_number or INITIAL_SEQUENCE_NUMBER @@ -378,6 +382,23 @@ def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool] evaluator = visitors.expression_evaluator(partition_schema, partition_expr, self.case_sensitive) return lambda data_file: evaluator(data_file.partition) + def _check_sequence_number(self, min_data_sequence_number: int, manifest: ManifestFile) -> bool: + """A helper function to make sure that no manifests are loaded that contain deletes + that are older than the data + + Args: + min_data_sequence_number (int): The minimal + manifest (ManifestFile): A ManifestFile that can be either data or deletes + + Returns: + Boolean indicating if it is either a data file, or a relevant delete file + """ + return (manifest.content is None or manifest.content == ManifestContent.DATA) or ( + # Not interested in deletes that are older than the data + manifest.content == ManifestContent.DELETES + and (manifest.sequence_number or INITIAL_SEQUENCE_NUMBER) >= min_data_sequence_number + ) + def plan_files(self) -> Iterable[FileScanTask]: """Plans the relevant files by filtering on the PartitionSpecs @@ -403,16 +424,17 @@ def plan_files(self) -> Iterable[FileScanTask]: # step 2: filter the data files in each manifest # this filter depends on the partition spec used to write the manifest file + partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) metrics_evaluator = _InclusiveMetricsEvaluator(self.table.schema(), self.row_filter, self.case_sensitive).eval - min_sequence_number = _min_sequence_number(manifests) + min_data_sequence_number = _min_data_file_sequence_number(manifests) - data_datafiles = [] - deletes_positional = [] + data_entries: List[ManifestEntry] = [] + positional_delete_entries = SortedList(key=lambda entry: entry.sequence_number or INITIAL_SEQUENCE_NUMBER) with ThreadPool() as pool: - for datafile in chain( + for manifest_entry in chain( *pool.starmap( func=_open_manifest, iterable=[ @@ -423,38 +445,56 @@ def plan_files(self) -> Iterable[FileScanTask]: metrics_evaluator, ) for manifest in manifests - if (manifest.content is None or manifest.content == ManifestContent.DATA) - or ( - # Not interested in deletes that are older than the data - manifest.content == ManifestContent.DELETES - and (manifest.sequence_number or INITIAL_SEQUENCE_NUMBER) >= min_sequence_number - ) + if self._check_sequence_number(min_data_sequence_number, manifest) ], ) ): - if datafile.content is None or datafile.content == DataFileContent.DATA: - data_datafiles.append(datafile) - elif datafile.content == DataFileContent.POSITION_DELETES: - deletes_positional.append(datafile) - elif datafile.content == DataFileContent.EQUALITY_DELETES: + data_file = manifest_entry.data_file + if data_file.content is None or data_file.content == DataFileContent.DATA: + data_entries.append(manifest_entry) + elif data_file.content == DataFileContent.POSITION_DELETES: + positional_delete_entries.add(manifest_entry) + elif data_file.content == DataFileContent.EQUALITY_DELETES: raise ValueError( "PyIceberg does not yet support equality deletes: https://github.com/apache/iceberg/issues/6568" ) else: - raise ValueError(f"Unknown DataFileContent: {datafile.content}") + raise ValueError(f"Unknown DataFileContent ({data_file.content}): {manifest_entry}") return [ - FileScanTask(data_file, delete_files=self._match_deletes_to_datafile(data_file, deletes_positional)) - for data_file in data_datafiles + FileScanTask( + data_entry.data_file, + delete_files=self._match_deletes_to_datafile( + data_entry.data_file.file_path, + positional_delete_entries[positional_delete_entries.bisect_right(data_entry) :], + ), + ) + for data_entry in data_entries ] - def _match_deletes_to_datafile(self, data_file: DataFile, positional_delete_files: List[DataFile]) -> Set[DataFile]: - return set( - filter( - _InclusiveMetricsEvaluator(POSITIONAL_DELETE_SCHEMA, EqualTo("file_path", data_file.file_path)).eval, - positional_delete_files, - ) - ) + def _match_deletes_to_datafile(self, data_file_path: str, positional_delete_entries: List[ManifestEntry]) -> Set[DataFile]: + """This method will check if the delete file is relevant for the data file + + It will: + - Check if the delete file has been written after the data file + - Use the column metrics to see if the filename is part of the metrics + + Args: + data_file_path (str): The manifest entry path of the datafile + positional_delete_entries (List[ManifestEntry]): All the candidate positional deletes manifest entries + + Returns: + A set of files that are relevant for the data file. + """ + if len(positional_delete_entries) > 0: + evaluator = _InclusiveMetricsEvaluator(POSITIONAL_DELETE_SCHEMA, EqualTo("file_path", data_file_path)) + return { + positional_delete_entry.data_file + for positional_delete_entry in positional_delete_entries + if evaluator.eval(positional_delete_entry.data_file) + } + else: + return set() def to_arrow(self) -> pa.Table: from pyiceberg.io.pyarrow import project_table diff --git a/python/pyproject.toml b/python/pyproject.toml index faa6482addfc..5f2c16bdb741 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -49,6 +49,7 @@ click = ">=7.1.1,<9.0.0" rich = ">=10.11.0,<14.0.0" pyyaml = ">=5.4.0,<7.0.0" # CVE-2020-14343 was fixed in 5.4. pydantic = ">=1.9.0,<2.0.0" +sortedcontainers = "2.4.0" fsspec = ">=2021.09.0,<2024.1.0" # `lexists()` was implemented in 2021.09.0. Upper bound set arbitrarily, to be reassessed in early 2024. pyparsing = ">=3.0.7,<4.0.0" # The `min` keyword argument for `delimited_list()` was added in 3.0.7. zstandard = ">=0.13.0,<1.0.0" @@ -234,5 +235,9 @@ ignore_missing_imports = true module = "pyspark.*" ignore_missing_imports = true +[[tool.mypy.overrides]] +module = "sortedcontainers.*" +ignore_missing_imports = true + [tool.coverage.run] source = ['pyiceberg/'] diff --git a/python/tests/io/test_pyarrow.py b/python/tests/io/test_pyarrow.py index b9df3fab3e47..3792aea03e9d 100644 --- a/python/tests/io/test_pyarrow.py +++ b/python/tests/io/test_pyarrow.py @@ -1235,7 +1235,48 @@ def test_delete(deletes_file: str, example_task: FileScanTask, table_schema_simp ), row_filter=AlwaysTrue(), projected_schema=table_schema_simple, - case_sensitive=True, + ) + + assert ( + str(with_deletes) + == """pyarrow.Table +foo: string +bar: int64 not null +baz: bool +---- +foo: [["a","c"]] +bar: [[1,3]] +baz: [[true,null]]""" + ) + + +def test_delete_duplicates(deletes_file: str, example_task: FileScanTask, table_schema_simple: Schema) -> None: + metadata_location = "file://a/b/c.json" + example_task_with_delete = FileScanTask( + data_file=example_task.file, + delete_files={ + DataFile(content=DataFileContent.POSITION_DELETES, file_path=deletes_file, file_format=FileFormat.PARQUET), + DataFile(content=DataFileContent.POSITION_DELETES, file_path=deletes_file, file_format=FileFormat.PARQUET), + }, + ) + + with_deletes = project_table( + tasks=[example_task_with_delete], + table=Table( + ("namespace", "table"), + metadata=TableMetadataV2( + location=metadata_location, + last_column_id=1, + format_version=2, + current_schema_id=1, + schemas=[table_schema_simple], + partition_specs=[PartitionSpec()], + ), + metadata_location=metadata_location, + io=load_file_io(), + ), + row_filter=AlwaysTrue(), + projected_schema=table_schema_simple, ) assert ( diff --git a/python/tests/test_integration.py b/python/tests/test_integration.py index 5577f3d84a2b..fe2f515081df 100644 --- a/python/tests/test_integration.py +++ b/python/tests/test_integration.py @@ -19,12 +19,19 @@ import math from urllib.parse import urlparse +import pyarrow as pa import pyarrow.parquet as pq import pytest from pyarrow.fs import S3FileSystem from pyiceberg.catalog import Catalog, load_catalog -from pyiceberg.expressions import IsNaN, NotNaN +from pyiceberg.expressions import ( + And, + GreaterThanOrEqual, + IsNaN, + LessThan, + NotNaN, +) from pyiceberg.io.pyarrow import pyarrow_to_schema from pyiceberg.schema import Schema from pyiceberg.table import Table @@ -64,6 +71,18 @@ def table_test_all_types(catalog: Catalog) -> Table: return catalog.load_table("default.test_all_types") +@pytest.fixture() +def test_positional_mor_deletes(catalog: Catalog) -> Table: + """Table that has positional deletes""" + return catalog.load_table("default.test_positional_mor_deletes") + + +@pytest.fixture() +def test_positional_mor_double_deletes(catalog: Catalog) -> Table: + """Table that has multiple positional deletes""" + return catalog.load_table("default.test_positional_mor_double_deletes") + + @pytest.mark.integration def test_pyarrow_nan(table_test_null_nan: Table) -> None: arrow_table = table_test_null_nan.scan(row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")).to_arrow() @@ -159,3 +178,55 @@ def test_pyarrow_to_iceberg_all_types(table_test_all_types: Table) -> None: stored_iceberg_schema = Schema.parse_raw(parquet_schema.metadata.get(b"iceberg.schema")) converted_iceberg_schema = pyarrow_to_schema(parquet_schema) assert converted_iceberg_schema == stored_iceberg_schema + + +@pytest.mark.integration +def test_pyarrow_deletes(test_positional_mor_deletes: Table) -> None: + arrow_table = test_positional_mor_deletes.scan().to_arrow() + assert len(arrow_table) == 6 + assert arrow_table["number"] == pa.Array([2, 4, 6, 8, 10, 12]) + + # Checking the filter + arrow_table = test_positional_mor_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) + ).to_arrow() + assert len(arrow_table) == 3 + assert arrow_table["number"] == pa.Array([6, 8, 10]) + + # Testing the combination of a filter and a limit + arrow_table = test_positional_mor_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 + ).to_arrow() + assert len(arrow_table) == 1 + assert arrow_table["number"] == pa.Array([6]) + + # Testing the slicing of indices + arrow_table = test_positional_mor_deletes.scan(limit=3).to_arrow() + assert len(arrow_table) == 3 + assert arrow_table["number"] == pa.Array([2, 4, 6]) + + +@pytest.mark.integration +def test_pyarrow_deletes_double(test_positional_mor_double_deletes: Table) -> None: + arrow_table = test_positional_mor_double_deletes.scan().to_arrow() + assert len(arrow_table) == 5 + assert arrow_table["number"] == pa.Array([2, 4, 8, 10, 12]) + + # Checking the filter + arrow_table = test_positional_mor_double_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) + ).to_arrow() + assert len(arrow_table) == 2 + assert arrow_table["number"] == pa.Array([8, 10]) + + # Testing the combination of a filter and a limit + arrow_table = test_positional_mor_double_deletes.scan( + row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 + ).to_arrow() + assert len(arrow_table) == 1 + assert arrow_table["number"] == pa.Array([8]) + + # Testing the slicing of indices + arrow_table = test_positional_mor_double_deletes.scan(limit=3).to_arrow() + assert len(arrow_table) == 3 + assert arrow_table["number"] == pa.Array([2, 4, 8]) From 16c251e58ea8e3214ae54c777bfff1ea8b56346e Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 22 May 2023 12:01:02 -0500 Subject: [PATCH 30/36] WIP --- python/Makefile | 3 +- python/dev/entrypoint.sh | 2 + python/dev/provision.py | 68 +++++++++++++++----------------- python/tests/test_integration.py | 52 ++++++++++++++++-------- 4 files changed, 71 insertions(+), 54 deletions(-) diff --git a/python/Makefile b/python/Makefile index ea0a3e82a8fd..2796f7799c82 100644 --- a/python/Makefile +++ b/python/Makefile @@ -34,7 +34,8 @@ test-s3: test-integration: docker-compose -f dev/docker-compose-integration.yml kill - docker-compose -f dev/docker-compose-integration.yml build + docker-compose -f dev/docker-compose-integration.yml rm -f + docker-compose -f dev/docker-compose-integration.yml build --no-cache docker-compose -f dev/docker-compose-integration.yml up -d sleep 30 poetry run pytest tests/ -m integration ${PYTEST_ARGS} diff --git a/python/dev/entrypoint.sh b/python/dev/entrypoint.sh index d777f8f5a284..55c16c43daea 100755 --- a/python/dev/entrypoint.sh +++ b/python/dev/entrypoint.sh @@ -18,6 +18,8 @@ # under the License. # +set -ex + start-master.sh -p 7077 start-worker.sh spark://spark-iceberg:7077 start-history-server.sh diff --git a/python/dev/provision.py b/python/dev/provision.py index 8f11a253f6c4..7a76027f812c 100644 --- a/python/dev/provision.py +++ b/python/dev/provision.py @@ -104,52 +104,47 @@ """ ) -spark.sql( - """ - DROP TABLE IF EXISTS test_deletes; +spark.sql(""" +DROP TABLE IF EXISTS test_positional_mor_deletes; """ ) -spark.sql( - """ - CREATE TABLE test_positional_mor_deletes ( +spark.sql(""" +CREATE TABLE test_positional_mor_deletes ( number integer, letter string - ) - USING iceberg - TBLPROPERTIES ( +) +USING iceberg +TBLPROPERTIES ( 'write.delete.mode'='merge-on-read', 'write.update.mode'='merge-on-read', - 'write.merge.mode'='merge-on-read' - ); + 'write.merge.mode'='merge-on-read', + 'format-version'='2' +); """ ) -spark.sql( - """ - INSERT INTO test_positional_mor_deletes - VALUES ( - (1, 'a'), - (2, 'b'), - (3, 'c'), - (4, 'd'), - (5, 'e'), - (6, 'f'), - (7, 'g'), - (8, 'h'), - (9, 'i'), - (10, 'j'), - (11, 'k'), - (12, 'l'), - ) +spark.sql(""" +INSERT INTO test_positional_mor_deletes +VALUES + (1, 'a'), + (2, 'b'), + (3, 'c'), + (4, 'd'), + (5, 'e'), + (6, 'f'), + (7, 'g'), + (8, 'h'), + (9, 'i'), + (10, 'j'), + (11, 'k'), + (12, 'l'); """ ) -spark.sql( - """ - DELETE FROM test_positional_mor_deletes WHERE number % 2 = 0 -""" -) +spark.sql(""" +DELETE FROM test_positional_mor_deletes WHERE number = 9 +""") spark.sql( @@ -162,7 +157,8 @@ TBLPROPERTIES ( 'write.delete.mode'='merge-on-read', 'write.update.mode'='merge-on-read', - 'write.merge.mode'='merge-on-read' + 'write.merge.mode'='merge-on-read', + 'format-version'='2' ); """ ) @@ -182,14 +178,14 @@ (9, 'i'), (10, 'j'), (11, 'k'), - (12, 'l'), + (12, 'l') ) """ ) spark.sql( """ - DELETE FROM test_positional_mor_double_deletes WHERE number % 2 = 1 + DELETE FROM test_positional_mor_double_deletes WHERE number = 9 """ ) diff --git a/python/tests/test_integration.py b/python/tests/test_integration.py index fe2f515081df..f41f65cfc3bb 100644 --- a/python/tests/test_integration.py +++ b/python/tests/test_integration.py @@ -182,51 +182,69 @@ def test_pyarrow_to_iceberg_all_types(table_test_all_types: Table) -> None: @pytest.mark.integration def test_pyarrow_deletes(test_positional_mor_deletes: Table) -> None: + # number, letter + # (1, 'a'), + # (2, 'b'), + # (3, 'c'), + # (4, 'd'), + # (5, 'e'), + # (6, 'f'), + # (7, 'g'), + # (8, 'h'), + # (9, 'i'), <- deleted + # (10, 'j'), + # (11, 'k'), + # (12, 'l') arrow_table = test_positional_mor_deletes.scan().to_arrow() - assert len(arrow_table) == 6 - assert arrow_table["number"] == pa.Array([2, 4, 6, 8, 10, 12]) + assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 6, 7, 8, 10, 11, 12] # Checking the filter arrow_table = test_positional_mor_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) ).to_arrow() - assert len(arrow_table) == 3 - assert arrow_table["number"] == pa.Array([6, 8, 10]) + assert arrow_table["number"] == pa.array([6, 7, 8, 10]) # Testing the combination of a filter and a limit arrow_table = test_positional_mor_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 ).to_arrow() - assert len(arrow_table) == 1 - assert arrow_table["number"] == pa.Array([6]) + assert arrow_table["number"] == pa.array([6]) # Testing the slicing of indices arrow_table = test_positional_mor_deletes.scan(limit=3).to_arrow() - assert len(arrow_table) == 3 - assert arrow_table["number"] == pa.Array([2, 4, 6]) + assert arrow_table["number"] == pa.array([2, 3, 4]) @pytest.mark.integration def test_pyarrow_deletes_double(test_positional_mor_double_deletes: Table) -> None: + # number, letter + # (1, 'a'), + # (2, 'b'), + # (3, 'c'), + # (4, 'd'), + # (5, 'e'), + # (6, 'f'), <- second delete + # (7, 'g'), + # (8, 'h'), + # (9, 'i'), <- first delete + # (10, 'j'), + # (11, 'k'), + # (12, 'l') arrow_table = test_positional_mor_double_deletes.scan().to_arrow() - assert len(arrow_table) == 5 - assert arrow_table["number"] == pa.Array([2, 4, 8, 10, 12]) + assert arrow_table["number"] == pa.array([1, 2, 3, 4, 5, 7, 8, 10, 11, 12]) # Checking the filter arrow_table = test_positional_mor_double_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) ).to_arrow() - assert len(arrow_table) == 2 - assert arrow_table["number"] == pa.Array([8, 10]) + assert arrow_table["number"] == pa.array([5, 7, 8, 10]) # Testing the combination of a filter and a limit arrow_table = test_positional_mor_double_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 ).to_arrow() - assert len(arrow_table) == 1 - assert arrow_table["number"] == pa.Array([8]) + assert arrow_table["number"] == pa.array([5]) # Testing the slicing of indices - arrow_table = test_positional_mor_double_deletes.scan(limit=3).to_arrow() - assert len(arrow_table) == 3 - assert arrow_table["number"] == pa.Array([2, 4, 8]) + arrow_table = test_positional_mor_double_deletes.scan(limit=8).to_arrow() + assert arrow_table["number"] == pa.array([1, 2, 3, 4, 5, 7, 8, 10]) From ea09514aed8f8bda255dbac764e71a5d68376373 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 22 May 2023 13:15:51 -0500 Subject: [PATCH 31/36] Add moar tests --- python/dev/provision.py | 15 ++++--- python/pyiceberg/table/__init__.py | 55 +++++++++++++------------- python/tests/table/test_init.py | 63 +++++++++++++++++++++++++++++- 3 files changed, 99 insertions(+), 34 deletions(-) diff --git a/python/dev/provision.py b/python/dev/provision.py index 7a30c9b8d873..dcb14f80c502 100644 --- a/python/dev/provision.py +++ b/python/dev/provision.py @@ -102,12 +102,14 @@ """ ) -spark.sql(""" +spark.sql( + """ DROP TABLE IF EXISTS test_positional_mor_deletes; """ ) -spark.sql(""" +spark.sql( + """ CREATE TABLE test_positional_mor_deletes ( number integer, letter string @@ -122,7 +124,8 @@ """ ) -spark.sql(""" +spark.sql( + """ INSERT INTO test_positional_mor_deletes VALUES (1, 'a'), @@ -140,9 +143,11 @@ """ ) -spark.sql(""" +spark.sql( + """ DELETE FROM test_positional_mor_deletes WHERE number = 9 -""") +""" +) spark.sql( diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index b44a4ec680cb..097a45264d4e 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -348,6 +348,30 @@ def _min_data_file_sequence_number(manifests: List[ManifestFile]) -> int: return INITIAL_SEQUENCE_NUMBER +def _match_deletes_to_datafile(data_entry: ManifestEntry, positional_delete_entries: SortedList[ManifestEntry]) -> Set[DataFile]: + """This method will check if the delete file is relevant for the data file + by using the column metrics to see if the filename is in the lower and upper bound + + Args: + data_entry (ManifestEntry): The manifest entry path of the datafile + positional_delete_entries (List[ManifestEntry]): All the candidate positional deletes manifest entries + + Returns: + A set of files that are relevant for the data file. + """ + relevant_entries = positional_delete_entries[positional_delete_entries.bisect_right(data_entry) :] + + if len(relevant_entries) > 0: + evaluator = _InclusiveMetricsEvaluator(POSITIONAL_DELETE_SCHEMA, EqualTo("file_path", data_entry.data_file.file_path)) + return { + positional_delete_entry.data_file + for positional_delete_entry in relevant_entries + if evaluator.eval(positional_delete_entry.data_file) + } + else: + return set() + + class DataScan(TableScan): def __init__( self, @@ -414,6 +438,7 @@ def plan_files(self) -> Iterable[FileScanTask]: # step 1: filter manifests using partition summaries # the filter depends on the partition spec used to write the manifest file, so create a cache of filters for each spec id + manifest_evaluators: Dict[int, Callable[[ManifestFile], bool]] = KeyDefaultDict(self._build_manifest_evaluator) manifests = [ @@ -464,38 +489,14 @@ def plan_files(self) -> Iterable[FileScanTask]: return [ FileScanTask( data_entry.data_file, - delete_files=self._match_deletes_to_datafile( - data_entry.data_file.file_path, - positional_delete_entries[positional_delete_entries.bisect_right(data_entry) :], + delete_files=_match_deletes_to_datafile( + data_entry, + positional_delete_entries, ), ) for data_entry in data_entries ] - def _match_deletes_to_datafile(self, data_file_path: str, positional_delete_entries: List[ManifestEntry]) -> Set[DataFile]: - """This method will check if the delete file is relevant for the data file - - It will: - - Check if the delete file has been written after the data file - - Use the column metrics to see if the filename is part of the metrics - - Args: - data_file_path (str): The manifest entry path of the datafile - positional_delete_entries (List[ManifestEntry]): All the candidate positional deletes manifest entries - - Returns: - A set of files that are relevant for the data file. - """ - if len(positional_delete_entries) > 0: - evaluator = _InclusiveMetricsEvaluator(POSITIONAL_DELETE_SCHEMA, EqualTo("file_path", data_file_path)) - return { - positional_delete_entry.data_file - for positional_delete_entry in positional_delete_entries - if evaluator.eval(positional_delete_entry.data_file) - } - else: - return set() - def to_arrow(self) -> pa.Table: from pyiceberg.io.pyarrow import project_table diff --git a/python/tests/table/test_init.py b/python/tests/table/test_init.py index 2b2f26e5ceb5..9049f2ac3fd0 100644 --- a/python/tests/table/test_init.py +++ b/python/tests/table/test_init.py @@ -18,6 +18,7 @@ from typing import Any, Dict import pytest +from sortedcontainers import SortedList from pyiceberg.expressions import ( AlwaysTrue, @@ -26,10 +27,17 @@ In, ) from pyiceberg.io import PY_IO_IMPL, load_file_io +from pyiceberg.manifest import ( + DataFile, + DataFileContent, + FileFormat, + ManifestEntry, + ManifestEntryStatus, +) from pyiceberg.partitioning import PartitionField, PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import StaticTable, Table -from pyiceberg.table.metadata import TableMetadataV2 +from pyiceberg.table import StaticTable, Table, _match_deletes_to_datafile +from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadataV2 from pyiceberg.table.snapshots import ( Operation, Snapshot, @@ -258,3 +266,54 @@ def test_static_table_same_as_table(table: Table, static_table: StaticTable) -> def test_static_table_io_does_not_exist(metadata_location: str) -> None: with pytest.raises(ValueError): StaticTable.from_metadata(metadata_location, {PY_IO_IMPL: "pyiceberg.does.not.exist.FileIO"}) + + +def test_match_deletes_to_datafile() -> None: + data_entry = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=1, + data_file=DataFile( + content=DataFileContent.DATA, + file_path="s3://bucket/0000.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ), + ) + delete_entry_1 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=0, # Older than the data + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0001-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ), + ) + delete_entry_2 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=3, + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0002-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + # We don't really care about the tests here + value_counts={}, + null_value_counts={}, + nan_value_counts={}, + lower_bounds={}, + upper_bounds={}, + ), + ) + assert _match_deletes_to_datafile( + data_entry, + SortedList(iterable=[delete_entry_1, delete_entry_2], key=lambda entry: entry.sequence_number or INITIAL_SEQUENCE_NUMBER), + ) == { + delete_entry_2.data_file, + } From 5f1538ada065289c96fd2f6012ec3afd77b2d46c Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 22 May 2023 13:50:41 -0500 Subject: [PATCH 32/36] WIP --- python/Makefile | 3 +- python/dev/Dockerfile | 2 +- python/dev/entrypoint.sh | 4 +- python/dev/provision.py | 119 ++++++++++++++------------------------- 4 files changed, 47 insertions(+), 81 deletions(-) diff --git a/python/Makefile b/python/Makefile index 2796f7799c82..9346d236b5c3 100644 --- a/python/Makefile +++ b/python/Makefile @@ -37,7 +37,8 @@ test-integration: docker-compose -f dev/docker-compose-integration.yml rm -f docker-compose -f dev/docker-compose-integration.yml build --no-cache docker-compose -f dev/docker-compose-integration.yml up -d - sleep 30 + sleep 10 + docker-compose -f dev/docker-compose-integration.yml exec spark-iceberg ipython ./provision.py poetry run pytest tests/ -m integration ${PYTEST_ARGS} test-adlfs: diff --git a/python/dev/Dockerfile b/python/dev/Dockerfile index 76c96e39793e..c6bbe543d328 100644 --- a/python/dev/Dockerfile +++ b/python/dev/Dockerfile @@ -62,7 +62,7 @@ ENV PATH="/opt/spark/sbin:/opt/spark/bin:${PATH}" RUN chmod u+x /opt/spark/sbin/* && \ chmod u+x /opt/spark/bin/* -RUN pip3 install ipython +RUN pip3 install -q ipython COPY entrypoint.sh . COPY provision.py . diff --git a/python/dev/entrypoint.sh b/python/dev/entrypoint.sh index b0a613ae4177..574e876c7702 100755 --- a/python/dev/entrypoint.sh +++ b/python/dev/entrypoint.sh @@ -18,10 +18,8 @@ # under the License. # -set -ex - start-master.sh -p 7077 start-worker.sh spark://spark-iceberg:7077 start-history-server.sh -ipython -i ./provision.py +tail -f /dev/null diff --git a/python/dev/provision.py b/python/dev/provision.py index dcb14f80c502..567d3daeee41 100644 --- a/python/dev/provision.py +++ b/python/dev/provision.py @@ -14,7 +14,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -import time from pyspark.sql import SparkSession from pyspark.sql.functions import current_date, date_add, expr @@ -29,19 +28,7 @@ spark.sql( """ - use default; -""" -) - -spark.sql( - """ - DROP TABLE IF EXISTS test_null_nan; -""" -) - -spark.sql( - """ - CREATE TABLE test_null_nan + CREATE OR REPLACE TABLE default.test_null_nan USING iceberg AS SELECT 1 AS idx, @@ -57,60 +44,43 @@ spark.sql( """ - DROP TABLE IF EXISTS test_null_nan_rewritten; -""" -) - -spark.sql( - """ - CREATE TABLE test_null_nan_rewritten + CREATE OR REPLACE TABLE default.test_null_nan_rewritten USING iceberg AS SELECT * FROM test_null_nan """ ) -spark.sql( - """ - DROP TABLE IF EXISTS test_limit; -""" -) spark.sql( """ - CREATE TABLE test_limit - USING iceberg - AS SELECT - 1 AS idx - UNION ALL SELECT - 2 AS idx - UNION ALL SELECT - 3 AS idx - UNION ALL SELECT - 4 AS idx - UNION ALL SELECT - 5 AS idx - UNION ALL SELECT - 6 AS idx - UNION ALL SELECT - 7 AS idx - UNION ALL SELECT - 8 AS idx - UNION ALL SELECT - 9 AS idx - UNION ALL SELECT - 10 AS idx - """ -) - -spark.sql( - """ -DROP TABLE IF EXISTS test_positional_mor_deletes; +CREATE OR REPLACE TABLE default.test_limit +USING iceberg + AS SELECT + 1 AS idx + UNION ALL SELECT + 2 AS idx + UNION ALL SELECT + 3 AS idx + UNION ALL SELECT + 4 AS idx + UNION ALL SELECT + 5 AS idx + UNION ALL SELECT + 6 AS idx + UNION ALL SELECT + 7 AS idx + UNION ALL SELECT + 8 AS idx + UNION ALL SELECT + 9 AS idx + UNION ALL SELECT + 10 AS idx """ ) spark.sql( """ -CREATE TABLE test_positional_mor_deletes ( +CREATE OR REPLACE TABLE default.test_positional_mor_deletes ( number integer, letter string ) @@ -126,7 +96,7 @@ spark.sql( """ -INSERT INTO test_positional_mor_deletes +INSERT INTO default.test_positional_mor_deletes VALUES (1, 'a'), (2, 'b'), @@ -145,14 +115,14 @@ spark.sql( """ -DELETE FROM test_positional_mor_deletes WHERE number = 9 +DELETE FROM default.test_positional_mor_deletes WHERE number = 9 """ ) spark.sql( """ - CREATE TABLE test_positional_mor_double_deletes ( + CREATE OR REPLACE TABLE test_positional_mor_double_deletes ( number integer, letter string ) @@ -168,21 +138,21 @@ spark.sql( """ - INSERT INTO test_positional_mor_double_deletes - VALUES ( - (1, 'a'), - (2, 'b'), - (3, 'c'), - (4, 'd'), - (5, 'e'), - (6, 'f'), - (7, 'g'), - (8, 'h'), - (9, 'i'), - (10, 'j'), - (11, 'k'), - (12, 'l') - ) +INSERT INTO test_positional_mor_double_deletes +VALUES ( + (1, 'a'), + (2, 'b'), + (3, 'c'), + (4, 'd'), + (5, 'e'), + (6, 'f'), + (7, 'g'), + (8, 'h'), + (9, 'i'), + (10, 'j'), + (11, 'k'), + (12, 'l') +) """ ) @@ -222,6 +192,3 @@ all_types_dataframe.writeTo("default.test_all_types").tableProperty("format-version", "2").partitionedBy( "intCol" ).createOrReplace() - -while True: - time.sleep(1) From ce8dc22f7cfb136a0a4811872130e7ab9b0c5b86 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 24 May 2023 07:44:10 -0500 Subject: [PATCH 33/36] WIP --- python/Makefile | 2 +- python/dev/provision.py | 81 ++++++++++++++++++++------------ python/tests/table/test_init.py | 58 +++++++++++++++++++++++ python/tests/test_integration.py | 2 +- 4 files changed, 111 insertions(+), 32 deletions(-) diff --git a/python/Makefile b/python/Makefile index 9346d236b5c3..f20e57d444c6 100644 --- a/python/Makefile +++ b/python/Makefile @@ -39,7 +39,7 @@ test-integration: docker-compose -f dev/docker-compose-integration.yml up -d sleep 10 docker-compose -f dev/docker-compose-integration.yml exec spark-iceberg ipython ./provision.py - poetry run pytest tests/ -m integration ${PYTEST_ARGS} + poetry run pytest tests/ -v -m integration ${PYTEST_ARGS} test-adlfs: sh ./dev/run-azurite.sh diff --git a/python/dev/provision.py b/python/dev/provision.py index 567d3daeee41..b2871c322404 100644 --- a/python/dev/provision.py +++ b/python/dev/provision.py @@ -46,7 +46,7 @@ """ CREATE OR REPLACE TABLE default.test_null_nan_rewritten USING iceberg - AS SELECT * FROM test_null_nan + AS SELECT * FROM default.test_null_nan """ ) @@ -81,6 +81,7 @@ spark.sql( """ CREATE OR REPLACE TABLE default.test_positional_mor_deletes ( + dt date, number integer, letter string ) @@ -94,22 +95,31 @@ """ ) +# Partitioning is not really needed, but there is a bug: +# https://github.com/apache/iceberg/pull/7685 +spark.sql( + """ + ALTER TABLE default.test_positional_mor_deletes ADD PARTITION FIELD years(dt) AS dt_years +""" +) + + spark.sql( """ INSERT INTO default.test_positional_mor_deletes VALUES - (1, 'a'), - (2, 'b'), - (3, 'c'), - (4, 'd'), - (5, 'e'), - (6, 'f'), - (7, 'g'), - (8, 'h'), - (9, 'i'), - (10, 'j'), - (11, 'k'), - (12, 'l'); + (CAST('2023-03-01' AS date), 1, 'a'), + (CAST('2023-03-02' AS date), 2, 'b'), + (CAST('2023-03-03' AS date), 3, 'c'), + (CAST('2023-03-04' AS date), 4, 'd'), + (CAST('2023-03-05' AS date), 5, 'e'), + (CAST('2023-03-06' AS date), 6, 'f'), + (CAST('2023-03-07' AS date), 7, 'g'), + (CAST('2023-03-08' AS date), 8, 'h'), + (CAST('2023-03-09' AS date), 9, 'i'), + (CAST('2023-03-10' AS date), 10, 'j'), + (CAST('2023-03-11' AS date), 11, 'k'), + (CAST('2023-03-12' AS date), 12, 'l'); """ ) @@ -122,7 +132,8 @@ spark.sql( """ - CREATE OR REPLACE TABLE test_positional_mor_double_deletes ( + CREATE OR REPLACE TABLE default.test_positional_mor_double_deletes ( + dt date, number integer, letter string ) @@ -136,36 +147,46 @@ """ ) + + +# Partitioning is not really needed, but there is a bug: +# https://github.com/apache/iceberg/pull/7685 spark.sql( """ -INSERT INTO test_positional_mor_double_deletes -VALUES ( - (1, 'a'), - (2, 'b'), - (3, 'c'), - (4, 'd'), - (5, 'e'), - (6, 'f'), - (7, 'g'), - (8, 'h'), - (9, 'i'), - (10, 'j'), - (11, 'k'), - (12, 'l') + ALTER TABLE default.test_positional_mor_double_deletes ADD PARTITION FIELD years(dt) AS dt_years +""" ) + + +spark.sql( + """ +INSERT INTO default.test_positional_mor_double_deletes +VALUES + (CAST('2023-03-01' AS date), 1, 'a'), + (CAST('2023-03-02' AS date), 2, 'b'), + (CAST('2023-03-03' AS date), 3, 'c'), + (CAST('2023-03-04' AS date), 4, 'd'), + (CAST('2023-03-05' AS date), 5, 'e'), + (CAST('2023-03-06' AS date), 6, 'f'), + (CAST('2023-03-07' AS date), 7, 'g'), + (CAST('2023-03-08' AS date), 8, 'h'), + (CAST('2023-03-09' AS date), 9, 'i'), + (CAST('2023-03-10' AS date), 10, 'j'), + (CAST('2023-03-11' AS date), 11, 'k'), + (CAST('2023-03-12' AS date), 12, 'l'); """ ) spark.sql( """ - DELETE FROM test_positional_mor_double_deletes WHERE number = 9 + DELETE FROM default.test_positional_mor_double_deletes WHERE number = 9 """ ) spark.sql( """ - DELETE FROM test_positional_mor_double_deletes WHERE letter == 'f' + DELETE FROM default.test_positional_mor_double_deletes WHERE letter == 'f' """ ) diff --git a/python/tests/table/test_init.py b/python/tests/table/test_init.py index 9049f2ac3fd0..36a518076b0c 100644 --- a/python/tests/table/test_init.py +++ b/python/tests/table/test_init.py @@ -317,3 +317,61 @@ def test_match_deletes_to_datafile() -> None: ) == { delete_entry_2.data_file, } + + +def test_match_deletes_to_datafile_duplicate_number() -> None: + data_entry = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=1, + data_file=DataFile( + content=DataFileContent.DATA, + file_path="s3://bucket/0000.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + ), + ) + delete_entry_1 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=3, + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0001-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + # We don't really care about the tests here + value_counts={}, + null_value_counts={}, + nan_value_counts={}, + lower_bounds={}, + upper_bounds={}, + ), + ) + delete_entry_2 = ManifestEntry( + status=ManifestEntryStatus.ADDED, + sequence_number=3, + data_file=DataFile( + content=DataFileContent.POSITION_DELETES, + file_path="s3://bucket/0002-delete.parquet", + file_format=FileFormat.PARQUET, + partition={}, + record_count=3, + file_size_in_bytes=3, + # We don't really care about the tests here + value_counts={}, + null_value_counts={}, + nan_value_counts={}, + lower_bounds={}, + upper_bounds={}, + ), + ) + assert _match_deletes_to_datafile( + data_entry, + SortedList(iterable=[delete_entry_1, delete_entry_2], key=lambda entry: entry.sequence_number or INITIAL_SEQUENCE_NUMBER), + ) == { + delete_entry_1.data_file, + delete_entry_2.data_file, + } diff --git a/python/tests/test_integration.py b/python/tests/test_integration.py index f41f65cfc3bb..030887568a52 100644 --- a/python/tests/test_integration.py +++ b/python/tests/test_integration.py @@ -231,7 +231,7 @@ def test_pyarrow_deletes_double(test_positional_mor_double_deletes: Table) -> No # (11, 'k'), # (12, 'l') arrow_table = test_positional_mor_double_deletes.scan().to_arrow() - assert arrow_table["number"] == pa.array([1, 2, 3, 4, 5, 7, 8, 10, 11, 12]) + assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10, 11, 12] # Checking the filter arrow_table = test_positional_mor_double_deletes.scan( From 8b4890d2c60243d96d1730d68cf142e2639ad3d8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 24 May 2023 16:31:17 -0500 Subject: [PATCH 34/36] Cleanup --- python/Makefile | 6 +++++- python/dev/provision.py | 1 - python/pyiceberg/io/pyarrow.py | 5 ++++- python/pyiceberg/table/__init__.py | 23 ++++++++++++++++++++++- python/tests/test_integration.py | 13 ++++++------- 5 files changed, 37 insertions(+), 11 deletions(-) diff --git a/python/Makefile b/python/Makefile index f20e57d444c6..ab28cbfa78a2 100644 --- a/python/Makefile +++ b/python/Makefile @@ -35,12 +35,16 @@ test-s3: test-integration: docker-compose -f dev/docker-compose-integration.yml kill docker-compose -f dev/docker-compose-integration.yml rm -f - docker-compose -f dev/docker-compose-integration.yml build --no-cache docker-compose -f dev/docker-compose-integration.yml up -d sleep 10 docker-compose -f dev/docker-compose-integration.yml exec spark-iceberg ipython ./provision.py poetry run pytest tests/ -v -m integration ${PYTEST_ARGS} +test-integration-rebuild: + docker-compose -f dev/docker-compose-integration.yml kill + docker-compose -f dev/docker-compose-integration.yml rm -f + docker-compose -f dev/docker-compose-integration.yml build --no-cache + test-adlfs: sh ./dev/run-azurite.sh poetry run pytest tests/ -m adlfs ${PYTEST_ARGS} diff --git a/python/dev/provision.py b/python/dev/provision.py index b2871c322404..d1250bdc3aa5 100644 --- a/python/dev/provision.py +++ b/python/dev/provision.py @@ -148,7 +148,6 @@ ) - # Partitioning is not really needed, but there is a bug: # https://github.com/apache/iceberg/pull/7685 spark.sql( diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index 0ea9d37dc489..f9c947a510ec 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -530,7 +530,10 @@ def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedAr def _create_positional_deletes_indices(positional_deletes: List[pa.ChunkedArray], fn_rows: Callable[[], int]) -> pa.Array: - sorted_deleted = merge(*positional_deletes) + # This is not ideal, looking for a native PyArrow implementation :) + # Ideally with uniqueness as well + # https://github.com/apache/arrow/issues/35748 + sorted_deleted = merge(*positional_deletes, key=lambda e: e.as_py()) def generator() -> Generator[int, None, None]: deleted_pos = next(sorted_deleted).as_py() # type: ignore diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 097a45264d4e..936e9cdd0859 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -54,6 +54,7 @@ DataFileContent, ManifestContent, ManifestEntry, + ManifestEntryStatus, ManifestFile, live_entries, ) @@ -323,6 +324,26 @@ def __init__( self.length = length or data_file.file_size_in_bytes +def _inherit_sequence_numbers(manifest_entry: ManifestEntry, manifest: ManifestFile) -> ManifestEntry: + # The snapshot_id is required in V1, inherit with V2 + if manifest_entry.snapshot_id is None: + manifest_entry.snapshot_id = manifest.added_snapshot_id + + if manifest_entry.sequence_number is None and ( + manifest.sequence_number == 0 or manifest_entry.status == ManifestEntryStatus.ADDED + ): + # Only available in V2, always 0 in V1 + manifest_entry.sequence_number = manifest.sequence_number + + if manifest_entry.file_sequence_number is None and ( + manifest.sequence_number == 0 or manifest_entry.status == ManifestEntryStatus.ADDED + ): + # Only available in V2, always 0 in V1 + manifest_entry.file_sequence_number = manifest.sequence_number + + return manifest_entry + + def _open_manifest( io: FileIO, manifest: ManifestFile, @@ -330,7 +351,7 @@ def _open_manifest( metrics_evaluator: Callable[[DataFile], bool], ) -> List[ManifestEntry]: return [ - manifest_entry + _inherit_sequence_numbers(manifest_entry, manifest) for manifest_entry in live_entries(io.new_input(manifest.manifest_path)) if partition_filter(manifest_entry.data_file) and metrics_evaluator(manifest_entry.data_file) ] diff --git a/python/tests/test_integration.py b/python/tests/test_integration.py index 030887568a52..14b3a36bdae6 100644 --- a/python/tests/test_integration.py +++ b/python/tests/test_integration.py @@ -19,7 +19,6 @@ import math from urllib.parse import urlparse -import pyarrow as pa import pyarrow.parquet as pq import pytest from pyarrow.fs import S3FileSystem @@ -202,17 +201,17 @@ def test_pyarrow_deletes(test_positional_mor_deletes: Table) -> None: arrow_table = test_positional_mor_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) ).to_arrow() - assert arrow_table["number"] == pa.array([6, 7, 8, 10]) + assert arrow_table["number"].to_pylist() == [5, 6, 7, 8, 10] # Testing the combination of a filter and a limit arrow_table = test_positional_mor_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 ).to_arrow() - assert arrow_table["number"] == pa.array([6]) + assert arrow_table["number"].to_pylist() == [5] # Testing the slicing of indices arrow_table = test_positional_mor_deletes.scan(limit=3).to_arrow() - assert arrow_table["number"] == pa.array([2, 3, 4]) + assert arrow_table["number"].to_pylist() == [1, 2, 3] @pytest.mark.integration @@ -237,14 +236,14 @@ def test_pyarrow_deletes_double(test_positional_mor_double_deletes: Table) -> No arrow_table = test_positional_mor_double_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")) ).to_arrow() - assert arrow_table["number"] == pa.array([5, 7, 8, 10]) + assert arrow_table["number"].to_pylist() == [5, 7, 8, 10] # Testing the combination of a filter and a limit arrow_table = test_positional_mor_double_deletes.scan( row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1 ).to_arrow() - assert arrow_table["number"] == pa.array([5]) + assert arrow_table["number"].to_pylist() == [5] # Testing the slicing of indices arrow_table = test_positional_mor_double_deletes.scan(limit=8).to_arrow() - assert arrow_table["number"] == pa.array([1, 2, 3, 4, 5, 7, 8, 10]) + assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10] From 2e47f245d03a459b9661462a68850ae28a7a4f78 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 6 Jun 2023 00:51:24 +0200 Subject: [PATCH 35/36] WIP --- python/dev/provision.py | 2 +- python/poetry.lock | 1674 +++++++++++++-------------------------- 2 files changed, 547 insertions(+), 1129 deletions(-) diff --git a/python/dev/provision.py b/python/dev/provision.py index 4c405358c7dc..73ec34fdc109 100644 --- a/python/dev/provision.py +++ b/python/dev/provision.py @@ -184,4 +184,4 @@ all_types_dataframe.writeTo("default.test_all_types").tableProperty("format-version", "2").partitionedBy( "intCol" -).createOrReplace() \ No newline at end of file +).createOrReplace() diff --git a/python/poetry.lock b/python/poetry.lock index a8ac8deeaf21..b726a504e0d7 100644 --- a/python/poetry.lock +++ b/python/poetry.lock @@ -1,10 +1,9 @@ -# This file is automatically @generated by Poetry and should not be changed by hand. +# This file is automatically @generated by Poetry 1.5.1 and should not be changed by hand. [[package]] name = "adlfs" version = "2023.4.0" description = "Access Azure Datalake Gen1 with fsspec and dask" -category = "main" optional = true python-versions = ">=3.8" files = [ @@ -27,7 +26,6 @@ docs = ["furo", "myst-parser", "numpydoc", "sphinx"] name = "aiobotocore" version = "2.5.0" description = "Async client for aws services using botocore and aiohttp" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -49,23 +47,17 @@ boto3 = ["boto3 (>=1.26.76,<1.26.77)"] name = "aiohttp" version = "3.8.4" description = "Async http client/server framework (asyncio)" -category = "main" optional = true python-versions = ">=3.6" files = [ {file = "aiohttp-3.8.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5ce45967538fb747370308d3145aa68a074bdecb4f3a300869590f725ced69c1"}, {file = "aiohttp-3.8.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b744c33b6f14ca26b7544e8d8aadff6b765a80ad6164fb1a430bbadd593dfb1a"}, {file = "aiohttp-3.8.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1a45865451439eb320784918617ba54b7a377e3501fb70402ab84d38c2cd891b"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:a86d42d7cba1cec432d47ab13b6637bee393a10f664c425ea7b305d1301ca1a3"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:ee3c36df21b5714d49fc4580247947aa64bcbe2939d1b77b4c8dcb8f6c9faecc"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:176a64b24c0935869d5bbc4c96e82f89f643bcdf08ec947701b9dbb3c956b7dd"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:c844fd628851c0bc309f3c801b3a3d58ce430b2ce5b359cd918a5a76d0b20cb5"}, - {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:5393fb786a9e23e4799fec788e7e735de18052f83682ce2dfcabaf1c00c2c08e"}, + {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a86d42d7cba1cec432d47ab13b6637bee393a10f664c425ea7b305d1301ca1a3"}, + {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ee3c36df21b5714d49fc4580247947aa64bcbe2939d1b77b4c8dcb8f6c9faecc"}, + {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:176a64b24c0935869d5bbc4c96e82f89f643bcdf08ec947701b9dbb3c956b7dd"}, + {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c844fd628851c0bc309f3c801b3a3d58ce430b2ce5b359cd918a5a76d0b20cb5"}, + {file = "aiohttp-3.8.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5393fb786a9e23e4799fec788e7e735de18052f83682ce2dfcabaf1c00c2c08e"}, {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:e4b09863aae0dc965c3ef36500d891a3ff495a2ea9ae9171e4519963c12ceefd"}, {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:adfbc22e87365a6e564c804c58fc44ff7727deea782d175c33602737b7feadb6"}, {file = "aiohttp-3.8.4-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:147ae376f14b55f4f3c2b118b95be50a369b89b38a971e80a17c3fd623f280c9"}, @@ -76,16 +68,11 @@ files = [ {file = "aiohttp-3.8.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:3d8ef1a630519a26d6760bc695842579cb09e373c5f227a21b67dc3eb16cfea4"}, {file = "aiohttp-3.8.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:5b3f2e06a512e94722886c0827bee9807c86a9f698fac6b3aee841fab49bbfb4"}, {file = "aiohttp-3.8.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:3a80464982d41b1fbfe3154e440ba4904b71c1a53e9cd584098cd41efdb188ef"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:8b631e26df63e52f7cce0cce6507b7a7f1bc9b0c501fcde69742130b32e8782f"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:3f43255086fe25e36fd5ed8f2ee47477408a73ef00e804cb2b5cba4bf2ac7f5e"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:4d347a172f866cd1d93126d9b239fcbe682acb39b48ee0873c73c933dd23bd0f"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:a3fec6a4cb5551721cdd70473eb009d90935b4063acc5f40905d40ecfea23e05"}, - {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:80a37fe8f7c1e6ce8f2d9c411676e4bc633a8462844e38f46156d07a7d401654"}, + {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8b631e26df63e52f7cce0cce6507b7a7f1bc9b0c501fcde69742130b32e8782f"}, + {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3f43255086fe25e36fd5ed8f2ee47477408a73ef00e804cb2b5cba4bf2ac7f5e"}, + {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4d347a172f866cd1d93126d9b239fcbe682acb39b48ee0873c73c933dd23bd0f"}, + {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a3fec6a4cb5551721cdd70473eb009d90935b4063acc5f40905d40ecfea23e05"}, + {file = "aiohttp-3.8.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:80a37fe8f7c1e6ce8f2d9c411676e4bc633a8462844e38f46156d07a7d401654"}, {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:d1e6a862b76f34395a985b3cd39a0d949ca80a70b6ebdea37d3ab39ceea6698a"}, {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:cd468460eefef601ece4428d3cf4562459157c0f6523db89365202c31b6daebb"}, {file = "aiohttp-3.8.4-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:618c901dd3aad4ace71dfa0f5e82e88b46ef57e3239fc7027773cb6d4ed53531"}, @@ -94,16 +81,11 @@ files = [ {file = "aiohttp-3.8.4-cp311-cp311-win32.whl", hash = "sha256:bbcf1a76cf6f6dacf2c7f4d2ebd411438c275faa1dc0c68e46eb84eebd05dd7d"}, {file = "aiohttp-3.8.4-cp311-cp311-win_amd64.whl", hash = "sha256:6e74dd54f7239fcffe07913ff8b964e28b712f09846e20de78676ce2a3dc0bfc"}, {file = "aiohttp-3.8.4-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:880e15bb6dad90549b43f796b391cfffd7af373f4646784795e20d92606b7a51"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:bb96fa6b56bb536c42d6a4a87dfca570ff8e52de2d63cabebfd6fb67049c34b6"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:4a6cadebe132e90cefa77e45f2d2f1a4b2ce5c6b1bfc1656c1ddafcfe4ba8131"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:f352b62b45dff37b55ddd7b9c0c8672c4dd2eb9c0f9c11d395075a84e2c40f75"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:7ab43061a0c81198d88f39aaf90dae9a7744620978f7ef3e3708339b8ed2ef01"}, - {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:c9cb1565a7ad52e096a6988e2ee0397f72fe056dadf75d17fa6b5aebaea05622"}, + {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb96fa6b56bb536c42d6a4a87dfca570ff8e52de2d63cabebfd6fb67049c34b6"}, + {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4a6cadebe132e90cefa77e45f2d2f1a4b2ce5c6b1bfc1656c1ddafcfe4ba8131"}, + {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f352b62b45dff37b55ddd7b9c0c8672c4dd2eb9c0f9c11d395075a84e2c40f75"}, + {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7ab43061a0c81198d88f39aaf90dae9a7744620978f7ef3e3708339b8ed2ef01"}, + {file = "aiohttp-3.8.4-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c9cb1565a7ad52e096a6988e2ee0397f72fe056dadf75d17fa6b5aebaea05622"}, {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:1b3ea7edd2d24538959c1c1abf97c744d879d4e541d38305f9bd7d9b10c9ec41"}, {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:7c7837fe8037e96b6dd5cfcf47263c1620a9d332a87ec06a6ca4564e56bd0f36"}, {file = "aiohttp-3.8.4-cp36-cp36m-musllinux_1_1_ppc64le.whl", hash = "sha256:3b90467ebc3d9fa5b0f9b6489dfb2c304a1db7b9946fa92aa76a831b9d587e99"}, @@ -112,16 +94,11 @@ files = [ {file = "aiohttp-3.8.4-cp36-cp36m-win32.whl", hash = "sha256:5e14f25765a578a0a634d5f0cd1e2c3f53964553a00347998dfdf96b8137f777"}, {file = "aiohttp-3.8.4-cp36-cp36m-win_amd64.whl", hash = "sha256:4c745b109057e7e5f1848c689ee4fb3a016c8d4d92da52b312f8a509f83aa05e"}, {file = "aiohttp-3.8.4-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:aede4df4eeb926c8fa70de46c340a1bc2c6079e1c40ccf7b0eae1313ffd33519"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:4ddaae3f3d32fc2cb4c53fab020b69a05c8ab1f02e0e59665c6f7a0d3a5be54f"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:c4eb3b82ca349cf6fadcdc7abcc8b3a50ab74a62e9113ab7a8ebc268aad35bb9"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:9bcb89336efa095ea21b30f9e686763f2be4478f1b0a616969551982c4ee4c3b"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:6c08e8ed6fa3d477e501ec9db169bfac8140e830aa372d77e4a43084d8dd91ab"}, - {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:c6cd05ea06daca6ad6a4ca3ba7fe7dc5b5de063ff4daec6170ec0f9979f6c332"}, + {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4ddaae3f3d32fc2cb4c53fab020b69a05c8ab1f02e0e59665c6f7a0d3a5be54f"}, + {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c4eb3b82ca349cf6fadcdc7abcc8b3a50ab74a62e9113ab7a8ebc268aad35bb9"}, + {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9bcb89336efa095ea21b30f9e686763f2be4478f1b0a616969551982c4ee4c3b"}, + {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c08e8ed6fa3d477e501ec9db169bfac8140e830aa372d77e4a43084d8dd91ab"}, + {file = "aiohttp-3.8.4-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c6cd05ea06daca6ad6a4ca3ba7fe7dc5b5de063ff4daec6170ec0f9979f6c332"}, {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:b7a00a9ed8d6e725b55ef98b1b35c88013245f35f68b1b12c5cd4100dddac333"}, {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:de04b491d0e5007ee1b63a309956eaed959a49f5bb4e84b26c8f5d49de140fa9"}, {file = "aiohttp-3.8.4-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:40653609b3bf50611356e6b6554e3a331f6879fa7116f3959b20e3528783e699"}, @@ -132,16 +109,11 @@ files = [ {file = "aiohttp-3.8.4-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:fabb87dd8850ef0f7fe2b366d44b77d7e6fa2ea87861ab3844da99291e81e60f"}, {file = "aiohttp-3.8.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:91f6d540163f90bbaef9387e65f18f73ffd7c79f5225ac3d3f61df7b0d01ad15"}, {file = "aiohttp-3.8.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:d265f09a75a79a788237d7f9054f929ced2e69eb0bb79de3798c468d8a90f945"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:3d89efa095ca7d442a6d0cbc755f9e08190ba40069b235c9886a8763b03785da"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:4dac314662f4e2aa5009977b652d9b8db7121b46c38f2073bfeed9f4049732cd"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:fe11310ae1e4cd560035598c3f29d86cef39a83d244c7466f95c27ae04850f10"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:6ddb2a2026c3f6a68c3998a6c47ab6795e4127315d2e35a09997da21865757f8"}, - {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:e75b89ac3bd27d2d043b234aa7b734c38ba1b0e43f07787130a0ecac1e12228a"}, + {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3d89efa095ca7d442a6d0cbc755f9e08190ba40069b235c9886a8763b03785da"}, + {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4dac314662f4e2aa5009977b652d9b8db7121b46c38f2073bfeed9f4049732cd"}, + {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fe11310ae1e4cd560035598c3f29d86cef39a83d244c7466f95c27ae04850f10"}, + {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6ddb2a2026c3f6a68c3998a6c47ab6795e4127315d2e35a09997da21865757f8"}, + {file = "aiohttp-3.8.4-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e75b89ac3bd27d2d043b234aa7b734c38ba1b0e43f07787130a0ecac1e12228a"}, {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:6e601588f2b502c93c30cd5a45bfc665faaf37bbe835b7cfd461753068232074"}, {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:a5d794d1ae64e7753e405ba58e08fcfa73e3fad93ef9b7e31112ef3c9a0efb52"}, {file = "aiohttp-3.8.4-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:a1f4689c9a1462f3df0a1f7e797791cd6b124ddbee2b570d34e7f38ade0e2c71"}, @@ -152,16 +124,11 @@ files = [ {file = "aiohttp-3.8.4-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:5a7bdf9e57126dc345b683c3632e8ba317c31d2a41acd5800c10640387d193ed"}, {file = "aiohttp-3.8.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:22f6eab15b6db242499a16de87939a342f5a950ad0abaf1532038e2ce7d31567"}, {file = "aiohttp-3.8.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7235604476a76ef249bd64cb8274ed24ccf6995c4a8b51a237005ee7a57e8643"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:ea9eb976ffdd79d0e893869cfe179a8f60f152d42cb64622fca418cd9b18dc2a"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:92c0cea74a2a81c4c76b62ea1cac163ecb20fb3ba3a75c909b9fa71b4ad493cf"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:493f5bc2f8307286b7799c6d899d388bbaa7dfa6c4caf4f97ef7521b9cb13719"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:0a63f03189a6fa7c900226e3ef5ba4d3bd047e18f445e69adbd65af433add5a2"}, - {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:10c8cefcff98fd9168cdd86c4da8b84baaa90bf2da2269c6161984e6737bf23e"}, + {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ea9eb976ffdd79d0e893869cfe179a8f60f152d42cb64622fca418cd9b18dc2a"}, + {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:92c0cea74a2a81c4c76b62ea1cac163ecb20fb3ba3a75c909b9fa71b4ad493cf"}, + {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:493f5bc2f8307286b7799c6d899d388bbaa7dfa6c4caf4f97ef7521b9cb13719"}, + {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0a63f03189a6fa7c900226e3ef5ba4d3bd047e18f445e69adbd65af433add5a2"}, + {file = "aiohttp-3.8.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:10c8cefcff98fd9168cdd86c4da8b84baaa90bf2da2269c6161984e6737bf23e"}, {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:bca5f24726e2919de94f047739d0a4fc01372801a3672708260546aa2601bf57"}, {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:03baa76b730e4e15a45f81dfe29a8d910314143414e528737f8589ec60cf7391"}, {file = "aiohttp-3.8.4-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:8c29c77cc57e40f84acef9bfb904373a4e89a4e8b74e71aa8075c021ec9078c2"}, @@ -188,7 +155,6 @@ speedups = ["Brotli", "aiodns", "cchardet"] name = "aioitertools" version = "0.11.0" description = "itertools and builtins for AsyncIO and mixed iterables" -category = "main" optional = true python-versions = ">=3.6" files = [ @@ -203,7 +169,6 @@ typing_extensions = {version = ">=4.0", markers = "python_version < \"3.10\""} name = "aiosignal" version = "1.3.1" description = "aiosignal: a list of registered asynchronous callbacks" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -218,7 +183,6 @@ frozenlist = ">=1.1.0" name = "async-timeout" version = "4.0.2" description = "Timeout context manager for asyncio programs" -category = "main" optional = true python-versions = ">=3.6" files = [ @@ -230,7 +194,6 @@ files = [ name = "attrs" version = "23.1.0" description = "Classes Without Boilerplate" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -247,14 +210,13 @@ tests-no-zope = ["cloudpickle", "hypothesis", "mypy (>=1.1.1)", "pympler", "pyte [[package]] name = "azure-core" -version = "1.26.4" +version = "1.27.0" description = "Microsoft Azure Core Library for Python" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "azure-core-1.26.4.zip", hash = "sha256:075fe06b74c3007950dd93d49440c2f3430fd9b4a5a2756ec8c79454afc989c6"}, - {file = "azure_core-1.26.4-py3-none-any.whl", hash = "sha256:d9664b4bc2675d72fba461a285ac43ae33abb2967014a955bf136d9703a2ab3c"}, + {file = "azure-core-1.27.0.zip", hash = "sha256:068ea8b61888165b1e749892785936e293e35070a10ea10c8c11ec9f5186a2f8"}, + {file = "azure_core-1.27.0-py3-none-any.whl", hash = "sha256:8ec1b607d11ab0dc762606c4804b52b6b2fae83524e89ed575055046b69f1afe"}, ] [package.dependencies] @@ -269,7 +231,6 @@ aio = ["aiohttp (>=3.0)"] name = "azure-datalake-store" version = "0.0.53" description = "Azure Data Lake Store Filesystem Client Library for Python" -category = "main" optional = true python-versions = "*" files = [ @@ -286,7 +247,6 @@ requests = ">=2.20.0" name = "azure-identity" version = "1.13.0" description = "Microsoft Azure Identity Library for Python" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -305,7 +265,6 @@ six = ">=1.12.0" name = "azure-storage-blob" version = "12.16.0" description = "Microsoft Azure Blob Storage Client Library for Python" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -326,7 +285,6 @@ aio = ["azure-core[aio] (>=1.26.0,<2.0.0)"] name = "boto3" version = "1.26.76" description = "The AWS SDK for Python" -category = "main" optional = false python-versions = ">= 3.7" files = [ @@ -346,7 +304,6 @@ crt = ["botocore[crt] (>=1.21.0,<2.0a0)"] name = "botocore" version = "1.29.76" description = "Low-level, data-driven core of boto 3." -category = "main" optional = false python-versions = ">= 3.7" files = [ @@ -366,7 +323,6 @@ crt = ["awscrt (==0.16.9)"] name = "build" version = "0.10.0" description = "A simple, correct Python build frontend" -category = "dev" optional = false python-versions = ">= 3.7" files = [ @@ -382,8 +338,7 @@ tomli = {version = ">=1.1.0", markers = "python_version < \"3.11\""} [package.extras] docs = ["furo (>=2021.08.31)", "sphinx (>=4.0,<5.0)", "sphinx-argparse-cli (>=1.5)", "sphinx-autodoc-typehints (>=1.10)"] -test = ["filelock (>=3)", "pytest (>=6.2.4)", "pytest-cov (>=2.12)", "pytest-mock (>=2)", "pytest-rerunfailures (>=9.1)", "pytest-xdist (>=1.34)", -"setuptools (>=42.0.0)", "setuptools (>=56.0.0)", "toml (>=0.10.0)", "wheel (>=0.36.0)"] +test = ["filelock (>=3)", "pytest (>=6.2.4)", "pytest-cov (>=2.12)", "pytest-mock (>=2)", "pytest-rerunfailures (>=9.1)", "pytest-xdist (>=1.34)", "setuptools (>=42.0.0)", "setuptools (>=56.0.0)", "toml (>=0.10.0)", "wheel (>=0.36.0)"] typing = ["importlib-metadata (>=5.1)", "mypy (==0.991)", "tomli", "typing-extensions (>=3.7.4.3)"] virtualenv = ["virtualenv (>=20.0.35)"] @@ -391,7 +346,6 @@ virtualenv = ["virtualenv (>=20.0.35)"] name = "certifi" version = "2023.5.7" description = "Python package for providing Mozilla's CA Bundle." -category = "main" optional = false python-versions = ">=3.6" files = [ @@ -403,7 +357,6 @@ files = [ name = "cffi" version = "1.15.1" description = "Foreign Function Interface for Python calling C code." -category = "main" optional = false python-versions = "*" files = [ @@ -416,85 +369,56 @@ files = [ {file = "cffi-1.15.1-cp27-cp27mu-manylinux1_x86_64.whl", hash = "sha256:ed9cb427ba5504c1dc15ede7d516b84757c3e3d7868ccc85121d9310d27eed0b"}, {file = "cffi-1.15.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:39d39875251ca8f612b6f33e6b1195af86d1b3e60086068be9cc053aa4376e21"}, {file = "cffi-1.15.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:285d29981935eb726a4399badae8f0ffdff4f5050eaa6d0cfc3f64b857b77185"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:3eb6971dcff08619f8d91607cfc726518b6fa2a9eba42856be181c6d0d9515fd"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:21157295583fe8943475029ed5abdcf71eb3911894724e360acff1d61c1d54bc"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:5635bd9cb9731e6d4a1132a498dd34f764034a8ce60cef4f5319c0541159392f"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:2012c72d854c2d03e45d06ae57f40d78e5770d252f195b93f581acf3ba44496e"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:dd86c085fae2efd48ac91dd7ccffcfc0571387fe1193d33b6394db7ef31fe2a4"}, + {file = "cffi-1.15.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3eb6971dcff08619f8d91607cfc726518b6fa2a9eba42856be181c6d0d9515fd"}, + {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:21157295583fe8943475029ed5abdcf71eb3911894724e360acff1d61c1d54bc"}, + {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5635bd9cb9731e6d4a1132a498dd34f764034a8ce60cef4f5319c0541159392f"}, + {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2012c72d854c2d03e45d06ae57f40d78e5770d252f195b93f581acf3ba44496e"}, + {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd86c085fae2efd48ac91dd7ccffcfc0571387fe1193d33b6394db7ef31fe2a4"}, {file = "cffi-1.15.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:fa6693661a4c91757f4412306191b6dc88c1703f780c8234035eac011922bc01"}, {file = "cffi-1.15.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:59c0b02d0a6c384d453fece7566d1c7e6b7bae4fc5874ef2ef46d56776d61c9e"}, {file = "cffi-1.15.1-cp310-cp310-win32.whl", hash = "sha256:cba9d6b9a7d64d4bd46167096fc9d2f835e25d7e4c121fb2ddfc6528fb0413b2"}, {file = "cffi-1.15.1-cp310-cp310-win_amd64.whl", hash = "sha256:ce4bcc037df4fc5e3d184794f27bdaab018943698f4ca31630bc7f84a7b69c6d"}, {file = "cffi-1.15.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3d08afd128ddaa624a48cf2b859afef385b720bb4b43df214f85616922e6a5ac"}, {file = "cffi-1.15.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:3799aecf2e17cf585d977b780ce79ff0dc9b78d799fc694221ce814c2c19db83"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:a591fe9e525846e4d154205572a029f653ada1a78b93697f3b5a8f1f2bc055b9"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:3548db281cd7d2561c9ad9984681c95f7b0e38881201e157833a2342c30d5e8c"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:91fc98adde3d7881af9b59ed0294046f3806221863722ba7d8d120c575314325"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:94411f22c3985acaec6f83c6df553f2dbe17b698cc7f8ae751ff2237d96b9e3c"}, + {file = "cffi-1.15.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a591fe9e525846e4d154205572a029f653ada1a78b93697f3b5a8f1f2bc055b9"}, + {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3548db281cd7d2561c9ad9984681c95f7b0e38881201e157833a2342c30d5e8c"}, + {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:91fc98adde3d7881af9b59ed0294046f3806221863722ba7d8d120c575314325"}, + {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:94411f22c3985acaec6f83c6df553f2dbe17b698cc7f8ae751ff2237d96b9e3c"}, {file = "cffi-1.15.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:03425bdae262c76aad70202debd780501fabeaca237cdfddc008987c0e0f59ef"}, {file = "cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:cc4d65aeeaa04136a12677d3dd0b1c0c94dc43abac5860ab33cceb42b801c1e8"}, {file = "cffi-1.15.1-cp311-cp311-win32.whl", hash = "sha256:a0f100c8912c114ff53e1202d0078b425bee3649ae34d7b070e9697f93c5d52d"}, {file = "cffi-1.15.1-cp311-cp311-win_amd64.whl", hash = "sha256:04ed324bda3cda42b9b695d51bb7d54b680b9719cfab04227cdd1e04e5de3104"}, {file = "cffi-1.15.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:50a74364d85fd319352182ef59c5c790484a336f6db772c1a9231f1c3ed0cbd7"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:e263d77ee3dd201c3a142934a086a4450861778baaeeb45db4591ef65550b0a6"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:cec7d9412a9102bdc577382c3929b337320c4c4c4849f2c5cdd14d7368c5562d"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:4289fc34b2f5316fbb762d75362931e351941fa95fa18789191b33fc4cf9504a"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.whl", hash = -"sha256:173379135477dc8cac4bc58f45db08ab45d228b3363adb7af79436135d028405"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = -"sha256:6975a3fac6bc83c4a65c9f9fcab9e47019a11d3d2cf7f3c0d03431bf145a941e"}, + {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e263d77ee3dd201c3a142934a086a4450861778baaeeb45db4591ef65550b0a6"}, + {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:cec7d9412a9102bdc577382c3929b337320c4c4c4849f2c5cdd14d7368c5562d"}, + {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4289fc34b2f5316fbb762d75362931e351941fa95fa18789191b33fc4cf9504a"}, + {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:173379135477dc8cac4bc58f45db08ab45d228b3363adb7af79436135d028405"}, + {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:6975a3fac6bc83c4a65c9f9fcab9e47019a11d3d2cf7f3c0d03431bf145a941e"}, {file = "cffi-1.15.1-cp36-cp36m-win32.whl", hash = "sha256:2470043b93ff09bf8fb1d46d1cb756ce6132c54826661a32d4e4d132e1977adf"}, {file = "cffi-1.15.1-cp36-cp36m-win_amd64.whl", hash = "sha256:30d78fbc8ebf9c92c9b7823ee18eb92f2e6ef79b45ac84db507f52fbe3ec4497"}, {file = "cffi-1.15.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:198caafb44239b60e252492445da556afafc7d1e3ab7a1fb3f0584ef6d742375"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:5ef34d190326c3b1f822a5b7a45f6c4535e2f47ed06fec77d3d799c450b2651e"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:8102eaf27e1e448db915d08afa8b41d6c7ca7a04b7d73af6514df10a3e74bd82"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:5df2768244d19ab7f60546d0c7c63ce1581f7af8b5de3eb3004b9b6fc8a9f84b"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:a8c4917bd7ad33e8eb21e9a5bbba979b49d9a97acb3a803092cbc1133e20343c"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:0e2642fe3142e4cc4af0799748233ad6da94c62a8bec3a6648bf8ee68b1c7426"}, + {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5ef34d190326c3b1f822a5b7a45f6c4535e2f47ed06fec77d3d799c450b2651e"}, + {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8102eaf27e1e448db915d08afa8b41d6c7ca7a04b7d73af6514df10a3e74bd82"}, + {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5df2768244d19ab7f60546d0c7c63ce1581f7af8b5de3eb3004b9b6fc8a9f84b"}, + {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a8c4917bd7ad33e8eb21e9a5bbba979b49d9a97acb3a803092cbc1133e20343c"}, + {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e2642fe3142e4cc4af0799748233ad6da94c62a8bec3a6648bf8ee68b1c7426"}, {file = "cffi-1.15.1-cp37-cp37m-win32.whl", hash = "sha256:e229a521186c75c8ad9490854fd8bbdd9a0c9aa3a524326b55be83b54d4e0ad9"}, {file = "cffi-1.15.1-cp37-cp37m-win_amd64.whl", hash = "sha256:a0b71b1b8fbf2b96e41c4d990244165e2c9be83d54962a9a1d118fd8657d2045"}, {file = "cffi-1.15.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:320dab6e7cb2eacdf0e658569d2575c4dad258c0fcc794f46215e1e39f90f2c3"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:1e74c6b51a9ed6589199c787bf5f9875612ca4a8a0785fb2d4a84429badaf22a"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:a5c84c68147988265e60416b57fc83425a78058853509c1b0629c180094904a5"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:3b926aa83d1edb5aa5b427b4053dc420ec295a08e40911296b9eb1b6170f6cca"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:87c450779d0914f2861b8526e035c5e6da0a3199d8f1add1a665e1cbc6fc6d02"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:4f2c9f67e9821cad2e5f480bc8d83b8742896f1242dba247911072d4fa94c192"}, + {file = "cffi-1.15.1-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e74c6b51a9ed6589199c787bf5f9875612ca4a8a0785fb2d4a84429badaf22a"}, + {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5c84c68147988265e60416b57fc83425a78058853509c1b0629c180094904a5"}, + {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3b926aa83d1edb5aa5b427b4053dc420ec295a08e40911296b9eb1b6170f6cca"}, + {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:87c450779d0914f2861b8526e035c5e6da0a3199d8f1add1a665e1cbc6fc6d02"}, + {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4f2c9f67e9821cad2e5f480bc8d83b8742896f1242dba247911072d4fa94c192"}, {file = "cffi-1.15.1-cp38-cp38-win32.whl", hash = "sha256:8b7ee99e510d7b66cdb6c593f21c043c248537a32e0bedf02e01e9553a172314"}, {file = "cffi-1.15.1-cp38-cp38-win_amd64.whl", hash = "sha256:00a9ed42e88df81ffae7a8ab6d9356b371399b91dbdf0c3cb1e84c03a13aceb5"}, {file = "cffi-1.15.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:54a2db7b78338edd780e7ef7f9f6c442500fb0d41a5a4ea24fff1c929d5af585"}, {file = "cffi-1.15.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:fcd131dd944808b5bdb38e6f5b53013c5aa4f334c5cad0c72742f6eba4b73db0"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:7473e861101c9e72452f9bf8acb984947aa1661a7704553a9f6e4baa5ba64415"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:6c9a799e985904922a4d207a94eae35c78ebae90e128f0c4e521ce339396be9d"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:3bcde07039e586f91b45c88f8583ea7cf7a0770df3a1649627bf598332cb6984"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:33ab79603146aace82c2427da5ca6e58f2b3f2fb5da893ceac0c42218a40be35"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:5d598b938678ebf3c67377cdd45e09d431369c3b1a5b331058c338e201f12b27"}, + {file = "cffi-1.15.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7473e861101c9e72452f9bf8acb984947aa1661a7704553a9f6e4baa5ba64415"}, + {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6c9a799e985904922a4d207a94eae35c78ebae90e128f0c4e521ce339396be9d"}, + {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3bcde07039e586f91b45c88f8583ea7cf7a0770df3a1649627bf598332cb6984"}, + {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:33ab79603146aace82c2427da5ca6e58f2b3f2fb5da893ceac0c42218a40be35"}, + {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5d598b938678ebf3c67377cdd45e09d431369c3b1a5b331058c338e201f12b27"}, {file = "cffi-1.15.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:db0fbb9c62743ce59a9ff687eb5f4afbe77e5e8403d6697f7446e5f609976f76"}, {file = "cffi-1.15.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:98d85c6a2bef81588d9227dde12db8a7f47f639f4a17c9ae08e773aa9c697bf3"}, {file = "cffi-1.15.1-cp39-cp39-win32.whl", hash = "sha256:40f4774f5a9d4f5e344f31a32b5096977b5d48560c5592e2f3d2c4374bd543ee"}, @@ -509,7 +433,6 @@ pycparser = "*" name = "cfgv" version = "3.3.1" description = "Validate configuration and produce human readable error messages." -category = "dev" optional = false python-versions = ">=3.6.1" files = [ @@ -521,145 +444,81 @@ files = [ name = "charset-normalizer" version = "3.1.0" description = "The Real First Universal Charset Detector. Open, modern and actively maintained alternative to Chardet." -category = "main" optional = false python-versions = ">=3.7.0" files = [ {file = "charset-normalizer-3.1.0.tar.gz", hash = "sha256:34e0a2f9c370eb95597aae63bf85eb5e96826d81e3dcf88b8886012906f509b5"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_10_9_universal2.whl", hash = -"sha256:e0ac8959c929593fee38da1c2b64ee9778733cdf03c482c9ff1d508b6b593b2b"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = -"sha256:d7fc3fca01da18fbabe4625d64bb612b533533ed10045a2ac3dd194bfa656b60"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = -"sha256:04eefcee095f58eaabe6dc3cc2262f3bcd776d2c67005880894f447b3f2cb9c1"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:20064ead0717cf9a73a6d1e779b23d149b53daf971169289ed2ed43a71e8d3b0"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:1435ae15108b1cb6fffbcea2af3d468683b7afed0169ad718451f8db5d1aff6f"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:c84132a54c750fda57729d1e2599bb598f5fa0344085dbde5003ba429a4798c0"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:75f2568b4189dda1c567339b48cba4ac7384accb9c2a7ed655cd86b04055c795"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:11d3bcb7be35e7b1bba2c23beedac81ee893ac9871d0ba79effc7fc01167db6c"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = -"sha256:891cf9b48776b5c61c700b55a598621fdb7b1e301a550365571e9624f270c203"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_i686.whl", hash = -"sha256:5f008525e02908b20e04707a4f704cd286d94718f48bb33edddc7d7b584dddc1"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = -"sha256:b06f0d3bf045158d2fb8837c5785fe9ff9b8c93358be64461a1089f5da983137"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_s390x.whl", hash = -"sha256:49919f8400b5e49e961f320c735388ee686a62327e773fa5b3ce6721f7e785ce"}, - {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = -"sha256:22908891a380d50738e1f978667536f6c6b526a2064156203d418f4856d6e86a"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e0ac8959c929593fee38da1c2b64ee9778733cdf03c482c9ff1d508b6b593b2b"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d7fc3fca01da18fbabe4625d64bb612b533533ed10045a2ac3dd194bfa656b60"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:04eefcee095f58eaabe6dc3cc2262f3bcd776d2c67005880894f447b3f2cb9c1"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:20064ead0717cf9a73a6d1e779b23d149b53daf971169289ed2ed43a71e8d3b0"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1435ae15108b1cb6fffbcea2af3d468683b7afed0169ad718451f8db5d1aff6f"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c84132a54c750fda57729d1e2599bb598f5fa0344085dbde5003ba429a4798c0"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75f2568b4189dda1c567339b48cba4ac7384accb9c2a7ed655cd86b04055c795"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:11d3bcb7be35e7b1bba2c23beedac81ee893ac9871d0ba79effc7fc01167db6c"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:891cf9b48776b5c61c700b55a598621fdb7b1e301a550365571e9624f270c203"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:5f008525e02908b20e04707a4f704cd286d94718f48bb33edddc7d7b584dddc1"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:b06f0d3bf045158d2fb8837c5785fe9ff9b8c93358be64461a1089f5da983137"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:49919f8400b5e49e961f320c735388ee686a62327e773fa5b3ce6721f7e785ce"}, + {file = "charset_normalizer-3.1.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:22908891a380d50738e1f978667536f6c6b526a2064156203d418f4856d6e86a"}, {file = "charset_normalizer-3.1.0-cp310-cp310-win32.whl", hash = "sha256:12d1a39aa6b8c6f6248bb54550efcc1c38ce0d8096a146638fd4738e42284448"}, {file = "charset_normalizer-3.1.0-cp310-cp310-win_amd64.whl", hash = "sha256:65ed923f84a6844de5fd29726b888e58c62820e0769b76565480e1fdc3d062f8"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_10_9_universal2.whl", hash = -"sha256:9a3267620866c9d17b959a84dd0bd2d45719b817245e49371ead79ed4f710d19"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = -"sha256:6734e606355834f13445b6adc38b53c0fd45f1a56a9ba06c2058f86893ae8017"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = -"sha256:f8303414c7b03f794347ad062c0516cee0e15f7a612abd0ce1e25caf6ceb47df"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:aaf53a6cebad0eae578f062c7d462155eada9c172bd8c4d250b8c1d8eb7f916a"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:3dc5b6a8ecfdc5748a7e429782598e4f17ef378e3e272eeb1340ea57c9109f41"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:e1b25e3ad6c909f398df8921780d6a3d120d8c09466720226fc621605b6f92b1"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:0ca564606d2caafb0abe6d1b5311c2649e8071eb241b2d64e75a0d0065107e62"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:b82fab78e0b1329e183a65260581de4375f619167478dddab510c6c6fb04d9b6"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = -"sha256:bd7163182133c0c7701b25e604cf1611c0d87712e56e88e7ee5d72deab3e76b5"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_i686.whl", hash = -"sha256:11d117e6c63e8f495412d37e7dc2e2fff09c34b2d09dbe2bee3c6229577818be"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = -"sha256:cf6511efa4801b9b38dc5546d7547d5b5c6ef4b081c60b23e4d941d0eba9cbeb"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_s390x.whl", hash = -"sha256:abc1185d79f47c0a7aaf7e2412a0eb2c03b724581139193d2d82b3ad8cbb00ac"}, - {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = -"sha256:cb7b2ab0188829593b9de646545175547a70d9a6e2b63bf2cd87a0a391599324"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9a3267620866c9d17b959a84dd0bd2d45719b817245e49371ead79ed4f710d19"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:6734e606355834f13445b6adc38b53c0fd45f1a56a9ba06c2058f86893ae8017"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f8303414c7b03f794347ad062c0516cee0e15f7a612abd0ce1e25caf6ceb47df"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aaf53a6cebad0eae578f062c7d462155eada9c172bd8c4d250b8c1d8eb7f916a"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3dc5b6a8ecfdc5748a7e429782598e4f17ef378e3e272eeb1340ea57c9109f41"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e1b25e3ad6c909f398df8921780d6a3d120d8c09466720226fc621605b6f92b1"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0ca564606d2caafb0abe6d1b5311c2649e8071eb241b2d64e75a0d0065107e62"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b82fab78e0b1329e183a65260581de4375f619167478dddab510c6c6fb04d9b6"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:bd7163182133c0c7701b25e604cf1611c0d87712e56e88e7ee5d72deab3e76b5"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:11d117e6c63e8f495412d37e7dc2e2fff09c34b2d09dbe2bee3c6229577818be"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:cf6511efa4801b9b38dc5546d7547d5b5c6ef4b081c60b23e4d941d0eba9cbeb"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:abc1185d79f47c0a7aaf7e2412a0eb2c03b724581139193d2d82b3ad8cbb00ac"}, + {file = "charset_normalizer-3.1.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:cb7b2ab0188829593b9de646545175547a70d9a6e2b63bf2cd87a0a391599324"}, {file = "charset_normalizer-3.1.0-cp311-cp311-win32.whl", hash = "sha256:c36bcbc0d5174a80d6cccf43a0ecaca44e81d25be4b7f90f0ed7bcfbb5a00909"}, {file = "charset_normalizer-3.1.0-cp311-cp311-win_amd64.whl", hash = "sha256:cca4def576f47a09a943666b8f829606bcb17e2bc2d5911a46c8f8da45f56755"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = -"sha256:0c95f12b74681e9ae127728f7e5409cbbef9cd914d5896ef238cc779b8152373"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:fca62a8301b605b954ad2e9c3666f9d97f63872aa4efcae5492baca2056b74ab"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:ac0aa6cd53ab9a31d397f8303f92c42f534693528fafbdb997c82bae6e477ad9"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:c3af8e0f07399d3176b179f2e2634c3ce9c1301379a6b8c9c9aeecd481da494f"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:3a5fc78f9e3f501a1614a98f7c54d3969f3ad9bba8ba3d9b438c3bc5d047dd28"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:628c985afb2c7d27a4800bfb609e03985aaecb42f955049957814e0491d4006d"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = -"sha256:74db0052d985cf37fa111828d0dd230776ac99c740e1a758ad99094be4f1803d"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = -"sha256:1e8fcdd8f672a1c4fc8d0bd3a2b576b152d2a349782d1eb0f6b8e52e9954731d"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = -"sha256:04afa6387e2b282cf78ff3dbce20f0cc071c12dc8f685bd40960cc68644cfea6"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_s390x.whl", hash = -"sha256:dd5653e67b149503c68c4018bf07e42eeed6b4e956b24c00ccdf93ac79cdff84"}, - {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = -"sha256:d2686f91611f9e17f4548dbf050e75b079bbc2a82be565832bc8ea9047b61c8c"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:0c95f12b74681e9ae127728f7e5409cbbef9cd914d5896ef238cc779b8152373"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fca62a8301b605b954ad2e9c3666f9d97f63872aa4efcae5492baca2056b74ab"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ac0aa6cd53ab9a31d397f8303f92c42f534693528fafbdb997c82bae6e477ad9"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c3af8e0f07399d3176b179f2e2634c3ce9c1301379a6b8c9c9aeecd481da494f"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a5fc78f9e3f501a1614a98f7c54d3969f3ad9bba8ba3d9b438c3bc5d047dd28"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:628c985afb2c7d27a4800bfb609e03985aaecb42f955049957814e0491d4006d"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:74db0052d985cf37fa111828d0dd230776ac99c740e1a758ad99094be4f1803d"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:1e8fcdd8f672a1c4fc8d0bd3a2b576b152d2a349782d1eb0f6b8e52e9954731d"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:04afa6387e2b282cf78ff3dbce20f0cc071c12dc8f685bd40960cc68644cfea6"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:dd5653e67b149503c68c4018bf07e42eeed6b4e956b24c00ccdf93ac79cdff84"}, + {file = "charset_normalizer-3.1.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:d2686f91611f9e17f4548dbf050e75b079bbc2a82be565832bc8ea9047b61c8c"}, {file = "charset_normalizer-3.1.0-cp37-cp37m-win32.whl", hash = "sha256:4155b51ae05ed47199dc5b2a4e62abccb274cee6b01da5b895099b61b1982974"}, {file = "charset_normalizer-3.1.0-cp37-cp37m-win_amd64.whl", hash = "sha256:322102cdf1ab682ecc7d9b1c5eed4ec59657a65e1c146a0da342b78f4112db23"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_10_9_universal2.whl", hash = -"sha256:e633940f28c1e913615fd624fcdd72fdba807bf53ea6925d6a588e84e1151531"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = -"sha256:3a06f32c9634a8705f4ca9946d667609f52cf130d5548881401f1eb2c39b1e2c"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_11_0_arm64.whl", hash = -"sha256:7381c66e0561c5757ffe616af869b916c8b4e42b367ab29fedc98481d1e74e14"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:3573d376454d956553c356df45bb824262c397c6e26ce43e8203c4c540ee0acb"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:e89df2958e5159b811af9ff0f92614dabf4ff617c03a4c1c6ff53bf1c399e0e1"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:78cacd03e79d009d95635e7d6ff12c21eb89b894c354bd2b2ed0b4763373693b"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:de5695a6f1d8340b12a5d6d4484290ee74d61e467c39ff03b39e30df62cf83a0"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:1c60b9c202d00052183c9be85e5eaf18a4ada0a47d188a83c8f5c5b23252f649"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = -"sha256:f645caaf0008bacf349875a974220f1f1da349c5dbe7c4ec93048cdc785a3326"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_i686.whl", hash = -"sha256:ea9f9c6034ea2d93d9147818f17c2a0860d41b71c38b9ce4d55f21b6f9165a11"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = -"sha256:80d1543d58bd3d6c271b66abf454d437a438dff01c3e62fdbcd68f2a11310d4b"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_s390x.whl", hash = -"sha256:73dc03a6a7e30b7edc5b01b601e53e7fc924b04e1835e8e407c12c037e81adbd"}, - {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = -"sha256:6f5c2e7bc8a4bf7c426599765b1bd33217ec84023033672c1e9a8b35eaeaaaf8"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:e633940f28c1e913615fd624fcdd72fdba807bf53ea6925d6a588e84e1151531"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:3a06f32c9634a8705f4ca9946d667609f52cf130d5548881401f1eb2c39b1e2c"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7381c66e0561c5757ffe616af869b916c8b4e42b367ab29fedc98481d1e74e14"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3573d376454d956553c356df45bb824262c397c6e26ce43e8203c4c540ee0acb"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e89df2958e5159b811af9ff0f92614dabf4ff617c03a4c1c6ff53bf1c399e0e1"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:78cacd03e79d009d95635e7d6ff12c21eb89b894c354bd2b2ed0b4763373693b"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:de5695a6f1d8340b12a5d6d4484290ee74d61e467c39ff03b39e30df62cf83a0"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1c60b9c202d00052183c9be85e5eaf18a4ada0a47d188a83c8f5c5b23252f649"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:f645caaf0008bacf349875a974220f1f1da349c5dbe7c4ec93048cdc785a3326"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ea9f9c6034ea2d93d9147818f17c2a0860d41b71c38b9ce4d55f21b6f9165a11"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:80d1543d58bd3d6c271b66abf454d437a438dff01c3e62fdbcd68f2a11310d4b"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:73dc03a6a7e30b7edc5b01b601e53e7fc924b04e1835e8e407c12c037e81adbd"}, + {file = "charset_normalizer-3.1.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6f5c2e7bc8a4bf7c426599765b1bd33217ec84023033672c1e9a8b35eaeaaaf8"}, {file = "charset_normalizer-3.1.0-cp38-cp38-win32.whl", hash = "sha256:12a2b561af122e3d94cdb97fe6fb2bb2b82cef0cdca131646fdb940a1eda04f0"}, {file = "charset_normalizer-3.1.0-cp38-cp38-win_amd64.whl", hash = "sha256:3160a0fd9754aab7d47f95a6b63ab355388d890163eb03b2d2b87ab0a30cfa59"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_10_9_universal2.whl", hash = -"sha256:38e812a197bf8e71a59fe55b757a84c1f946d0ac114acafaafaf21667a7e169e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = -"sha256:6baf0baf0d5d265fa7944feb9f7451cc316bfe30e8df1a61b1bb08577c554f31"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_11_0_arm64.whl", hash = -"sha256:8f25e17ab3039b05f762b0a55ae0b3632b2e073d9c8fc88e89aca31a6198e88f"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:3747443b6a904001473370d7810aa19c3a180ccd52a7157aacc264a5ac79265e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:b116502087ce8a6b7a5f1814568ccbd0e9f6cfd99948aa59b0e241dc57cf739f"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:d16fd5252f883eb074ca55cb622bc0bee49b979ae4e8639fff6ca3ff44f9f854"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:21fa558996782fc226b529fdd2ed7866c2c6ec91cee82735c98a197fae39f706"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:6f6c7a8a57e9405cad7485f4c9d3172ae486cfef1344b5ddd8e5239582d7355e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = -"sha256:ac3775e3311661d4adace3697a52ac0bab17edd166087d493b52d4f4f553f9f0"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_i686.whl", hash = -"sha256:10c93628d7497c81686e8e5e557aafa78f230cd9e77dd0c40032ef90c18f2230"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = -"sha256:6f4f4668e1831850ebcc2fd0b1cd11721947b6dc7c00bf1c6bd3c929ae14f2c7"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_s390x.whl", hash = -"sha256:0be65ccf618c1e7ac9b849c315cc2e8a8751d9cfdaa43027d4f6624bd587ab7e"}, - {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = -"sha256:53d0a3fa5f8af98a1e261de6a3943ca631c526635eb5817a87a59d9a57ebf48f"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:38e812a197bf8e71a59fe55b757a84c1f946d0ac114acafaafaf21667a7e169e"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6baf0baf0d5d265fa7944feb9f7451cc316bfe30e8df1a61b1bb08577c554f31"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:8f25e17ab3039b05f762b0a55ae0b3632b2e073d9c8fc88e89aca31a6198e88f"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3747443b6a904001473370d7810aa19c3a180ccd52a7157aacc264a5ac79265e"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b116502087ce8a6b7a5f1814568ccbd0e9f6cfd99948aa59b0e241dc57cf739f"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d16fd5252f883eb074ca55cb622bc0bee49b979ae4e8639fff6ca3ff44f9f854"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:21fa558996782fc226b529fdd2ed7866c2c6ec91cee82735c98a197fae39f706"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6f6c7a8a57e9405cad7485f4c9d3172ae486cfef1344b5ddd8e5239582d7355e"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:ac3775e3311661d4adace3697a52ac0bab17edd166087d493b52d4f4f553f9f0"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:10c93628d7497c81686e8e5e557aafa78f230cd9e77dd0c40032ef90c18f2230"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:6f4f4668e1831850ebcc2fd0b1cd11721947b6dc7c00bf1c6bd3c929ae14f2c7"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:0be65ccf618c1e7ac9b849c315cc2e8a8751d9cfdaa43027d4f6624bd587ab7e"}, + {file = "charset_normalizer-3.1.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:53d0a3fa5f8af98a1e261de6a3943ca631c526635eb5817a87a59d9a57ebf48f"}, {file = "charset_normalizer-3.1.0-cp39-cp39-win32.whl", hash = "sha256:a04f86f41a8916fe45ac5024ec477f41f886b3c435da2d4e3d2709b22ab02af1"}, {file = "charset_normalizer-3.1.0-cp39-cp39-win_amd64.whl", hash = "sha256:830d2948a5ec37c386d3170c483063798d7879037492540f10a475e3fd6f244b"}, {file = "charset_normalizer-3.1.0-py3-none-any.whl", hash = "sha256:3d9098b479e78c85080c98e1e35ff40b4a31d8953102bb0fd7d1b6f8a2111a3d"}, @@ -669,7 +528,6 @@ files = [ name = "click" version = "8.1.3" description = "Composable command line interface toolkit" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -684,7 +542,6 @@ colorama = {version = "*", markers = "platform_system == \"Windows\""} name = "colorama" version = "0.4.6" description = "Cross-platform colored terminal text." -category = "main" optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*,>=2.7" files = [ @@ -696,18 +553,14 @@ files = [ name = "coverage" version = "7.2.7" description = "Code coverage measurement for Python" -category = "dev" optional = false python-versions = ">=3.7" files = [ {file = "coverage-7.2.7-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d39b5b4f2a66ccae8b7263ac3c8170994b65266797fb96cbbfd3fb5b23921db8"}, {file = "coverage-7.2.7-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6d040ef7c9859bb11dfeb056ff5b3872436e3b5e401817d87a31e1750b9ae2fb"}, - {file = "coverage-7.2.7-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:ba90a9563ba44a72fda2e85302c3abc71c5589cea608ca16c22b9804262aaeb6"}, - {file = "coverage-7.2.7-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:e7d9405291c6928619403db1d10bd07888888ec1abcbd9748fdaa971d7d661b2"}, - {file = "coverage-7.2.7-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:31563e97dae5598556600466ad9beea39fb04e0229e61c12eaa206e0aa202063"}, + {file = "coverage-7.2.7-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ba90a9563ba44a72fda2e85302c3abc71c5589cea608ca16c22b9804262aaeb6"}, + {file = "coverage-7.2.7-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e7d9405291c6928619403db1d10bd07888888ec1abcbd9748fdaa971d7d661b2"}, + {file = "coverage-7.2.7-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:31563e97dae5598556600466ad9beea39fb04e0229e61c12eaa206e0aa202063"}, {file = "coverage-7.2.7-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:ebba1cd308ef115925421d3e6a586e655ca5a77b5bf41e02eb0e4562a111f2d1"}, {file = "coverage-7.2.7-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:cb017fd1b2603ef59e374ba2063f593abe0fc45f2ad9abdde5b4d83bd922a353"}, {file = "coverage-7.2.7-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d62a5c7dad11015c66fbb9d881bc4caa5b12f16292f857842d9d1871595f4495"}, @@ -715,36 +568,27 @@ files = [ {file = "coverage-7.2.7-cp310-cp310-win_amd64.whl", hash = "sha256:f75f7168ab25dd93110c8a8117a22450c19976afbc44234cbf71481094c1b850"}, {file = "coverage-7.2.7-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:06a9a2be0b5b576c3f18f1a241f0473575c4a26021b52b2a85263a00f034d51f"}, {file = "coverage-7.2.7-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5baa06420f837184130752b7c5ea0808762083bf3487b5038d68b012e5937dbe"}, - {file = "coverage-7.2.7-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:fdec9e8cbf13a5bf63290fc6013d216a4c7232efb51548594ca3631a7f13c3a3"}, - {file = "coverage-7.2.7-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:52edc1a60c0d34afa421c9c37078817b2e67a392cab17d97283b64c5833f427f"}, - {file = "coverage-7.2.7-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:63426706118b7f5cf6bb6c895dc215d8a418d5952544042c8a2d9fe87fcf09cb"}, + {file = "coverage-7.2.7-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fdec9e8cbf13a5bf63290fc6013d216a4c7232efb51548594ca3631a7f13c3a3"}, + {file = "coverage-7.2.7-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:52edc1a60c0d34afa421c9c37078817b2e67a392cab17d97283b64c5833f427f"}, + {file = "coverage-7.2.7-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:63426706118b7f5cf6bb6c895dc215d8a418d5952544042c8a2d9fe87fcf09cb"}, {file = "coverage-7.2.7-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:afb17f84d56068a7c29f5fa37bfd38d5aba69e3304af08ee94da8ed5b0865833"}, {file = "coverage-7.2.7-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:48c19d2159d433ccc99e729ceae7d5293fbffa0bdb94952d3579983d1c8c9d97"}, {file = "coverage-7.2.7-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:0e1f928eaf5469c11e886fe0885ad2bf1ec606434e79842a879277895a50942a"}, {file = "coverage-7.2.7-cp311-cp311-win32.whl", hash = "sha256:33d6d3ea29d5b3a1a632b3c4e4f4ecae24ef170b0b9ee493883f2df10039959a"}, {file = "coverage-7.2.7-cp311-cp311-win_amd64.whl", hash = "sha256:5b7540161790b2f28143191f5f8ec02fb132660ff175b7747b95dcb77ac26562"}, {file = "coverage-7.2.7-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:f2f67fe12b22cd130d34d0ef79206061bfb5eda52feb6ce0dba0644e20a03cf4"}, - {file = "coverage-7.2.7-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:a342242fe22407f3c17f4b499276a02b01e80f861f1682ad1d95b04018e0c0d4"}, - {file = "coverage-7.2.7-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:171717c7cb6b453aebac9a2ef603699da237f341b38eebfee9be75d27dc38e01"}, - {file = "coverage-7.2.7-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:49969a9f7ffa086d973d91cec8d2e31080436ef0fb4a359cae927e742abfaaa6"}, + {file = "coverage-7.2.7-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a342242fe22407f3c17f4b499276a02b01e80f861f1682ad1d95b04018e0c0d4"}, + {file = "coverage-7.2.7-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:171717c7cb6b453aebac9a2ef603699da237f341b38eebfee9be75d27dc38e01"}, + {file = "coverage-7.2.7-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:49969a9f7ffa086d973d91cec8d2e31080436ef0fb4a359cae927e742abfaaa6"}, {file = "coverage-7.2.7-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:b46517c02ccd08092f4fa99f24c3b83d8f92f739b4657b0f146246a0ca6a831d"}, {file = "coverage-7.2.7-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:a3d33a6b3eae87ceaefa91ffdc130b5e8536182cd6dfdbfc1aa56b46ff8c86de"}, {file = "coverage-7.2.7-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:976b9c42fb2a43ebf304fa7d4a310e5f16cc99992f33eced91ef6f908bd8f33d"}, {file = "coverage-7.2.7-cp312-cp312-win32.whl", hash = "sha256:8de8bb0e5ad103888d65abef8bca41ab93721647590a3f740100cd65c3b00511"}, {file = "coverage-7.2.7-cp312-cp312-win_amd64.whl", hash = "sha256:9e31cb64d7de6b6f09702bb27c02d1904b3aebfca610c12772452c4e6c21a0d3"}, {file = "coverage-7.2.7-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:58c2ccc2f00ecb51253cbe5d8d7122a34590fac9646a960d1430d5b15321d95f"}, - {file = "coverage-7.2.7-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:d22656368f0e6189e24722214ed8d66b8022db19d182927b9a248a2a8a2f67eb"}, - {file = "coverage-7.2.7-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:a895fcc7b15c3fc72beb43cdcbdf0ddb7d2ebc959edac9cef390b0d14f39f8a9"}, - {file = "coverage-7.2.7-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:e84606b74eb7de6ff581a7915e2dab7a28a0517fbe1c9239eb227e1354064dcd"}, + {file = "coverage-7.2.7-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d22656368f0e6189e24722214ed8d66b8022db19d182927b9a248a2a8a2f67eb"}, + {file = "coverage-7.2.7-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a895fcc7b15c3fc72beb43cdcbdf0ddb7d2ebc959edac9cef390b0d14f39f8a9"}, + {file = "coverage-7.2.7-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e84606b74eb7de6ff581a7915e2dab7a28a0517fbe1c9239eb227e1354064dcd"}, {file = "coverage-7.2.7-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:0a5f9e1dbd7fbe30196578ca36f3fba75376fb99888c395c5880b355e2875f8a"}, {file = "coverage-7.2.7-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:419bfd2caae268623dd469eff96d510a920c90928b60f2073d79f8fe2bbc5959"}, {file = "coverage-7.2.7-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:2aee274c46590717f38ae5e4650988d1af340fe06167546cc32fe2f58ed05b02"}, @@ -752,12 +596,9 @@ files = [ {file = "coverage-7.2.7-cp37-cp37m-win_amd64.whl", hash = "sha256:b1c546aca0ca4d028901d825015dc8e4d56aac4b541877690eb76490f1dc8ed0"}, {file = "coverage-7.2.7-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:54b896376ab563bd38453cecb813c295cf347cf5906e8b41d340b0321a5433e5"}, {file = "coverage-7.2.7-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:3d376df58cc111dc8e21e3b6e24606b5bb5dee6024f46a5abca99124b2229ef5"}, - {file = "coverage-7.2.7-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:5e330fc79bd7207e46c7d7fd2bb4af2963f5f635703925543a70b99574b0fea9"}, - {file = "coverage-7.2.7-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:1e9d683426464e4a252bf70c3498756055016f99ddaec3774bf368e76bbe02b6"}, - {file = "coverage-7.2.7-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:8d13c64ee2d33eccf7437961b6ea7ad8673e2be040b4f7fd4fd4d4d28d9ccb1e"}, + {file = "coverage-7.2.7-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5e330fc79bd7207e46c7d7fd2bb4af2963f5f635703925543a70b99574b0fea9"}, + {file = "coverage-7.2.7-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e9d683426464e4a252bf70c3498756055016f99ddaec3774bf368e76bbe02b6"}, + {file = "coverage-7.2.7-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8d13c64ee2d33eccf7437961b6ea7ad8673e2be040b4f7fd4fd4d4d28d9ccb1e"}, {file = "coverage-7.2.7-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:b7aa5f8a41217360e600da646004f878250a0d6738bcdc11a0a39928d7dc2050"}, {file = "coverage-7.2.7-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:8fa03bce9bfbeeef9f3b160a8bed39a221d82308b4152b27d82d8daa7041fee5"}, {file = "coverage-7.2.7-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:245167dd26180ab4c91d5e1496a30be4cd721a5cf2abf52974f965f10f11419f"}, @@ -765,12 +606,9 @@ files = [ {file = "coverage-7.2.7-cp38-cp38-win_amd64.whl", hash = "sha256:2e07b54284e381531c87f785f613b833569c14ecacdcb85d56b25c4622c16c3c"}, {file = "coverage-7.2.7-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:537891ae8ce59ef63d0123f7ac9e2ae0fc8b72c7ccbe5296fec45fd68967b6c9"}, {file = "coverage-7.2.7-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:06fb182e69f33f6cd1d39a6c597294cff3143554b64b9825d1dc69d18cc2fff2"}, - {file = "coverage-7.2.7-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:201e7389591af40950a6480bd9edfa8ed04346ff80002cec1a66cac4549c1ad7"}, - {file = "coverage-7.2.7-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:f6951407391b639504e3b3be51b7ba5f3528adbf1a8ac3302b687ecababf929e"}, - {file = "coverage-7.2.7-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:6f48351d66575f535669306aa7d6d6f71bc43372473b54a832222803eb956fd1"}, + {file = "coverage-7.2.7-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:201e7389591af40950a6480bd9edfa8ed04346ff80002cec1a66cac4549c1ad7"}, + {file = "coverage-7.2.7-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f6951407391b639504e3b3be51b7ba5f3528adbf1a8ac3302b687ecababf929e"}, + {file = "coverage-7.2.7-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6f48351d66575f535669306aa7d6d6f71bc43372473b54a832222803eb956fd1"}, {file = "coverage-7.2.7-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:b29019c76039dc3c0fd815c41392a044ce555d9bcdd38b0fb60fb4cd8e475ba9"}, {file = "coverage-7.2.7-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:81c13a1fc7468c40f13420732805a4c38a105d89848b7c10af65a90beff25250"}, {file = "coverage-7.2.7-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:975d70ab7e3c80a3fe86001d8751f6778905ec723f5b110aed1e450da9d4b7f2"}, @@ -788,43 +626,30 @@ toml = ["tomli"] [[package]] name = "cryptography" -version = "41.0.0" +version = "41.0.1" description = "cryptography is a package which provides cryptographic recipes and primitives to Python developers." -category = "main" optional = false python-versions = ">=3.7" files = [ - {file = "cryptography-41.0.0-cp37-abi3-macosx_10_12_universal2.whl", hash = -"sha256:3c5ef25d060c80d6d9f7f9892e1d41bb1c79b78ce74805b8cb4aa373cb7d5ec8"}, - {file = "cryptography-41.0.0-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:8362565b3835ceacf4dc8f3b56471a2289cf51ac80946f9087e66dc283a810e0"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:3680248309d340fda9611498a5319b0193a8dbdb73586a1acf8109d06f25b92d"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:84a165379cb9d411d58ed739e4af3396e544eac190805a54ba2e0322feb55c46"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_28_aarch64.whl", hash = -"sha256:4ab14d567f7bbe7f1cdff1c53d5324ed4d3fc8bd17c481b395db224fb405c237"}, - {file = "cryptography-41.0.0-cp37-abi3-manylinux_2_28_x86_64.whl", hash = -"sha256:9f65e842cb02550fac96536edb1d17f24c0a338fd84eaf582be25926e993dde4"}, - {file = "cryptography-41.0.0-cp37-abi3-musllinux_1_1_aarch64.whl", hash = -"sha256:b7f2f5c525a642cecad24ee8670443ba27ac1fab81bba4cc24c7b6b41f2d0c75"}, - {file = "cryptography-41.0.0-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:7d92f0248d38faa411d17f4107fc0bce0c42cae0b0ba5415505df72d751bf62d"}, - {file = "cryptography-41.0.0-cp37-abi3-win32.whl", hash = "sha256:34d405ea69a8b34566ba3dfb0521379b210ea5d560fafedf9f800a9a94a41928"}, - {file = "cryptography-41.0.0-cp37-abi3-win_amd64.whl", hash = "sha256:344c6de9f8bda3c425b3a41b319522ba3208551b70c2ae00099c205f0d9fd3be"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-macosx_10_12_x86_64.whl", hash = -"sha256:88ff107f211ea696455ea8d911389f6d2b276aabf3231bf72c8853d22db755c5"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-manylinux_2_28_aarch64.whl", hash = -"sha256:b846d59a8d5a9ba87e2c3d757ca019fa576793e8758174d3868aecb88d6fc8eb"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-manylinux_2_28_x86_64.whl", hash = -"sha256:f5d0bf9b252f30a31664b6f64432b4730bb7038339bd18b1fafe129cfc2be9be"}, - {file = "cryptography-41.0.0-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:5c1f7293c31ebc72163a9a0df246f890d65f66b4a40d9ec80081969ba8c78cc9"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = -"sha256:bf8fc66012ca857d62f6a347007e166ed59c0bc150cefa49f28376ebe7d992a2"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = -"sha256:a4fc68d1c5b951cfb72dfd54702afdbbf0fb7acdc9b7dc4301bbf2225a27714d"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = -"sha256:14754bcdae909d66ff24b7b5f166d69340ccc6cb15731670435efd5719294895"}, - {file = "cryptography-41.0.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:0ddaee209d1cf1f180f1efa338a68c4621154de0afaef92b89486f5f96047c55"}, - {file = "cryptography-41.0.0.tar.gz", hash = "sha256:6b71f64beeea341c9b4f963b48ee3b62d62d57ba93eb120e1196b31dc1025e78"}, + {file = "cryptography-41.0.1-cp37-abi3-macosx_10_12_universal2.whl", hash = "sha256:f73bff05db2a3e5974a6fd248af2566134d8981fd7ab012e5dd4ddb1d9a70699"}, + {file = "cryptography-41.0.1-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:1a5472d40c8f8e91ff7a3d8ac6dfa363d8e3138b961529c996f3e2df0c7a411a"}, + {file = "cryptography-41.0.1-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7fa01527046ca5facdf973eef2535a27fec4cb651e4daec4d043ef63f6ecd4ca"}, + {file = "cryptography-41.0.1-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b46e37db3cc267b4dea1f56da7346c9727e1209aa98487179ee8ebed09d21e43"}, + {file = "cryptography-41.0.1-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:d198820aba55660b4d74f7b5fd1f17db3aa5eb3e6893b0a41b75e84e4f9e0e4b"}, + {file = "cryptography-41.0.1-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:948224d76c4b6457349d47c0c98657557f429b4e93057cf5a2f71d603e2fc3a3"}, + {file = "cryptography-41.0.1-cp37-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:059e348f9a3c1950937e1b5d7ba1f8e968508ab181e75fc32b879452f08356db"}, + {file = "cryptography-41.0.1-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:b4ceb5324b998ce2003bc17d519080b4ec8d5b7b70794cbd2836101406a9be31"}, + {file = "cryptography-41.0.1-cp37-abi3-win32.whl", hash = "sha256:8f4ab7021127a9b4323537300a2acfb450124b2def3756f64dc3a3d2160ee4b5"}, + {file = "cryptography-41.0.1-cp37-abi3-win_amd64.whl", hash = "sha256:1fee5aacc7367487b4e22484d3c7e547992ed726d14864ee33c0176ae43b0d7c"}, + {file = "cryptography-41.0.1-pp38-pypy38_pp73-macosx_10_12_x86_64.whl", hash = "sha256:9a6c7a3c87d595608a39980ebaa04d5a37f94024c9f24eb7d10262b92f739ddb"}, + {file = "cryptography-41.0.1-pp38-pypy38_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:5d092fdfedaec4cbbffbf98cddc915ba145313a6fdaab83c6e67f4e6c218e6f3"}, + {file = "cryptography-41.0.1-pp38-pypy38_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:1a8e6c2de6fbbcc5e14fd27fb24414507cb3333198ea9ab1258d916f00bc3039"}, + {file = "cryptography-41.0.1-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:cb33ccf15e89f7ed89b235cff9d49e2e62c6c981a6061c9c8bb47ed7951190bc"}, + {file = "cryptography-41.0.1-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:5f0ff6e18d13a3de56f609dd1fd11470918f770c6bd5d00d632076c727d35485"}, + {file = "cryptography-41.0.1-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:7bfc55a5eae8b86a287747053140ba221afc65eb06207bedf6e019b8934b477c"}, + {file = "cryptography-41.0.1-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:eb8163f5e549a22888c18b0d53d6bb62a20510060a22fd5a995ec8a05268df8a"}, + {file = "cryptography-41.0.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:8dde71c4169ec5ccc1087bb7521d54251c016f126f922ab2dfe6649170a3b8c5"}, + {file = "cryptography-41.0.1.tar.gz", hash = "sha256:d34579085401d3f49762d2f7d6634d6b6c2ae1242202e860f4d26b046e3a1006"}, ] [package.dependencies] @@ -844,7 +669,6 @@ test-randomorder = ["pytest-randomly"] name = "distlib" version = "0.3.6" description = "Distribution utilities" -category = "main" optional = false python-versions = "*" files = [ @@ -856,7 +680,6 @@ files = [ name = "docutils" version = "0.20.1" description = "Docutils -- Python Documentation Utilities" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -868,17 +691,14 @@ files = [ name = "duckdb" version = "0.8.0" description = "DuckDB embedded database" -category = "main" optional = true python-versions = "*" files = [ {file = "duckdb-0.8.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6455aee00af30770c20f4a8c5e4347918cf59b578f49ee996a13807b12911871"}, {file = "duckdb-0.8.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b8cf0622ae7f86d4ce72791f8928af4357a46824aadf1b6879c7936b3db65344"}, {file = "duckdb-0.8.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6132e8183ca3ae08a593e43c97cb189794077dedd48546e27ce43bd6a51a9c33"}, - {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:fe29e5343fa2a95f2cde4519a4f4533f4fd551a48d2d9a8ab5220d40ebf53610"}, - {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:945165987ca87c097dc0e578dcf47a100cad77e1c29f5dd8443d53ce159dc22e"}, + {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fe29e5343fa2a95f2cde4519a4f4533f4fd551a48d2d9a8ab5220d40ebf53610"}, + {file = "duckdb-0.8.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:945165987ca87c097dc0e578dcf47a100cad77e1c29f5dd8443d53ce159dc22e"}, {file = "duckdb-0.8.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:673c60daf7ada1d9a8518286a6893ec45efabb64602954af5f3d98f42912fda6"}, {file = "duckdb-0.8.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d5075fe1ff97ae62331ca5c61e3597e6e9f7682a6fdd418c23ba5c4873ed5cd1"}, {file = "duckdb-0.8.0-cp310-cp310-win32.whl", hash = "sha256:001f5102f45d3d67f389fa8520046c8f55a99e2c6d43b8e68b38ea93261c5395"}, @@ -886,10 +706,8 @@ files = [ {file = "duckdb-0.8.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:b2707096d6df4321044fcde2c9f04da632d11a8be60957fd09d49a42fae71a29"}, {file = "duckdb-0.8.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b27df1b70ae74d2c88efb5ffca8490954fdc678099509a9c4404ca30acc53426"}, {file = "duckdb-0.8.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:75a97c800271b52dd0f37696d074c50576dcb4b2750b6115932a98696a268070"}, - {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:804cac261a5e016506a6d67838a65d19b06a237f7949f1704f0e800eb708286a"}, - {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:c6b9abca7fa6713e1d031c18485343b4de99742c7e1b85c10718aa2f31a4e2c6"}, + {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:804cac261a5e016506a6d67838a65d19b06a237f7949f1704f0e800eb708286a"}, + {file = "duckdb-0.8.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c6b9abca7fa6713e1d031c18485343b4de99742c7e1b85c10718aa2f31a4e2c6"}, {file = "duckdb-0.8.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:51aa6d606d49072abcfeb3be209eb559ac94c1b5e70f58ac3adbb94aca9cd69f"}, {file = "duckdb-0.8.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:7c8dc769aaf2be0a1c57995ca657e5b92c1c56fc8437edb720ca6cab571adf14"}, {file = "duckdb-0.8.0-cp311-cp311-win32.whl", hash = "sha256:c4207d18b42387c4a035846d8878eb967070198be8ac26fd77797ce320d1a400"}, @@ -898,10 +716,8 @@ files = [ {file = "duckdb-0.8.0-cp36-cp36m-win32.whl", hash = "sha256:914896526f7caba86b170f2c4f17f11fd06540325deeb0000cb4fb24ec732966"}, {file = "duckdb-0.8.0-cp36-cp36m-win_amd64.whl", hash = "sha256:022ebda86d0e3204cdc206e4af45aa9f0ae0668b34c2c68cf88e08355af4a372"}, {file = "duckdb-0.8.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:96a31c0f3f4ccbf0f5b18f94319f37691205d82f80aae48c6fe04860d743eb2c"}, - {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:a07c73c6e6a8cf4ce1a634625e0d1b17e5b817242a8a530d26ed84508dfbdc26"}, - {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:424acbd6e857531b06448d757d7c2557938dbddbff0632092090efbf413b4699"}, + {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a07c73c6e6a8cf4ce1a634625e0d1b17e5b817242a8a530d26ed84508dfbdc26"}, + {file = "duckdb-0.8.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:424acbd6e857531b06448d757d7c2557938dbddbff0632092090efbf413b4699"}, {file = "duckdb-0.8.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:c83cfd2a868f1acb0692b9c3fd5ef1d7da8faa1348c6eabf421fbf5d8c2f3eb8"}, {file = "duckdb-0.8.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:5c6f6b2d8db56936f662c649539df81856b5a8cb769a31f9544edf18af2a11ff"}, {file = "duckdb-0.8.0-cp37-cp37m-win32.whl", hash = "sha256:0bd6376b40a512172eaf4aa816813b1b9d68994292ca436ce626ccd5f77f8184"}, @@ -909,10 +725,8 @@ files = [ {file = "duckdb-0.8.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:42e7853d963d68e72403ea208bcf806b0f28c7b44db0aa85ce49bb124d56c133"}, {file = "duckdb-0.8.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:fcc338399175be3d43366576600aef7d72e82114d415992a7a95aded98a0f3fd"}, {file = "duckdb-0.8.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:03dd08a4624d6b581a59f9f9dbfd34902416398d16795ad19f92361cf21fd9b5"}, - {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:0c7c24ea0c9d8563dbd5ad49ccb54b7a9a3c7b8c2833d35e5d32a08549cacea5"}, - {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:cb58f6505cc0f34b4e976154302d26563d2e5d16b206758daaa04b65e55d9dd8"}, + {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0c7c24ea0c9d8563dbd5ad49ccb54b7a9a3c7b8c2833d35e5d32a08549cacea5"}, + {file = "duckdb-0.8.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cb58f6505cc0f34b4e976154302d26563d2e5d16b206758daaa04b65e55d9dd8"}, {file = "duckdb-0.8.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ef37ac7880100c4b3f913c8483a29a13f8289313b9a07df019fadfa8e7427544"}, {file = "duckdb-0.8.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:c2a4f5ee913ca8a6a069c78f8944b9934ffdbc71fd935f9576fdcea2a6f476f1"}, {file = "duckdb-0.8.0-cp38-cp38-win32.whl", hash = "sha256:73831c6d7aefcb5f4072cd677b9efebecbf6c578946d21710791e10a1fc41b9a"}, @@ -920,10 +734,8 @@ files = [ {file = "duckdb-0.8.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:50a31ec237ed619e50f9ab79eb0ec5111eb9697d4475da6e0ab22c08495ce26b"}, {file = "duckdb-0.8.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:351abb4cc2d229d043920c4bc2a4c29ca31a79fef7d7ef8f6011cf4331f297bf"}, {file = "duckdb-0.8.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:568550a163aca6a787bef8313e358590254de3f4019025a8d68c3a61253fedc1"}, - {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:2b82617f0e7f9fc080eda217090d82b42d4fad083bc9f6d58dfda9cecb7e3b29"}, - {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:d01c9be34d272532b75e8faedda0ff77fa76d1034cde60b8f5768ae85680d6d3"}, + {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2b82617f0e7f9fc080eda217090d82b42d4fad083bc9f6d58dfda9cecb7e3b29"}, + {file = "duckdb-0.8.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d01c9be34d272532b75e8faedda0ff77fa76d1034cde60b8f5768ae85680d6d3"}, {file = "duckdb-0.8.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:8549d6a6bf5f00c012b6916f605416226507e733a3ffc57451682afd6e674d1b"}, {file = "duckdb-0.8.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:8d145c6d51e55743c3ed1a74cffa109d9e72f82b07e203b436cfa453c925313a"}, {file = "duckdb-0.8.0-cp39-cp39-win32.whl", hash = "sha256:f8610dfd21e90d7b04e8598b244bf3ad68599fd6ba0daad3428c03cbfd74dced"}, @@ -935,7 +747,6 @@ files = [ name = "exceptiongroup" version = "1.1.1" description = "Backport of PEP 654 (exception groups)" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -950,47 +761,36 @@ test = ["pytest (>=6)"] name = "fastavro" version = "1.7.4" description = "Fast read/write of AVRO files" -category = "dev" optional = false python-versions = ">=3.7" files = [ {file = "fastavro-1.7.4-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:7568e621b94e061974b2a96d70670d09910e0a71482dd8610b153c07bd768497"}, - {file = "fastavro-1.7.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:d4ec994faf64b743647f0027fcc56b01dc15d46c0e48fa15828277cb02dbdcd6"}, - {file = "fastavro-1.7.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:727fdc1ddd12fcc6addab0b6df12ef999a6babe4b753db891f78aa2ee33edc77"}, + {file = "fastavro-1.7.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d4ec994faf64b743647f0027fcc56b01dc15d46c0e48fa15828277cb02dbdcd6"}, + {file = "fastavro-1.7.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:727fdc1ddd12fcc6addab0b6df12ef999a6babe4b753db891f78aa2ee33edc77"}, {file = "fastavro-1.7.4-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:b2f0cb3f7795fcb0042e0bbbe51204c28338a455986d68409b26dcbde64dd69a"}, {file = "fastavro-1.7.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:bb0a8b5016a99be4b8ce3550889a1bd968c0fb3f521bcfbae24210c6342aee0c"}, {file = "fastavro-1.7.4-cp310-cp310-win_amd64.whl", hash = "sha256:1d2040b2bf3dc1a75170ea44d1e7e09f84fb77f40ef2e6c6b9f2eaf710557083"}, {file = "fastavro-1.7.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:5542423f46bb7fc9699c467cbf151c2713aa6976ef14f4f5ec3532d80d0bb616"}, - {file = "fastavro-1.7.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:ec396e6ab6b272708c8b9a0142df01fff4c7a1f168050f292ab92fdaee0b0257"}, - {file = "fastavro-1.7.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:39b10d68c03371b79f461feca1c6c7e9d3f6aea2e9c7472b25cd749c57562aa1"}, + {file = "fastavro-1.7.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ec396e6ab6b272708c8b9a0142df01fff4c7a1f168050f292ab92fdaee0b0257"}, + {file = "fastavro-1.7.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:39b10d68c03371b79f461feca1c6c7e9d3f6aea2e9c7472b25cd749c57562aa1"}, {file = "fastavro-1.7.4-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:f94d5168ec72f3cfcf2181df1c46ad240dc1fcf361717447d2c5237121b9df55"}, {file = "fastavro-1.7.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:bad3dc279ed4ce747989259035cb3607f189ef7aff40339202f9321ca7f83d0b"}, {file = "fastavro-1.7.4-cp311-cp311-win_amd64.whl", hash = "sha256:8480ff444d9c7abd0bf121dd68656bd2115caca8ed28e71936eff348fde706e0"}, {file = "fastavro-1.7.4-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:bd3d669f4ec6915c88bb80b7c14e01d2c3ceb93a61de5dcf33ff13972bba505e"}, - {file = "fastavro-1.7.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:1a312b128536b81bdb79f27076f513b998abe7d13ee6fe52e99bc01f7ad9b06a"}, - {file = "fastavro-1.7.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:487054d1419f1bfa41e7f19c718cbdbbb254319d3fd5b9ac411054d6432b9d40"}, + {file = "fastavro-1.7.4-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1a312b128536b81bdb79f27076f513b998abe7d13ee6fe52e99bc01f7ad9b06a"}, + {file = "fastavro-1.7.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:487054d1419f1bfa41e7f19c718cbdbbb254319d3fd5b9ac411054d6432b9d40"}, {file = "fastavro-1.7.4-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:d2897fe7d1d5b27dcd33c43d68480de36e55a0e651d7731004a36162cd3eed9e"}, {file = "fastavro-1.7.4-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:6d318b49fd648a1fd93394411fe23761b486ac65dadea7c52dbeb0d0bef30221"}, {file = "fastavro-1.7.4-cp37-cp37m-win_amd64.whl", hash = "sha256:a117c3b122a8110c6ab99b3e66736790b4be19ceefb1edf0e732c33b3dc411c8"}, {file = "fastavro-1.7.4-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:0cca15e1a1f829e40524004342e425acfb594cefbd3388b0a5d13542750623ac"}, - {file = "fastavro-1.7.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:f9211ec7a18a46a2aee01a2a979fd79f05f36b11fdb1bc469c9d9fd8cec32579"}, - {file = "fastavro-1.7.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:f16bde6b5fb51e15233bfcee0378f48d4221201ba45e497a8063f6d216b7aad7"}, + {file = "fastavro-1.7.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f9211ec7a18a46a2aee01a2a979fd79f05f36b11fdb1bc469c9d9fd8cec32579"}, + {file = "fastavro-1.7.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f16bde6b5fb51e15233bfcee0378f48d4221201ba45e497a8063f6d216b7aad7"}, {file = "fastavro-1.7.4-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:aeca55c905ff4c667f2158564654a778918988811ae3eb28592767edcf5f5c4a"}, {file = "fastavro-1.7.4-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:b244f3abc024fc043d6637284ba2ffee5a1291c08a0f361ea1af4d829f66f303"}, {file = "fastavro-1.7.4-cp38-cp38-win_amd64.whl", hash = "sha256:b64e394c87cb99d0681727e1ae5d3633906a72abeab5ea0c692394aeb5a56607"}, {file = "fastavro-1.7.4-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:8c8115bdb1c862354d9abd0ea23eab85793bbff139087f2607bd4b83e8ae07ab"}, - {file = "fastavro-1.7.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:b27dd08f2338a478185c6ba23308002f334642ce83a6aeaf8308271efef88062"}, - {file = "fastavro-1.7.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:f087c246afab8bac08d86ef21be87cbf4f3779348fb960c081863fc3d570412c"}, + {file = "fastavro-1.7.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b27dd08f2338a478185c6ba23308002f334642ce83a6aeaf8308271efef88062"}, + {file = "fastavro-1.7.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f087c246afab8bac08d86ef21be87cbf4f3779348fb960c081863fc3d570412c"}, {file = "fastavro-1.7.4-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:b4077e17a2bab37af96e5ca52e61b6f2b85e4577e7a2903f6814642eb6a834f7"}, {file = "fastavro-1.7.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:776511cecf2ea9da4edd0de5015c1562cd9063683cf94f79bc9e20bab8f06923"}, {file = "fastavro-1.7.4-cp39-cp39-win_amd64.whl", hash = "sha256:a7ea5565fe2c145e074ce9ba75fafd5479a86b34a8dbd00dd1835cf192290e14"}, @@ -1007,7 +807,6 @@ zstandard = ["zstandard"] name = "filelock" version = "3.12.0" description = "A platform independent file lock." -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1017,31 +816,23 @@ files = [ [package.extras] docs = ["furo (>=2023.3.27)", "sphinx (>=6.1.3)", "sphinx-autodoc-typehints (>=1.23,!=1.23.4)"] -testing = ["covdefaults (>=2.3)", "coverage (>=7.2.3)", "diff-cover (>=7.5)", "pytest (>=7.3.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)", -"pytest-timeout (>=2.1)"] +testing = ["covdefaults (>=2.3)", "coverage (>=7.2.3)", "diff-cover (>=7.5)", "pytest (>=7.3.1)", "pytest-cov (>=4)", "pytest-mock (>=3.10)", "pytest-timeout (>=2.1)"] [[package]] name = "frozenlist" version = "1.3.3" description = "A list-like structure which implements collections.abc.MutableSequence" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "frozenlist-1.3.3-cp310-cp310-macosx_10_9_universal2.whl", hash = -"sha256:ff8bf625fe85e119553b5383ba0fb6aa3d0ec2ae980295aaefa552374926b3f4"}, + {file = "frozenlist-1.3.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:ff8bf625fe85e119553b5383ba0fb6aa3d0ec2ae980295aaefa552374926b3f4"}, {file = "frozenlist-1.3.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:dfbac4c2dfcc082fcf8d942d1e49b6aa0766c19d3358bd86e2000bf0fa4a9cf0"}, {file = "frozenlist-1.3.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:b1c63e8d377d039ac769cd0926558bb7068a1f7abb0f003e3717ee003ad85530"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:7fdfc24dcfce5b48109867c13b4cb15e4660e7bd7661741a391f821f23dfdca7"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:2c926450857408e42f0bbc295e84395722ce74bae69a3b2aa2a65fe22cb14b99"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:1841e200fdafc3d51f974d9d377c079a0694a8f06de2e67b48150328d66d5483"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:f470c92737afa7d4c3aacc001e335062d582053d4dbe73cda126f2d7031068dd"}, - {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:783263a4eaad7c49983fe4b2e7b53fa9770c136c270d2d4bbb6d2192bf4d9caf"}, + {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7fdfc24dcfce5b48109867c13b4cb15e4660e7bd7661741a391f821f23dfdca7"}, + {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2c926450857408e42f0bbc295e84395722ce74bae69a3b2aa2a65fe22cb14b99"}, + {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1841e200fdafc3d51f974d9d377c079a0694a8f06de2e67b48150328d66d5483"}, + {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f470c92737afa7d4c3aacc001e335062d582053d4dbe73cda126f2d7031068dd"}, + {file = "frozenlist-1.3.3-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:783263a4eaad7c49983fe4b2e7b53fa9770c136c270d2d4bbb6d2192bf4d9caf"}, {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:924620eef691990dfb56dc4709f280f40baee568c794b5c1885800c3ecc69816"}, {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:ae4dc05c465a08a866b7a1baf360747078b362e6a6dbeb0c57f234db0ef88ae0"}, {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:bed331fe18f58d844d39ceb398b77d6ac0b010d571cba8267c2e7165806b00ce"}, @@ -1049,20 +840,14 @@ files = [ {file = "frozenlist-1.3.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:9545a33965d0d377b0bc823dcabf26980e77f1b6a7caa368a365a9497fb09420"}, {file = "frozenlist-1.3.3-cp310-cp310-win32.whl", hash = "sha256:d5cd3ab21acbdb414bb6c31958d7b06b85eeb40f66463c264a9b343a4e238642"}, {file = "frozenlist-1.3.3-cp310-cp310-win_amd64.whl", hash = "sha256:b756072364347cb6aa5b60f9bc18e94b2f79632de3b0190253ad770c5df17db1"}, - {file = "frozenlist-1.3.3-cp311-cp311-macosx_10_9_universal2.whl", hash = -"sha256:b4395e2f8d83fbe0c627b2b696acce67868793d7d9750e90e39592b3626691b7"}, + {file = "frozenlist-1.3.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:b4395e2f8d83fbe0c627b2b696acce67868793d7d9750e90e39592b3626691b7"}, {file = "frozenlist-1.3.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:14143ae966a6229350021384870458e4777d1eae4c28d1a7aa47f24d030e6678"}, {file = "frozenlist-1.3.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5d8860749e813a6f65bad8285a0520607c9500caa23fea6ee407e63debcdbef6"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:23d16d9f477bb55b6154654e0e74557040575d9d19fe78a161bd33d7d76808e8"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:eb82dbba47a8318e75f679690190c10a5e1f447fbf9df41cbc4c3afd726d88cb"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:9309869032abb23d196cb4e4db574232abe8b8be1339026f489eeb34a4acfd91"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:a97b4fe50b5890d36300820abd305694cb865ddb7885049587a5678215782a6b"}, - {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:c188512b43542b1e91cadc3c6c915a82a5eb95929134faf7fd109f14f9892ce4"}, + {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:23d16d9f477bb55b6154654e0e74557040575d9d19fe78a161bd33d7d76808e8"}, + {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:eb82dbba47a8318e75f679690190c10a5e1f447fbf9df41cbc4c3afd726d88cb"}, + {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9309869032abb23d196cb4e4db574232abe8b8be1339026f489eeb34a4acfd91"}, + {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a97b4fe50b5890d36300820abd305694cb865ddb7885049587a5678215782a6b"}, + {file = "frozenlist-1.3.3-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c188512b43542b1e91cadc3c6c915a82a5eb95929134faf7fd109f14f9892ce4"}, {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:303e04d422e9b911a09ad499b0368dc551e8c3cd15293c99160c7f1f07b59a48"}, {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:0771aed7f596c7d73444c847a1c16288937ef988dc04fb9f7be4b2aa91db609d"}, {file = "frozenlist-1.3.3-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:66080ec69883597e4d026f2f71a231a1ee9887835902dbe6b6467d5a89216cf6"}, @@ -1071,16 +856,11 @@ files = [ {file = "frozenlist-1.3.3-cp311-cp311-win32.whl", hash = "sha256:f30f1928162e189091cf4d9da2eac617bfe78ef907a761614ff577ef4edfb3c8"}, {file = "frozenlist-1.3.3-cp311-cp311-win_amd64.whl", hash = "sha256:a6394d7dadd3cfe3f4b3b186e54d5d8504d44f2d58dcc89d693698e8b7132b32"}, {file = "frozenlist-1.3.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:8df3de3a9ab8325f94f646609a66cbeeede263910c5c0de0101079ad541af332"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:0693c609e9742c66ba4870bcee1ad5ff35462d5ffec18710b4ac89337ff16e27"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:cd4210baef299717db0a600d7a3cac81d46ef0e007f88c9335db79f8979c0d3d"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:394c9c242113bfb4b9aa36e2b80a05ffa163a30691c7b5a29eba82e937895d5e"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:6327eb8e419f7d9c38f333cde41b9ae348bec26d840927332f17e887a8dcb70d"}, - {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:2e24900aa13212e75e5b366cb9065e78bbf3893d4baab6052d1aca10d46d944c"}, + {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0693c609e9742c66ba4870bcee1ad5ff35462d5ffec18710b4ac89337ff16e27"}, + {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:cd4210baef299717db0a600d7a3cac81d46ef0e007f88c9335db79f8979c0d3d"}, + {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:394c9c242113bfb4b9aa36e2b80a05ffa163a30691c7b5a29eba82e937895d5e"}, + {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6327eb8e419f7d9c38f333cde41b9ae348bec26d840927332f17e887a8dcb70d"}, + {file = "frozenlist-1.3.3-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e24900aa13212e75e5b366cb9065e78bbf3893d4baab6052d1aca10d46d944c"}, {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:3843f84a6c465a36559161e6c59dce2f2ac10943040c2fd021cfb70d58c4ad56"}, {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:84610c1502b2461255b4c9b7d5e9c48052601a8957cd0aea6ec7a7a1e1fb9420"}, {file = "frozenlist-1.3.3-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:c21b9aa40e08e4f63a2f92ff3748e6b6c84d717d033c7b3438dd3123ee18f70e"}, @@ -1091,16 +871,11 @@ files = [ {file = "frozenlist-1.3.3-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:034a5c08d36649591be1cbb10e09da9f531034acfe29275fc5454a3b101ce41a"}, {file = "frozenlist-1.3.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:ba64dc2b3b7b158c6660d49cdb1d872d1d0bf4e42043ad8d5006099479a194e5"}, {file = "frozenlist-1.3.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:47df36a9fe24054b950bbc2db630d508cca3aa27ed0566c0baf661225e52c18e"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:008a054b75d77c995ea26629ab3a0c0d7281341f2fa7e1e85fa6153ae29ae99c"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:841ea19b43d438a80b4de62ac6ab21cfe6827bb8a9dc62b896acc88eaf9cecba"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:e235688f42b36be2b6b06fc37ac2126a73b75fb8d6bc66dd632aa35286238703"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:ca713d4af15bae6e5d79b15c10c8522859a9a89d3b361a50b817c98c2fb402a2"}, - {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:9ac5995f2b408017b0be26d4a1d7c61bce106ff3d9e3324374d66b5964325448"}, + {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:008a054b75d77c995ea26629ab3a0c0d7281341f2fa7e1e85fa6153ae29ae99c"}, + {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:841ea19b43d438a80b4de62ac6ab21cfe6827bb8a9dc62b896acc88eaf9cecba"}, + {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e235688f42b36be2b6b06fc37ac2126a73b75fb8d6bc66dd632aa35286238703"}, + {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca713d4af15bae6e5d79b15c10c8522859a9a89d3b361a50b817c98c2fb402a2"}, + {file = "frozenlist-1.3.3-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9ac5995f2b408017b0be26d4a1d7c61bce106ff3d9e3324374d66b5964325448"}, {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:a4ae8135b11652b08a8baf07631d3ebfe65a4c87909dbef5fa0cdde440444ee4"}, {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:4ea42116ceb6bb16dbb7d526e242cb6747b08b7710d9782aa3d6732bd8d27649"}, {file = "frozenlist-1.3.3-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:810860bb4bdce7557bc0febb84bbd88198b9dbc2022d8eebe5b3590b2ad6c842"}, @@ -1111,16 +886,11 @@ files = [ {file = "frozenlist-1.3.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:2b07ae0c1edaa0a36339ec6cce700f51b14a3fc6545fdd32930d2c83917332cf"}, {file = "frozenlist-1.3.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:ebb86518203e12e96af765ee89034a1dbb0c3c65052d1b0c19bbbd6af8a145e1"}, {file = "frozenlist-1.3.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:5cf820485f1b4c91e0417ea0afd41ce5cf5965011b3c22c400f6d144296ccbc0"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:5c11e43016b9024240212d2a65043b70ed8dfd3b52678a1271972702d990ac6d"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:8fa3c6e3305aa1146b59a09b32b2e04074945ffcfb2f0931836d103a2c38f936"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:352bd4c8c72d508778cf05ab491f6ef36149f4d0cb3c56b1b4302852255d05d5"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:65a5e4d3aa679610ac6e3569e865425b23b372277f89b5ef06cf2cdaf1ebf22b"}, - {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:b1e2c1185858d7e10ff045c496bbf90ae752c28b365fef2c09cf0fa309291669"}, + {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c11e43016b9024240212d2a65043b70ed8dfd3b52678a1271972702d990ac6d"}, + {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8fa3c6e3305aa1146b59a09b32b2e04074945ffcfb2f0931836d103a2c38f936"}, + {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:352bd4c8c72d508778cf05ab491f6ef36149f4d0cb3c56b1b4302852255d05d5"}, + {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:65a5e4d3aa679610ac6e3569e865425b23b372277f89b5ef06cf2cdaf1ebf22b"}, + {file = "frozenlist-1.3.3-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b1e2c1185858d7e10ff045c496bbf90ae752c28b365fef2c09cf0fa309291669"}, {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:f163d2fd041c630fed01bc48d28c3ed4a3b003c00acd396900e11ee5316b56bb"}, {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:05cdb16d09a0832eedf770cb7bd1fe57d8cf4eaf5aced29c4e41e3f20b30a784"}, {file = "frozenlist-1.3.3-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:8bae29d60768bfa8fb92244b74502b18fae55a80eac13c88eb0b496d4268fd2d"}, @@ -1135,7 +905,6 @@ files = [ name = "fsspec" version = "2023.5.0" description = "File-system specification" -category = "main" optional = false python-versions = ">=3.8" files = [ @@ -1150,8 +919,7 @@ arrow = ["pyarrow (>=1)"] dask = ["dask", "distributed"] devel = ["pytest", "pytest-cov"] dropbox = ["dropbox", "dropboxdrivefs", "requests"] -full = ["adlfs", "aiohttp (!=4.0.0a0,!=4.0.0a1)", "dask", "distributed", "dropbox", "dropboxdrivefs", "fusepy", "gcsfs", "libarchive-c", "ocifs", -"panel", "paramiko", "pyarrow (>=1)", "pygit2", "requests", "s3fs", "smbprotocol", "tqdm"] +full = ["adlfs", "aiohttp (!=4.0.0a0,!=4.0.0a1)", "dask", "distributed", "dropbox", "dropboxdrivefs", "fusepy", "gcsfs", "libarchive-c", "ocifs", "panel", "paramiko", "pyarrow (>=1)", "pygit2", "requests", "s3fs", "smbprotocol", "tqdm"] fuse = ["fusepy"] gcs = ["gcsfs"] git = ["pygit2"] @@ -1172,27 +940,22 @@ tqdm = ["tqdm"] name = "grpcio" version = "1.49.1" description = "HTTP/2-based RPC framework" -category = "main" optional = true python-versions = ">=3.7" files = [ {file = "grpcio-1.49.1-cp310-cp310-linux_armv7l.whl", hash = "sha256:fd86040232e805b8e6378b2348c928490ee595b058ce9aaa27ed8e4b0f172b20"}, {file = "grpcio-1.49.1-cp310-cp310-macosx_10_10_x86_64.whl", hash = "sha256:6fd0c9cede9552bf00f8c5791d257d5bf3790d7057b26c59df08be5e7a1e021d"}, {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:d0d402e158d4e84e49c158cb5204119d55e1baf363ee98d6cb5dce321c3a065d"}, - {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:822ceec743d42a627e64ea266059a62d214c5a3cdfcd0d7fe2b7a8e4e82527c7"}, - {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:2106d9c16527f0a85e2eea6e6b91a74fc99579c60dd810d8690843ea02bc0f5f"}, + {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:822ceec743d42a627e64ea266059a62d214c5a3cdfcd0d7fe2b7a8e4e82527c7"}, + {file = "grpcio-1.49.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2106d9c16527f0a85e2eea6e6b91a74fc99579c60dd810d8690843ea02bc0f5f"}, {file = "grpcio-1.49.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:52dd02b7e7868233c571b49bc38ebd347c3bb1ff8907bb0cb74cb5f00c790afc"}, {file = "grpcio-1.49.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:120fecba2ec5d14b5a15d11063b39783fda8dc8d24addd83196acb6582cabd9b"}, {file = "grpcio-1.49.1-cp310-cp310-win32.whl", hash = "sha256:f1a3b88e3c53c1a6e6bed635ec1bbb92201bb6a1f2db186179f7f3f244829788"}, {file = "grpcio-1.49.1-cp310-cp310-win_amd64.whl", hash = "sha256:a7d0017b92d3850abea87c1bdec6ea41104e71c77bca44c3e17f175c6700af62"}, {file = "grpcio-1.49.1-cp311-cp311-linux_armv7l.whl", hash = "sha256:9fb17ff8c0d56099ac6ebfa84f670c5a62228d6b5c695cf21c02160c2ac1446b"}, {file = "grpcio-1.49.1-cp311-cp311-macosx_10_10_x86_64.whl", hash = "sha256:075f2d06e3db6b48a2157a1bcd52d6cbdca980dd18988fe6afdb41795d51625f"}, - {file = "grpcio-1.49.1-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:46d93a1b4572b461a227f1db6b8d35a88952db1c47e5fadcf8b8a2f0e1dd9201"}, - {file = "grpcio-1.49.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:dc79b2b37d779ac42341ddef40ad5bf0966a64af412c89fc2b062e3ddabb093f"}, + {file = "grpcio-1.49.1-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:46d93a1b4572b461a227f1db6b8d35a88952db1c47e5fadcf8b8a2f0e1dd9201"}, + {file = "grpcio-1.49.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dc79b2b37d779ac42341ddef40ad5bf0966a64af412c89fc2b062e3ddabb093f"}, {file = "grpcio-1.49.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:5f8b3a971c7820ea9878f3fd70086240a36aeee15d1b7e9ecbc2743b0e785568"}, {file = "grpcio-1.49.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:49b301740cf5bc8fed4fee4c877570189ae3951432d79fa8e524b09353659811"}, {file = "grpcio-1.49.1-cp311-cp311-win32.whl", hash = "sha256:1c66a25afc6c71d357867b341da594a5587db5849b48f4b7d5908d236bb62ede"}, @@ -1200,10 +963,8 @@ files = [ {file = "grpcio-1.49.1-cp37-cp37m-linux_armv7l.whl", hash = "sha256:1cc400c8a2173d1c042997d98a9563e12d9bb3fb6ad36b7f355bc77c7663b8af"}, {file = "grpcio-1.49.1-cp37-cp37m-macosx_10_10_x86_64.whl", hash = "sha256:34f736bd4d0deae90015c0e383885b431444fe6b6c591dea288173df20603146"}, {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:196082b9c89ebf0961dcd77cb114bed8171964c8e3063b9da2fb33536a6938ed"}, - {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:8c9f89c42749890618cd3c2464e1fbf88446e3d2f67f1e334c8e5db2f3272bbd"}, - {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:64419cb8a5b612cdb1550c2fd4acbb7d4fb263556cf4625f25522337e461509e"}, + {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8c9f89c42749890618cd3c2464e1fbf88446e3d2f67f1e334c8e5db2f3272bbd"}, + {file = "grpcio-1.49.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:64419cb8a5b612cdb1550c2fd4acbb7d4fb263556cf4625f25522337e461509e"}, {file = "grpcio-1.49.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:8a5272061826e6164f96e3255405ef6f73b88fd3e8bef464c7d061af8585ac62"}, {file = "grpcio-1.49.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:ea9d0172445241ad7cb49577314e39d0af2c5267395b3561d7ced5d70458a9f3"}, {file = "grpcio-1.49.1-cp37-cp37m-win32.whl", hash = "sha256:2070e87d95991473244c72d96d13596c751cb35558e11f5df5414981e7ed2492"}, @@ -1211,10 +972,8 @@ files = [ {file = "grpcio-1.49.1-cp38-cp38-linux_armv7l.whl", hash = "sha256:afbb3475cf7f4f7d380c2ca37ee826e51974f3e2665613996a91d6a58583a534"}, {file = "grpcio-1.49.1-cp38-cp38-macosx_10_10_x86_64.whl", hash = "sha256:a4f9ba141380abde6c3adc1727f21529137a2552002243fa87c41a07e528245c"}, {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:cf0a1fb18a7204b9c44623dfbd1465b363236ce70c7a4ed30402f9f60d8b743b"}, - {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:17bb6fe72784b630728c6cff9c9d10ccc3b6d04e85da6e0a7b27fb1d135fac62"}, - {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:18305d5a082d1593b005a895c10041f833b16788e88b02bb81061f5ebcc465df"}, + {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:17bb6fe72784b630728c6cff9c9d10ccc3b6d04e85da6e0a7b27fb1d135fac62"}, + {file = "grpcio-1.49.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:18305d5a082d1593b005a895c10041f833b16788e88b02bb81061f5ebcc465df"}, {file = "grpcio-1.49.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:b6a1b39e59ac5a3067794a0e498911cf2e37e4b19ee9e9977dc5e7051714f13f"}, {file = "grpcio-1.49.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:0e20d59aafc086b1cc68400463bddda6e41d3e5ed30851d1e2e0f6a2e7e342d3"}, {file = "grpcio-1.49.1-cp38-cp38-win32.whl", hash = "sha256:e1e83233d4680863a421f3ee4a7a9b80d33cd27ee9ed7593bc93f6128302d3f2"}, @@ -1222,10 +981,8 @@ files = [ {file = "grpcio-1.49.1-cp39-cp39-linux_armv7l.whl", hash = "sha256:fa9e6e61391e99708ac87fc3436f6b7b9c6b845dc4639b406e5e61901e1aacde"}, {file = "grpcio-1.49.1-cp39-cp39-macosx_10_10_x86_64.whl", hash = "sha256:9b449e966ef518ce9c860d21f8afe0b0f055220d95bc710301752ac1db96dd6a"}, {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:aa34d2ad9f24e47fa9a3172801c676e4037d862247e39030165fe83821a7aafd"}, - {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:5207f4eed1b775d264fcfe379d8541e1c43b878f2b63c0698f8f5c56c40f3d68"}, - {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:0b24a74651438d45619ac67004638856f76cc13d78b7478f2457754cbcb1c8ad"}, + {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5207f4eed1b775d264fcfe379d8541e1c43b878f2b63c0698f8f5c56c40f3d68"}, + {file = "grpcio-1.49.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0b24a74651438d45619ac67004638856f76cc13d78b7478f2457754cbcb1c8ad"}, {file = "grpcio-1.49.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:fe763781669790dc8b9618e7e677c839c87eae6cf28b655ee1fa69ae04eea03f"}, {file = "grpcio-1.49.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:2f2ff7ba0f8f431f32d4b4bc3a3713426949d3533b08466c4ff1b2b475932ca8"}, {file = "grpcio-1.49.1-cp39-cp39-win32.whl", hash = "sha256:08ff74aec8ff457a89b97152d36cb811dcc1d17cd5a92a65933524e363327394"}, @@ -1243,27 +1000,22 @@ protobuf = ["grpcio-tools (>=1.49.1)"] name = "grpcio" version = "1.51.3" description = "HTTP/2-based RPC framework" -category = "main" optional = true python-versions = ">=3.7" files = [ {file = "grpcio-1.51.3-cp310-cp310-linux_armv7l.whl", hash = "sha256:f601aaeae18dab81930fb8d4f916b0da21e89bb4b5f7367ef793f46b4a76b7b0"}, {file = "grpcio-1.51.3-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:eef0450a4b5ed11feab639bf3eb1b6e23d0efa9b911bf7b06fb60e14f5f8a585"}, {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_aarch64.whl", hash = "sha256:82b0ad8ac825d4bb31bff9f638557c045f4a6d824d84b21e893968286f88246b"}, - {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:3667c06e37d6cd461afdd51cefe6537702f3d1dc5ff4cac07e88d8b4795dc16f"}, - {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:3709048fe0aa23dda09b3e69849a12055790171dab9e399a72ea8f9dfbf9ac80"}, + {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3667c06e37d6cd461afdd51cefe6537702f3d1dc5ff4cac07e88d8b4795dc16f"}, + {file = "grpcio-1.51.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3709048fe0aa23dda09b3e69849a12055790171dab9e399a72ea8f9dfbf9ac80"}, {file = "grpcio-1.51.3-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:200d69857f9910f7458b39b9bcf83ee4a180591b40146ba9e49314e3a7419313"}, {file = "grpcio-1.51.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:cd9a5e68e79c5f031500e67793048a90209711e0854a9ddee8a3ce51728de4e5"}, {file = "grpcio-1.51.3-cp310-cp310-win32.whl", hash = "sha256:6604f614016127ae10969176bbf12eb0e03d2fb3d643f050b3b69e160d144fb4"}, {file = "grpcio-1.51.3-cp310-cp310-win_amd64.whl", hash = "sha256:e95c7ccd4c5807adef1602005513bf7c7d14e5a41daebcf9d8d30d8bf51b8f81"}, {file = "grpcio-1.51.3-cp311-cp311-linux_armv7l.whl", hash = "sha256:5e77ee138100f0bb55cbd147840f87ee6241dbd25f09ea7cd8afe7efff323449"}, {file = "grpcio-1.51.3-cp311-cp311-macosx_10_10_universal2.whl", hash = "sha256:68a7514b754e38e8de9075f7bb4dee919919515ec68628c43a894027e40ddec4"}, - {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:3c1b9f8afa62ff265d86a4747a2990ec5a96e4efce5d5888f245a682d66eca47"}, - {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:8de30f0b417744288cec65ec8cf84b8a57995cf7f1e84ccad2704d93f05d0aae"}, + {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3c1b9f8afa62ff265d86a4747a2990ec5a96e4efce5d5888f245a682d66eca47"}, + {file = "grpcio-1.51.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8de30f0b417744288cec65ec8cf84b8a57995cf7f1e84ccad2704d93f05d0aae"}, {file = "grpcio-1.51.3-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:b69c7adc7ed60da1cb1b502853db61f453fc745f940cbcc25eb97c99965d8f41"}, {file = "grpcio-1.51.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d81528ffe0e973dc840ec73a4132fd18b8203ad129d7410155d951a0a7e4f5d0"}, {file = "grpcio-1.51.3-cp311-cp311-win32.whl", hash = "sha256:040eb421613b57c696063abde405916dd830203c184c9000fc8c3b3b3c950325"}, @@ -1271,10 +1023,8 @@ files = [ {file = "grpcio-1.51.3-cp37-cp37m-linux_armv7l.whl", hash = "sha256:d5cd1389669a847555df54177b911d9ff6f17345b2a6f19388707b7a9f724c88"}, {file = "grpcio-1.51.3-cp37-cp37m-macosx_10_10_universal2.whl", hash = "sha256:be1bf35ce82cdbcac14e39d5102d8de4079a1c1a6a06b68e41fcd9ef64f9dd28"}, {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_aarch64.whl", hash = "sha256:5eed34994c095e2bf7194ffac7381c6068b057ef1e69f8f08db77771350a7566"}, - {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:3f9a7d88082b2a17ae7bd3c2354d13bab0453899e0851733f6afa6918373f476"}, - {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:36c8abbc5f837111e7bd619612eedc223c290b0903b952ce0c7b00840ea70f14"}, + {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3f9a7d88082b2a17ae7bd3c2354d13bab0453899e0851733f6afa6918373f476"}, + {file = "grpcio-1.51.3-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:36c8abbc5f837111e7bd619612eedc223c290b0903b952ce0c7b00840ea70f14"}, {file = "grpcio-1.51.3-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:165b05af77e6aecb4210ae7663e25acf234ba78a7c1c157fa5f2efeb0d6ec53c"}, {file = "grpcio-1.51.3-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:54e36c2ee304ff15f2bfbdc43d2b56c63331c52d818c364e5b5214e5bc2ad9f6"}, {file = "grpcio-1.51.3-cp37-cp37m-win32.whl", hash = "sha256:cd0daac21d9ef5e033a5100c1d3aa055bbed28bfcf070b12d8058045c4e821b1"}, @@ -1282,10 +1032,8 @@ files = [ {file = "grpcio-1.51.3-cp38-cp38-linux_armv7l.whl", hash = "sha256:54b0c29bdd9a3b1e1b61443ab152f060fc719f1c083127ab08d03fac5efd51be"}, {file = "grpcio-1.51.3-cp38-cp38-macosx_10_10_universal2.whl", hash = "sha256:ffaaf7e93fcb437356b5a4b23bf36e8a3d0221399ff77fd057e4bc77776a24be"}, {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_aarch64.whl", hash = "sha256:eafbe7501a3268d05f2e450e1ddaffb950d842a8620c13ec328b501d25d2e2c3"}, - {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:881ecb34feabf31c6b3b9bbbddd1a5b57e69f805041e5a2c6c562a28574f71c4"}, - {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:e860a3222139b41d430939bbec2ec9c3f6c740938bf7a04471a9a8caaa965a2e"}, + {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:881ecb34feabf31c6b3b9bbbddd1a5b57e69f805041e5a2c6c562a28574f71c4"}, + {file = "grpcio-1.51.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e860a3222139b41d430939bbec2ec9c3f6c740938bf7a04471a9a8caaa965a2e"}, {file = "grpcio-1.51.3-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:49ede0528e9dac7e8a9fe30b16c73b630ddd9a576bf4b675eb6b0c53ee5ca00f"}, {file = "grpcio-1.51.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6972b009638b40a448d10e1bc18e2223143b8a7aa20d7def0d78dd4af4126d12"}, {file = "grpcio-1.51.3-cp38-cp38-win32.whl", hash = "sha256:5694448256e3cdfe5bd358f1574a3f2f51afa20cc834713c4b9788d60b7cc646"}, @@ -1293,10 +1041,8 @@ files = [ {file = "grpcio-1.51.3-cp39-cp39-linux_armv7l.whl", hash = "sha256:6c677581ce129f5fa228b8f418cee10bd28dd449f3a544ea73c8ba590ee49d0b"}, {file = "grpcio-1.51.3-cp39-cp39-macosx_10_10_universal2.whl", hash = "sha256:30e09b5e0531685e176f49679b6a3b190762cc225f4565e55a899f5e14b3aa62"}, {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_aarch64.whl", hash = "sha256:c831f31336e81243f85b6daff3e5e8a123302ce0ea1f2726ad752fd7a59f3aee"}, - {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:2cd2e4cefb724cab1ba2df4b7535a9980531b9ec51b4dbb5f137a1f3a3754ef0"}, - {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:f7a0d0bf44438869d307f85a54f25a896ad6b4b0ca12370f76892ad732928d87"}, + {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2cd2e4cefb724cab1ba2df4b7535a9980531b9ec51b4dbb5f137a1f3a3754ef0"}, + {file = "grpcio-1.51.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f7a0d0bf44438869d307f85a54f25a896ad6b4b0ca12370f76892ad732928d87"}, {file = "grpcio-1.51.3-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:c02abd55409bfb293371554adf6a4401197ec2133dd97727c01180889014ba4d"}, {file = "grpcio-1.51.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:2f8ff75e61e1227ba7a3f16b2eadbcc11d0a54096d52ab75a6b88cfbe56f55d1"}, {file = "grpcio-1.51.3-cp39-cp39-win32.whl", hash = "sha256:6c99a73a6260bdf844b2e5ddad02dcd530310f80e1fa72c300fa19c1c7496962"}, @@ -1311,7 +1057,6 @@ protobuf = ["grpcio-tools (>=1.51.3)"] name = "identify" version = "2.5.24" description = "File identification library for Python" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -1326,7 +1071,6 @@ license = ["ukkonen"] name = "idna" version = "3.4" description = "Internationalized Domain Names in Applications (IDNA)" -category = "main" optional = false python-versions = ">=3.5" files = [ @@ -1338,7 +1082,6 @@ files = [ name = "importlib-metadata" version = "6.6.0" description = "Read metadata from Python packages" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -1352,14 +1095,12 @@ zipp = ">=0.5" [package.extras] docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] perf = ["ipython"] -testing = ["flake8 (<5)", "flufl.flake8", "importlib-resources (>=1.3)", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7)", -"pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "pytest-perf (>=0.9.2)"] +testing = ["flake8 (<5)", "flufl.flake8", "importlib-resources (>=1.3)", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "pytest-perf (>=0.9.2)"] [[package]] name = "importlib-resources" version = "5.12.0" description = "Read resources from Python packages" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -1372,14 +1113,12 @@ zipp = {version = ">=3.1.0", markers = "python_version < \"3.10\""} [package.extras] docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] -testing = ["flake8 (<5)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", -"pytest-mypy (>=0.9.1)"] +testing = ["flake8 (<5)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"] [[package]] name = "iniconfig" version = "2.0.0" description = "brain-dead simple config-ini parsing" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -1391,7 +1130,6 @@ files = [ name = "isodate" version = "0.6.1" description = "An ISO 8601 date/time/duration parser and formatter" -category = "main" optional = true python-versions = "*" files = [ @@ -1406,7 +1144,6 @@ six = "*" name = "jinja2" version = "3.1.2" description = "A very fast and expressive template engine." -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -1424,7 +1161,6 @@ i18n = ["Babel (>=2.7)"] name = "jmespath" version = "1.0.1" description = "JSON Matching Expressions" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1436,7 +1172,6 @@ files = [ name = "jsonschema" version = "4.17.3" description = "An implementation of JSON Schema validation for Python" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -1452,14 +1187,12 @@ pyrsistent = ">=0.14.0,<0.17.0 || >0.17.0,<0.17.1 || >0.17.1,<0.17.2 || >0.17.2" [package.extras] format = ["fqdn", "idna", "isoduration", "jsonpointer (>1.13)", "rfc3339-validator", "rfc3987", "uri-template", "webcolors (>=1.11)"] -format-nongpl = ["fqdn", "idna", "isoduration", "jsonpointer (>1.13)", "rfc3339-validator", "rfc3986-validator (>0.1.0)", "uri-template", "webcolors -(>=1.11)"] +format-nongpl = ["fqdn", "idna", "isoduration", "jsonpointer (>1.13)", "rfc3339-validator", "rfc3986-validator (>0.1.0)", "uri-template", "webcolors (>=1.11)"] [[package]] name = "markdown-it-py" version = "2.2.0" description = "Python port of markdown-it. Markdown parsing, done right!" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1482,86 +1215,67 @@ testing = ["coverage", "pytest", "pytest-cov", "pytest-regressions"] [[package]] name = "markupsafe" -version = "2.1.2" +version = "2.1.3" description = "Safely add untrusted strings to HTML/XML markup." -category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "MarkupSafe-2.1.2-cp310-cp310-macosx_10_9_universal2.whl", hash = -"sha256:665a36ae6f8f20a4676b53224e33d456a6f5a72657d9c83c2aa00765072f31f7"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:340bea174e9761308703ae988e982005aedf427de816d1afe98147668cc03036"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:22152d00bf4a9c7c83960521fc558f55a1adbc0631fbb00a9471e097b19d72e1"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:28057e985dace2f478e042eaa15606c7efccb700797660629da387eb289b9323"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:ca244fa73f50a800cf8c3ebf7fd93149ec37f5cb9596aa8873ae2c1d23498601"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:d9d971ec1e79906046aa3ca266de79eac42f1dbf3612a05dc9368125952bd1a1"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:7e007132af78ea9df29495dbf7b5824cb71648d7133cf7848a2a5dd00d36f9ff"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:7313ce6a199651c4ed9d7e4cfb4aa56fe923b1adf9af3b420ee14e6d9a73df65"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-win32.whl", hash = "sha256:c4a549890a45f57f1ebf99c067a4ad0cb423a05544accaf2b065246827ed9603"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-win_amd64.whl", hash = "sha256:835fb5e38fd89328e9c81067fd642b3593c33e1e17e2fdbf77f5676abb14a156"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-macosx_10_9_universal2.whl", hash = -"sha256:2ec4f2d48ae59bbb9d1f9d7efb9236ab81429a764dedca114f5fdabbc3788013"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:608e7073dfa9e38a85d38474c082d4281f4ce276ac0010224eaba11e929dd53a"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:65608c35bfb8a76763f37036547f7adfd09270fbdbf96608be2bead319728fcd"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:f2bfb563d0211ce16b63c7cb9395d2c682a23187f54c3d79bfec33e6705473c6"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:da25303d91526aac3672ee6d49a2f3db2d9502a4a60b55519feb1a4c7714e07d"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:9cad97ab29dfc3f0249b483412c85c8ef4766d96cdf9dcf5a1e3caa3f3661cf1"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:085fd3201e7b12809f9e6e9bc1e5c96a368c8523fad5afb02afe3c051ae4afcc"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1bea30e9bf331f3fef67e0a3877b2288593c98a21ccb2cf29b74c581a4eb3af0"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-win32.whl", hash = "sha256:7df70907e00c970c60b9ef2938d894a9381f38e6b9db73c5be35e59d92e06625"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-win_amd64.whl", hash = "sha256:e55e40ff0cc8cc5c07996915ad367fa47da6b3fc091fdadca7f5403239c5fec3"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:a6e40afa7f45939ca356f348c8e23048e02cb109ced1eb8420961b2f40fb373a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:cf877ab4ed6e302ec1d04952ca358b381a882fbd9d1b07cccbfd61783561f98a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:63ba06c9941e46fa389d389644e2d8225e0e3e5ebcc4ff1ea8506dce646f8c8a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:f1cd098434e83e656abf198f103a8207a8187c0fc110306691a2e94a78d0abb2"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:55f44b440d491028addb3b88f72207d71eeebfb7b5dbf0643f7c023ae1fba619"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:a6f2fcca746e8d5910e18782f976489939d54a91f9411c32051b4aab2bd7c513"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:0b462104ba25f1ac006fdab8b6a01ebbfbce9ed37fd37fd4acd70c67c973e460"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-win32.whl", hash = "sha256:7668b52e102d0ed87cb082380a7e2e1e78737ddecdde129acadb0eccc5423859"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-win_amd64.whl", hash = "sha256:6d6607f98fcf17e534162f0709aaad3ab7a96032723d8ac8750ffe17ae5a0666"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:a806db027852538d2ad7555b203300173dd1b77ba116de92da9afbc3a3be3eed"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:a4abaec6ca3ad8660690236d11bfe28dfd707778e2442b45addd2f086d6ef094"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:f03a532d7dee1bed20bc4884194a16160a2de9ffc6354b3878ec9682bb623c54"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:4cf06cdc1dda95223e9d2d3c58d3b178aa5dacb35ee7e3bbac10e4e1faacb419"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:22731d79ed2eb25059ae3df1dfc9cb1546691cc41f4e3130fe6bfbc3ecbbecfa"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:f8ffb705ffcf5ddd0e80b65ddf7bed7ee4f5a441ea7d3419e861a12eaf41af58"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:8db032bf0ce9022a8e41a22598eefc802314e81b879ae093f36ce9ddf39ab1ba"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:2298c859cfc5463f1b64bd55cb3e602528db6fa0f3cfd568d3605c50678f8f03"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-win32.whl", hash = "sha256:50c42830a633fa0cf9e7d27664637532791bfc31c731a87b202d2d8ac40c3ea2"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-win_amd64.whl", hash = "sha256:bb06feb762bade6bf3c8b844462274db0c76acc95c52abe8dbed28ae3d44a147"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:99625a92da8229df6d44335e6fcc558a5037dd0a760e11d84be2260e6f37002f"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8bca7e26c1dd751236cfb0c6c72d4ad61d986e9a41bbf76cb445f69488b2a2bd"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:40627dcf047dadb22cd25ea7ecfe9cbf3bbbad0482ee5920b582f3809c97654f"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:40dfd3fefbef579ee058f139733ac336312663c6706d1163b82b3003fb1925c4"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:090376d812fb6ac5f171e5938e82e7f2d7adc2b629101cec0db8b267815c85e2"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:2e7821bffe00aa6bd07a23913b7f4e01328c3d5cc0b40b36c0bd81d362faeb65"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:c0a33bc9f02c2b17c3ea382f91b4db0e6cde90b63b296422a939886a7a80de1c"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:b8526c6d437855442cdd3d87eede9c425c4445ea011ca38d937db299382e6fa3"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-win32.whl", hash = "sha256:137678c63c977754abe9086a3ec011e8fd985ab90631145dfb9294ad09c102a7"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-win_amd64.whl", hash = "sha256:0576fe974b40a400449768941d5d0858cc624e3249dfd1e0c33674e5c7ca7aed"}, - {file = "MarkupSafe-2.1.2.tar.gz", hash = "sha256:abcabc8c2b26036d62d4c746381a6f7cf60aafcc653198ad678306986b09450d"}, + {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-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"}, ] [[package]] name = "mdurl" version = "0.1.2" description = "Markdown URL utilities" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1573,53 +1287,40 @@ files = [ name = "mmhash3" version = "3.0.1" description = "Python wrapper for MurmurHash (MurmurHash3), a set of fast and robust hash functions." -category = "main" optional = false python-versions = "*" files = [ {file = "mmhash3-3.0.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:47deea30cd8d3d5cd52dc740902a4c70383bfe8248eac29d0877fe63e03c2713"}, {file = "mmhash3-3.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:ecdaf4d1de617818bf05cd526ca558db6010beeba7ea9e19f695f2bdcac0e0a4"}, - {file = "mmhash3-3.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:4675585617584e9e1aafa3a90ac0ac9a437257c507748d97de8b21977e9d6745"}, - {file = "mmhash3-3.0.1-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:ebfd0c2af09b41f0fe1dd260799bda90a0fc7eba4477ccaeb3951527700cd58f"}, + {file = "mmhash3-3.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4675585617584e9e1aafa3a90ac0ac9a437257c507748d97de8b21977e9d6745"}, + {file = "mmhash3-3.0.1-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ebfd0c2af09b41f0fe1dd260799bda90a0fc7eba4477ccaeb3951527700cd58f"}, {file = "mmhash3-3.0.1-cp310-cp310-win32.whl", hash = "sha256:68587dec7b8acdb7528fd511e295d8b5ccfe26022923a69867e1822f0fdb4c44"}, {file = "mmhash3-3.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:54954ebe3a614f19ab4cfe24fa66ea093c493d9fac0533d002dd64c2540a0c99"}, {file = "mmhash3-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b172f3bd3220b0bf56fd7cc760fd8a9033def47103611d63fe867003079a1256"}, {file = "mmhash3-3.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:de7895eafabc32f7c0c09a81a624921f536768de6e438e3de69e3e954a4d7072"}, - {file = "mmhash3-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:b4b0914effe4ddd8d33149e3508564c17719465b0cc81691c4fa50d5e0e14f80"}, - {file = "mmhash3-3.0.1-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:c0575050ac691475938df1ad03d8738c5bd1eadef62093e76157ebb7f2df0946"}, + {file = "mmhash3-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b4b0914effe4ddd8d33149e3508564c17719465b0cc81691c4fa50d5e0e14f80"}, + {file = "mmhash3-3.0.1-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0575050ac691475938df1ad03d8738c5bd1eadef62093e76157ebb7f2df0946"}, {file = "mmhash3-3.0.1-cp311-cp311-win32.whl", hash = "sha256:22f92f0f88f28b215357acd346362fa9f7c9fffb436beb42cc4b442b676dbaa3"}, {file = "mmhash3-3.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:538240ab7936bf71b18304e5a7e7fd3c4c2fab103330ea99584bb4f777299a2b"}, {file = "mmhash3-3.0.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:ca791bfb311e36ce13998e4632262ed4b95da9d3461941e18b6690760171a045"}, - {file = "mmhash3-3.0.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:b41708f72c6aa2a49ada1f0b61e85c05cd8389ad31d463fd5bca29999a4d5f9c"}, - {file = "mmhash3-3.0.1-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:a3ce9b4533ddc0a88ba045a27309714c3b127bd05e57fd252d1d5a71d4247ea7"}, + {file = "mmhash3-3.0.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b41708f72c6aa2a49ada1f0b61e85c05cd8389ad31d463fd5bca29999a4d5f9c"}, + {file = "mmhash3-3.0.1-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a3ce9b4533ddc0a88ba045a27309714c3b127bd05e57fd252d1d5a71d4247ea7"}, {file = "mmhash3-3.0.1-cp36-cp36m-win32.whl", hash = "sha256:bfafeb96fdeb10db8767d06e1f07b6fdcddba4aaa0dd15058561a49f7ae45345"}, {file = "mmhash3-3.0.1-cp36-cp36m-win_amd64.whl", hash = "sha256:97fe077b24c948709ed2afc749bf6285d407bc54ff12c63d2dc86678c38a0b8e"}, {file = "mmhash3-3.0.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:0cfd91ccd5fca1ba7ee925297131a15dfb94c714bfe6ba0fb3b1ca78b12bbfec"}, - {file = "mmhash3-3.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:6d51b1005233141ce7394531af40a3f0fc1f274467bf8dff44dcf7987924fe58"}, - {file = "mmhash3-3.0.1-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:855c67b100e37df166acf79cdff58fa8f9f6c48be0d1e1b6e9ad0fa34a9661ef"}, + {file = "mmhash3-3.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6d51b1005233141ce7394531af40a3f0fc1f274467bf8dff44dcf7987924fe58"}, + {file = "mmhash3-3.0.1-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:855c67b100e37df166acf79cdff58fa8f9f6c48be0d1e1b6e9ad0fa34a9661ef"}, {file = "mmhash3-3.0.1-cp37-cp37m-win32.whl", hash = "sha256:bb3030df1334fd665427f8be8e8ce4f04aeab7f6010ce4f2c128f0099bdab96f"}, {file = "mmhash3-3.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:1545e1177294afe4912d5a5e401c7fa9b799dd109b30289e7af74d5b07e7c474"}, {file = "mmhash3-3.0.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:2479899e7dda834a671991a1098a691ab1c2eaa20c3e939d691ca4a19361cfe0"}, - {file = "mmhash3-3.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:c9056196d5e3d3d844433a63d806a683f710ab3aaf1c910550c7746464bc43ae"}, - {file = "mmhash3-3.0.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:a0d4c307af0bf70207305f70f131898be071d1b19a89f462b13487f5c25e8d4e"}, + {file = "mmhash3-3.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c9056196d5e3d3d844433a63d806a683f710ab3aaf1c910550c7746464bc43ae"}, + {file = "mmhash3-3.0.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a0d4c307af0bf70207305f70f131898be071d1b19a89f462b13487f5c25e8d4e"}, {file = "mmhash3-3.0.1-cp38-cp38-win32.whl", hash = "sha256:5f885f65e329fd14bc38debac4a79eacf381e856965d9c65c4d1c6946ea190d0"}, {file = "mmhash3-3.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:3b42d0bda5e1cd22c18b16887b0521060fb59d0aaaaf033feacbc0a2492d20fe"}, {file = "mmhash3-3.0.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:d3f333286bb87aa9dc6bd8e7960a55a27b011a401f24b889a50e6d219f65e7c9"}, {file = "mmhash3-3.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:6b7ef2eb95a18bcd02ce0d3e047adde3a025afd96c1d266a8a0d44574f44a307"}, - {file = "mmhash3-3.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:d6ac8a5f511c60f341bf9cae462bb4941abb149d98464ba5f4f4548875b601c6"}, - {file = "mmhash3-3.0.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:efef9e632e6e248e46f52d108a5ebd1f28edaf427b7fd47ebf97dbff4b2cab81"}, + {file = "mmhash3-3.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d6ac8a5f511c60f341bf9cae462bb4941abb149d98464ba5f4f4548875b601c6"}, + {file = "mmhash3-3.0.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:efef9e632e6e248e46f52d108a5ebd1f28edaf427b7fd47ebf97dbff4b2cab81"}, {file = "mmhash3-3.0.1-cp39-cp39-win32.whl", hash = "sha256:bdac06d72e448c67afb12e758b203d875e29d4097bb279a38a5649d44b518ba7"}, {file = "mmhash3-3.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:0baeaa20cac5f75ed14f28056826bd9d9c8b2354b382073f3fd5190708992a0d"}, {file = "mmhash3-3.0.1.tar.gz", hash = "sha256:a00d68f4a1cc434b9501513c8a29e18ed1ddad383677d72b41d71d0d862348af"}, @@ -1627,14 +1328,13 @@ files = [ [[package]] name = "moto" -version = "4.1.10" +version = "4.1.11" description = "" -category = "dev" optional = false python-versions = ">=3.7" files = [ - {file = "moto-4.1.10-py2.py3-none-any.whl", hash = "sha256:86ee1f6b23a0c3dcf5c829fb1210fa3f293a2e8f1c95412538c0ea4a2f437021"}, - {file = "moto-4.1.10.tar.gz", hash = "sha256:65e15fc3e80a9589f50f8a5267b9a99f327224ed4d5cc44fe20edcbdb9248e06"}, + {file = "moto-4.1.11-py2.py3-none-any.whl", hash = "sha256:5003126c46ce70fe351ff1cb67dc8d9a5983f403fae13b7628b0fb503d19039e"}, + {file = "moto-4.1.11.tar.gz", hash = "sha256:f3e966ba1460751e19eab5356545813b29c05478b47eb0da445d688949339be2"}, ] [package.dependencies] @@ -1649,21 +1349,17 @@ werkzeug = ">=0.5,<2.2.0 || >2.2.0,<2.2.1 || >2.2.1" xmltodict = "*" [package.extras] -all = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", "jsondiff -(>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.0)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", -"setuptools", "sshpubkeys (>=3.1.0)"] +all = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.3)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] apigateway = ["PyYAML (>=5.1)", "ecdsa (!=0.15)", "openapi-spec-validator (>=0.2.8)", "python-jose[cryptography] (>=3.1.0,<4.0.0)"] apigatewayv2 = ["PyYAML (>=5.1)"] appsync = ["graphql-core"] awslambda = ["docker (>=3.0.0)"] batch = ["docker (>=3.0.0)"] -cloudformation = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", -"jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.0)", "pyparsing (>=3.0.7)", "python-jose[cryptography] -(>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] +cloudformation = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.3)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] cognitoidp = ["ecdsa (!=0.15)", "python-jose[cryptography] (>=3.1.0,<4.0.0)"] ds = ["sshpubkeys (>=3.1.0)"] -dynamodb = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.0)"] -dynamodbstreams = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.0)"] +dynamodb = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.3)"] +dynamodbstreams = ["docker (>=3.0.0)", "py-partiql-parser (==0.3.3)"] ebs = ["sshpubkeys (>=3.1.0)"] ec2 = ["sshpubkeys (>=3.1.0)"] efs = ["sshpubkeys (>=3.1.0)"] @@ -1671,20 +1367,15 @@ eks = ["sshpubkeys (>=3.1.0)"] glue = ["pyparsing (>=3.0.7)"] iotdata = ["jsondiff (>=1.1.2)"] route53resolver = ["sshpubkeys (>=3.1.0)"] -s3 = ["PyYAML (>=5.1)", "py-partiql-parser (==0.3.0)"] -server = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "flask (!=2.2.0,!=2.2.1)", -"flask-cors", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.0)", "pyparsing (>=3.0.7)", -"python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] +s3 = ["PyYAML (>=5.1)", "py-partiql-parser (==0.3.3)"] +server = ["PyYAML (>=5.1)", "aws-xray-sdk (>=0.93,!=0.96)", "cfn-lint (>=0.40.0)", "docker (>=3.0.0)", "ecdsa (!=0.15)", "flask (!=2.2.0,!=2.2.1)", "flask-cors", "graphql-core", "jsondiff (>=1.1.2)", "openapi-spec-validator (>=0.2.8)", "py-partiql-parser (==0.3.3)", "pyparsing (>=3.0.7)", "python-jose[cryptography] (>=3.1.0,<4.0.0)", "setuptools", "sshpubkeys (>=3.1.0)"] ssm = ["PyYAML (>=5.1)"] xray = ["aws-xray-sdk (>=0.93,!=0.96)", "setuptools"] [[package]] name = "msal" version = "1.22.0" -description = "The Microsoft Authentication Library (MSAL) for Python library enables your app to access the Microsoft Cloud by supporting -authentication of users with Microsoft Azure Active Directory accounts (AAD) and Microsoft Accounts (MSA) using industry standard OAuth2 and OpenID -Connect." -category = "main" +description = "The Microsoft Authentication Library (MSAL) for Python library enables your app to access the Microsoft Cloud by supporting authentication of users with Microsoft Azure Active Directory accounts (AAD) and Microsoft Accounts (MSA) using industry standard OAuth2 and OpenID Connect." optional = true python-versions = "*" files = [ @@ -1703,9 +1394,7 @@ broker = ["pymsalruntime (>=0.13.2,<0.14)"] [[package]] name = "msal-extensions" version = "1.0.0" -description = "Microsoft Authentication Library extensions (MSAL EX) provides a persistence API that can save your data on disk, encrypted on Windows, -macOS and Linux. Concurrent data access will be coordinated by a file lock mechanism." -category = "main" +description = "Microsoft Authentication Library extensions (MSAL EX) provides a persistence API that can save your data on disk, encrypted on Windows, macOS and Linux. Concurrent data access will be coordinated by a file lock mechanism." optional = true python-versions = "*" files = [ @@ -1724,19 +1413,15 @@ portalocker = [ name = "msgpack" version = "1.0.5" description = "MessagePack serializer" -category = "main" optional = true python-versions = "*" files = [ {file = "msgpack-1.0.5-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:525228efd79bb831cf6830a732e2e80bc1b05436b086d4264814b4b2955b2fa9"}, {file = "msgpack-1.0.5-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:4f8d8b3bf1ff2672567d6b5c725a1b347fe838b912772aa8ae2bf70338d5a198"}, {file = "msgpack-1.0.5-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:cdc793c50be3f01106245a61b739328f7dccc2c648b501e237f0699fe1395b81"}, - {file = "msgpack-1.0.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:5cb47c21a8a65b165ce29f2bec852790cbc04936f502966768e4aae9fa763cb7"}, - {file = "msgpack-1.0.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:e42b9594cc3bf4d838d67d6ed62b9e59e201862a25e9a157019e171fbe672dd3"}, - {file = "msgpack-1.0.5-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:55b56a24893105dc52c1253649b60f475f36b3aa0fc66115bffafb624d7cb30b"}, + {file = "msgpack-1.0.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5cb47c21a8a65b165ce29f2bec852790cbc04936f502966768e4aae9fa763cb7"}, + {file = "msgpack-1.0.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e42b9594cc3bf4d838d67d6ed62b9e59e201862a25e9a157019e171fbe672dd3"}, + {file = "msgpack-1.0.5-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:55b56a24893105dc52c1253649b60f475f36b3aa0fc66115bffafb624d7cb30b"}, {file = "msgpack-1.0.5-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:1967f6129fc50a43bfe0951c35acbb729be89a55d849fab7686004da85103f1c"}, {file = "msgpack-1.0.5-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:20a97bf595a232c3ee6d57ddaadd5453d174a52594bf9c21d10407e2a2d9b3bd"}, {file = "msgpack-1.0.5-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d25dd59bbbbb996eacf7be6b4ad082ed7eacc4e8f3d2df1ba43822da9bfa122a"}, @@ -1745,36 +1430,27 @@ files = [ {file = "msgpack-1.0.5-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9f5ae84c5c8a857ec44dc180a8b0cc08238e021f57abdf51a8182e915e6299f0"}, {file = "msgpack-1.0.5-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:9e6ca5d5699bcd89ae605c150aee83b5321f2115695e741b99618f4856c50898"}, {file = "msgpack-1.0.5-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5494ea30d517a3576749cad32fa27f7585c65f5f38309c88c6d137877fa28a5a"}, - {file = "msgpack-1.0.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:1ab2f3331cb1b54165976a9d976cb251a83183631c88076613c6c780f0d6e45a"}, - {file = "msgpack-1.0.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:28592e20bbb1620848256ebc105fc420436af59515793ed27d5c77a217477705"}, - {file = "msgpack-1.0.5-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:fe5c63197c55bce6385d9aee16c4d0641684628f63ace85f73571e65ad1c1e8d"}, + {file = "msgpack-1.0.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1ab2f3331cb1b54165976a9d976cb251a83183631c88076613c6c780f0d6e45a"}, + {file = "msgpack-1.0.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:28592e20bbb1620848256ebc105fc420436af59515793ed27d5c77a217477705"}, + {file = "msgpack-1.0.5-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fe5c63197c55bce6385d9aee16c4d0641684628f63ace85f73571e65ad1c1e8d"}, {file = "msgpack-1.0.5-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:ed40e926fa2f297e8a653c954b732f125ef97bdd4c889f243182299de27e2aa9"}, {file = "msgpack-1.0.5-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:b2de4c1c0538dcb7010902a2b97f4e00fc4ddf2c8cda9749af0e594d3b7fa3d7"}, {file = "msgpack-1.0.5-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:bf22a83f973b50f9d38e55c6aade04c41ddda19b00c4ebc558930d78eecc64ed"}, {file = "msgpack-1.0.5-cp311-cp311-win32.whl", hash = "sha256:c396e2cc213d12ce017b686e0f53497f94f8ba2b24799c25d913d46c08ec422c"}, {file = "msgpack-1.0.5-cp311-cp311-win_amd64.whl", hash = "sha256:6c4c68d87497f66f96d50142a2b73b97972130d93677ce930718f68828b382e2"}, {file = "msgpack-1.0.5-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:a2b031c2e9b9af485d5e3c4520f4220d74f4d222a5b8dc8c1a3ab9448ca79c57"}, - {file = "msgpack-1.0.5-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:4f837b93669ce4336e24d08286c38761132bc7ab29782727f8557e1eb21b2080"}, - {file = "msgpack-1.0.5-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:b1d46dfe3832660f53b13b925d4e0fa1432b00f5f7210eb3ad3bb9a13c6204a6"}, - {file = "msgpack-1.0.5-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:366c9a7b9057e1547f4ad51d8facad8b406bab69c7d72c0eb6f529cf76d4b85f"}, + {file = "msgpack-1.0.5-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4f837b93669ce4336e24d08286c38761132bc7ab29782727f8557e1eb21b2080"}, + {file = "msgpack-1.0.5-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b1d46dfe3832660f53b13b925d4e0fa1432b00f5f7210eb3ad3bb9a13c6204a6"}, + {file = "msgpack-1.0.5-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:366c9a7b9057e1547f4ad51d8facad8b406bab69c7d72c0eb6f529cf76d4b85f"}, {file = "msgpack-1.0.5-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:4c075728a1095efd0634a7dccb06204919a2f67d1893b6aa8e00497258bf926c"}, {file = "msgpack-1.0.5-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:f933bbda5a3ee63b8834179096923b094b76f0c7a73c1cfe8f07ad608c58844b"}, {file = "msgpack-1.0.5-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:36961b0568c36027c76e2ae3ca1132e35123dcec0706c4b7992683cc26c1320c"}, {file = "msgpack-1.0.5-cp36-cp36m-win32.whl", hash = "sha256:b5ef2f015b95f912c2fcab19c36814963b5463f1fb9049846994b007962743e9"}, {file = "msgpack-1.0.5-cp36-cp36m-win_amd64.whl", hash = "sha256:288e32b47e67f7b171f86b030e527e302c91bd3f40fd9033483f2cacc37f327a"}, {file = "msgpack-1.0.5-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:137850656634abddfb88236008339fdaba3178f4751b28f270d2ebe77a563b6c"}, - {file = "msgpack-1.0.5-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:0c05a4a96585525916b109bb85f8cb6511db1c6f5b9d9cbcbc940dc6b4be944b"}, - {file = "msgpack-1.0.5-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:56a62ec00b636583e5cb6ad313bbed36bb7ead5fa3a3e38938503142c72cba4f"}, - {file = "msgpack-1.0.5-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:ef8108f8dedf204bb7b42994abf93882da1159728a2d4c5e82012edd92c9da9f"}, + {file = "msgpack-1.0.5-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0c05a4a96585525916b109bb85f8cb6511db1c6f5b9d9cbcbc940dc6b4be944b"}, + {file = "msgpack-1.0.5-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:56a62ec00b636583e5cb6ad313bbed36bb7ead5fa3a3e38938503142c72cba4f"}, + {file = "msgpack-1.0.5-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ef8108f8dedf204bb7b42994abf93882da1159728a2d4c5e82012edd92c9da9f"}, {file = "msgpack-1.0.5-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:1835c84d65f46900920b3708f5ba829fb19b1096c1800ad60bae8418652a951d"}, {file = "msgpack-1.0.5-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:e57916ef1bd0fee4f21c4600e9d1da352d8816b52a599c46460e93a6e9f17086"}, {file = "msgpack-1.0.5-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:17358523b85973e5f242ad74aa4712b7ee560715562554aa2134d96e7aa4cbbf"}, @@ -1783,12 +1459,9 @@ files = [ {file = "msgpack-1.0.5-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:b72d0698f86e8d9ddf9442bdedec15b71df3598199ba33322d9711a19f08145c"}, {file = "msgpack-1.0.5-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:379026812e49258016dd84ad79ac8446922234d498058ae1d415f04b522d5b2d"}, {file = "msgpack-1.0.5-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:332360ff25469c346a1c5e47cbe2a725517919892eda5cfaffe6046656f0b7bb"}, - {file = "msgpack-1.0.5-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:476a8fe8fae289fdf273d6d2a6cb6e35b5a58541693e8f9f019bfe990a51e4ba"}, - {file = "msgpack-1.0.5-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:a9985b214f33311df47e274eb788a5893a761d025e2b92c723ba4c63936b69b1"}, - {file = "msgpack-1.0.5-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:48296af57cdb1d885843afd73c4656be5c76c0c6328db3440c9601a98f303d87"}, + {file = "msgpack-1.0.5-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:476a8fe8fae289fdf273d6d2a6cb6e35b5a58541693e8f9f019bfe990a51e4ba"}, + {file = "msgpack-1.0.5-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a9985b214f33311df47e274eb788a5893a761d025e2b92c723ba4c63936b69b1"}, + {file = "msgpack-1.0.5-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:48296af57cdb1d885843afd73c4656be5c76c0c6328db3440c9601a98f303d87"}, {file = "msgpack-1.0.5-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:addab7e2e1fcc04bd08e4eb631c2a90960c340e40dfc4a5e24d2ff0d5a3b3edb"}, {file = "msgpack-1.0.5-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:916723458c25dfb77ff07f4c66aed34e47503b2eb3188b3adbec8d8aa6e00f48"}, {file = "msgpack-1.0.5-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:821c7e677cc6acf0fd3f7ac664c98803827ae6de594a9f99563e48c5a2f27eb0"}, @@ -1797,12 +1470,9 @@ files = [ {file = "msgpack-1.0.5-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:20c784e66b613c7f16f632e7b5e8a1651aa5702463d61394671ba07b2fc9e025"}, {file = "msgpack-1.0.5-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:266fa4202c0eb94d26822d9bfd7af25d1e2c088927fe8de9033d929dd5ba24c5"}, {file = "msgpack-1.0.5-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:18334484eafc2b1aa47a6d42427da7fa8f2ab3d60b674120bce7a895a0a85bdd"}, - {file = "msgpack-1.0.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:57e1f3528bd95cc44684beda696f74d3aaa8a5e58c816214b9046512240ef437"}, - {file = "msgpack-1.0.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:586d0d636f9a628ddc6a17bfd45aa5b5efaf1606d2b60fa5d87b8986326e933f"}, - {file = "msgpack-1.0.5-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:a740fa0e4087a734455f0fc3abf5e746004c9da72fbd541e9b113013c8dc3282"}, + {file = "msgpack-1.0.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:57e1f3528bd95cc44684beda696f74d3aaa8a5e58c816214b9046512240ef437"}, + {file = "msgpack-1.0.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:586d0d636f9a628ddc6a17bfd45aa5b5efaf1606d2b60fa5d87b8986326e933f"}, + {file = "msgpack-1.0.5-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a740fa0e4087a734455f0fc3abf5e746004c9da72fbd541e9b113013c8dc3282"}, {file = "msgpack-1.0.5-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:3055b0455e45810820db1f29d900bf39466df96ddca11dfa6d074fa47054376d"}, {file = "msgpack-1.0.5-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:a61215eac016f391129a013c9e46f3ab308db5f5ec9f25811e811f96962599a8"}, {file = "msgpack-1.0.5-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:362d9655cd369b08fda06b6657a303eb7172d5279997abe094512e919cf74b11"}, @@ -1815,23 +1485,17 @@ files = [ name = "multidict" version = "6.0.4" description = "multidict implementation" -category = "main" 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-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"}, @@ -1842,16 +1506,11 @@ files = [ {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-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"}, @@ -1860,16 +1519,11 @@ files = [ {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-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"}, @@ -1880,16 +1534,11 @@ files = [ {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-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"}, @@ -1900,16 +1549,11 @@ files = [ {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-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"}, @@ -1924,7 +1568,6 @@ files = [ name = "nodeenv" version = "1.8.0" description = "Node.js virtual environment builder" -category = "dev" optional = false python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*" files = [ @@ -1939,45 +1582,35 @@ setuptools = "*" name = "numpy" version = "1.24.3" description = "Fundamental package for array computing in Python" -category = "main" optional = true python-versions = ">=3.8" files = [ {file = "numpy-1.24.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3c1104d3c036fb81ab923f507536daedc718d0ad5a8707c6061cdfd6d184e570"}, {file = "numpy-1.24.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:202de8f38fc4a45a3eea4b63e2f376e5f2dc64ef0fa692838e31a808520efaf7"}, - {file = "numpy-1.24.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:8535303847b89aa6b0f00aa1dc62867b5a32923e4d1681a35b5eef2d9591a463"}, - {file = "numpy-1.24.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:2d926b52ba1367f9acb76b0df6ed21f0b16a1ad87c6720a1121674e5cf63e2b6"}, + {file = "numpy-1.24.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8535303847b89aa6b0f00aa1dc62867b5a32923e4d1681a35b5eef2d9591a463"}, + {file = "numpy-1.24.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2d926b52ba1367f9acb76b0df6ed21f0b16a1ad87c6720a1121674e5cf63e2b6"}, {file = "numpy-1.24.3-cp310-cp310-win32.whl", hash = "sha256:f21c442fdd2805e91799fbe044a7b999b8571bb0ab0f7850d0cb9641a687092b"}, {file = "numpy-1.24.3-cp310-cp310-win_amd64.whl", hash = "sha256:ab5f23af8c16022663a652d3b25dcdc272ac3f83c3af4c02eb8b824e6b3ab9d7"}, {file = "numpy-1.24.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:9a7721ec204d3a237225db3e194c25268faf92e19338a35f3a224469cb6039a3"}, {file = "numpy-1.24.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:d6cc757de514c00b24ae8cf5c876af2a7c3df189028d68c0cb4eaa9cd5afc2bf"}, - {file = "numpy-1.24.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:76e3f4e85fc5d4fd311f6e9b794d0c00e7002ec122be271f2019d63376f1d385"}, - {file = "numpy-1.24.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:a1d3c026f57ceaad42f8231305d4653d5f05dc6332a730ae5c0bea3513de0950"}, + {file = "numpy-1.24.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:76e3f4e85fc5d4fd311f6e9b794d0c00e7002ec122be271f2019d63376f1d385"}, + {file = "numpy-1.24.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a1d3c026f57ceaad42f8231305d4653d5f05dc6332a730ae5c0bea3513de0950"}, {file = "numpy-1.24.3-cp311-cp311-win32.whl", hash = "sha256:c91c4afd8abc3908e00a44b2672718905b8611503f7ff87390cc0ac3423fb096"}, {file = "numpy-1.24.3-cp311-cp311-win_amd64.whl", hash = "sha256:5342cf6aad47943286afa6f1609cad9b4266a05e7f2ec408e2cf7aea7ff69d80"}, {file = "numpy-1.24.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:7776ea65423ca6a15255ba1872d82d207bd1e09f6d0894ee4a64678dd2204078"}, {file = "numpy-1.24.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:ae8d0be48d1b6ed82588934aaaa179875e7dc4f3d84da18d7eae6eb3f06c242c"}, - {file = "numpy-1.24.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:ecde0f8adef7dfdec993fd54b0f78183051b6580f606111a6d789cd14c61ea0c"}, - {file = "numpy-1.24.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:4749e053a29364d3452c034827102ee100986903263e89884922ef01a0a6fd2f"}, + {file = "numpy-1.24.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ecde0f8adef7dfdec993fd54b0f78183051b6580f606111a6d789cd14c61ea0c"}, + {file = "numpy-1.24.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4749e053a29364d3452c034827102ee100986903263e89884922ef01a0a6fd2f"}, {file = "numpy-1.24.3-cp38-cp38-win32.whl", hash = "sha256:d933fabd8f6a319e8530d0de4fcc2e6a61917e0b0c271fded460032db42a0fe4"}, {file = "numpy-1.24.3-cp38-cp38-win_amd64.whl", hash = "sha256:56e48aec79ae238f6e4395886b5eaed058abb7231fb3361ddd7bfdf4eed54289"}, {file = "numpy-1.24.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:4719d5aefb5189f50887773699eaf94e7d1e02bf36c1a9d353d9f46703758ca4"}, {file = "numpy-1.24.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0ec87a7084caa559c36e0a2309e4ecb1baa03b687201d0a847c8b0ed476a7187"}, - {file = "numpy-1.24.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:ea8282b9bcfe2b5e7d491d0bf7f3e2da29700cec05b49e64d6246923329f2b02"}, - {file = "numpy-1.24.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:210461d87fb02a84ef243cac5e814aad2b7f4be953b32cb53327bb49fd77fbb4"}, + {file = "numpy-1.24.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ea8282b9bcfe2b5e7d491d0bf7f3e2da29700cec05b49e64d6246923329f2b02"}, + {file = "numpy-1.24.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:210461d87fb02a84ef243cac5e814aad2b7f4be953b32cb53327bb49fd77fbb4"}, {file = "numpy-1.24.3-cp39-cp39-win32.whl", hash = "sha256:784c6da1a07818491b0ffd63c6bbe5a33deaa0e25a20e1b3ea20cf0e43f8046c"}, {file = "numpy-1.24.3-cp39-cp39-win_amd64.whl", hash = "sha256:d5036197ecae68d7f491fcdb4df90082b0d4960ca6599ba2659957aafced7c17"}, {file = "numpy-1.24.3-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:352ee00c7f8387b44d19f4cada524586f07379c0d49270f87233983bc5087ca0"}, - {file = "numpy-1.24.3-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:1a7d6acc2e7524c9955e5c903160aa4ea083736fde7e91276b0e5d98e6332812"}, + {file = "numpy-1.24.3-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a7d6acc2e7524c9955e5c903160aa4ea083736fde7e91276b0e5d98e6332812"}, {file = "numpy-1.24.3-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:35400e6a8d102fd07c71ed7dcadd9eb62ee9a6e84ec159bd48c28235bbb0f8e4"}, {file = "numpy-1.24.3.tar.gz", hash = "sha256:ab344f1bf21f140adab8e47fdbc7c35a477dc01408791f8ba00d018dd0bc5155"}, ] @@ -1986,7 +1619,6 @@ files = [ name = "packaging" version = "23.1" description = "Core utilities for Python packages" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -1998,40 +1630,31 @@ files = [ name = "pandas" version = "2.0.2" description = "Powerful data structures for data analysis, time series, and statistics" -category = "main" optional = true python-versions = ">=3.8" files = [ {file = "pandas-2.0.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9ebb9f1c22ddb828e7fd017ea265a59d80461d5a79154b49a4207bd17514d122"}, {file = "pandas-2.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1eb09a242184092f424b2edd06eb2b99d06dc07eeddff9929e8667d4ed44e181"}, - {file = "pandas-2.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:c7319b6e68de14e6209460f72a8d1ef13c09fb3d3ef6c37c1e65b35d50b5c145"}, - {file = "pandas-2.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:dd46bde7309088481b1cf9c58e3f0e204b9ff9e3244f441accd220dd3365ce7c"}, + {file = "pandas-2.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c7319b6e68de14e6209460f72a8d1ef13c09fb3d3ef6c37c1e65b35d50b5c145"}, + {file = "pandas-2.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd46bde7309088481b1cf9c58e3f0e204b9ff9e3244f441accd220dd3365ce7c"}, {file = "pandas-2.0.2-cp310-cp310-win32.whl", hash = "sha256:51a93d422fbb1bd04b67639ba4b5368dffc26923f3ea32a275d2cc450f1d1c86"}, {file = "pandas-2.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:66d00300f188fa5de73f92d5725ced162488f6dc6ad4cecfe4144ca29debe3b8"}, {file = "pandas-2.0.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:02755de164da6827764ceb3bbc5f64b35cb12394b1024fdf88704d0fa06e0e2f"}, {file = "pandas-2.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:0a1e0576611641acde15c2322228d138258f236d14b749ad9af498ab69089e2d"}, - {file = "pandas-2.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:a6b5f14cd24a2ed06e14255ff40fe2ea0cfaef79a8dd68069b7ace74bd6acbba"}, - {file = "pandas-2.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:50e451932b3011b61d2961b4185382c92cc8c6ee4658dcd4f320687bb2d000ee"}, + {file = "pandas-2.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a6b5f14cd24a2ed06e14255ff40fe2ea0cfaef79a8dd68069b7ace74bd6acbba"}, + {file = "pandas-2.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:50e451932b3011b61d2961b4185382c92cc8c6ee4658dcd4f320687bb2d000ee"}, {file = "pandas-2.0.2-cp311-cp311-win32.whl", hash = "sha256:7b21cb72958fc49ad757685db1919021d99650d7aaba676576c9e88d3889d456"}, {file = "pandas-2.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:c4af689352c4fe3d75b2834933ee9d0ccdbf5d7a8a7264f0ce9524e877820c08"}, {file = "pandas-2.0.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:69167693cb8f9b3fc060956a5d0a0a8dbfed5f980d9fd2c306fb5b9c855c814c"}, {file = "pandas-2.0.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:30a89d0fec4263ccbf96f68592fd668939481854d2ff9da709d32a047689393b"}, - {file = "pandas-2.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:a18e5c72b989ff0f7197707ceddc99828320d0ca22ab50dd1b9e37db45b010c0"}, - {file = "pandas-2.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:7376e13d28eb16752c398ca1d36ccfe52bf7e887067af9a0474de6331dd948d2"}, + {file = "pandas-2.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a18e5c72b989ff0f7197707ceddc99828320d0ca22ab50dd1b9e37db45b010c0"}, + {file = "pandas-2.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7376e13d28eb16752c398ca1d36ccfe52bf7e887067af9a0474de6331dd948d2"}, {file = "pandas-2.0.2-cp38-cp38-win32.whl", hash = "sha256:6d6d10c2142d11d40d6e6c0a190b1f89f525bcf85564707e31b0a39e3b398e08"}, {file = "pandas-2.0.2-cp38-cp38-win_amd64.whl", hash = "sha256:e69140bc2d29a8556f55445c15f5794490852af3de0f609a24003ef174528b79"}, {file = "pandas-2.0.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b42b120458636a981077cfcfa8568c031b3e8709701315e2bfa866324a83efa8"}, {file = "pandas-2.0.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f908a77cbeef9bbd646bd4b81214cbef9ac3dda4181d5092a4aa9797d1bc7774"}, - {file = "pandas-2.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:713f2f70abcdade1ddd68fc91577cb090b3544b07ceba78a12f799355a13ee44"}, - {file = "pandas-2.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:cf3f0c361a4270185baa89ec7ab92ecaa355fe783791457077473f974f654df5"}, + {file = "pandas-2.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:713f2f70abcdade1ddd68fc91577cb090b3544b07ceba78a12f799355a13ee44"}, + {file = "pandas-2.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cf3f0c361a4270185baa89ec7ab92ecaa355fe783791457077473f974f654df5"}, {file = "pandas-2.0.2-cp39-cp39-win32.whl", hash = "sha256:598e9020d85a8cdbaa1815eb325a91cfff2bb2b23c1442549b8a3668e36f0f77"}, {file = "pandas-2.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:77550c8909ebc23e56a89f91b40ad01b50c42cfbfab49b3393694a50549295ea"}, {file = "pandas-2.0.2.tar.gz", hash = "sha256:dd5476b6c3fe410ee95926873f377b856dbc4e81a9c605a0dc05aaccc6a7c6c6"}, @@ -2048,12 +1671,7 @@ pytz = ">=2020.1" tzdata = ">=2022.1" [package.extras] -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.0.0)", "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)"] +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.0.0)", "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)"] @@ -2079,7 +1697,6 @@ xml = ["lxml (>=4.6.3)"] name = "pkgutil-resolve-name" version = "1.3.10" description = "Resolve a name to an object." -category = "main" optional = true python-versions = ">=3.6" files = [ @@ -2091,7 +1708,6 @@ files = [ name = "platformdirs" version = "3.5.1" description = "A small Python package for determining appropriate platform-specific dirs, e.g. a \"user data dir\"." -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -2107,7 +1723,6 @@ test = ["appdirs (==1.4.4)", "covdefaults (>=2.3)", "pytest (>=7.3.1)", "pytest- name = "pluggy" version = "1.0.0" description = "plugin and hook calling mechanisms for python" -category = "dev" optional = false python-versions = ">=3.6" files = [ @@ -2123,7 +1738,6 @@ testing = ["pytest", "pytest-benchmark"] name = "portalocker" version = "2.7.0" description = "Wraps the portalocker recipe for easy usage" -category = "main" optional = true python-versions = ">=3.5" files = [ @@ -2143,7 +1757,6 @@ tests = ["pytest (>=5.4.1)", "pytest-cov (>=2.8.1)", "pytest-mypy (>=0.8.0)", "p name = "pre-commit" version = "3.3.2" description = "A framework for managing and maintaining multi-language pre-commit hooks." -category = "dev" optional = false python-versions = ">=3.8" files = [ @@ -2162,7 +1775,6 @@ virtualenv = ">=20.10.0" name = "protobuf" version = "4.23.2" description = "" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -2185,43 +1797,32 @@ files = [ name = "pyarrow" version = "12.0.0" description = "Python library for Apache Arrow" -category = "main" optional = true python-versions = ">=3.7" files = [ {file = "pyarrow-12.0.0-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:3b97649c8a9a09e1d8dc76513054f1331bd9ece78ee39365e6bf6bc7503c1e94"}, {file = "pyarrow-12.0.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:bc4ea634dacb03936f50fcf59574a8e727f90c17c24527e488d8ceb52ae284de"}, - {file = "pyarrow-12.0.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:1d568acfca3faa565d663e53ee34173be8e23a95f78f2abfdad198010ec8f745"}, - {file = "pyarrow-12.0.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:1b50bb9a82dca38a002d7cbd802a16b1af0f8c50ed2ec94a319f5f2afc047ee9"}, + {file = "pyarrow-12.0.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d568acfca3faa565d663e53ee34173be8e23a95f78f2abfdad198010ec8f745"}, + {file = "pyarrow-12.0.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1b50bb9a82dca38a002d7cbd802a16b1af0f8c50ed2ec94a319f5f2afc047ee9"}, {file = "pyarrow-12.0.0-cp310-cp310-win_amd64.whl", hash = "sha256:3d1733b1ea086b3c101427d0e57e2be3eb964686e83c2363862a887bb5c41fa8"}, {file = "pyarrow-12.0.0-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:a7cd32fe77f967fe08228bc100433273020e58dd6caced12627bcc0a7675a513"}, {file = "pyarrow-12.0.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:92fb031e6777847f5c9b01eaa5aa0c9033e853ee80117dce895f116d8b0c3ca3"}, - {file = "pyarrow-12.0.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:280289ebfd4ac3570f6b776515baa01e4dcbf17122c401e4b7170a27c4be63fd"}, - {file = "pyarrow-12.0.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:272f147d4f8387bec95f17bb58dcfc7bc7278bb93e01cb7b08a0e93a8921e18e"}, + {file = "pyarrow-12.0.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:280289ebfd4ac3570f6b776515baa01e4dcbf17122c401e4b7170a27c4be63fd"}, + {file = "pyarrow-12.0.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:272f147d4f8387bec95f17bb58dcfc7bc7278bb93e01cb7b08a0e93a8921e18e"}, {file = "pyarrow-12.0.0-cp311-cp311-win_amd64.whl", hash = "sha256:0846ace49998825eda4722f8d7f83fa05601c832549c9087ea49d6d5397d8cec"}, {file = "pyarrow-12.0.0-cp37-cp37m-macosx_10_14_x86_64.whl", hash = "sha256:993287136369aca60005ee7d64130f9466489c4f7425f5c284315b0a5401ccd9"}, - {file = "pyarrow-12.0.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:7a7b6a765ee4f88efd7d8348d9a1f804487d60799d0428b6ddf3344eaef37282"}, - {file = "pyarrow-12.0.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:a1c4fce253d5bdc8d62f11cfa3da5b0b34b562c04ce84abb8bd7447e63c2b327"}, + {file = "pyarrow-12.0.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7a7b6a765ee4f88efd7d8348d9a1f804487d60799d0428b6ddf3344eaef37282"}, + {file = "pyarrow-12.0.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a1c4fce253d5bdc8d62f11cfa3da5b0b34b562c04ce84abb8bd7447e63c2b327"}, {file = "pyarrow-12.0.0-cp37-cp37m-win_amd64.whl", hash = "sha256:e6be4d85707fc8e7a221c8ab86a40449ce62559ce25c94321df7c8500245888f"}, {file = "pyarrow-12.0.0-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:ea830d9f66bfb82d30b5794642f83dd0e4a718846462d22328981e9eb149cba8"}, {file = "pyarrow-12.0.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:7b5b9f60d9ef756db59bec8d90e4576b7df57861e6a3d6a8bf99538f68ca15b3"}, - {file = "pyarrow-12.0.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:b99e559d27db36ad3a33868a475f03e3129430fc065accc839ef4daa12c6dab6"}, - {file = "pyarrow-12.0.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:5b0810864a593b89877120972d1f7af1d1c9389876dbed92b962ed81492d3ffc"}, + {file = "pyarrow-12.0.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b99e559d27db36ad3a33868a475f03e3129430fc065accc839ef4daa12c6dab6"}, + {file = "pyarrow-12.0.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5b0810864a593b89877120972d1f7af1d1c9389876dbed92b962ed81492d3ffc"}, {file = "pyarrow-12.0.0-cp38-cp38-win_amd64.whl", hash = "sha256:23a77d97f4d101ddfe81b9c2ee03a177f0e590a7e68af15eafa06e8f3cf05976"}, {file = "pyarrow-12.0.0-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:2cc63e746221cddb9001f7281dee95fd658085dd5b717b076950e1ccc607059c"}, {file = "pyarrow-12.0.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:d8c26912607e26c2991826bbaf3cf2b9c8c3e17566598c193b492f058b40d3a4"}, - {file = "pyarrow-12.0.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:0d8b90efc290e99a81d06015f3a46601c259ecc81ffb6d8ce288c91bd1b868c9"}, - {file = "pyarrow-12.0.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:2466be046b81863be24db370dffd30a2e7894b4f9823fb60ef0a733c31ac6256"}, + {file = "pyarrow-12.0.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0d8b90efc290e99a81d06015f3a46601c259ecc81ffb6d8ce288c91bd1b868c9"}, + {file = "pyarrow-12.0.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2466be046b81863be24db370dffd30a2e7894b4f9823fb60ef0a733c31ac6256"}, {file = "pyarrow-12.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:0e36425b1c1cbf5447718b3f1751bf86c58f2b3ad299f996cd9b1aa040967656"}, {file = "pyarrow-12.0.0.tar.gz", hash = "sha256:19c812d303610ab5d664b7b1de4051ae23565f9f94d04cbea9e50569746ae1ee"}, ] @@ -2233,7 +1834,6 @@ numpy = ">=1.16.6" name = "pycparser" version = "2.21" description = "C parser in Python" -category = "main" optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" files = [ @@ -2245,51 +1845,40 @@ files = [ name = "pydantic" version = "1.10.8" description = "Data validation and settings management using python type hints" -category = "main" optional = false python-versions = ">=3.7" files = [ {file = "pydantic-1.10.8-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:1243d28e9b05003a89d72e7915fdb26ffd1d39bdd39b00b7dbe4afae4b557f9d"}, {file = "pydantic-1.10.8-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c0ab53b609c11dfc0c060d94335993cc2b95b2150e25583bec37a49b2d6c6c3f"}, - {file = "pydantic-1.10.8-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:f9613fadad06b4f3bc5db2653ce2f22e0de84a7c6c293909b48f6ed37b83c61f"}, - {file = "pydantic-1.10.8-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:df7800cb1984d8f6e249351139667a8c50a379009271ee6236138a22a0c0f319"}, + {file = "pydantic-1.10.8-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f9613fadad06b4f3bc5db2653ce2f22e0de84a7c6c293909b48f6ed37b83c61f"}, + {file = "pydantic-1.10.8-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:df7800cb1984d8f6e249351139667a8c50a379009271ee6236138a22a0c0f319"}, {file = "pydantic-1.10.8-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:0c6fafa0965b539d7aab0a673a046466d23b86e4b0e8019d25fd53f4df62c277"}, {file = "pydantic-1.10.8-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:e82d4566fcd527eae8b244fa952d99f2ca3172b7e97add0b43e2d97ee77f81ab"}, {file = "pydantic-1.10.8-cp310-cp310-win_amd64.whl", hash = "sha256:ab523c31e22943713d80d8d342d23b6f6ac4b792a1e54064a8d0cf78fd64e800"}, {file = "pydantic-1.10.8-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:666bdf6066bf6dbc107b30d034615d2627e2121506c555f73f90b54a463d1f33"}, {file = "pydantic-1.10.8-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:35db5301b82e8661fa9c505c800d0990bc14e9f36f98932bb1d248c0ac5cada5"}, - {file = "pydantic-1.10.8-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:f90c1e29f447557e9e26afb1c4dbf8768a10cc676e3781b6a577841ade126b85"}, - {file = "pydantic-1.10.8-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:93e766b4a8226e0708ef243e843105bf124e21331694367f95f4e3b4a92bbb3f"}, + {file = "pydantic-1.10.8-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f90c1e29f447557e9e26afb1c4dbf8768a10cc676e3781b6a577841ade126b85"}, + {file = "pydantic-1.10.8-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:93e766b4a8226e0708ef243e843105bf124e21331694367f95f4e3b4a92bbb3f"}, {file = "pydantic-1.10.8-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:88f195f582851e8db960b4a94c3e3ad25692c1c1539e2552f3df7a9e972ef60e"}, {file = "pydantic-1.10.8-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:34d327c81e68a1ecb52fe9c8d50c8a9b3e90d3c8ad991bfc8f953fb477d42fb4"}, {file = "pydantic-1.10.8-cp311-cp311-win_amd64.whl", hash = "sha256:d532bf00f381bd6bc62cabc7d1372096b75a33bc197a312b03f5838b4fb84edd"}, {file = "pydantic-1.10.8-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:7d5b8641c24886d764a74ec541d2fc2c7fb19f6da2a4001e6d580ba4a38f7878"}, - {file = "pydantic-1.10.8-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:7b1f6cb446470b7ddf86c2e57cd119a24959af2b01e552f60705910663af09a4"}, - {file = "pydantic-1.10.8-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:c33b60054b2136aef8cf190cd4c52a3daa20b2263917c49adad20eaf381e823b"}, + {file = "pydantic-1.10.8-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7b1f6cb446470b7ddf86c2e57cd119a24959af2b01e552f60705910663af09a4"}, + {file = "pydantic-1.10.8-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c33b60054b2136aef8cf190cd4c52a3daa20b2263917c49adad20eaf381e823b"}, {file = "pydantic-1.10.8-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:1952526ba40b220b912cdc43c1c32bcf4a58e3f192fa313ee665916b26befb68"}, {file = "pydantic-1.10.8-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:bb14388ec45a7a0dc429e87def6396f9e73c8c77818c927b6a60706603d5f2ea"}, {file = "pydantic-1.10.8-cp37-cp37m-win_amd64.whl", hash = "sha256:16f8c3e33af1e9bb16c7a91fc7d5fa9fe27298e9f299cff6cb744d89d573d62c"}, {file = "pydantic-1.10.8-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1ced8375969673929809d7f36ad322934c35de4af3b5e5b09ec967c21f9f7887"}, {file = "pydantic-1.10.8-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:93e6bcfccbd831894a6a434b0aeb1947f9e70b7468f274154d03d71fabb1d7c6"}, - {file = "pydantic-1.10.8-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:191ba419b605f897ede9892f6c56fb182f40a15d309ef0142212200a10af4c18"}, - {file = "pydantic-1.10.8-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:052d8654cb65174d6f9490cc9b9a200083a82cf5c3c5d3985db765757eb3b375"}, + {file = "pydantic-1.10.8-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:191ba419b605f897ede9892f6c56fb182f40a15d309ef0142212200a10af4c18"}, + {file = "pydantic-1.10.8-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:052d8654cb65174d6f9490cc9b9a200083a82cf5c3c5d3985db765757eb3b375"}, {file = "pydantic-1.10.8-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:ceb6a23bf1ba4b837d0cfe378329ad3f351b5897c8d4914ce95b85fba96da5a1"}, {file = "pydantic-1.10.8-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:6f2e754d5566f050954727c77f094e01793bcb5725b663bf628fa6743a5a9108"}, {file = "pydantic-1.10.8-cp38-cp38-win_amd64.whl", hash = "sha256:6a82d6cda82258efca32b40040228ecf43a548671cb174a1e81477195ed3ed56"}, {file = "pydantic-1.10.8-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3e59417ba8a17265e632af99cc5f35ec309de5980c440c255ab1ca3ae96a3e0e"}, {file = "pydantic-1.10.8-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:84d80219c3f8d4cad44575e18404099c76851bc924ce5ab1c4c8bb5e2a2227d0"}, - {file = "pydantic-1.10.8-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:2e4148e635994d57d834be1182a44bdb07dd867fa3c2d1b37002000646cc5459"}, - {file = "pydantic-1.10.8-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:12f7b0bf8553e310e530e9f3a2f5734c68699f42218bf3568ef49cd9b0e44df4"}, + {file = "pydantic-1.10.8-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e4148e635994d57d834be1182a44bdb07dd867fa3c2d1b37002000646cc5459"}, + {file = "pydantic-1.10.8-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:12f7b0bf8553e310e530e9f3a2f5734c68699f42218bf3568ef49cd9b0e44df4"}, {file = "pydantic-1.10.8-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:42aa0c4b5c3025483240a25b09f3c09a189481ddda2ea3a831a9d25f444e03c1"}, {file = "pydantic-1.10.8-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:17aef11cc1b997f9d574b91909fed40761e13fac438d72b81f902226a69dac01"}, {file = "pydantic-1.10.8-cp39-cp39-win_amd64.whl", hash = "sha256:66a703d1983c675a6e0fed8953b0971c44dba48a929a2000a493c3772eb61a5a"}, @@ -2308,7 +1897,6 @@ email = ["email-validator (>=1.0.3)"] name = "pygments" version = "2.15.1" description = "Pygments is a syntax highlighting package written in Python." -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -2323,7 +1911,6 @@ plugins = ["importlib-metadata"] name = "pyjwt" version = "2.7.0" description = "JSON Web Token implementation in Python" -category = "main" optional = true python-versions = ">=3.7" files = [ @@ -2336,8 +1923,7 @@ cryptography = {version = ">=3.4.0", optional = true, markers = "extra == \"cryp [package.extras] crypto = ["cryptography (>=3.4.0)"] -dev = ["coverage[toml] (==5.0.4)", "cryptography (>=3.4.0)", "pre-commit", "pytest (>=6.0.0,<7.0.0)", "sphinx (>=4.5.0,<5.0.0)", "sphinx-rtd-theme", -"zope.interface"] +dev = ["coverage[toml] (==5.0.4)", "cryptography (>=3.4.0)", "pre-commit", "pytest (>=6.0.0,<7.0.0)", "sphinx (>=4.5.0,<5.0.0)", "sphinx-rtd-theme", "zope.interface"] docs = ["sphinx (>=4.5.0,<5.0.0)", "sphinx-rtd-theme", "zope.interface"] tests = ["coverage[toml] (==5.0.4)", "pytest (>=6.0.0,<7.0.0)"] @@ -2345,7 +1931,6 @@ tests = ["coverage[toml] (==5.0.4)", "pytest (>=6.0.0,<7.0.0)"] name = "pyparsing" version = "3.0.9" description = "pyparsing module - Classes and methods to define and execute parsing grammars" -category = "main" optional = false python-versions = ">=3.6.8" files = [ @@ -2360,7 +1945,6 @@ diagrams = ["jinja2", "railroad-diagrams"] name = "pyproject-hooks" version = "1.0.0" description = "Wrappers to call pyproject.toml-based build backend hooks." -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2375,45 +1959,32 @@ tomli = {version = ">=1.1.0", markers = "python_version < \"3.11\""} name = "pyrsistent" version = "0.19.3" description = "Persistent/Functional/Immutable data structures" -category = "main" optional = true python-versions = ">=3.7" files = [ - {file = "pyrsistent-0.19.3-cp310-cp310-macosx_10_9_universal2.whl", hash = -"sha256:20460ac0ea439a3e79caa1dbd560344b64ed75e85d8703943e0b66c2a6150e4a"}, - {file = "pyrsistent-0.19.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:4c18264cb84b5e68e7085a43723f9e4c1fd1d935ab240ce02c0324a8e01ccb64"}, - {file = "pyrsistent-0.19.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:4b774f9288dda8d425adb6544e5903f1fb6c273ab3128a355c6b972b7df39dcf"}, + {file = "pyrsistent-0.19.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:20460ac0ea439a3e79caa1dbd560344b64ed75e85d8703943e0b66c2a6150e4a"}, + {file = "pyrsistent-0.19.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4c18264cb84b5e68e7085a43723f9e4c1fd1d935ab240ce02c0324a8e01ccb64"}, + {file = "pyrsistent-0.19.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4b774f9288dda8d425adb6544e5903f1fb6c273ab3128a355c6b972b7df39dcf"}, {file = "pyrsistent-0.19.3-cp310-cp310-win32.whl", hash = "sha256:5a474fb80f5e0d6c9394d8db0fc19e90fa540b82ee52dba7d246a7791712f74a"}, {file = "pyrsistent-0.19.3-cp310-cp310-win_amd64.whl", hash = "sha256:49c32f216c17148695ca0e02a5c521e28a4ee6c5089f97e34fe24163113722da"}, - {file = "pyrsistent-0.19.3-cp311-cp311-macosx_10_9_universal2.whl", hash = -"sha256:f0774bf48631f3a20471dd7c5989657b639fd2d285b861237ea9e82c36a415a9"}, - {file = "pyrsistent-0.19.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:3ab2204234c0ecd8b9368dbd6a53e83c3d4f3cab10ecaf6d0e772f456c442393"}, - {file = "pyrsistent-0.19.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:e42296a09e83028b3476f7073fcb69ffebac0e66dbbfd1bd847d61f74db30f19"}, + {file = "pyrsistent-0.19.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:f0774bf48631f3a20471dd7c5989657b639fd2d285b861237ea9e82c36a415a9"}, + {file = "pyrsistent-0.19.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3ab2204234c0ecd8b9368dbd6a53e83c3d4f3cab10ecaf6d0e772f456c442393"}, + {file = "pyrsistent-0.19.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e42296a09e83028b3476f7073fcb69ffebac0e66dbbfd1bd847d61f74db30f19"}, {file = "pyrsistent-0.19.3-cp311-cp311-win32.whl", hash = "sha256:64220c429e42a7150f4bfd280f6f4bb2850f95956bde93c6fda1b70507af6ef3"}, {file = "pyrsistent-0.19.3-cp311-cp311-win_amd64.whl", hash = "sha256:016ad1afadf318eb7911baa24b049909f7f3bb2c5b1ed7b6a8f21db21ea3faa8"}, {file = "pyrsistent-0.19.3-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:c4db1bd596fefd66b296a3d5d943c94f4fac5bcd13e99bffe2ba6a759d959a28"}, - {file = "pyrsistent-0.19.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:aeda827381f5e5d65cced3024126529ddc4289d944f75e090572c77ceb19adbf"}, - {file = "pyrsistent-0.19.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:42ac0b2f44607eb92ae88609eda931a4f0dfa03038c44c772e07f43e738bcac9"}, + {file = "pyrsistent-0.19.3-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aeda827381f5e5d65cced3024126529ddc4289d944f75e090572c77ceb19adbf"}, + {file = "pyrsistent-0.19.3-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:42ac0b2f44607eb92ae88609eda931a4f0dfa03038c44c772e07f43e738bcac9"}, {file = "pyrsistent-0.19.3-cp37-cp37m-win32.whl", hash = "sha256:e8f2b814a3dc6225964fa03d8582c6e0b6650d68a232df41e3cc1b66a5d2f8d1"}, {file = "pyrsistent-0.19.3-cp37-cp37m-win_amd64.whl", hash = "sha256:c9bb60a40a0ab9aba40a59f68214eed5a29c6274c83b2cc206a359c4a89fa41b"}, {file = "pyrsistent-0.19.3-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:a2471f3f8693101975b1ff85ffd19bb7ca7dd7c38f8a81701f67d6b4f97b87d8"}, - {file = "pyrsistent-0.19.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:cc5d149f31706762c1f8bda2e8c4f8fead6e80312e3692619a75301d3dbb819a"}, - {file = "pyrsistent-0.19.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:3311cb4237a341aa52ab8448c27e3a9931e2ee09561ad150ba94e4cfd3fc888c"}, + {file = "pyrsistent-0.19.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cc5d149f31706762c1f8bda2e8c4f8fead6e80312e3692619a75301d3dbb819a"}, + {file = "pyrsistent-0.19.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3311cb4237a341aa52ab8448c27e3a9931e2ee09561ad150ba94e4cfd3fc888c"}, {file = "pyrsistent-0.19.3-cp38-cp38-win32.whl", hash = "sha256:f0e7c4b2f77593871e918be000b96c8107da48444d57005b6a6bc61fb4331b2c"}, {file = "pyrsistent-0.19.3-cp38-cp38-win_amd64.whl", hash = "sha256:c147257a92374fde8498491f53ffa8f4822cd70c0d85037e09028e478cababb7"}, {file = "pyrsistent-0.19.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:b735e538f74ec31378f5a1e3886a26d2ca6351106b4dfde376a26fc32a044edc"}, - {file = "pyrsistent-0.19.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:99abb85579e2165bd8522f0c0138864da97847875ecbd45f3e7e2af569bfc6f2"}, - {file = "pyrsistent-0.19.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:3a8cb235fa6d3fd7aae6a4f1429bbb1fec1577d978098da1252f0489937786f3"}, + {file = "pyrsistent-0.19.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:99abb85579e2165bd8522f0c0138864da97847875ecbd45f3e7e2af569bfc6f2"}, + {file = "pyrsistent-0.19.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3a8cb235fa6d3fd7aae6a4f1429bbb1fec1577d978098da1252f0489937786f3"}, {file = "pyrsistent-0.19.3-cp39-cp39-win32.whl", hash = "sha256:c74bed51f9b41c48366a286395c67f4e894374306b197e62810e0fdaf2364da2"}, {file = "pyrsistent-0.19.3-cp39-cp39-win_amd64.whl", hash = "sha256:878433581fc23e906d947a6814336eee031a00e6defba224234169ae3d3d6a98"}, {file = "pyrsistent-0.19.3-py3-none-any.whl", hash = "sha256:ccf0d6bd208f8111179f0c26fdf84ed7c3891982f2edaeae7422575f47e66b64"}, @@ -2424,7 +1995,6 @@ files = [ name = "pytest" version = "7.3.1" description = "pytest: simple powerful testing with Python" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2447,7 +2017,6 @@ testing = ["argcomplete", "attrs (>=19.2.0)", "hypothesis (>=3.56)", "mock", "no name = "pytest-checkdocs" version = "2.9.0" description = "check the README when running tests" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2462,14 +2031,12 @@ importlib-metadata = {version = ">=4", markers = "python_version < \"3.10\""} [package.extras] docs = ["furo", "jaraco.packaging (>=9)", "rst.linker (>=1.9)", "sphinx (>=3.5)"] -testing = ["flake8 (<5)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", -"pytest-mypy (>=0.9.1)", "types-docutils"] +testing = ["flake8 (<5)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)", "types-docutils"] [[package]] name = "python-dateutil" version = "2.8.2" description = "Extensions to the standard Python datetime module" -category = "main" optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,>=2.7" files = [ @@ -2484,88 +2051,56 @@ six = ">=1.5" name = "python-snappy" version = "0.6.1" description = "Python library for the snappy compression library from Google" -category = "main" optional = true python-versions = "*" files = [ {file = "python-snappy-0.6.1.tar.gz", hash = "sha256:b6a107ab06206acc5359d4c5632bd9b22d448702a79b3169b0c62e0fb808bb2a"}, - {file = "python_snappy-0.6.1-cp310-cp310-macosx_10_9_universal2.whl", hash = -"sha256:b7f920eaf46ebf41bd26f9df51c160d40f9e00b7b48471c3438cb8d027f7fb9b"}, + {file = "python_snappy-0.6.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:b7f920eaf46ebf41bd26f9df51c160d40f9e00b7b48471c3438cb8d027f7fb9b"}, {file = "python_snappy-0.6.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:4ec533a8c1f8df797bded662ec3e494d225b37855bb63eb0d75464a07947477c"}, - {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:6f8bf4708a11b47517baf962f9a02196478bbb10fdb9582add4aa1459fa82380"}, - {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:8d0c019ee7dcf2c60e240877107cddbd95a5b1081787579bf179938392d66480"}, - {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:cb18d9cd7b3f35a2f5af47bb8ed6a5bdbf4f3ddee37f3daade4ab7864c292f5b"}, - {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:b265cde49774752aec9ca7f5d272e3f98718164afc85521622a8a5394158a2b5"}, - {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:d017775851a778ec9cc32651c4464079d06d927303c2dde9ae9830ccf6fe94e1"}, + {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:6f8bf4708a11b47517baf962f9a02196478bbb10fdb9582add4aa1459fa82380"}, + {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:8d0c019ee7dcf2c60e240877107cddbd95a5b1081787579bf179938392d66480"}, + {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cb18d9cd7b3f35a2f5af47bb8ed6a5bdbf4f3ddee37f3daade4ab7864c292f5b"}, + {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b265cde49774752aec9ca7f5d272e3f98718164afc85521622a8a5394158a2b5"}, + {file = "python_snappy-0.6.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d017775851a778ec9cc32651c4464079d06d927303c2dde9ae9830ccf6fe94e1"}, {file = "python_snappy-0.6.1-cp310-cp310-win32.whl", hash = "sha256:8277d1f6282463c40761f802b742f833f9f2449fcdbb20a96579aa05c8feb614"}, {file = "python_snappy-0.6.1-cp310-cp310-win_amd64.whl", hash = "sha256:2aaaf618c68d8c9daebc23a20436bd01b09ee70d7fbf7072b7f38b06d2fab539"}, {file = "python_snappy-0.6.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:277757d5dad4e239dc1417438a0871b65b1b155beb108888e7438c27ffc6a8cc"}, - {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:e066a0586833d610c4bbddba0be5ba0e3e4f8e0bc5bb6d82103d8f8fc47bb59a"}, - {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:0d489b50f49433494160c45048fe806de6b3aeab0586e497ebd22a0bab56e427"}, - {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:463fd340a499d47b26ca42d2f36a639188738f6e2098c6dbf80aef0e60f461e1"}, - {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:9837ac1650cc68d22a3cf5f15fb62c6964747d16cecc8b22431f113d6e39555d"}, - {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:5e973e637112391f05581f427659c05b30b6843bc522a65be35ac7b18ce3dedd"}, + {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:e066a0586833d610c4bbddba0be5ba0e3e4f8e0bc5bb6d82103d8f8fc47bb59a"}, + {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:0d489b50f49433494160c45048fe806de6b3aeab0586e497ebd22a0bab56e427"}, + {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:463fd340a499d47b26ca42d2f36a639188738f6e2098c6dbf80aef0e60f461e1"}, + {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9837ac1650cc68d22a3cf5f15fb62c6964747d16cecc8b22431f113d6e39555d"}, + {file = "python_snappy-0.6.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5e973e637112391f05581f427659c05b30b6843bc522a65be35ac7b18ce3dedd"}, {file = "python_snappy-0.6.1-cp36-cp36m-win32.whl", hash = "sha256:c20498bd712b6e31a4402e1d027a1cd64f6a4a0066a3fe3c7344475886d07fdf"}, {file = "python_snappy-0.6.1-cp36-cp36m-win_amd64.whl", hash = "sha256:59e975be4206cc54d0a112ef72fa3970a57c2b1bcc2c97ed41d6df0ebe518228"}, {file = "python_snappy-0.6.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:2a7e528ab6e09c0d67dcb61a1730a292683e5ff9bb088950638d3170cf2a0a54"}, - {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:39692bedbe0b717001a99915ac0eb2d9d0bad546440d392a2042b96d813eede1"}, - {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:6a7620404da966f637b9ce8d4d3d543d363223f7a12452a575189c5355fc2d25"}, - {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:7778c224efc38a40d274da4eb82a04cac27aae20012372a7db3c4bbd8926c4d4"}, - {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:1d029f7051ec1bbeaa3e03030b6d8ed47ceb69cae9016f493c802a08af54e026"}, - {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:a0ad38bc98d0b0497a0b0dbc29409bcabfcecff4511ed7063403c86de16927bc"}, + {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:39692bedbe0b717001a99915ac0eb2d9d0bad546440d392a2042b96d813eede1"}, + {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:6a7620404da966f637b9ce8d4d3d543d363223f7a12452a575189c5355fc2d25"}, + {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7778c224efc38a40d274da4eb82a04cac27aae20012372a7db3c4bbd8926c4d4"}, + {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1d029f7051ec1bbeaa3e03030b6d8ed47ceb69cae9016f493c802a08af54e026"}, + {file = "python_snappy-0.6.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a0ad38bc98d0b0497a0b0dbc29409bcabfcecff4511ed7063403c86de16927bc"}, {file = "python_snappy-0.6.1-cp37-cp37m-win32.whl", hash = "sha256:5a453c45178d7864c1bdd6bfe0ee3ed2883f63b9ba2c9bb967c6b586bf763f96"}, {file = "python_snappy-0.6.1-cp37-cp37m-win_amd64.whl", hash = "sha256:9f0c0d88b84259f93c3aa46398680646f2c23e43394779758d9f739c34e15295"}, - {file = "python_snappy-0.6.1-cp38-cp38-macosx_10_9_universal2.whl", hash = -"sha256:5bb05c28298803a74add08ba496879242ef159c75bc86a5406fac0ffc7dd021b"}, + {file = "python_snappy-0.6.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:5bb05c28298803a74add08ba496879242ef159c75bc86a5406fac0ffc7dd021b"}, {file = "python_snappy-0.6.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:9eac51307c6a1a38d5f86ebabc26a889fddf20cbba7a116ccb54ba1446601d5b"}, - {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:88b6ea78b83d2796f330b0af1b70cdd3965dbdab02d8ac293260ec2c8fe340ee"}, - {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:8c07220408d3268e8268c9351c5c08041bc6f8c6172e59d398b71020df108541"}, - {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:4038019b1bcaadde726a57430718394076c5a21545ebc5badad2c045a09546cf"}, - {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:dc96668d9c7cc656609764275c5f8da58ef56d89bdd6810f6923d36497468ff7"}, - {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:cf5bb9254e1c38aacf253d510d3d9be631bba21f3d068b17672b38b5cbf2fff5"}, + {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:88b6ea78b83d2796f330b0af1b70cdd3965dbdab02d8ac293260ec2c8fe340ee"}, + {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:8c07220408d3268e8268c9351c5c08041bc6f8c6172e59d398b71020df108541"}, + {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4038019b1bcaadde726a57430718394076c5a21545ebc5badad2c045a09546cf"}, + {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:dc96668d9c7cc656609764275c5f8da58ef56d89bdd6810f6923d36497468ff7"}, + {file = "python_snappy-0.6.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:cf5bb9254e1c38aacf253d510d3d9be631bba21f3d068b17672b38b5cbf2fff5"}, {file = "python_snappy-0.6.1-cp38-cp38-win32.whl", hash = "sha256:eaf905a580f2747c4a474040a5063cd5e0cc3d1d2d6edb65f28196186493ad4a"}, {file = "python_snappy-0.6.1-cp38-cp38-win_amd64.whl", hash = "sha256:546c1a7470ecbf6239101e9aff0f709b68ca0f0268b34d9023019a55baa1f7c6"}, - {file = "python_snappy-0.6.1-cp39-cp39-macosx_10_9_universal2.whl", hash = -"sha256:e3a013895c64352b49d0d8e107a84f99631b16dbab156ded33ebf0becf56c8b2"}, + {file = "python_snappy-0.6.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:e3a013895c64352b49d0d8e107a84f99631b16dbab156ded33ebf0becf56c8b2"}, {file = "python_snappy-0.6.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:3fb9a88a4dd6336488f3de67ce75816d0d796dce53c2c6e4d70e0b565633c7fd"}, - {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:735cd4528c55dbe4516d6d2b403331a99fc304f8feded8ae887cf97b67d589bb"}, - {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:90b0186516b7a101c14764b0c25931b741fb0102f21253eff67847b4742dfc72"}, - {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:1a993dc8aadd901915a510fe6af5f20ae4256f527040066c22a154db8946751f"}, - {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:530bfb9efebcc1aab8bb4ebcbd92b54477eed11f6cf499355e882970a6d3aa7d"}, - {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:5843feb914796b1f0405ccf31ea0fb51034ceb65a7588edfd5a8250cb369e3b2"}, + {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:735cd4528c55dbe4516d6d2b403331a99fc304f8feded8ae887cf97b67d589bb"}, + {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:90b0186516b7a101c14764b0c25931b741fb0102f21253eff67847b4742dfc72"}, + {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1a993dc8aadd901915a510fe6af5f20ae4256f527040066c22a154db8946751f"}, + {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:530bfb9efebcc1aab8bb4ebcbd92b54477eed11f6cf499355e882970a6d3aa7d"}, + {file = "python_snappy-0.6.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5843feb914796b1f0405ccf31ea0fb51034ceb65a7588edfd5a8250cb369e3b2"}, {file = "python_snappy-0.6.1-cp39-cp39-win32.whl", hash = "sha256:66c80e9b366012dbee262bb1869e4fc5ba8786cda85928481528bc4a72ec2ee8"}, {file = "python_snappy-0.6.1-cp39-cp39-win_amd64.whl", hash = "sha256:4d3cafdf454354a621c8ab7408e45aa4e9d5c0b943b61ff4815f71ca6bdf0130"}, - {file = "python_snappy-0.6.1-pp37-pypy37_pp73-manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:586724a0276d7a6083a17259d0b51622e492289a9998848a1b01b6441ca12b2f"}, - {file = "python_snappy-0.6.1-pp37-pypy37_pp73-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:2be4f4550acd484912441f5f1209ba611ac399aac9355fee73611b9a0d4f949c"}, - {file = "python_snappy-0.6.1-pp37-pypy37_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:0bdb6942180660bda7f7d01f4c0def3cfc72b1c6d99aad964801775a3e379aba"}, + {file = "python_snappy-0.6.1-pp37-pypy37_pp73-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:586724a0276d7a6083a17259d0b51622e492289a9998848a1b01b6441ca12b2f"}, + {file = "python_snappy-0.6.1-pp37-pypy37_pp73-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:2be4f4550acd484912441f5f1209ba611ac399aac9355fee73611b9a0d4f949c"}, + {file = "python_snappy-0.6.1-pp37-pypy37_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0bdb6942180660bda7f7d01f4c0def3cfc72b1c6d99aad964801775a3e379aba"}, {file = "python_snappy-0.6.1-pp37-pypy37_pp73-win_amd64.whl", hash = "sha256:03bb511380fca2a13325b6f16fe8234c8e12da9660f0258cd45d9a02ffc916af"}, ] @@ -2573,7 +2108,6 @@ files = [ name = "pytz" version = "2023.3" description = "World timezone definitions, modern and historical" -category = "main" optional = true python-versions = "*" files = [ @@ -2585,7 +2119,6 @@ files = [ name = "pywin32" version = "306" description = "Python for Window Extensions" -category = "main" optional = true python-versions = "*" files = [ @@ -2609,65 +2142,46 @@ files = [ name = "pyyaml" version = "6.0" description = "YAML parser and emitter for Python" -category = "main" optional = false python-versions = ">=3.6" files = [ {file = "PyYAML-6.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4db7c7aef085872ef65a8fd7d6d09a14ae91f691dec3e87ee5ee0539d516f53"}, {file = "PyYAML-6.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9df7ed3b3d2e0ecfe09e14741b857df43adb5a3ddadc919a2d94fbdf78fea53c"}, - {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:77f396e6ef4c73fdc33a9157446466f1cff553d979bd00ecb64385760c6babdc"}, - {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:a80a78046a72361de73f8f395f1f1e49f956c6be882eed58505a15f3e430962b"}, - {file = "PyYAML-6.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:f84fbc98b019fef2ee9a1cb3ce93e3187a6df0b2538a651bfb890254ba9f90b5"}, + {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:77f396e6ef4c73fdc33a9157446466f1cff553d979bd00ecb64385760c6babdc"}, + {file = "PyYAML-6.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a80a78046a72361de73f8f395f1f1e49f956c6be882eed58505a15f3e430962b"}, + {file = "PyYAML-6.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f84fbc98b019fef2ee9a1cb3ce93e3187a6df0b2538a651bfb890254ba9f90b5"}, {file = "PyYAML-6.0-cp310-cp310-win32.whl", hash = "sha256:2cd5df3de48857ed0544b34e2d40e9fac445930039f3cfe4bcc592a1f836d513"}, {file = "PyYAML-6.0-cp310-cp310-win_amd64.whl", hash = "sha256:daf496c58a8c52083df09b80c860005194014c3698698d1a57cbcfa182142a3a"}, {file = "PyYAML-6.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:d4b0ba9512519522b118090257be113b9468d804b19d63c71dbcf4a48fa32358"}, {file = "PyYAML-6.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:81957921f441d50af23654aa6c5e5eaf9b06aba7f0a19c18a538dc7ef291c5a1"}, - {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:afa17f5bc4d1b10afd4466fd3a44dc0e245382deca5b3c353d8b757f9e3ecb8d"}, - {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:dbad0e9d368bb989f4515da330b88a057617d16b6a8245084f1b05400f24609f"}, - {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:432557aa2c09802be39460360ddffd48156e30721f5e8d917f01d31694216782"}, + {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:afa17f5bc4d1b10afd4466fd3a44dc0e245382deca5b3c353d8b757f9e3ecb8d"}, + {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dbad0e9d368bb989f4515da330b88a057617d16b6a8245084f1b05400f24609f"}, + {file = "PyYAML-6.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:432557aa2c09802be39460360ddffd48156e30721f5e8d917f01d31694216782"}, {file = "PyYAML-6.0-cp311-cp311-win32.whl", hash = "sha256:bfaef573a63ba8923503d27530362590ff4f576c626d86a9fed95822a8255fd7"}, {file = "PyYAML-6.0-cp311-cp311-win_amd64.whl", hash = "sha256:01b45c0191e6d66c470b6cf1b9531a771a83c1c4208272ead47a3ae4f2f603bf"}, {file = "PyYAML-6.0-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:897b80890765f037df3403d22bab41627ca8811ae55e9a722fd0392850ec4d86"}, - {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:50602afada6d6cbfad699b0c7bb50d5ccffa7e46a3d738092afddc1f9758427f"}, - {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:48c346915c114f5fdb3ead70312bd042a953a8ce5c7106d5bfb1a5254e47da92"}, - {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:98c4d36e99714e55cfbaaee6dd5badbc9a1ec339ebfc3b1f52e293aee6bb71a4"}, + {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:50602afada6d6cbfad699b0c7bb50d5ccffa7e46a3d738092afddc1f9758427f"}, + {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:48c346915c114f5fdb3ead70312bd042a953a8ce5c7106d5bfb1a5254e47da92"}, + {file = "PyYAML-6.0-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:98c4d36e99714e55cfbaaee6dd5badbc9a1ec339ebfc3b1f52e293aee6bb71a4"}, {file = "PyYAML-6.0-cp36-cp36m-win32.whl", hash = "sha256:0283c35a6a9fbf047493e3a0ce8d79ef5030852c51e9d911a27badfde0605293"}, {file = "PyYAML-6.0-cp36-cp36m-win_amd64.whl", hash = "sha256:07751360502caac1c067a8132d150cf3d61339af5691fe9e87803040dbc5db57"}, {file = "PyYAML-6.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:819b3830a1543db06c4d4b865e70ded25be52a2e0631ccd2f6a47a2822f2fd7c"}, - {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:473f9edb243cb1935ab5a084eb238d842fb8f404ed2193a915d1784b5a6b5fc0"}, - {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:0ce82d761c532fe4ec3f87fc45688bdd3a4c1dc5e0b4a19814b9009a29baefd4"}, - {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:231710d57adfd809ef5d34183b8ed1eeae3f76459c18fb4a0b373ad56bedcdd9"}, + {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:473f9edb243cb1935ab5a084eb238d842fb8f404ed2193a915d1784b5a6b5fc0"}, + {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0ce82d761c532fe4ec3f87fc45688bdd3a4c1dc5e0b4a19814b9009a29baefd4"}, + {file = "PyYAML-6.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:231710d57adfd809ef5d34183b8ed1eeae3f76459c18fb4a0b373ad56bedcdd9"}, {file = "PyYAML-6.0-cp37-cp37m-win32.whl", hash = "sha256:c5687b8d43cf58545ade1fe3e055f70eac7a5a1a0bf42824308d868289a95737"}, {file = "PyYAML-6.0-cp37-cp37m-win_amd64.whl", hash = "sha256:d15a181d1ecd0d4270dc32edb46f7cb7733c7c508857278d3d378d14d606db2d"}, {file = "PyYAML-6.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0b4624f379dab24d3725ffde76559cff63d9ec94e1736b556dacdfebe5ab6d4b"}, - {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:213c60cd50106436cc818accf5baa1aba61c0189ff610f64f4a3e8c6726218ba"}, - {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:9fa600030013c4de8165339db93d182b9431076eb98eb40ee068700c9c813e34"}, - {file = "PyYAML-6.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:277a0ef2981ca40581a47093e9e2d13b3f1fbbeffae064c1d21bfceba2030287"}, + {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:213c60cd50106436cc818accf5baa1aba61c0189ff610f64f4a3e8c6726218ba"}, + {file = "PyYAML-6.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9fa600030013c4de8165339db93d182b9431076eb98eb40ee068700c9c813e34"}, + {file = "PyYAML-6.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:277a0ef2981ca40581a47093e9e2d13b3f1fbbeffae064c1d21bfceba2030287"}, {file = "PyYAML-6.0-cp38-cp38-win32.whl", hash = "sha256:d4eccecf9adf6fbcc6861a38015c2a64f38b9d94838ac1810a9023a0609e1b78"}, {file = "PyYAML-6.0-cp38-cp38-win_amd64.whl", hash = "sha256:1e4747bc279b4f613a09eb64bba2ba602d8a6664c6ce6396a4d0cd413a50ce07"}, {file = "PyYAML-6.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:055d937d65826939cb044fc8c9b08889e8c743fdc6a32b33e2390f66013e449b"}, {file = "PyYAML-6.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:e61ceaab6f49fb8bdfaa0f92c4b57bcfbea54c09277b1b4f7ac376bfb7a7c174"}, - {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:d67d839ede4ed1b28a4e8909735fc992a923cdb84e618544973d7dfc71540803"}, - {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:cba8c411ef271aa037d7357a2bc8f9ee8b58b9965831d9e51baf703280dc73d3"}, - {file = "PyYAML-6.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:40527857252b61eacd1d9af500c3337ba8deb8fc298940291486c465c8b46ec0"}, + {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d67d839ede4ed1b28a4e8909735fc992a923cdb84e618544973d7dfc71540803"}, + {file = "PyYAML-6.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:cba8c411ef271aa037d7357a2bc8f9ee8b58b9965831d9e51baf703280dc73d3"}, + {file = "PyYAML-6.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:40527857252b61eacd1d9af500c3337ba8deb8fc298940291486c465c8b46ec0"}, {file = "PyYAML-6.0-cp39-cp39-win32.whl", hash = "sha256:b5b9eccad747aabaaffbc6064800670f0c297e52c12754eb1d976c57e4f74dcb"}, {file = "PyYAML-6.0-cp39-cp39-win_amd64.whl", hash = "sha256:b3d267842bf12586ba6c734f89d1f5b871df0273157918b0ccefa29deb05c21c"}, {file = "PyYAML-6.0.tar.gz", hash = "sha256:68fb519c14306fec9720a2a5b45bc9f0c8d1b9c72adf45c37baedfcd949c35a2"}, @@ -2677,7 +2191,6 @@ files = [ name = "ray" version = "2.4.0" description = "Ray provides a simple, universal API for building distributed applications." -category = "main" optional = true python-versions = "*" files = [ @@ -2732,23 +2245,15 @@ requests = "*" virtualenv = ">=20.0.24,<20.21.1" [package.extras] -air = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "fsspec", "gpustat (>=1.0.0)", "numpy (>=1.20)", "opencensus", "pandas", -"pandas (>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic", "requests", "smart-open", "starlette", "tabulate", -"tensorboardX (>=1.9)", "uvicorn"] -all = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "dm-tree", "fastapi", "fsspec", "gpustat (>=1.0.0)", "gymnasium (==0.26.3)", -"kubernetes", "lz4", "numpy (>=1.20)", "opencensus", "opentelemetry-api", "opentelemetry-exporter-otlp", "opentelemetry-sdk", "pandas", "pandas -(>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic", "pyyaml", "ray-cpp (==2.4.0)", "requests", "rich", -"scikit-image", "scipy", "smart-open", "starlette", "tabulate", "tensorboardX (>=1.9)", "typer", "urllib3", "uvicorn"] +air = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "fsspec", "gpustat (>=1.0.0)", "numpy (>=1.20)", "opencensus", "pandas", "pandas (>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic", "requests", "smart-open", "starlette", "tabulate", "tensorboardX (>=1.9)", "uvicorn"] +all = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "dm-tree", "fastapi", "fsspec", "gpustat (>=1.0.0)", "gymnasium (==0.26.3)", "kubernetes", "lz4", "numpy (>=1.20)", "opencensus", "opentelemetry-api", "opentelemetry-exporter-otlp", "opentelemetry-sdk", "pandas", "pandas (>=1.3)", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pyarrow (>=6.0.1)", "pydantic", "pyyaml", "ray-cpp (==2.4.0)", "requests", "rich", "scikit-image", "scipy", "smart-open", "starlette", "tabulate", "tensorboardX (>=1.9)", "typer", "urllib3", "uvicorn"] cpp = ["ray-cpp (==2.4.0)"] data = ["fsspec", "numpy (>=1.20)", "pandas (>=1.3)", "pyarrow (>=6.0.1)"] -default = ["aiohttp (>=3.7)", "aiohttp-cors", "colorful", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", -"pydantic", "requests", "smart-open"] +default = ["aiohttp (>=3.7)", "aiohttp-cors", "colorful", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pydantic", "requests", "smart-open"] k8s = ["kubernetes", "urllib3"] observability = ["opentelemetry-api", "opentelemetry-exporter-otlp", "opentelemetry-sdk"] -rllib = ["dm-tree", "gymnasium (==0.26.3)", "lz4", "pandas", "pyyaml", "requests", "rich", "scikit-image", "scipy", "tabulate", "tensorboardX (>=1.9)", -"typer"] -serve = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", -"py-spy (>=0.2.0)", "pydantic", "requests", "smart-open", "starlette", "uvicorn"] +rllib = ["dm-tree", "gymnasium (==0.26.3)", "lz4", "pandas", "pyyaml", "requests", "rich", "scikit-image", "scipy", "tabulate", "tensorboardX (>=1.9)", "typer"] +serve = ["aiohttp (>=3.7)", "aiohttp-cors", "aiorwlock", "colorful", "fastapi", "gpustat (>=1.0.0)", "opencensus", "prometheus-client (>=0.7.1)", "py-spy (>=0.2.0)", "pydantic", "requests", "smart-open", "starlette", "uvicorn"] train = ["pandas", "requests", "tabulate", "tensorboardX (>=1.9)"] tune = ["pandas", "requests", "tabulate", "tensorboardX (>=1.9)"] @@ -2756,7 +2261,6 @@ tune = ["pandas", "requests", "tabulate", "tensorboardX (>=1.9)"] name = "requests" version = "2.31.0" description = "Python HTTP for Humans." -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -2778,7 +2282,6 @@ use-chardet-on-py3 = ["chardet (>=3.0.2,<6)"] name = "requests-mock" version = "1.10.0" description = "Mock out responses from the requests package" -category = "dev" optional = false python-versions = "*" files = [ @@ -2798,7 +2301,6 @@ test = ["fixtures", "mock", "purl", "pytest", "requests-futures", "sphinx", "tes name = "responses" version = "0.23.1" description = "A utility library for mocking out the `requests` Python library." -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2813,14 +2315,12 @@ types-PyYAML = "*" urllib3 = ">=1.25.10" [package.extras] -tests = ["coverage (>=6.0.0)", "flake8", "mypy", "pytest (>=7.0.0)", "pytest-asyncio", "pytest-cov", "pytest-httpserver", "tomli", "tomli-w", -"types-requests"] +tests = ["coverage (>=6.0.0)", "flake8", "mypy", "pytest (>=7.0.0)", "pytest-asyncio", "pytest-cov", "pytest-httpserver", "tomli", "tomli-w", "types-requests"] [[package]] name = "rich" version = "13.4.1" description = "Render rich text, tables, progress bars, syntax highlighting, markdown and more to the terminal" -category = "main" optional = false python-versions = ">=3.7.0" files = [ @@ -2840,7 +2340,6 @@ jupyter = ["ipywidgets (>=7.5.1,<9)"] name = "s3fs" version = "2023.5.0" description = "Convenient Filesystem interface over S3" -category = "main" optional = true python-versions = ">= 3.8" files = [ @@ -2861,7 +2360,6 @@ boto3 = ["aiobotocore[boto3] (>=2.5.0,<2.6.0)"] name = "s3transfer" version = "0.6.1" description = "An Amazon S3 Transfer Manager" -category = "main" optional = false python-versions = ">= 3.7" files = [ @@ -2879,7 +2377,6 @@ crt = ["botocore[crt] (>=1.20.29,<2.0a.0)"] name = "setuptools" version = "67.8.0" description = "Easily download, build, install, upgrade, and uninstall Python packages" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2888,20 +2385,14 @@ files = [ ] [package.extras] -docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "rst.linker (>=1.9)", "sphinx (>=3.5)", -"sphinx-favicon", "sphinx-hoverxref (<2)", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (==0.8.3)", "sphinx-reredirects", -"sphinxcontrib-towncrier"] -testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip -(>=19.1)", "pip-run (>=8.8)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-mypy -(>=0.9.1)", "pytest-perf", "pytest-ruff", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] -testing-integration = ["build[virtualenv]", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pytest", "pytest-enabler", -"pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"] +docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-hoverxref (<2)", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (==0.8.3)", "sphinx-reredirects", "sphinxcontrib-towncrier"] +testing = ["build[virtualenv]", "filelock (>=3.4.0)", "flake8-2020", "ini2toml[lite] (>=0.9)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pip (>=19.1)", "pip-run (>=8.8)", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-mypy (>=0.9.1)", "pytest-perf", "pytest-ruff", "pytest-timeout", "pytest-xdist", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel"] +testing-integration = ["build[virtualenv]", "filelock (>=3.4.0)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "pytest", "pytest-enabler", "pytest-xdist", "tomli", "virtualenv (>=13.0.0)", "wheel"] [[package]] name = "six" version = "1.16.0" description = "Python 2 and 3 compatibility utilities" -category = "main" optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*" files = [ @@ -2909,11 +2400,21 @@ files = [ {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"}, ] +[[package]] +name = "sortedcontainers" +version = "2.4.0" +description = "Sorted Containers -- Sorted List, Sorted Dict, Sorted Set" +optional = false +python-versions = "*" +files = [ + {file = "sortedcontainers-2.4.0-py2.py3-none-any.whl", hash = "sha256:a163dcaede0f1c021485e957a39245190e74249897e2ae4b2aa38595db237ee0"}, + {file = "sortedcontainers-2.4.0.tar.gz", hash = "sha256:25caa5a06cc30b6b83d11423433f65d1f9d76c4c6a0c90e3379eaa43b9bfdb88"}, +] + [[package]] name = "strictyaml" version = "1.7.3" description = "Strict, typed YAML parser" -category = "main" optional = false python-versions = ">=3.7.0" files = [ @@ -2928,7 +2429,6 @@ python-dateutil = ">=2.6.0" name = "thrift" version = "0.16.0" description = "Python bindings for the Apache Thrift RPC system" -category = "main" optional = true python-versions = "*" files = [ @@ -2947,7 +2447,6 @@ twisted = ["twisted"] name = "tomli" version = "2.0.1" description = "A lil' TOML parser" -category = "dev" optional = false python-versions = ">=3.7" files = [ @@ -2959,7 +2458,6 @@ files = [ name = "types-pyyaml" version = "6.0.12.10" description = "Typing stubs for PyYAML" -category = "dev" optional = false python-versions = "*" files = [ @@ -2971,7 +2469,6 @@ files = [ name = "typing-extensions" version = "4.6.3" description = "Backported and Experimental Type Hints for Python 3.7+" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -2983,7 +2480,6 @@ files = [ name = "tzdata" version = "2023.3" description = "Provider of IANA time zone data" -category = "main" optional = true python-versions = ">=2" files = [ @@ -2995,7 +2491,6 @@ files = [ name = "urllib3" version = "1.26.16" description = "HTTP library with thread-safe connection pooling, file post, and more." -category = "main" optional = false python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*" files = [ @@ -3012,7 +2507,6 @@ socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"] name = "virtualenv" version = "20.21.0" description = "Virtual Python Environment builder" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -3026,16 +2520,13 @@ filelock = ">=3.4.1,<4" platformdirs = ">=2.4,<4" [package.extras] -docs = ["furo (>=2022.12.7)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier -(>=22.12)"] -test = ["covdefaults (>=2.2.2)", "coverage (>=7.1)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23)", "pytest (>=7.2.1)", -"pytest-env (>=0.8.1)", "pytest-freezegun (>=0.4.2)", "pytest-mock (>=3.10)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)"] +docs = ["furo (>=2022.12.7)", "proselint (>=0.13)", "sphinx (>=6.1.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=22.12)"] +test = ["covdefaults (>=2.2.2)", "coverage (>=7.1)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23)", "pytest (>=7.2.1)", "pytest-env (>=0.8.1)", "pytest-freezegun (>=0.4.2)", "pytest-mock (>=3.10)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)"] [[package]] name = "werkzeug" version = "2.3.4" description = "The comprehensive WSGI web application library." -category = "dev" optional = false python-versions = ">=3.8" files = [ @@ -3053,7 +2544,6 @@ watchdog = ["watchdog (>=2.3)"] name = "wrapt" version = "1.15.0" description = "Module for decorators, wrappers and monkey patching." -category = "main" optional = true python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,>=2.7" files = [ @@ -3068,12 +2558,9 @@ files = [ {file = "wrapt-1.15.0-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:58d7a75d731e8c63614222bcb21dd992b4ab01a399f1f09dd82af17bbfc2368a"}, {file = "wrapt-1.15.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:21f6d9a0d5b3a207cdf7acf8e58d7d13d463e639f0c7e01d82cdb671e6cb7923"}, {file = "wrapt-1.15.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:ce42618f67741d4697684e501ef02f29e758a123aa2d669e2d964ff734ee00ee"}, - {file = "wrapt-1.15.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:41d07d029dd4157ae27beab04d22b8e261eddfc6ecd64ff7000b10dc8b3a5727"}, - {file = "wrapt-1.15.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:54accd4b8bc202966bafafd16e69da9d5640ff92389d33d28555c5fd4f25ccb7"}, - {file = "wrapt-1.15.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:2fbfbca668dd15b744418265a9607baa970c347eefd0db6a518aaf0cfbd153c0"}, + {file = "wrapt-1.15.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:41d07d029dd4157ae27beab04d22b8e261eddfc6ecd64ff7000b10dc8b3a5727"}, + {file = "wrapt-1.15.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:54accd4b8bc202966bafafd16e69da9d5640ff92389d33d28555c5fd4f25ccb7"}, + {file = "wrapt-1.15.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2fbfbca668dd15b744418265a9607baa970c347eefd0db6a518aaf0cfbd153c0"}, {file = "wrapt-1.15.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:76e9c727a874b4856d11a32fb0b389afc61ce8aaf281ada613713ddeadd1cfec"}, {file = "wrapt-1.15.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:e20076a211cd6f9b44a6be58f7eeafa7ab5720eb796975d0c03f05b47d89eb90"}, {file = "wrapt-1.15.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:a74d56552ddbde46c246b5b89199cb3fd182f9c346c784e1a93e4dc3f5ec9975"}, @@ -3081,12 +2568,9 @@ files = [ {file = "wrapt-1.15.0-cp310-cp310-win_amd64.whl", hash = "sha256:75760a47c06b5974aa5e01949bf7e66d2af4d08cb8c1d6516af5e39595397f5e"}, {file = "wrapt-1.15.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ba1711cda2d30634a7e452fc79eabcadaffedf241ff206db2ee93dd2c89a60e7"}, {file = "wrapt-1.15.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:56374914b132c702aa9aa9959c550004b8847148f95e1b824772d453ac204a72"}, - {file = "wrapt-1.15.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:a89ce3fd220ff144bd9d54da333ec0de0399b52c9ac3d2ce34b569cf1a5748fb"}, - {file = "wrapt-1.15.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:3bbe623731d03b186b3d6b0d6f51865bf598587c38d6f7b0be2e27414f7f214e"}, - {file = "wrapt-1.15.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:3abbe948c3cbde2689370a262a8d04e32ec2dd4f27103669a45c6929bcdbfe7c"}, + {file = "wrapt-1.15.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a89ce3fd220ff144bd9d54da333ec0de0399b52c9ac3d2ce34b569cf1a5748fb"}, + {file = "wrapt-1.15.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3bbe623731d03b186b3d6b0d6f51865bf598587c38d6f7b0be2e27414f7f214e"}, + {file = "wrapt-1.15.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3abbe948c3cbde2689370a262a8d04e32ec2dd4f27103669a45c6929bcdbfe7c"}, {file = "wrapt-1.15.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:b67b819628e3b748fd3c2192c15fb951f549d0f47c0449af0764d7647302fda3"}, {file = "wrapt-1.15.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:7eebcdbe3677e58dd4c0e03b4f2cfa346ed4049687d839adad68cc38bb559c92"}, {file = "wrapt-1.15.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:74934ebd71950e3db69960a7da29204f89624dde411afbfb3b4858c1409b1e98"}, @@ -3099,24 +2583,18 @@ files = [ {file = "wrapt-1.15.0-cp35-cp35m-win32.whl", hash = "sha256:fbec11614dba0424ca72f4e8ba3c420dba07b4a7c206c8c8e4e73f2e98f4c559"}, {file = "wrapt-1.15.0-cp35-cp35m-win_amd64.whl", hash = "sha256:fd69666217b62fa5d7c6aa88e507493a34dec4fa20c5bd925e4bc12fce586639"}, {file = "wrapt-1.15.0-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:b0724f05c396b0a4c36a3226c31648385deb6a65d8992644c12a4963c70326ba"}, - {file = "wrapt-1.15.0-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:bbeccb1aa40ab88cd29e6c7d8585582c99548f55f9b2581dfc5ba68c59a85752"}, - {file = "wrapt-1.15.0-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:38adf7198f8f154502883242f9fe7333ab05a5b02de7d83aa2d88ea621f13364"}, - {file = "wrapt-1.15.0-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:578383d740457fa790fdf85e6d346fda1416a40549fe8db08e5e9bd281c6a475"}, + {file = "wrapt-1.15.0-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bbeccb1aa40ab88cd29e6c7d8585582c99548f55f9b2581dfc5ba68c59a85752"}, + {file = "wrapt-1.15.0-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:38adf7198f8f154502883242f9fe7333ab05a5b02de7d83aa2d88ea621f13364"}, + {file = "wrapt-1.15.0-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:578383d740457fa790fdf85e6d346fda1416a40549fe8db08e5e9bd281c6a475"}, {file = "wrapt-1.15.0-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:a4cbb9ff5795cd66f0066bdf5947f170f5d63a9274f99bdbca02fd973adcf2a8"}, {file = "wrapt-1.15.0-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:af5bd9ccb188f6a5fdda9f1f09d9f4c86cc8a539bd48a0bfdc97723970348418"}, {file = "wrapt-1.15.0-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:b56d5519e470d3f2fe4aa7585f0632b060d532d0696c5bdfb5e8319e1d0f69a2"}, {file = "wrapt-1.15.0-cp36-cp36m-win32.whl", hash = "sha256:77d4c1b881076c3ba173484dfa53d3582c1c8ff1f914c6461ab70c8428b796c1"}, {file = "wrapt-1.15.0-cp36-cp36m-win_amd64.whl", hash = "sha256:077ff0d1f9d9e4ce6476c1a924a3332452c1406e59d90a2cf24aeb29eeac9420"}, {file = "wrapt-1.15.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:5c5aa28df055697d7c37d2099a7bc09f559d5053c3349b1ad0c39000e611d317"}, - {file = "wrapt-1.15.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:3a8564f283394634a7a7054b7983e47dbf39c07712d7b177b37e03f2467a024e"}, - {file = "wrapt-1.15.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:780c82a41dc493b62fc5884fb1d3a3b81106642c5c5c78d6a0d4cbe96d62ba7e"}, - {file = "wrapt-1.15.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:e169e957c33576f47e21864cf3fc9ff47c223a4ebca8960079b8bd36cb014fd0"}, + {file = "wrapt-1.15.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3a8564f283394634a7a7054b7983e47dbf39c07712d7b177b37e03f2467a024e"}, + {file = "wrapt-1.15.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:780c82a41dc493b62fc5884fb1d3a3b81106642c5c5c78d6a0d4cbe96d62ba7e"}, + {file = "wrapt-1.15.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e169e957c33576f47e21864cf3fc9ff47c223a4ebca8960079b8bd36cb014fd0"}, {file = "wrapt-1.15.0-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:b02f21c1e2074943312d03d243ac4388319f2456576b2c6023041c4d57cd7019"}, {file = "wrapt-1.15.0-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:f2e69b3ed24544b0d3dbe2c5c0ba5153ce50dcebb576fdc4696d52aa22db6034"}, {file = "wrapt-1.15.0-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:d787272ed958a05b2c86311d3a4135d3c2aeea4fc655705f074130aa57d71653"}, @@ -3124,12 +2602,9 @@ files = [ {file = "wrapt-1.15.0-cp37-cp37m-win_amd64.whl", hash = "sha256:abd52a09d03adf9c763d706df707c343293d5d106aea53483e0ec8d9e310ad5e"}, {file = "wrapt-1.15.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:cdb4f085756c96a3af04e6eca7f08b1345e94b53af8921b25c72f096e704e145"}, {file = "wrapt-1.15.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:230ae493696a371f1dbffaad3dafbb742a4d27a0afd2b1aecebe52b740167e7f"}, - {file = "wrapt-1.15.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:63424c681923b9f3bfbc5e3205aafe790904053d42ddcc08542181a30a7a51bd"}, - {file = "wrapt-1.15.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:d6bcbfc99f55655c3d93feb7ef3800bd5bbe963a755687cbf1f490a71fb7794b"}, - {file = "wrapt-1.15.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:c99f4309f5145b93eca6e35ac1a988f0dc0a7ccf9ccdcd78d3c0adf57224e62f"}, + {file = "wrapt-1.15.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:63424c681923b9f3bfbc5e3205aafe790904053d42ddcc08542181a30a7a51bd"}, + {file = "wrapt-1.15.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d6bcbfc99f55655c3d93feb7ef3800bd5bbe963a755687cbf1f490a71fb7794b"}, + {file = "wrapt-1.15.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c99f4309f5145b93eca6e35ac1a988f0dc0a7ccf9ccdcd78d3c0adf57224e62f"}, {file = "wrapt-1.15.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:b130fe77361d6771ecf5a219d8e0817d61b236b7d8b37cc045172e574ed219e6"}, {file = "wrapt-1.15.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:96177eb5645b1c6985f5c11d03fc2dbda9ad24ec0f3a46dcce91445747e15094"}, {file = "wrapt-1.15.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:d5fe3e099cf07d0fb5a1e23d399e5d4d1ca3e6dfcbe5c8570ccff3e9208274f7"}, @@ -3137,12 +2612,9 @@ files = [ {file = "wrapt-1.15.0-cp38-cp38-win_amd64.whl", hash = "sha256:b06fa97478a5f478fb05e1980980a7cdf2712015493b44d0c87606c1513ed5b1"}, {file = "wrapt-1.15.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:2e51de54d4fb8fb50d6ee8327f9828306a959ae394d3e01a1ba8b2f937747d86"}, {file = "wrapt-1.15.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0970ddb69bba00670e58955f8019bec4a42d1785db3faa043c33d81de2bf843c"}, - {file = "wrapt-1.15.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:76407ab327158c510f44ded207e2f76b657303e17cb7a572ffe2f5a8a48aa04d"}, - {file = "wrapt-1.15.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:cd525e0e52a5ff16653a3fc9e3dd827981917d34996600bbc34c05d048ca35cc"}, - {file = "wrapt-1.15.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:9d37ac69edc5614b90516807de32d08cb8e7b12260a285ee330955604ed9dd29"}, + {file = "wrapt-1.15.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:76407ab327158c510f44ded207e2f76b657303e17cb7a572ffe2f5a8a48aa04d"}, + {file = "wrapt-1.15.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cd525e0e52a5ff16653a3fc9e3dd827981917d34996600bbc34c05d048ca35cc"}, + {file = "wrapt-1.15.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9d37ac69edc5614b90516807de32d08cb8e7b12260a285ee330955604ed9dd29"}, {file = "wrapt-1.15.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:078e2a1a86544e644a68422f881c48b84fef6d18f8c7a957ffd3f2e0a74a0d4a"}, {file = "wrapt-1.15.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:2cf56d0e237280baed46f0b5316661da892565ff58309d4d2ed7dba763d984b8"}, {file = "wrapt-1.15.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:7dc0713bf81287a00516ef43137273b23ee414fe41a3c14be10dd95ed98a2df9"}, @@ -3156,7 +2628,6 @@ files = [ name = "xmltodict" version = "0.13.0" description = "Makes working with XML feel like you are working with JSON" -category = "dev" optional = false python-versions = ">=3.4" files = [ @@ -3168,23 +2639,17 @@ files = [ name = "yarl" version = "1.9.2" description = "Yet another URL library" -category = "main" optional = true python-versions = ">=3.7" files = [ {file = "yarl-1.9.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:8c2ad583743d16ddbdf6bb14b5cd76bf43b0d0006e918809d5d4ddf7bde8dd82"}, {file = "yarl-1.9.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:82aa6264b36c50acfb2424ad5ca537a2060ab6de158a5bd2a72a032cc75b9eb8"}, {file = "yarl-1.9.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c0c77533b5ed4bcc38e943178ccae29b9bcf48ffd1063f5821192f23a1bd27b9"}, - {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:ee4afac41415d52d53a9833ebae7e32b344be72835bbb589018c9e938045a560"}, - {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:9bf345c3a4f5ba7f766430f97f9cc1320786f19584acc7086491f45524a551ac"}, - {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:2a96c19c52ff442a808c105901d0bdfd2e28575b3d5f82e2f5fd67e20dc5f4ea"}, - {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:891c0e3ec5ec881541f6c5113d8df0315ce5440e244a716b95f2525b7b9f3608"}, - {file = "yarl-1.9.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:c3a53ba34a636a256d767c086ceb111358876e1fb6b50dfc4d3f4951d40133d5"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ee4afac41415d52d53a9833ebae7e32b344be72835bbb589018c9e938045a560"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9bf345c3a4f5ba7f766430f97f9cc1320786f19584acc7086491f45524a551ac"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2a96c19c52ff442a808c105901d0bdfd2e28575b3d5f82e2f5fd67e20dc5f4ea"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:891c0e3ec5ec881541f6c5113d8df0315ce5440e244a716b95f2525b7b9f3608"}, + {file = "yarl-1.9.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c3a53ba34a636a256d767c086ceb111358876e1fb6b50dfc4d3f4951d40133d5"}, {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:566185e8ebc0898b11f8026447eacd02e46226716229cea8db37496c8cdd26e0"}, {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:2b0738fb871812722a0ac2154be1f049c6223b9f6f22eec352996b69775b36d4"}, {file = "yarl-1.9.2-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:32f1d071b3f362c80f1a7d322bfd7b2d11e33d2adf395cc1dd4df36c9c243095"}, @@ -3195,16 +2660,11 @@ files = [ {file = "yarl-1.9.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:646d663eb2232d7909e6601f1a9107e66f9791f290a1b3dc7057818fe44fc2b6"}, {file = "yarl-1.9.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:aff634b15beff8902d1f918012fc2a42e0dbae6f469fce134c8a0dc51ca423bb"}, {file = "yarl-1.9.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a83503934c6273806aed765035716216cc9ab4e0364f7f066227e1aaea90b8d0"}, - {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:b25322201585c69abc7b0e89e72790469f7dad90d26754717f3310bfe30331c2"}, - {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:22a94666751778629f1ec4280b08eb11815783c63f52092a5953faf73be24191"}, - {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:8ec53a0ea2a80c5cd1ab397925f94bff59222aa3cf9c6da938ce05c9ec20428d"}, - {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:159d81f22d7a43e6eabc36d7194cb53f2f15f498dbbfa8edc8a3239350f59fe7"}, - {file = "yarl-1.9.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:832b7e711027c114d79dffb92576acd1bd2decc467dec60e1cac96912602d0e6"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b25322201585c69abc7b0e89e72790469f7dad90d26754717f3310bfe30331c2"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:22a94666751778629f1ec4280b08eb11815783c63f52092a5953faf73be24191"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8ec53a0ea2a80c5cd1ab397925f94bff59222aa3cf9c6da938ce05c9ec20428d"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:159d81f22d7a43e6eabc36d7194cb53f2f15f498dbbfa8edc8a3239350f59fe7"}, + {file = "yarl-1.9.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:832b7e711027c114d79dffb92576acd1bd2decc467dec60e1cac96912602d0e6"}, {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:95d2ecefbcf4e744ea952d073c6922e72ee650ffc79028eb1e320e732898d7e8"}, {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:d4e2c6d555e77b37288eaf45b8f60f0737c9efa3452c6c44626a5455aeb250b9"}, {file = "yarl-1.9.2-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:783185c75c12a017cc345015ea359cc801c3b29a2966c2655cd12b233bf5a2be"}, @@ -3213,16 +2673,11 @@ files = [ {file = "yarl-1.9.2-cp311-cp311-win32.whl", hash = "sha256:a60347f234c2212a9f0361955007fcf4033a75bf600a33c88a0a8e91af77c0e8"}, {file = "yarl-1.9.2-cp311-cp311-win_amd64.whl", hash = "sha256:be6b3fdec5c62f2a67cb3f8c6dbf56bbf3f61c0f046f84645cd1ca73532ea051"}, {file = "yarl-1.9.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:38a3928ae37558bc1b559f67410df446d1fbfa87318b124bf5032c31e3447b74"}, - {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:ac9bb4c5ce3975aeac288cfcb5061ce60e0d14d92209e780c93954076c7c4367"}, - {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:3da8a678ca8b96c8606bbb8bfacd99a12ad5dd288bc6f7979baddd62f71c63ef"}, - {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:13414591ff516e04fcdee8dc051c13fd3db13b673c7a4cb1350e6b2ad9639ad3"}, - {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:bf74d08542c3a9ea97bb8f343d4fcbd4d8f91bba5ec9d5d7f792dbe727f88938"}, - {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:6e7221580dc1db478464cfeef9b03b95c5852cc22894e418562997df0d074ccc"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ac9bb4c5ce3975aeac288cfcb5061ce60e0d14d92209e780c93954076c7c4367"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3da8a678ca8b96c8606bbb8bfacd99a12ad5dd288bc6f7979baddd62f71c63ef"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:13414591ff516e04fcdee8dc051c13fd3db13b673c7a4cb1350e6b2ad9639ad3"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bf74d08542c3a9ea97bb8f343d4fcbd4d8f91bba5ec9d5d7f792dbe727f88938"}, + {file = "yarl-1.9.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6e7221580dc1db478464cfeef9b03b95c5852cc22894e418562997df0d074ccc"}, {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:494053246b119b041960ddcd20fd76224149cfea8ed8777b687358727911dd33"}, {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:52a25809fcbecfc63ac9ba0c0fb586f90837f5425edfd1ec9f3372b119585e45"}, {file = "yarl-1.9.2-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:e65610c5792870d45d7b68c677681376fcf9cc1c289f23e8e8b39c1485384185"}, @@ -3233,16 +2688,11 @@ files = [ {file = "yarl-1.9.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:5610f80cf43b6202e2c33ba3ec2ee0a2884f8f423c8f4f62906731d876ef4fac"}, {file = "yarl-1.9.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:b9a4e67ad7b646cd6f0938c7ebfd60e481b7410f574c560e455e938d2da8e0f4"}, {file = "yarl-1.9.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:83fcc480d7549ccebe9415d96d9263e2d4226798c37ebd18c930fce43dfb9574"}, - {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:5fcd436ea16fee7d4207c045b1e340020e58a2597301cfbcfdbe5abd2356c2fb"}, - {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:84e0b1599334b1e1478db01b756e55937d4614f8654311eb26012091be109d59"}, - {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:3458a24e4ea3fd8930e934c129b676c27452e4ebda80fbe47b56d8c6c7a63a9e"}, - {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:838162460b3a08987546e881a2bfa573960bb559dfa739e7800ceeec92e64417"}, - {file = "yarl-1.9.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:f4e2d08f07a3d7d3e12549052eb5ad3eab1c349c53ac51c209a0e5991bbada78"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5fcd436ea16fee7d4207c045b1e340020e58a2597301cfbcfdbe5abd2356c2fb"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:84e0b1599334b1e1478db01b756e55937d4614f8654311eb26012091be109d59"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3458a24e4ea3fd8930e934c129b676c27452e4ebda80fbe47b56d8c6c7a63a9e"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:838162460b3a08987546e881a2bfa573960bb559dfa739e7800ceeec92e64417"}, + {file = "yarl-1.9.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f4e2d08f07a3d7d3e12549052eb5ad3eab1c349c53ac51c209a0e5991bbada78"}, {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:de119f56f3c5f0e2fb4dee508531a32b069a5f2c6e827b272d1e0ff5ac040333"}, {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:149ddea5abf329752ea5051b61bd6c1d979e13fbf122d3a1f9f0c8be6cb6f63c"}, {file = "yarl-1.9.2-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:674ca19cbee4a82c9f54e0d1eee28116e63bc6fd1e96c43031d11cbab8b2afd5"}, @@ -3253,16 +2703,11 @@ files = [ {file = "yarl-1.9.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:75df5ef94c3fdc393c6b19d80e6ef1ecc9ae2f4263c09cacb178d871c02a5ba9"}, {file = "yarl-1.9.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:c027a6e96ef77d401d8d5a5c8d6bc478e8042f1e448272e8d9752cb0aff8b5c8"}, {file = "yarl-1.9.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f3b078dbe227f79be488ffcfc7a9edb3409d018e0952cf13f15fd6512847f3f7"}, - {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:59723a029760079b7d991a401386390c4be5bfec1e7dd83e25a6a0881859e716"}, - {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = -"sha256:b03917871bf859a81ccb180c9a2e6c1e04d2f6a51d953e6a5cdd70c93d4e5a2a"}, - {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = -"sha256:c1012fa63eb6c032f3ce5d2171c267992ae0c00b9e164efe4d73db818465fac3"}, - {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:a74dcbfe780e62f4b5a062714576f16c2f3493a0394e555ab141bf0d746bb955"}, - {file = "yarl-1.9.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:8c56986609b057b4839968ba901944af91b8e92f1725d1a2d77cbac6972b9ed1"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:59723a029760079b7d991a401386390c4be5bfec1e7dd83e25a6a0881859e716"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b03917871bf859a81ccb180c9a2e6c1e04d2f6a51d953e6a5cdd70c93d4e5a2a"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c1012fa63eb6c032f3ce5d2171c267992ae0c00b9e164efe4d73db818465fac3"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a74dcbfe780e62f4b5a062714576f16c2f3493a0394e555ab141bf0d746bb955"}, + {file = "yarl-1.9.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8c56986609b057b4839968ba901944af91b8e92f1725d1a2d77cbac6972b9ed1"}, {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:2c315df3293cd521033533d242d15eab26583360b58f7ee5d9565f15fee1bef4"}, {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:b7232f8dfbd225d57340e441d8caf8652a6acd06b389ea2d3222b8bc89cbfca6"}, {file = "yarl-1.9.2-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:53338749febd28935d55b41bf0bcc79d634881195a39f6b2f767870b72514caf"}, @@ -3281,7 +2726,6 @@ multidict = ">=4.0" name = "zipp" version = "3.15.0" description = "Backport of pathlib-compatible object wrapper for zip files" -category = "main" optional = false python-versions = ">=3.7" files = [ @@ -3291,80 +2735,55 @@ files = [ [package.extras] docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] -testing = ["big-O", "flake8 (<5)", "jaraco.functools", "jaraco.itertools", "more-itertools", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs -(>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"] +testing = ["big-O", "flake8 (<5)", "jaraco.functools", "jaraco.itertools", "more-itertools", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"] [[package]] name = "zstandard" version = "0.21.0" description = "Zstandard bindings for Python" -category = "main" optional = false python-versions = ">=3.7" files = [ {file = "zstandard-0.21.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:649a67643257e3b2cff1c0a73130609679a5673bf389564bc6d4b164d822a7ce"}, {file = "zstandard-0.21.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:144a4fe4be2e747bf9c646deab212666e39048faa4372abb6a250dab0f347a29"}, - {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:b72060402524ab91e075881f6b6b3f37ab715663313030d0ce983da44960a86f"}, - {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:8257752b97134477fb4e413529edaa04fc0457361d304c1319573de00ba796b1"}, - {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:c053b7c4cbf71cc26808ed67ae955836232f7638444d709bfc302d3e499364fa"}, - {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:2769730c13638e08b7a983b32cb67775650024632cd0476bf1ba0e6360f5ac7d"}, - {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:7d3bc4de588b987f3934ca79140e226785d7b5e47e31756761e48644a45a6766"}, + {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b72060402524ab91e075881f6b6b3f37ab715663313030d0ce983da44960a86f"}, + {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8257752b97134477fb4e413529edaa04fc0457361d304c1319573de00ba796b1"}, + {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:c053b7c4cbf71cc26808ed67ae955836232f7638444d709bfc302d3e499364fa"}, + {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2769730c13638e08b7a983b32cb67775650024632cd0476bf1ba0e6360f5ac7d"}, + {file = "zstandard-0.21.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:7d3bc4de588b987f3934ca79140e226785d7b5e47e31756761e48644a45a6766"}, {file = "zstandard-0.21.0-cp310-cp310-win32.whl", hash = "sha256:67829fdb82e7393ca68e543894cd0581a79243cc4ec74a836c305c70a5943f07"}, {file = "zstandard-0.21.0-cp310-cp310-win_amd64.whl", hash = "sha256:e6048a287f8d2d6e8bc67f6b42a766c61923641dd4022b7fd3f7439e17ba5a4d"}, {file = "zstandard-0.21.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:7f2afab2c727b6a3d466faee6974a7dad0d9991241c498e7317e5ccf53dbc766"}, {file = "zstandard-0.21.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ff0852da2abe86326b20abae912d0367878dd0854b8931897d44cfeb18985472"}, - {file = "zstandard-0.21.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:d12fa383e315b62630bd407477d750ec96a0f438447d0e6e496ab67b8b451d39"}, - {file = "zstandard-0.21.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:f1b9703fe2e6b6811886c44052647df7c37478af1b4a1a9078585806f42e5b15"}, - {file = "zstandard-0.21.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:df28aa5c241f59a7ab524f8ad8bb75d9a23f7ed9d501b0fed6d40ec3064784e8"}, + {file = "zstandard-0.21.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d12fa383e315b62630bd407477d750ec96a0f438447d0e6e496ab67b8b451d39"}, + {file = "zstandard-0.21.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f1b9703fe2e6b6811886c44052647df7c37478af1b4a1a9078585806f42e5b15"}, + {file = "zstandard-0.21.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:df28aa5c241f59a7ab524f8ad8bb75d9a23f7ed9d501b0fed6d40ec3064784e8"}, {file = "zstandard-0.21.0-cp311-cp311-win32.whl", hash = "sha256:0aad6090ac164a9d237d096c8af241b8dcd015524ac6dbec1330092dba151657"}, {file = "zstandard-0.21.0-cp311-cp311-win_amd64.whl", hash = "sha256:48b6233b5c4cacb7afb0ee6b4f91820afbb6c0e3ae0fa10abbc20000acdf4f11"}, {file = "zstandard-0.21.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:e7d560ce14fd209db6adacce8908244503a009c6c39eee0c10f138996cd66d3e"}, - {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:1e6e131a4df2eb6f64961cea6f979cdff22d6e0d5516feb0d09492c8fd36f3bc"}, - {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:e1e0c62a67ff425927898cf43da2cf6b852289ebcc2054514ea9bf121bec10a5"}, - {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:1545fb9cb93e043351d0cb2ee73fa0ab32e61298968667bb924aac166278c3fc"}, - {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:fe6c821eb6870f81d73bf10e5deed80edcac1e63fbc40610e61f340723fd5f7c"}, - {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:ddb086ea3b915e50f6604be93f4f64f168d3fc3cef3585bb9a375d5834392d4f"}, + {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1e6e131a4df2eb6f64961cea6f979cdff22d6e0d5516feb0d09492c8fd36f3bc"}, + {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e1e0c62a67ff425927898cf43da2cf6b852289ebcc2054514ea9bf121bec10a5"}, + {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:1545fb9cb93e043351d0cb2ee73fa0ab32e61298968667bb924aac166278c3fc"}, + {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fe6c821eb6870f81d73bf10e5deed80edcac1e63fbc40610e61f340723fd5f7c"}, + {file = "zstandard-0.21.0-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:ddb086ea3b915e50f6604be93f4f64f168d3fc3cef3585bb9a375d5834392d4f"}, {file = "zstandard-0.21.0-cp37-cp37m-win32.whl", hash = "sha256:57ac078ad7333c9db7a74804684099c4c77f98971c151cee18d17a12649bc25c"}, {file = "zstandard-0.21.0-cp37-cp37m-win_amd64.whl", hash = "sha256:1243b01fb7926a5a0417120c57d4c28b25a0200284af0525fddba812d575f605"}, {file = "zstandard-0.21.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:ea68b1ba4f9678ac3d3e370d96442a6332d431e5050223626bdce748692226ea"}, {file = "zstandard-0.21.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:8070c1cdb4587a8aa038638acda3bd97c43c59e1e31705f2766d5576b329e97c"}, - {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:4af612c96599b17e4930fe58bffd6514e6c25509d120f4eae6031b7595912f85"}, - {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:cff891e37b167bc477f35562cda1248acc115dbafbea4f3af54ec70821090965"}, - {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:a9fec02ce2b38e8b2e86079ff0b912445495e8ab0b137f9c0505f88ad0d61296"}, - {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:0bdbe350691dec3078b187b8304e6a9c4d9db3eb2d50ab5b1d748533e746d099"}, - {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:b69cccd06a4a0a1d9fb3ec9a97600055cf03030ed7048d4bcb88c574f7895773"}, + {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4af612c96599b17e4930fe58bffd6514e6c25509d120f4eae6031b7595912f85"}, + {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cff891e37b167bc477f35562cda1248acc115dbafbea4f3af54ec70821090965"}, + {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:a9fec02ce2b38e8b2e86079ff0b912445495e8ab0b137f9c0505f88ad0d61296"}, + {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0bdbe350691dec3078b187b8304e6a9c4d9db3eb2d50ab5b1d748533e746d099"}, + {file = "zstandard-0.21.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:b69cccd06a4a0a1d9fb3ec9a97600055cf03030ed7048d4bcb88c574f7895773"}, {file = "zstandard-0.21.0-cp38-cp38-win32.whl", hash = "sha256:9980489f066a391c5572bc7dc471e903fb134e0b0001ea9b1d3eff85af0a6f1b"}, {file = "zstandard-0.21.0-cp38-cp38-win_amd64.whl", hash = "sha256:0e1e94a9d9e35dc04bf90055e914077c80b1e0c15454cc5419e82529d3e70728"}, {file = "zstandard-0.21.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:d2d61675b2a73edcef5e327e38eb62bdfc89009960f0e3991eae5cc3d54718de"}, {file = "zstandard-0.21.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:25fbfef672ad798afab12e8fd204d122fca3bc8e2dcb0a2ba73bf0a0ac0f5f07"}, - {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = -"sha256:62957069a7c2626ae80023998757e27bd28d933b165c487ab6f83ad3337f773d"}, - {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = -"sha256:14e10ed461e4807471075d4b7a2af51f5234c8f1e2a0c1d37d5ca49aaaad49e8"}, - {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = -"sha256:9cff89a036c639a6a9299bf19e16bfb9ac7def9a7634c52c257166db09d950e7"}, - {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = -"sha256:52b2b5e3e7670bd25835e0e0730a236f2b0df87672d99d3bf4bf87248aa659fb"}, - {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = -"sha256:b1367da0dde8ae5040ef0413fb57b5baeac39d8931c70536d5f013b11d3fc3a5"}, + {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:62957069a7c2626ae80023998757e27bd28d933b165c487ab6f83ad3337f773d"}, + {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:14e10ed461e4807471075d4b7a2af51f5234c8f1e2a0c1d37d5ca49aaaad49e8"}, + {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:9cff89a036c639a6a9299bf19e16bfb9ac7def9a7634c52c257166db09d950e7"}, + {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:52b2b5e3e7670bd25835e0e0730a236f2b0df87672d99d3bf4bf87248aa659fb"}, + {file = "zstandard-0.21.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:b1367da0dde8ae5040ef0413fb57b5baeac39d8931c70536d5f013b11d3fc3a5"}, {file = "zstandard-0.21.0-cp39-cp39-win32.whl", hash = "sha256:db62cbe7a965e68ad2217a056107cc43d41764c66c895be05cf9c8b19578ce9c"}, {file = "zstandard-0.21.0-cp39-cp39-win_amd64.whl", hash = "sha256:a8d200617d5c876221304b0e3fe43307adde291b4a897e7b0617a61611dfff6a"}, {file = "zstandard-0.21.0.tar.gz", hash = "sha256:f08e3a10d01a247877e4cb61a82a319ea746c356a3786558bed2481e6c405546"}, @@ -3384,7 +2803,7 @@ glue = ["boto3"] hive = ["thrift"] pandas = ["pandas", "pyarrow"] pyarrow = ["pyarrow"] -ray = ["ray", "pyarrow", "pandas"] +ray = ["pandas", "pyarrow", "ray"] s3fs = ["s3fs"] snappy = ["python-snappy"] zstandard = ["zstandard"] @@ -3392,5 +2811,4 @@ zstandard = ["zstandard"] [metadata] lock-version = "2.0" python-versions = "^3.8" -content-hash = "e11ad53a6e619deaf308393a4df3d3b3ba4ed142c90fda6dda0c7b04c86bf4cb" - +content-hash = "2cc0c58478acda9d65b8c7683d82433fc7237b947ce2db0a8bc2b3694693d1d0" From 60c83b64e9cb74f602ff50973e841be2b374543f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 14 Jun 2023 15:09:46 +0200 Subject: [PATCH 36/36] Optimizations --- python/pyiceberg/io/pyarrow.py | 36 ++++++++---------------------- python/pyiceberg/table/__init__.py | 23 +++++++++---------- python/pyproject.toml | 4 ++++ 3 files changed, 24 insertions(+), 39 deletions(-) diff --git a/python/pyiceberg/io/pyarrow.py b/python/pyiceberg/io/pyarrow.py index e04ae10e249b..89ddf805eb73 100644 --- a/python/pyiceberg/io/pyarrow.py +++ b/python/pyiceberg/io/pyarrow.py @@ -28,7 +28,6 @@ import os from abc import ABC, abstractmethod from functools import lru_cache, singledispatch -from heapq import merge from itertools import chain from multiprocessing.pool import ThreadPool from multiprocessing.sharedctypes import Synchronized @@ -37,7 +36,6 @@ Any, Callable, Dict, - Generator, Generic, Iterable, List, @@ -50,6 +48,7 @@ ) from urllib.parse import urlparse +import numpy as np import pyarrow as pa import pyarrow.compute as pc import pyarrow.dataset as ds @@ -529,26 +528,12 @@ def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedAr } -def _create_positional_deletes_indices(positional_deletes: List[pa.ChunkedArray], fn_rows: Callable[[], int]) -> pa.Array: - # This is not ideal, looking for a native PyArrow implementation :) - # Ideally with uniqueness as well - # https://github.com/apache/arrow/issues/35748 - sorted_deleted = merge(*positional_deletes, key=lambda e: e.as_py()) - - def generator() -> Generator[int, None, None]: - deleted_pos = next(sorted_deleted).as_py() # type: ignore - for pos in range(fn_rows()): - if deleted_pos == pos: - while deleted_pos == pos: - try: - deleted_pos = next(sorted_deleted).as_py() # type: ignore - except StopIteration: - deleted_pos = -1 - else: - yield pos - - # Filter on the positions - return pa.array(generator(), type=pa.int64()) +def _combine_positional_deletes(positional_deletes: List[pa.ChunkedArray], rows: int) -> pa.Array: + if len(positional_deletes) == 1: + all_chunks = positional_deletes[0] + else: + all_chunks = pa.chunked_array(chain(*[arr.chunks for arr in positional_deletes])) + return np.setdiff1d(np.arange(rows), all_chunks, assume_unique=False) def pyarrow_to_schema(schema: pa.Schema) -> Schema: @@ -782,10 +767,8 @@ def _task_to_table( ) if positional_deletes: - # In the case of a mask, it is a bit awkward because we first - # need to go to a table to apply the bitwise mask, and then - # the table is warped into a dataset to apply the expression - indices = _create_positional_deletes_indices(positional_deletes, fragment.count_rows) + # Create the mask of indices that we're interested in + indices = _combine_positional_deletes(positional_deletes, fragment.count_rows()) if limit: if pyarrow_filter is not None: @@ -802,7 +785,6 @@ def _task_to_table( # Apply the user filter if pyarrow_filter is not None: arrow_table = arrow_table.filter(pyarrow_filter) - else: # If there are no deletes, we can just take the head # and the user-filter is already applied diff --git a/python/pyiceberg/table/__init__.py b/python/pyiceberg/table/__init__.py index 2c62d222ce49..9c095a5319e5 100644 --- a/python/pyiceberg/table/__init__.py +++ b/python/pyiceberg/table/__init__.py @@ -348,12 +348,13 @@ def _min_data_file_sequence_number(manifests: List[ManifestFile]) -> int: def _match_deletes_to_datafile(data_entry: ManifestEntry, positional_delete_entries: SortedList[ManifestEntry]) -> Set[DataFile]: - """This method will check if the delete file is relevant for the data file - by using the column metrics to see if the filename is in the lower and upper bound + """This method will check if the delete file is relevant for the data file. + + Using the column metrics to see if the filename is in the lower and upper bound. Args: - data_entry (ManifestEntry): The manifest entry path of the datafile - positional_delete_entries (List[ManifestEntry]): All the candidate positional deletes manifest entries + data_entry (ManifestEntry): The manifest entry path of the datafile. + positional_delete_entries (List[ManifestEntry]): All the candidate positional deletes manifest entries. Returns: A set of files that are relevant for the data file. @@ -406,15 +407,14 @@ def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool] return lambda data_file: evaluator(data_file.partition) def _check_sequence_number(self, min_data_sequence_number: int, manifest: ManifestFile) -> bool: - """A helper function to make sure that no manifests are loaded that contain deletes - that are older than the data + """A helper function to make sure that no manifests are loaded that contain deletes that are older than the data. Args: - min_data_sequence_number (int): The minimal - manifest (ManifestFile): A ManifestFile that can be either data or deletes + min_data_sequence_number (int): The minimal sequence number. + manifest (ManifestFile): A ManifestFile that can be either data or deletes. Returns: - Boolean indicating if it is either a data file, or a relevant delete file + Boolean indicating if it is either a data file, or a relevant delete file. """ return manifest.content == ManifestContent.DATA or ( # Not interested in deletes that are older than the data @@ -423,12 +423,11 @@ def _check_sequence_number(self, min_data_sequence_number: int, manifest: Manife ) def plan_files(self) -> Iterable[FileScanTask]: - """Plans the relevant files by filtering on the PartitionSpecs + """Plans the relevant files by filtering on the PartitionSpecs. Returns: - List of FileScanTasks that contain both data and delete files + List of FileScanTasks that contain both data and delete files. """ - snapshot = self.snapshot() if not snapshot: return iter([]) diff --git a/python/pyproject.toml b/python/pyproject.toml index cfda50efff47..cad82f5bdeaa 100644 --- a/python/pyproject.toml +++ b/python/pyproject.toml @@ -243,5 +243,9 @@ ignore_missing_imports = true module = "sortedcontainers.*" ignore_missing_imports = true +[[tool.mypy.overrides]] +module = "numpy.*" +ignore_missing_imports = true + [tool.coverage.run] source = ['pyiceberg/']