From 14c27cfc0f32c6f1fd927f492f93daf732b359a3 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Mon, 10 Feb 2025 19:52:51 +0200
Subject: [PATCH 001/112] first attempt at init_manifests
---
pyiceberg/manifest.py | 22 +-
pyiceberg/table/__init__.py | 57 ++++-
pyiceberg/table/update/snapshot.py | 315 +++++++++++++++++++-----
tests/integration/test_inspect_table.py | 42 ++++
4 files changed, 373 insertions(+), 63 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 61cb87e3d8..bc139a764b 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -693,6 +693,24 @@ def has_added_files(self) -> bool:
def has_existing_files(self) -> bool:
return self.existing_files_count is None or self.existing_files_count > 0
+ def copy_with_snapshot_id(self, snapshot_id: int) -> ManifestFile:
+ return ManifestFile(
+ manifest_path=self.manifest_path,
+ manifest_length=self.manifest_length,
+ partition_spec_id=self.partition_spec_id,
+ content=self.content,
+ sequence_number=self.sequence_number,
+ min_sequence_number=self.min_sequence_number,
+ added_snapshot_id=snapshot_id,
+ added_files_count=self.added_files_count,
+ existing_files_count=self.existing_files_count,
+ deleted_files_count=self.deleted_files_count,
+ added_rows_count=self.added_rows_count,
+ existing_rows_count=self.existing_rows_count,
+ deleted_rows_count=self.deleted_rows_count,
+ partitions=self.partitions,
+ key_metadata=self.key_metadata,
+ )
def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List[ManifestEntry]:
"""
Read the manifest entries from the manifest file.
@@ -716,7 +734,9 @@ def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List
for entry in reader
if not discard_deleted or entry.status != ManifestEntryStatus.DELETED
]
-
+ def __hash__(self) -> int:
+ """Return the hash of the file path."""
+ return hash(self.manifest_path)
@cached(cache=LRUCache(maxsize=128), key=lambda io, manifest_list: hashkey(manifest_list))
def _manifests(io: FileIO, manifest_list: str) -> Tuple[ManifestFile, ...]:
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 78676a774a..9d2d271c56 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -119,6 +119,7 @@
ManageSnapshots,
UpdateSnapshot,
_FastAppendFiles,
+ _ManifestMergeManager,
)
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
@@ -138,7 +139,7 @@
)
from pyiceberg.utils.concurrent import ExecutorFactory
from pyiceberg.utils.config import Config
-from pyiceberg.utils.properties import property_as_bool
+from pyiceberg.utils.properties import property_as_bool, property_as_int
if TYPE_CHECKING:
import daft
@@ -438,6 +439,9 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> U
A new UpdateSnapshot
"""
return UpdateSnapshot(self, io=self._table.io, snapshot_properties=snapshot_properties)
+ def rewrite_manifests(self) -> None:
+ with self.update_snapshot().rewrite() as rewrite:
+ rewrite.commit()
def update_statistics(self) -> UpdateStatistics:
"""
@@ -1233,6 +1237,10 @@ def name_mapping(self) -> Optional[NameMapping]:
"""Return the table's field-id NameMapping."""
return self.metadata.name_mapping()
+ def rewrite_manifests(self) -> None:
+ with self.transaction() as tx:
+ tx.rewrite_manifests()
+
def upsert(
self,
df: pa.Table,
@@ -1368,6 +1376,53 @@ def add_files(
file_paths=file_paths, snapshot_properties=snapshot_properties, check_duplicate_files=check_duplicate_files
)
+ def rewrite_manifests(
+ self,
+ spec_id: Optional[int] = None,
+ rewrite_all: bool = False,
+ max_manifest_size: Optional[int] = None,
+ ) -> "Table":
+
+ with self.transaction() as tx:
+ tx.rewrite_manifests()
+ ...
+ """Rewrite manifests in the table.
+
+ Args:
+ spec_id: Spec ID to be used for the rewritten manifests
+ rewrite_all: If True, rewrite all manifests. If False, only rewrite small manifests
+ max_manifest_size: Target size for manifests in bytes
+
+ Returns:
+ An updated version of the table with rewritten manifests
+ #"""
+ # return RewriteManifests(
+ # self,
+ # spec_id=spec_id,
+ # rewrite_all=rewrite_all,
+ # max_manifest_size=max_manifest_size,
+ # ).commit()
+
+ # snapshot = self.current_snapshot()
+ # manifests = []
+ # for manifest in snapshot.manifests(self.io):
+ # if manifest.content == ManifestContent.DATA:
+ # manifests.append(manifest)
+ #
+ # data_manifest_merge_manager = _ManifestMergeManager(
+ # target_size_bytes=property_as_int(
+ # self.properties,
+ # TableProperties.MANIFEST_TARGET_SIZE_BYTES,
+ # TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
+ # ),
+ # min_count_to_merge=2,
+ # merge_enabled=True,
+ # snapshot_producer=self,
+ # )
+ #
+ # data_manifest_merge_manager.merge_manifests(manifests)
+ # entries = self.inspect.entries().filter("status < 2").selectExpr("input_file_name() as manifest")
+
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index a82167744d..7c89004007 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -106,12 +106,12 @@ class _SnapshotProducer(UpdateTableMetadata[U], Generic[U]):
_deleted_data_files: Set[DataFile]
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
super().__init__(transaction)
self.commit_uuid = commit_uuid or uuid.uuid4()
@@ -136,10 +136,12 @@ def delete_data_file(self, data_file: DataFile) -> _SnapshotProducer[U]:
return self
@abstractmethod
- def _deleted_entries(self) -> List[ManifestEntry]: ...
+ def _deleted_entries(self) -> List[ManifestEntry]:
+ ...
@abstractmethod
- def _existing_manifests(self) -> List[ManifestFile]: ...
+ def _existing_manifests(self) -> List[ManifestFile]:
+ ...
def _process_manifests(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
"""To perform any post-processing on the manifests before writing them to the new snapshot."""
@@ -149,11 +151,11 @@ def _manifests(self) -> List[ManifestFile]:
def _write_added_manifest() -> List[ManifestFile]:
if self._added_data_files:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.spec(),
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.spec(),
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
for data_file in self._added_data_files:
writer.add(
@@ -179,11 +181,11 @@ def _write_delete_manifest() -> List[ManifestFile]:
partition_groups[deleted_entry.data_file.spec_id].append(deleted_entry)
for spec_id, entries in partition_groups.items():
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
for entry in entries:
writer.add_entry(entry)
@@ -198,7 +200,8 @@ def _write_delete_manifest() -> List[ManifestFile]:
delete_manifests = executor.submit(_write_delete_manifest)
existing_manifests = executor.submit(self._existing_manifests)
- return self._process_manifests(added_manifests.result() + delete_manifests.result() + existing_manifests.result())
+ return self._process_manifests(
+ added_manifests.result() + delete_manifests.result() + existing_manifests.result())
def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
from pyiceberg.table import TableProperties
@@ -250,11 +253,11 @@ def _commit(self) -> UpdatesAndRequirements:
location_provider = self._transaction._table.location_provider()
manifest_list_file_path = location_provider.new_metadata_location(file_name)
with write_manifest_list(
- format_version=self._transaction.table_metadata.format_version,
- output_file=self._io.new_output(manifest_list_file_path),
- snapshot_id=self._snapshot_id,
- parent_snapshot_id=self._parent_snapshot_id,
- sequence_number=next_sequence_number,
+ format_version=self._transaction.table_metadata.format_version,
+ output_file=self._io.new_output(manifest_list_file_path),
+ snapshot_id=self._snapshot_id,
+ parent_snapshot_id=self._parent_snapshot_id,
+ sequence_number=next_sequence_number,
) as writer:
writer.add_manifests(new_manifests)
@@ -271,7 +274,8 @@ def _commit(self) -> UpdatesAndRequirements:
(
AddSnapshotUpdate(snapshot=snapshot),
SetSnapshotRefUpdate(
- snapshot_id=self._snapshot_id, parent_snapshot_id=self._parent_snapshot_id, ref_name="main", type="branch"
+ snapshot_id=self._snapshot_id, parent_snapshot_id=self._parent_snapshot_id, ref_name="main",
+ type="branch"
),
),
(AssertRefSnapshotId(snapshot_id=self._transaction.table_metadata.current_snapshot_id, ref="main"),),
@@ -317,12 +321,12 @@ class _DeleteFiles(_SnapshotProducer["_DeleteFiles"]):
_case_sensitive: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
):
super().__init__(operation, transaction, io, commit_uuid, snapshot_properties)
self._predicate = AlwaysFalse()
@@ -375,7 +379,8 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
)
manifest_evaluators: Dict[int, Callable[[ManifestFile], bool]] = KeyDefaultDict(self._build_manifest_evaluator)
- strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate, case_sensitive=self._case_sensitive).eval
+ strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate,
+ case_sensitive=self._case_sensitive).eval
inclusive_metrics_evaluator = _InclusiveMetricsEvaluator(
schema, self._predicate, case_sensitive=self._case_sensitive
).eval
@@ -411,11 +416,11 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
# Rewrite the manifest
if len(existing_entries) > 0:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
for existing_entry in existing_entries:
writer.add_entry(existing_entry)
@@ -444,6 +449,182 @@ def files_affected(self) -> bool:
return len(self._deleted_entries()) > 0
+class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
+ """Rewrite manifest files based on the predicate."""
+
+ KEPT_MANIFESTS_COUNT = "manifests-kept"
+ CREATED_MANIFESTS_COUNT = "manifests-created"
+ REPLACED_MANIFESTS_COUNT = "manifests-replaced"
+ PROCESSED_ENTRY_COUNT = "entries-processed"
+
+ def __init__(
+ self,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ ):
+ from pyiceberg.table import TableProperties
+
+ super().__init__(Operation.REPLACE, transaction, io, commit_uuid, snapshot_properties)
+
+ self.specs_by_id = self._transaction.table_metadata.spec().spec_id
+ self.manifest_target_size_bytes = property_as_int(
+ self._transaction.table_metadata.properties,
+ TableProperties.MANIFEST_TARGET_SIZE_BYTES,
+ TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
+ )
+ self.deleted_manifests: Set[ManifestFile] = set()
+ self.added_manifests: List[ManifestFile] = []
+ self.rewritten_added_manifests: List[ManifestFile] = []
+ self.kept_manifests: Set[ManifestFile] = set()
+ self.new_manifests: Set[ManifestFile] = set()
+ self.rewritten_manifests: Set[ManifestFile] = set()
+
+ def copy_manifest(self, manifest: ManifestFile) -> ManifestFile:
+ return ManifestFile(
+ manifest_path=manifest.manifest_path,
+ manifest_length=manifest.manifest_length,
+ partition_spec_id=manifest.partition_spec_id,
+ content=manifest.content,
+ sequence_number=manifest.sequence_number,
+ min_sequence_number=manifest.min_sequence_number,
+ added_snapshot_id=manifest.added_snapshot_id,
+ added_files_count=manifest.added_files_count,
+ existing_files_count=manifest.existing_files_count,
+ deleted_files_count=manifest.deleted_files_count,
+ added_rows_count=manifest.added_rows_count,
+ existing_rows_count=manifest.existing_rows_count,
+ deleted_rows_count=manifest.deleted_rows_count,
+ partitions=manifest.partitions,
+ key_metadata=manifest.key_metadata,
+ )
+
+ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
+ summary = {
+ self.CREATED_MANIFESTS_COUNT: str(
+ len(self.new_manifests) + len(self.added_manifests) + len(self.rewritten_added_manifests)
+ ),
+ self.KEPT_MANIFESTS_COUNT: str(len(self.kept_manifests)),
+ self.REPLACED_MANIFESTS_COUNT: str(len(self.rewritten_manifests) + len(self.deleted_manifests)),
+ self.PROCESSED_ENTRY_COUNT: str(self.entry_count),
+ }
+ return super()._summary(summary)
+
+ def delete_manifest(self, manifest):
+ self.deleted_manifests.add(manifest)
+ return self
+
+ def requires_rewrite(self, current_manifests):
+
+ if not self.rewritten_manifests:
+ # nothing yet processed so perform a full rewrite
+ return True
+
+ # if any processed manifest is not in the current manifest list, perform a full rewrite
+ return any(manifest not in current_manifests for manifest in self.rewritten_manifests)
+
+ def keep_active_manifests(self, current_manifests: List[ManifestFile]) -> None:
+ # keep any existing manifests as-is that were not processed
+ self.kept_manifests.clear()
+ for manifest in current_manifests:
+ if manifest not in self.rewritten_manifests and manifest not in self.deleted_manifests:
+ self.kept_manifests.add(manifest)
+
+ def active_files_count(self, manifests):
+ active_files_count = 0
+
+ for manifest in manifests:
+ if manifest.added_files_count is None:
+ raise ValueError("Missing file counts in {}".format(manifest.path()))
+ if manifest.existing_files_count is None:
+ raise ValueError("Missing file counts in {}".format(manifest.path()))
+ active_files_count += manifest.added_files_count
+ active_files_count += manifest.existing_files_count
+
+ return active_files_count
+
+ def validate_files_counts(self):
+ created_manifests = itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
+ created_manifests_files_count = self.active_files_count(created_manifests)
+
+ replaced_manifests = itertools.chain(self.rewritten_manifests, self.deleted_manifests)
+ replaced_manifests_files_count = self.active_files_count(replaced_manifests)
+
+ if created_manifests_files_count != replaced_manifests_files_count:
+ raise ValueError(
+ "Replaced and created manifests must have the same number of active files: {} (new), {} (old)".format(
+ created_manifests_files_count, replaced_manifests_files_count
+ )
+ )
+
+ def _existing_manifests(self) -> List[ManifestFile]:
+ """Returns the list of manifests to include in the next snapshot."""
+ return self.apply()
+
+ def _deleted_entries(self) -> List[ManifestEntry]:
+ """No entries are deleted during manifest rewriting."""
+ return []
+ def apply(self) -> List[ManifestFile]:
+ snapshot = self._transaction.table_metadata.current_snapshot()
+ current_manifests = snapshot.manifests(io=self._io)
+ current_manifest_set = set(current_manifests)
+ data_manifest_merge_manager = _ManifestMergeManager(
+ target_size_bytes=self.manifest_target_size_bytes,
+ min_count_to_merge=2,
+ merge_enabled=True,
+ snapshot_producer=self,
+ )
+ # self.validate_deleted_manifests(current_manifest_set, base.current_snapshot().snapshot_id())
+
+ if self.requires_rewrite(current_manifest_set):
+ new_manifests = data_manifest_merge_manager.merge_manifests(manifests=current_manifests)
+ # self.rewritten_manifests.add(new_manifests)
+ self.rewritten_manifests.update(new_manifests)
+ else:
+ self.keep_active_manifests(current_manifests)
+
+ self.validate_files_counts()
+
+ new_manifests_with_metadata = [
+ manifest.copy_with_snapshot_id(self.snapshot_id)
+ for manifest in itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
+ ]
+
+ # put new manifests at the beginning
+ applyi = list(new_manifests_with_metadata)
+ applyi.extend(self.kept_manifests)
+
+ return applyi
+
+ # def execute(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
+ # data_manifest_merge_manager = _ManifestMergeManager(
+ # target_size_bytes=self.manifest_target_size_bytes,
+ # min_count_to_merge=2,
+ # merge_enabled=True,
+ # snapshot_producer=self,
+ # )
+ # return data_manifest_merge_manager.merge_manifests(manifests=manifests)
+
+ def add_manifest(self, manifest):
+ if manifest.has_added_files():
+ raise ValueError("Cannot add manifest with added files")
+ if manifest.has_deleted_files():
+ raise ValueError("Cannot add manifest with deleted files")
+ if manifest.snapshot_id() is not None and manifest.snapshot_id() != -1:
+ raise ValueError("Snapshot id must be assigned during commit")
+ if manifest.sequence_number() != -1:
+ raise ValueError("Sequence must be assigned during commit")
+
+ if manifest.snapshot_id() is None:
+ self.added_manifests.append(manifest)
+ else:
+ copied_manifest = self.copy_manifest(manifest)
+ self.rewritten_added_manifests.append(copied_manifest)
+
+ return self
+
+
class _FastAppendFiles(_SnapshotProducer["_FastAppendFiles"]):
def _existing_manifests(self) -> List[ManifestFile]:
"""To determine if there are any existing manifest files.
@@ -479,12 +660,12 @@ class _MergeAppendFiles(_FastAppendFiles):
_merge_enabled: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
from pyiceberg.table import TableProperties
@@ -537,7 +718,8 @@ def _existing_manifests(self) -> List[ManifestFile]:
if snapshot := self._transaction.table_metadata.current_snapshot():
for manifest_file in snapshot.manifests(io=self._io):
entries = manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True)
- found_deleted_data_files = [entry.data_file for entry in entries if entry.data_file in self._deleted_data_files]
+ found_deleted_data_files = [entry.data_file for entry in entries if
+ entry.data_file in self._deleted_data_files]
if len(found_deleted_data_files) == 0:
existing_files.append(manifest_file)
@@ -545,11 +727,11 @@ def _existing_manifests(self) -> List[ManifestFile]:
# We have to rewrite the manifest file without the deleted data files
if any(entry.data_file not in found_deleted_data_files for entry in entries):
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
[
writer.add_entry(
@@ -613,12 +795,19 @@ def __init__(self, transaction: Transaction, io: FileIO, snapshot_properties: Di
def fast_append(self) -> _FastAppendFiles:
return _FastAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
+ operation=Operation.APPEND, transaction=self._transaction, io=self._io,
+ snapshot_properties=self._snapshot_properties
)
def merge_append(self) -> _MergeAppendFiles:
return _MergeAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
+ operation=Operation.APPEND, transaction=self._transaction, io=self._io,
+ snapshot_properties=self._snapshot_properties
+ )
+
+ def rewrite(self) -> _RewriteManifests:
+ return _RewriteManifests(
+ transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
)
def overwrite(self, commit_uuid: Optional[uuid.UUID] = None) -> _OverwriteFiles:
@@ -648,7 +837,8 @@ class _ManifestMergeManager(Generic[U]):
_snapshot_producer: _SnapshotProducer[U]
def __init__(
- self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool, snapshot_producer: _SnapshotProducer[U]
+ self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool,
+ snapshot_producer: _SnapshotProducer[U]
) -> None:
self._target_size_bytes = target_size_bytes
self._min_count_to_merge = min_count_to_merge
@@ -677,8 +867,10 @@ def _create_manifest(self, spec_id: int, manifest_bin: List[ManifestFile]) -> Ma
return writer.to_manifest_file()
- def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[ManifestFile]:
- packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1, largest_bin_first=False)
+ def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[
+ ManifestFile]:
+ packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1,
+ largest_bin_first=False)
bins: List[List[ManifestFile]] = packer.pack_end(manifests, lambda m: m.manifest_length)
def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
@@ -700,7 +892,8 @@ def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
# for consistent ordering, we need to maintain future order
futures_index = {f: i for i, f in enumerate(futures)}
- completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[], key=lambda f: futures_index[f])
+ completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[],
+ key=lambda f: futures_index[f])
for future in concurrent.futures.as_completed(futures):
completed_futures.add(future)
@@ -801,12 +994,12 @@ def remove_tag(self, tag_name: str) -> ManageSnapshots:
return self._remove_ref_snapshot(ref_name=tag_name)
def create_branch(
- self,
- snapshot_id: int,
- branch_name: str,
- max_ref_age_ms: Optional[int] = None,
- max_snapshot_age_ms: Optional[int] = None,
- min_snapshots_to_keep: Optional[int] = None,
+ self,
+ snapshot_id: int,
+ branch_name: str,
+ max_ref_age_ms: Optional[int] = None,
+ max_snapshot_age_ms: Optional[int] = None,
+ min_snapshots_to_keep: Optional[int] = None,
) -> ManageSnapshots:
"""
Create a new branch pointing to the given snapshot id.
diff --git a/tests/integration/test_inspect_table.py b/tests/integration/test_inspect_table.py
index e81050a81c..7c805bae13 100644
--- a/tests/integration/test_inspect_table.py
+++ b/tests/integration/test_inspect_table.py
@@ -1101,3 +1101,45 @@ def test_inspect_files_partitioned(spark: SparkSession, session_catalog: Catalog
.reset_index()
)
assert_frame_equal(lhs, rhs, check_dtype=False)
+
+def test_inspect_all_example(spark: SparkSession, session_catalog: Catalog, format_version: int) -> None:
+ from pandas.testing import assert_frame_equal
+
+ identifier = "default.table_metadata_all_manifests"
+ try:
+ session_catalog.drop_table(identifier=identifier)
+ except NoSuchTableError:
+ pass
+
+ spark.sql(
+ f"""
+ CREATE TABLE {identifier} (
+ id int,
+ data string
+ )
+ PARTITIONED BY (data)
+ TBLPROPERTIES ('write.update.mode'='merge-on-read',
+ 'write.delete.mode'='merge-on-read')
+ """
+ )
+ tbl = session_catalog.load_table(identifier)
+ # check all_manifests when there are no snapshots
+ lhs = tbl.inspect.all_manifests().to_pandas()
+ rhs = spark.table(f"{identifier}.all_manifests").toPandas()
+ assert_frame_equal(lhs, rhs, check_dtype=False)
+
+ spark.sql(f"INSERT INTO {identifier} VALUES (1, 'a')")
+
+ spark.sql(f"INSERT INTO {identifier} VALUES (2, 'b')")
+
+ spark.sql(f"UPDATE {identifier} SET data = 'c' WHERE id = 1")
+
+ spark.sql(f"DELETE FROM {identifier} WHERE id = 2")
+
+ spark.sql(f"INSERT OVERWRITE {identifier} VALUES (1, 'a')")
+
+ tbl.refresh()
+
+
+ tbl.rewrite_manifests()
+ print("efd")
\ No newline at end of file
From ee0392d17a5a917f5ec74ebc337516624136acf6 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Thu, 13 Feb 2025 22:19:36 +0200
Subject: [PATCH 002/112] initial implementation of rewrite_manifests
---
pyiceberg/manifest.py | 3 +
pyiceberg/table/__init__.py | 60 +--
pyiceberg/table/snapshots.py | 2 +-
pyiceberg/table/update/snapshot.py | 409 +++++++-----------
.../test_writes/test_rewrite_manifests.py | 116 +++++
5 files changed, 290 insertions(+), 300 deletions(-)
create mode 100644 tests/integration/test_writes/test_rewrite_manifests.py
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index bc139a764b..dc66371aa3 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -711,6 +711,7 @@ def copy_with_snapshot_id(self, snapshot_id: int) -> ManifestFile:
partitions=self.partitions,
key_metadata=self.key_metadata,
)
+
def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List[ManifestEntry]:
"""
Read the manifest entries from the manifest file.
@@ -734,10 +735,12 @@ def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List
for entry in reader
if not discard_deleted or entry.status != ManifestEntryStatus.DELETED
]
+
def __hash__(self) -> int:
"""Return the hash of the file path."""
return hash(self.manifest_path)
+
@cached(cache=LRUCache(maxsize=128), key=lambda io, manifest_list: hashkey(manifest_list))
def _manifests(io: FileIO, manifest_list: str) -> Tuple[ManifestFile, ...]:
"""Read and cache manifests from the given manifest list, returning a tuple to prevent modification."""
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 9d2d271c56..d831e8fd74 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -115,12 +115,7 @@
update_table_metadata,
)
from pyiceberg.table.update.schema import UpdateSchema
-from pyiceberg.table.update.snapshot import (
- ManageSnapshots,
- UpdateSnapshot,
- _FastAppendFiles,
- _ManifestMergeManager,
-)
+from pyiceberg.table.update.snapshot import ManageSnapshots, UpdateSnapshot, _FastAppendFiles, RewriteManifestsResult
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
from pyiceberg.transforms import IdentityTransform
@@ -139,7 +134,7 @@
)
from pyiceberg.utils.concurrent import ExecutorFactory
from pyiceberg.utils.config import Config
-from pyiceberg.utils.properties import property_as_bool, property_as_int
+from pyiceberg.utils.properties import property_as_bool
if TYPE_CHECKING:
import daft
@@ -439,9 +434,10 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> U
A new UpdateSnapshot
"""
return UpdateSnapshot(self, io=self._table.io, snapshot_properties=snapshot_properties)
+
def rewrite_manifests(self) -> None:
with self.update_snapshot().rewrite() as rewrite:
- rewrite.commit()
+ rewrite.rewrite_manifests()
def update_statistics(self) -> UpdateStatistics:
"""
@@ -1379,49 +1375,17 @@ def add_files(
def rewrite_manifests(
self,
spec_id: Optional[int] = None,
- rewrite_all: bool = False,
- max_manifest_size: Optional[int] = None,
- ) -> "Table":
-
- with self.transaction() as tx:
- tx.rewrite_manifests()
- ...
- """Rewrite manifests in the table.
+ ) -> RewriteManifestsResult:
+ """
+ Shorthand API for Rewriting manifests for the table.
Args:
- spec_id: Spec ID to be used for the rewritten manifests
- rewrite_all: If True, rewrite all manifests. If False, only rewrite small manifests
- max_manifest_size: Target size for manifests in bytes
+ spec_id: Spec id of the manifests to rewrite (defaults to current spec id)
- Returns:
- An updated version of the table with rewritten manifests
- #"""
- # return RewriteManifests(
- # self,
- # spec_id=spec_id,
- # rewrite_all=rewrite_all,
- # max_manifest_size=max_manifest_size,
- # ).commit()
-
- # snapshot = self.current_snapshot()
- # manifests = []
- # for manifest in snapshot.manifests(self.io):
- # if manifest.content == ManifestContent.DATA:
- # manifests.append(manifest)
- #
- # data_manifest_merge_manager = _ManifestMergeManager(
- # target_size_bytes=property_as_int(
- # self.properties,
- # TableProperties.MANIFEST_TARGET_SIZE_BYTES,
- # TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
- # ),
- # min_count_to_merge=2,
- # merge_enabled=True,
- # snapshot_producer=self,
- # )
- #
- # data_manifest_merge_manager.merge_manifests(manifests)
- # entries = self.inspect.entries().filter("status < 2").selectExpr("input_file_name() as manifest")
+ """
+ with self.transaction() as tx:
+ rewrite_results = tx.rewrite_manifests(spec_id=spec_id)
+ return rewrite_results
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/snapshots.py b/pyiceberg/table/snapshots.py
index 8d1a24c420..683ab1f3dc 100644
--- a/pyiceberg/table/snapshots.py
+++ b/pyiceberg/table/snapshots.py
@@ -353,7 +353,7 @@ def get_prop(prop: str) -> int:
def update_snapshot_summaries(
summary: Summary, previous_summary: Optional[Mapping[str, str]] = None, truncate_full_table: bool = False
) -> Summary:
- if summary.operation not in {Operation.APPEND, Operation.OVERWRITE, Operation.DELETE}:
+ if summary.operation not in {Operation.APPEND, Operation.OVERWRITE, Operation.DELETE, Operation.REPLACE}:
raise ValueError(f"Operation not implemented: {summary.operation}")
if truncate_full_table and summary.operation == Operation.OVERWRITE and previous_summary is not None:
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 7c89004007..07f5c3e1f1 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -22,6 +22,7 @@
from abc import abstractmethod
from collections import defaultdict
from concurrent.futures import Future
+from dataclasses import dataclass
from functools import cached_property
from typing import TYPE_CHECKING, Callable, Dict, Generic, List, Optional, Set, Tuple
@@ -82,7 +83,7 @@
from pyiceberg.utils.properties import property_as_bool, property_as_int
if TYPE_CHECKING:
- from pyiceberg.table import Transaction
+ from pyiceberg.table import Table, Transaction
def _new_manifest_file_name(num: int, commit_uuid: uuid.UUID) -> str:
@@ -106,12 +107,12 @@ class _SnapshotProducer(UpdateTableMetadata[U], Generic[U]):
_deleted_data_files: Set[DataFile]
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
super().__init__(transaction)
self.commit_uuid = commit_uuid or uuid.uuid4()
@@ -136,12 +137,10 @@ def delete_data_file(self, data_file: DataFile) -> _SnapshotProducer[U]:
return self
@abstractmethod
- def _deleted_entries(self) -> List[ManifestEntry]:
- ...
+ def _deleted_entries(self) -> List[ManifestEntry]: ...
@abstractmethod
- def _existing_manifests(self) -> List[ManifestFile]:
- ...
+ def _existing_manifests(self) -> List[ManifestFile]: ...
def _process_manifests(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
"""To perform any post-processing on the manifests before writing them to the new snapshot."""
@@ -151,11 +150,11 @@ def _manifests(self) -> List[ManifestFile]:
def _write_added_manifest() -> List[ManifestFile]:
if self._added_data_files:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.spec(),
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.spec(),
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
for data_file in self._added_data_files:
writer.add(
@@ -181,11 +180,11 @@ def _write_delete_manifest() -> List[ManifestFile]:
partition_groups[deleted_entry.data_file.spec_id].append(deleted_entry)
for spec_id, entries in partition_groups.items():
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
for entry in entries:
writer.add_entry(entry)
@@ -200,8 +199,7 @@ def _write_delete_manifest() -> List[ManifestFile]:
delete_manifests = executor.submit(_write_delete_manifest)
existing_manifests = executor.submit(self._existing_manifests)
- return self._process_manifests(
- added_manifests.result() + delete_manifests.result() + existing_manifests.result())
+ return self._process_manifests(added_manifests.result() + delete_manifests.result() + existing_manifests.result())
def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
from pyiceberg.table import TableProperties
@@ -253,11 +251,11 @@ def _commit(self) -> UpdatesAndRequirements:
location_provider = self._transaction._table.location_provider()
manifest_list_file_path = location_provider.new_metadata_location(file_name)
with write_manifest_list(
- format_version=self._transaction.table_metadata.format_version,
- output_file=self._io.new_output(manifest_list_file_path),
- snapshot_id=self._snapshot_id,
- parent_snapshot_id=self._parent_snapshot_id,
- sequence_number=next_sequence_number,
+ format_version=self._transaction.table_metadata.format_version,
+ output_file=self._io.new_output(manifest_list_file_path),
+ snapshot_id=self._snapshot_id,
+ parent_snapshot_id=self._parent_snapshot_id,
+ sequence_number=next_sequence_number,
) as writer:
writer.add_manifests(new_manifests)
@@ -274,8 +272,7 @@ def _commit(self) -> UpdatesAndRequirements:
(
AddSnapshotUpdate(snapshot=snapshot),
SetSnapshotRefUpdate(
- snapshot_id=self._snapshot_id, parent_snapshot_id=self._parent_snapshot_id, ref_name="main",
- type="branch"
+ snapshot_id=self._snapshot_id, parent_snapshot_id=self._parent_snapshot_id, ref_name="main", type="branch"
),
),
(AssertRefSnapshotId(snapshot_id=self._transaction.table_metadata.current_snapshot_id, ref="main"),),
@@ -321,12 +318,12 @@ class _DeleteFiles(_SnapshotProducer["_DeleteFiles"]):
_case_sensitive: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
):
super().__init__(operation, transaction, io, commit_uuid, snapshot_properties)
self._predicate = AlwaysFalse()
@@ -379,8 +376,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
)
manifest_evaluators: Dict[int, Callable[[ManifestFile], bool]] = KeyDefaultDict(self._build_manifest_evaluator)
- strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate,
- case_sensitive=self._case_sensitive).eval
+ strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate, case_sensitive=self._case_sensitive).eval
inclusive_metrics_evaluator = _InclusiveMetricsEvaluator(
schema, self._predicate, case_sensitive=self._case_sensitive
).eval
@@ -416,11 +412,11 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
# Rewrite the manifest
if len(existing_entries) > 0:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
for existing_entry in existing_entries:
writer.add_entry(existing_entry)
@@ -449,182 +445,6 @@ def files_affected(self) -> bool:
return len(self._deleted_entries()) > 0
-class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
- """Rewrite manifest files based on the predicate."""
-
- KEPT_MANIFESTS_COUNT = "manifests-kept"
- CREATED_MANIFESTS_COUNT = "manifests-created"
- REPLACED_MANIFESTS_COUNT = "manifests-replaced"
- PROCESSED_ENTRY_COUNT = "entries-processed"
-
- def __init__(
- self,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
- ):
- from pyiceberg.table import TableProperties
-
- super().__init__(Operation.REPLACE, transaction, io, commit_uuid, snapshot_properties)
-
- self.specs_by_id = self._transaction.table_metadata.spec().spec_id
- self.manifest_target_size_bytes = property_as_int(
- self._transaction.table_metadata.properties,
- TableProperties.MANIFEST_TARGET_SIZE_BYTES,
- TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
- )
- self.deleted_manifests: Set[ManifestFile] = set()
- self.added_manifests: List[ManifestFile] = []
- self.rewritten_added_manifests: List[ManifestFile] = []
- self.kept_manifests: Set[ManifestFile] = set()
- self.new_manifests: Set[ManifestFile] = set()
- self.rewritten_manifests: Set[ManifestFile] = set()
-
- def copy_manifest(self, manifest: ManifestFile) -> ManifestFile:
- return ManifestFile(
- manifest_path=manifest.manifest_path,
- manifest_length=manifest.manifest_length,
- partition_spec_id=manifest.partition_spec_id,
- content=manifest.content,
- sequence_number=manifest.sequence_number,
- min_sequence_number=manifest.min_sequence_number,
- added_snapshot_id=manifest.added_snapshot_id,
- added_files_count=manifest.added_files_count,
- existing_files_count=manifest.existing_files_count,
- deleted_files_count=manifest.deleted_files_count,
- added_rows_count=manifest.added_rows_count,
- existing_rows_count=manifest.existing_rows_count,
- deleted_rows_count=manifest.deleted_rows_count,
- partitions=manifest.partitions,
- key_metadata=manifest.key_metadata,
- )
-
- def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
- summary = {
- self.CREATED_MANIFESTS_COUNT: str(
- len(self.new_manifests) + len(self.added_manifests) + len(self.rewritten_added_manifests)
- ),
- self.KEPT_MANIFESTS_COUNT: str(len(self.kept_manifests)),
- self.REPLACED_MANIFESTS_COUNT: str(len(self.rewritten_manifests) + len(self.deleted_manifests)),
- self.PROCESSED_ENTRY_COUNT: str(self.entry_count),
- }
- return super()._summary(summary)
-
- def delete_manifest(self, manifest):
- self.deleted_manifests.add(manifest)
- return self
-
- def requires_rewrite(self, current_manifests):
-
- if not self.rewritten_manifests:
- # nothing yet processed so perform a full rewrite
- return True
-
- # if any processed manifest is not in the current manifest list, perform a full rewrite
- return any(manifest not in current_manifests for manifest in self.rewritten_manifests)
-
- def keep_active_manifests(self, current_manifests: List[ManifestFile]) -> None:
- # keep any existing manifests as-is that were not processed
- self.kept_manifests.clear()
- for manifest in current_manifests:
- if manifest not in self.rewritten_manifests and manifest not in self.deleted_manifests:
- self.kept_manifests.add(manifest)
-
- def active_files_count(self, manifests):
- active_files_count = 0
-
- for manifest in manifests:
- if manifest.added_files_count is None:
- raise ValueError("Missing file counts in {}".format(manifest.path()))
- if manifest.existing_files_count is None:
- raise ValueError("Missing file counts in {}".format(manifest.path()))
- active_files_count += manifest.added_files_count
- active_files_count += manifest.existing_files_count
-
- return active_files_count
-
- def validate_files_counts(self):
- created_manifests = itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
- created_manifests_files_count = self.active_files_count(created_manifests)
-
- replaced_manifests = itertools.chain(self.rewritten_manifests, self.deleted_manifests)
- replaced_manifests_files_count = self.active_files_count(replaced_manifests)
-
- if created_manifests_files_count != replaced_manifests_files_count:
- raise ValueError(
- "Replaced and created manifests must have the same number of active files: {} (new), {} (old)".format(
- created_manifests_files_count, replaced_manifests_files_count
- )
- )
-
- def _existing_manifests(self) -> List[ManifestFile]:
- """Returns the list of manifests to include in the next snapshot."""
- return self.apply()
-
- def _deleted_entries(self) -> List[ManifestEntry]:
- """No entries are deleted during manifest rewriting."""
- return []
- def apply(self) -> List[ManifestFile]:
- snapshot = self._transaction.table_metadata.current_snapshot()
- current_manifests = snapshot.manifests(io=self._io)
- current_manifest_set = set(current_manifests)
- data_manifest_merge_manager = _ManifestMergeManager(
- target_size_bytes=self.manifest_target_size_bytes,
- min_count_to_merge=2,
- merge_enabled=True,
- snapshot_producer=self,
- )
- # self.validate_deleted_manifests(current_manifest_set, base.current_snapshot().snapshot_id())
-
- if self.requires_rewrite(current_manifest_set):
- new_manifests = data_manifest_merge_manager.merge_manifests(manifests=current_manifests)
- # self.rewritten_manifests.add(new_manifests)
- self.rewritten_manifests.update(new_manifests)
- else:
- self.keep_active_manifests(current_manifests)
-
- self.validate_files_counts()
-
- new_manifests_with_metadata = [
- manifest.copy_with_snapshot_id(self.snapshot_id)
- for manifest in itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
- ]
-
- # put new manifests at the beginning
- applyi = list(new_manifests_with_metadata)
- applyi.extend(self.kept_manifests)
-
- return applyi
-
- # def execute(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
- # data_manifest_merge_manager = _ManifestMergeManager(
- # target_size_bytes=self.manifest_target_size_bytes,
- # min_count_to_merge=2,
- # merge_enabled=True,
- # snapshot_producer=self,
- # )
- # return data_manifest_merge_manager.merge_manifests(manifests=manifests)
-
- def add_manifest(self, manifest):
- if manifest.has_added_files():
- raise ValueError("Cannot add manifest with added files")
- if manifest.has_deleted_files():
- raise ValueError("Cannot add manifest with deleted files")
- if manifest.snapshot_id() is not None and manifest.snapshot_id() != -1:
- raise ValueError("Snapshot id must be assigned during commit")
- if manifest.sequence_number() != -1:
- raise ValueError("Sequence must be assigned during commit")
-
- if manifest.snapshot_id() is None:
- self.added_manifests.append(manifest)
- else:
- copied_manifest = self.copy_manifest(manifest)
- self.rewritten_added_manifests.append(copied_manifest)
-
- return self
-
-
class _FastAppendFiles(_SnapshotProducer["_FastAppendFiles"]):
def _existing_manifests(self) -> List[ManifestFile]:
"""To determine if there are any existing manifest files.
@@ -654,18 +474,32 @@ def _deleted_entries(self) -> List[ManifestEntry]:
return []
+@dataclass(init=False)
+class RewriteManifestsResult:
+ rewritten_manifests: List[ManifestFile]
+ added_manifests: List[ManifestFile]
+
+ def __init__(
+ self,
+ rewritten_manifests: Optional[List[ManifestFile]],
+ added_manifests: Optional[List[ManifestFile]],
+ ) -> None:
+ self.rewritten_manifests = rewritten_manifests or []
+ self.added_manifests = added_manifests or []
+
+
class _MergeAppendFiles(_FastAppendFiles):
_target_size_bytes: int
_min_count_to_merge: int
_merge_enabled: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
from pyiceberg.table import TableProperties
@@ -705,6 +539,83 @@ def _process_manifests(self, manifests: List[ManifestFile]) -> List[ManifestFile
return data_manifest_merge_manager.merge_manifests(unmerged_data_manifests) + unmerged_deletes_manifests
+class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
+ _target_size_bytes: int
+ rewritten_manifests: List[ManifestFile] = []
+ added_manifests: List[ManifestFile] = []
+
+ def __init__(
+ self,
+ table: Table,
+ transaction: Transaction,
+ io: FileIO,
+ spec_id: Optional[int] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ ):
+ from pyiceberg.table import TableProperties
+
+ _table: Table
+ _spec: PartitionSpec
+
+ super().__init__(Operation.REPLACE, transaction, io, snapshot_properties=snapshot_properties)
+ self._target_size_bytes = property_as_int(
+ self._transaction.table_metadata.properties,
+ TableProperties.MANIFEST_TARGET_SIZE_BYTES,
+ TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
+ ) # type: ignore
+ self._table = table
+ self._spec_id = spec_id
+
+ def rewrite_manifests(self) -> RewriteManifestsResult:
+ data_result = self._find_matching_manifests(ManifestContent.DATA)
+ self.rewritten_manifests.extend(data_result.rewritten_manifests)
+ self.added_manifests.extend(data_result.added_manifests)
+
+ deletes_result = self._find_matching_manifests(ManifestContent.DELETES)
+ self.rewritten_manifests.extend(deletes_result.rewritten_manifests)
+ self.added_manifests.extend(deletes_result.added_manifests)
+
+ if not self.rewritten_manifests:
+ return RewriteManifestsResult(rewritten_manifests=[], added_manifests=[])
+
+ return RewriteManifestsResult(rewritten_manifests=self.rewritten_manifests, added_manifests=self.added_manifests)
+
+ def _find_matching_manifests(self, content: ManifestContent) -> RewriteManifestsResult:
+ snapshot = self._table.current_snapshot()
+ if self._spec_id and self._spec_id not in self._table.specs():
+ raise ValueError(f"Cannot find spec with id: {self._spec_id}")
+
+ if not snapshot:
+ raise ValueError("Cannot rewrite manifests without a current snapshot")
+
+ manifests = [
+ manifest
+ for manifest in snapshot.manifests(io=self._io)
+ if manifest.partition_spec_id == self._spec_id and manifest.content == content
+ ]
+
+ data_manifest_merge_manager = _ManifestMergeManager(
+ target_size_bytes=self._target_size_bytes,
+ min_count_to_merge=2,
+ merge_enabled=True,
+ snapshot_producer=self,
+ )
+ new_manifests = data_manifest_merge_manager.merge_manifests(manifests=manifests)
+
+ return RewriteManifestsResult(rewritten_manifests=manifests, added_manifests=new_manifests)
+
+ def _existing_manifests(self) -> List[ManifestFile]:
+ """Determine if there are any existing manifest files."""
+ return []
+
+ def _deleted_entries(self) -> List[ManifestEntry]:
+ """To determine if we need to record any deleted manifest entries.
+
+ In case of an append, nothing is deleted.
+ """
+ return []
+
+
class _OverwriteFiles(_SnapshotProducer["_OverwriteFiles"]):
"""Overwrites data from the table. This will produce an OVERWRITE snapshot.
@@ -718,8 +629,7 @@ def _existing_manifests(self) -> List[ManifestFile]:
if snapshot := self._transaction.table_metadata.current_snapshot():
for manifest_file in snapshot.manifests(io=self._io):
entries = manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True)
- found_deleted_data_files = [entry.data_file for entry in entries if
- entry.data_file in self._deleted_data_files]
+ found_deleted_data_files = [entry.data_file for entry in entries if entry.data_file in self._deleted_data_files]
if len(found_deleted_data_files) == 0:
existing_files.append(manifest_file)
@@ -727,11 +637,11 @@ def _existing_manifests(self) -> List[ManifestFile]:
# We have to rewrite the manifest file without the deleted data files
if any(entry.data_file not in found_deleted_data_files for entry in entries):
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
) as writer:
[
writer.add_entry(
@@ -795,19 +705,20 @@ def __init__(self, transaction: Transaction, io: FileIO, snapshot_properties: Di
def fast_append(self) -> _FastAppendFiles:
return _FastAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io,
- snapshot_properties=self._snapshot_properties
+ operation=Operation.APPEND, transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
)
def merge_append(self) -> _MergeAppendFiles:
return _MergeAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io,
- snapshot_properties=self._snapshot_properties
+ operation=Operation.APPEND, transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
)
def rewrite(self) -> _RewriteManifests:
return _RewriteManifests(
- transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
+ table=self._transaction._table,
+ transaction=self._transaction,
+ io=self._io,
+ snapshot_properties=self._snapshot_properties,
)
def overwrite(self, commit_uuid: Optional[uuid.UUID] = None) -> _OverwriteFiles:
@@ -837,8 +748,7 @@ class _ManifestMergeManager(Generic[U]):
_snapshot_producer: _SnapshotProducer[U]
def __init__(
- self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool,
- snapshot_producer: _SnapshotProducer[U]
+ self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool, snapshot_producer: _SnapshotProducer[U]
) -> None:
self._target_size_bytes = target_size_bytes
self._min_count_to_merge = min_count_to_merge
@@ -867,10 +777,8 @@ def _create_manifest(self, spec_id: int, manifest_bin: List[ManifestFile]) -> Ma
return writer.to_manifest_file()
- def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[
- ManifestFile]:
- packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1,
- largest_bin_first=False)
+ def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[ManifestFile]:
+ packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1, largest_bin_first=False)
bins: List[List[ManifestFile]] = packer.pack_end(manifests, lambda m: m.manifest_length)
def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
@@ -892,8 +800,7 @@ def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
# for consistent ordering, we need to maintain future order
futures_index = {f: i for i, f in enumerate(futures)}
- completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[],
- key=lambda f: futures_index[f])
+ completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[], key=lambda f: futures_index[f])
for future in concurrent.futures.as_completed(futures):
completed_futures.add(future)
@@ -994,12 +901,12 @@ def remove_tag(self, tag_name: str) -> ManageSnapshots:
return self._remove_ref_snapshot(ref_name=tag_name)
def create_branch(
- self,
- snapshot_id: int,
- branch_name: str,
- max_ref_age_ms: Optional[int] = None,
- max_snapshot_age_ms: Optional[int] = None,
- min_snapshots_to_keep: Optional[int] = None,
+ self,
+ snapshot_id: int,
+ branch_name: str,
+ max_ref_age_ms: Optional[int] = None,
+ max_snapshot_age_ms: Optional[int] = None,
+ min_snapshots_to_keep: Optional[int] = None,
) -> ManageSnapshots:
"""
Create a new branch pointing to the given snapshot id.
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
new file mode 100644
index 0000000000..6d0d753227
--- /dev/null
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -0,0 +1,116 @@
+# 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.
+# pylint:disable=redefined-outer-name
+
+import pyarrow as pa
+import pytest
+from pyspark.sql import SparkSession
+
+from pyiceberg.catalog import Catalog
+from utils import _create_table
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v1_with_null(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.arrow_table_v1_with_null"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "1"}, [arrow_table_with_null])
+ assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v1_without_data(session_catalog: Catalog, arrow_table_without_data: pa.Table) -> None:
+ identifier = "default.arrow_table_v1_without_data"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "1"}, [arrow_table_without_data])
+ assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v1_with_only_nulls(session_catalog: Catalog, arrow_table_with_only_nulls: pa.Table) -> None:
+ identifier = "default.arrow_table_v1_with_only_nulls"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "1"}, [arrow_table_with_only_nulls])
+ assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v1_appended_with_null(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.arrow_table_v1_appended_with_null"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "1"}, 2 * [arrow_table_with_null])
+ assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v2_with_null(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.arrow_table_v2_with_null"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_null])
+ assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v2_without_data(session_catalog: Catalog, arrow_table_without_data: pa.Table) -> None:
+ identifier = "default.arrow_table_v2_without_data"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_without_data])
+ assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v2_with_only_nulls(session_catalog: Catalog, arrow_table_with_only_nulls: pa.Table) -> None:
+ identifier = "default.arrow_table_v2_with_only_nulls"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_only_nulls])
+ assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v2_appended_with_null(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.arrow_table_v2_appended_with_null"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, 2 * [arrow_table_with_null])
+ assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
+
+
+@pytest.fixture(scope="session", autouse=True)
+def table_v1_v2_appended_with_null(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.arrow_table_v1_v2_appended_with_null"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "1"}, [arrow_table_with_null])
+ assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
+
+ with tbl.transaction() as tx:
+ tx.upgrade_table_version(format_version=2)
+
+ tbl.append(arrow_table_with_null)
+
+ assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
+
+
+@pytest.mark.integration
+def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.arrow_table_summaries"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
+ tbl.append(arrow_table_with_null)
+ tbl.append(arrow_table_with_null)
+
+ # tbl.rewrite_manifests()
+
+ # records1 = [ThreeColumnRecord(1, None, "AAAA")]
+ # write_records(spark, table_location, records1)
+
+ tbl.refresh()
+ manifests = tbl.inspect.all_manifests().to_pylist()
+ assert len(manifests) == 3, "Should have 3 manifests before rewrite"
+
+ result = tbl.rewrite_manifests()
+ tbl.refresh()
+ manifests = tbl.inspect.all_manifests().to_pylist()
+ assert len(manifests) == 1, "Should have 1 manifests before rewrite"
From 7b82638383a6260512c87a1d3b2d1c2e457423b3 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Thu, 13 Feb 2025 22:21:07 +0200
Subject: [PATCH 003/112] fix lint
---
pyiceberg/table/__init__.py | 7 +++----
tests/integration/test_writes/test_rewrite_manifests.py | 2 +-
2 files changed, 4 insertions(+), 5 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index d831e8fd74..19a1d2d1b7 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -115,7 +115,7 @@
update_table_metadata,
)
from pyiceberg.table.update.schema import UpdateSchema
-from pyiceberg.table.update.snapshot import ManageSnapshots, UpdateSnapshot, _FastAppendFiles, RewriteManifestsResult
+from pyiceberg.table.update.snapshot import ManageSnapshots, UpdateSnapshot, _FastAppendFiles
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
from pyiceberg.transforms import IdentityTransform
@@ -1375,7 +1375,7 @@ def add_files(
def rewrite_manifests(
self,
spec_id: Optional[int] = None,
- ) -> RewriteManifestsResult:
+ ) -> None:
"""
Shorthand API for Rewriting manifests for the table.
@@ -1384,8 +1384,7 @@ def rewrite_manifests(
"""
with self.transaction() as tx:
- rewrite_results = tx.rewrite_manifests(spec_id=spec_id)
- return rewrite_results
+ tx.rewrite_manifests(spec_id=spec_id)
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index 6d0d753227..1946ef542d 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -110,7 +110,7 @@ def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_wi
manifests = tbl.inspect.all_manifests().to_pylist()
assert len(manifests) == 3, "Should have 3 manifests before rewrite"
- result = tbl.rewrite_manifests()
+ tbl.rewrite_manifests()
tbl.refresh()
manifests = tbl.inspect.all_manifests().to_pylist()
assert len(manifests) == 1, "Should have 1 manifests before rewrite"
From 15a8f6adcf8c1065c898910d8aee3bf5e2489b95 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Thu, 13 Feb 2025 22:42:52 +0200
Subject: [PATCH 004/112] remove unused code
---
pyiceberg/manifest.py | 19 -------------------
1 file changed, 19 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index dc66371aa3..26f11792c0 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -693,25 +693,6 @@ def has_added_files(self) -> bool:
def has_existing_files(self) -> bool:
return self.existing_files_count is None or self.existing_files_count > 0
- def copy_with_snapshot_id(self, snapshot_id: int) -> ManifestFile:
- return ManifestFile(
- manifest_path=self.manifest_path,
- manifest_length=self.manifest_length,
- partition_spec_id=self.partition_spec_id,
- content=self.content,
- sequence_number=self.sequence_number,
- min_sequence_number=self.min_sequence_number,
- added_snapshot_id=snapshot_id,
- added_files_count=self.added_files_count,
- existing_files_count=self.existing_files_count,
- deleted_files_count=self.deleted_files_count,
- added_rows_count=self.added_rows_count,
- existing_rows_count=self.existing_rows_count,
- deleted_rows_count=self.deleted_rows_count,
- partitions=self.partitions,
- key_metadata=self.key_metadata,
- )
-
def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List[ManifestEntry]:
"""
Read the manifest entries from the manifest file.
From 443ffda777ebdfb198596bf6d314c62ad51060a8 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Mon, 17 Feb 2025 19:34:19 +0200
Subject: [PATCH 005/112] commit
---
pyiceberg/table/__init__.py | 3 +
pyiceberg/table/update/snapshot.py | 299 ++++++++++++++++++++++++++++-
2 files changed, 300 insertions(+), 2 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 19a1d2d1b7..b0c19c1aed 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -237,6 +237,9 @@ class TableProperties:
MIN_SNAPSHOTS_TO_KEEP = "history.expire.min-snapshots-to-keep"
MIN_SNAPSHOTS_TO_KEEP_DEFAULT = 1
+ SNAPSHOT_ID_INHERITANCE_ENABLED = "compatibility.snapshot-id-inheritance.enabled"
+ SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT = False
+
class Transaction:
_table: Table
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 07f5c3e1f1..2f21baeaa1 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -564,7 +564,7 @@ def __init__(
TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
) # type: ignore
self._table = table
- self._spec_id = spec_id
+ self._spec_id = spec_id or table.spec().spec_id
def rewrite_manifests(self) -> RewriteManifestsResult:
data_result = self._find_matching_manifests(ManifestContent.DATA)
@@ -604,9 +604,33 @@ def _find_matching_manifests(self, content: ManifestContent) -> RewriteManifests
return RewriteManifestsResult(rewritten_manifests=manifests, added_manifests=new_manifests)
+ def _copy_manifest_file(self, manifest_file:ManifestFile, snapshot_id:int) -> ManifestFile:
+ return ManifestFile(
+ manifest_path=manifest_file.manifest_path,
+ manifest_length=manifest_file.manifest_length,
+ partition_spec_id=manifest_file.partition_spec_id,
+ content=manifest_file.content,
+ sequence_number=manifest_file.sequence_number,
+ min_sequence_number=manifest_file.min_sequence_number,
+ added_snapshot_id=snapshot_id,
+ added_files_count=manifest_file.added_files_count,
+ existing_files_count=manifest_file.existing_files_count,
+ deleted_files_count=manifest_file.deleted_files_count,
+ added_rows_count=manifest_file.added_rows_count,
+ existing_rows_count=manifest_file.existing_rows_count,
+ deleted_rows_count=manifest_file.deleted_rows_count,
+ partitions=manifest_file.partitions,
+ key_metadata=manifest_file.key_metadata,
+ )
+
def _existing_manifests(self) -> List[ManifestFile]:
"""Determine if there are any existing manifest files."""
- return []
+ new_manifests = [
+ self._copy_manifest_file(manifest, self.snapshot_id)
+ for manifest in self.added_manifests + self.rewritten_manifests
+ ]
+ return new_manifests
+ # return []
def _deleted_entries(self) -> List[ManifestEntry]:
"""To determine if we need to record any deleted manifest entries.
@@ -616,6 +640,277 @@ def _deleted_entries(self) -> List[ManifestEntry]:
return []
+# class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
+# KEPT_MANIFESTS_COUNT = "manifests-kept"
+# CREATED_MANIFESTS_COUNT = "manifests-created"
+# REPLACED_MANIFESTS_COUNT = "manifests-replaced"
+# PROCESSED_ENTRY_COUNT = "entries-processed"
+# _target_size_bytes: int
+# _min_count_to_merge: int
+#
+# def __init__(
+# self,
+# table: Table,
+# transaction: Transaction,
+# io: FileIO,
+# commit_uuid: Optional[uuid.UUID] = None,
+# snapshot_properties: Dict[str, str] = EMPTY_DICT,
+# ) -> None:
+# from pyiceberg.table import TableProperties
+#
+# super().__init__(Operation.REPLACE, transaction, io, commit_uuid, snapshot_properties)
+# self._table = table
+# self.specs_by_id = self._table.spec() # ops.current().specs_by_id()
+# self.manifest_target_size_bytes = 8388608 # Default value
+# self.deleted_manifests: Set[ManifestFile] = set()
+# self.added_manifests: List[ManifestFile] = []
+# self.rewritten_added_manifests: List[ManifestFile] = []
+# self.kept_manifests: List[ManifestFile] = []
+# self.new_manifests: List[ManifestFile] = []
+# self.rewritten_manifests: Set[ManifestFile] = set()
+# self.snapshot_Id_Inheritance_enabled = property_as_bool(
+# self._transaction.table_metadata.properties,
+# TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED,
+# TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT,
+# ) # type: ignore
+# self.can_inherit_snapshot_id = table.metadata.format_version > 1 or self.snapshot_Id_Inheritance_enabled
+# # self.writers: Dict[Tuple[Any, int], 'WriterWrapper'] = {}
+# self.entry_count = 0
+# self.cluster_by_func: Optional[Callable[[DataFile], Any]] = None
+# self.predicate: Optional[Callable[[ManifestFile], bool]] = None
+# # self.summary_builder = SnapshotSummary.Builder()
+# # self.lock = threading.Lock()
+# self._target_size_bytes = property_as_int(
+# self._transaction.table_metadata.properties,
+# TableProperties.MANIFEST_TARGET_SIZE_BYTES,
+# TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
+# ) # type: ignore
+# self._min_count_to_merge = property_as_int(
+# self._transaction.table_metadata.properties,
+# TableProperties.MANIFEST_MIN_MERGE_COUNT,
+# TableProperties.MANIFEST_MIN_MERGE_COUNT_DEFAULT,
+# ) # type: ignore
+#
+# # def cluster_by(self, func: Callable[[DataFile], Any]]) -> RewriteManifests:
+# # self.cluster_by_func = func
+# # return self
+#
+# # def rewrite_if(self, predicate: Callable[[ManifestFile], bool]]) -> RewriteManifests:
+# # self.predicate = predicate
+# # return self
+#
+# # def delete_manifest(self, manifest: ManifestFile) -> "_RewriteManifests":
+# # self.deleted_manifests.add(manifest)
+# # return self
+# #
+# # def add_manifest(self, manifest: ManifestFile) -> "_RewriteManifests":
+# # if self.can_inherit_snapshot_id and manifest.added_snapshot_id is None:
+# # self.added_manifests.append(manifest)
+# # else:
+# # copied_manifest = self.copy_manifest(manifest)
+# # self.rewritten_added_manifests.append(copied_manifest)
+# # return self
+#
+# # def copy_manifest(self, manifest: ManifestFile) -> ManifestFile:
+# # """
+# # Copies a manifest file to a new location, updating its metadata (e.g., snapshot ID).
+# #
+# # Args:
+# # manifest: The manifest file to copy.
+# #
+# # Returns:
+# # A new ManifestFile object representing the copied manifest.
+# # """
+# # # Get the current table metadata
+# # current_metadata = self._table.metadata
+# #
+# # # Create an input file for the manifest to be copied
+# # input_file = self.ops.io().new_input_file(manifest.path)
+# #
+# # # Create an output file for the new manifest
+# # output_file = self.new_manifest_output_file()
+# #
+# # # Copy the manifest file, updating its metadata
+# # new_manifest = ManifestFile(
+# # format_version=current_metadata.format_version,
+# # spec_id=manifest.partition_spec_id,
+# # input_file=input_file,
+# # specs_by_id=self.specs_by_id,
+# # output_file=output_file,
+# # snapshot_id=self.snapshot_id(),
+# # summary_builder=self.summary_builder,
+# # )
+# #
+# # return new_manifest
+#
+# def _existing_manifests(self) -> List[ManifestFile]:
+# """To determine if there are any existing manifest files.
+#
+# A fast append will add another ManifestFile to the ManifestList.
+# All the existing manifest files are considered existing.
+# """
+# existing_manifests = []
+#
+# if self._parent_snapshot_id is not None:
+# previous_snapshot = self._transaction.table_metadata.snapshot_by_id(self._parent_snapshot_id)
+#
+# if previous_snapshot is None:
+# raise ValueError(f"Snapshot could not be found: {self._parent_snapshot_id}")
+#
+# for manifest in previous_snapshot.manifests(io=self._io):
+# if manifest.has_added_files() or manifest.has_existing_files() or manifest.added_snapshot_id == self._snapshot_id:
+# existing_manifests.append(manifest)
+#
+# return existing_manifests
+# def _deleted_entries(self) -> List[ManifestEntry]:
+# """To determine if we need to record any deleted manifest entries.
+#
+# In case of an append, nothing is deleted.
+# """
+# return []
+#
+#
+# def rewrite_manifests(self) -> List[ManifestFile]:
+# snapshot = self._table.current_snapshot()
+# if not snapshot:
+# raise ValueError("Cannot rewrite manifests without a current snapshot")
+# current_manifests = snapshot.manifests(self._io)
+# current_manifest_set = set(current_manifests)
+# self.validate_deleted_manifests(current_manifest_set, snapshot.snapshot_id)
+#
+# if self.requires_rewrite(current_manifest_set):
+# self.perform_rewrite(current_manifests)
+# else:
+# self.keep_active_manifests(current_manifests)
+#
+# self.validate_files_counts()
+#
+# new_manifests = [
+# self.with_snapshot_id(manifest)
+# for manifest in list(self.new_manifests) + self.added_manifests + self.rewritten_added_manifests
+# ]
+# return new_manifests + list(self.kept_manifests)
+#
+# def perform_rewrite(self, current_manifests: List[ManifestFile]) -> None:
+# # self.reset()
+# remaining_manifests = [m for m in current_manifests if m not in self.deleted_manifests]
+# data_manifest_merge_manager = _ManifestMergeManager(
+# target_size_bytes=self._target_size_bytes,
+# min_count_to_merge=self._min_count_to_merge,
+# merge_enabled=True,
+# snapshot_producer=self,
+# )
+#
+# new_manifests = data_manifest_merge_manager.merge_manifests(remaining_manifests)
+# self.new_manifests.extend(new_manifests)
+#
+# # def process_manifest(self, manifest: ManifestFile):
+# # if not self.contains_deletes(manifest) and self.matches_predicate(manifest):
+# # self.rewritten_manifests.add(manifest)
+# # try:
+# # reader = ManifestReader(manifest, self.ops.io(), self.ops.current().specs_by_id())
+# # for entry in reader.live_entries():
+# # self.append_entry(entry, self.cluster_by_func(entry.file), manifest.partition_spec_id)
+# # reader.close()
+# # except IOError as e:
+# # raise RuntimeIOException from e
+# # else:
+# # self.kept_manifests.put(manifest)
+#
+# # def append_entry(self, entry: ManifestEntry, key: Any, partition_spec_id: int):
+# # with self.lock:
+# # writer = self.get_writer(key, partition_spec_id)
+# # writer.add_entry(entry)
+# # self.entry_count += 1
+#
+# # def get_writer(self, key: Any, partition_spec_id: int) -> 'WriterWrapper':
+# # return self.writers.setdefault((key, partition_spec_id), WriterWrapper(self, self.specs_by_id[partition_spec_id]))
+#
+# def validate_deleted_manifests(self, current_manifests: Set[ManifestFile], current_snapshot_id: int) -> None:
+# for manifest in self.deleted_manifests:
+# if manifest not in current_manifests:
+# raise ValueError(
+# f"Deleted manifest {manifest.manifest_path} could not be found in the latest snapshot {current_snapshot_id}"
+# )
+#
+# def requires_rewrite(self, current_manifests: Set[ManifestFile]) -> bool:
+# # if self.cluster_by_func is None:
+# # return False
+# return len(self.rewritten_manifests) == 0 or any(
+# manifest not in current_manifests for manifest in self.rewritten_manifests
+# )
+#
+# def keep_active_manifests(self, current_manifests: List[ManifestFile]) -> None:
+# for manifest in current_manifests:
+# if manifest not in self.rewritten_manifests and manifest not in self.deleted_manifests:
+# self.kept_manifests.append(manifest)
+#
+# def validate_files_counts(self) -> None:
+# created_manifests = list(self.new_manifests) + self.added_manifests + self.rewritten_added_manifests
+# created_files_count = self.active_files_count(created_manifests)
+# replaced_manifests = list(self.rewritten_manifests) + list(self.deleted_manifests)
+# replaced_files_count = self.active_files_count(replaced_manifests)
+#
+# if created_files_count != replaced_files_count:
+# raise ValueError(
+# f"Replaced and created manifests must have the same number of active files: {created_files_count} (new), {replaced_files_count} (old)"
+# )
+#
+# def active_files_count(self, manifests: List[ManifestFile]) -> int:
+# count = 0
+# for manifest in manifests:
+# count += manifest.existing_files_count + manifest.added_files_count
+# return count
+#
+# # def reset(self) -> None:
+# # self.clean_uncommitted(self.new_manifests, set())
+# # self.entry_count = 0
+# # self.kept_manifests.clear()
+# # self.rewritten_manifests.clear()
+# # self.new_manifests.clear()
+# #
+# # def clean_uncommitted(self, manifests: List[ManifestFile], committed: Set[ManifestFile]) -> None:
+# # for manifest in manifests:
+# # if manifest not in committed:
+# # self.delete_file(manifest.manifest_path)
+# #
+# # def delete_file(self, path: str):
+# # # Mock implementation
+# # if os.path.exists(path):
+# # os.remove(path)
+#
+# def with_snapshot_id(self, manifest: ManifestFile) -> ManifestFile:
+# # Mock implementation
+# return ManifestFile(manifest.manifest_path, snapshot_id=0)
+#
+# def new_manifest_writer(self, spec: PartitionSpec) -> ManifestWriter:
+# # Mock implementation
+# return ManifestWriter(spec)
+#
+# # class WriterWrapper:
+# # def __init__(self, outer: 'BaseRewriteManifests', spec: PartitionSpec):
+# # self.outer = outer
+# # self.spec = spec
+# # self.writer: Optional[ManifestWriter] = None
+# # self.lock = threading.Lock()
+# #
+# # def add_entry(self, entry: ManifestEntry):
+# # with self.lock:
+# # if self.writer is None or self.writer.length() >= self.outer.manifest_target_size_bytes:
+# # self._close_writer()
+# # self.writer = self.outer.new_manifest_writer(self.spec)
+# # self.writer.existing(entry)
+# #
+# # def _close_writer(self):
+# # if self.writer:
+# # self.writer.close()
+# # self.outer.new_manifests.put(self.writer.to_manifest_file())
+# #
+# # def close(self):
+# # with self.lock:
+# # self._close_writer()
+
+
class _OverwriteFiles(_SnapshotProducer["_OverwriteFiles"]):
"""Overwrites data from the table. This will produce an OVERWRITE snapshot.
From 10affa8349174e49cd825336638c5d4614ec48fc Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Sun, 23 Feb 2025 09:05:46 +0200
Subject: [PATCH 006/112] 1. support returning rewrite results 2. write tests
for rewrite manifests
---
pyiceberg/table/__init__.py | 13 +-
pyiceberg/table/update/snapshot.py | 320 +++---------------
.../test_writes/test_rewrite_manifests.py | 125 ++++++-
3 files changed, 162 insertions(+), 296 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index b0c19c1aed..5b0e1da10f 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -115,7 +115,7 @@
update_table_metadata,
)
from pyiceberg.table.update.schema import UpdateSchema
-from pyiceberg.table.update.snapshot import ManageSnapshots, UpdateSnapshot, _FastAppendFiles
+from pyiceberg.table.update.snapshot import ManageSnapshots, RewriteManifestsResult, UpdateSnapshot, _FastAppendFiles
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
from pyiceberg.transforms import IdentityTransform
@@ -438,9 +438,12 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> U
"""
return UpdateSnapshot(self, io=self._table.io, snapshot_properties=snapshot_properties)
- def rewrite_manifests(self) -> None:
+ def rewrite_manifests(self) -> RewriteManifestsResult:
+ if self._table.current_snapshot() is None:
+ return RewriteManifestsResult(rewritten_manifests=[], added_manifests=[])
with self.update_snapshot().rewrite() as rewrite:
- rewrite.rewrite_manifests()
+ rewritten = rewrite.rewrite_manifests()
+ return rewritten
def update_statistics(self) -> UpdateStatistics:
"""
@@ -1378,7 +1381,7 @@ def add_files(
def rewrite_manifests(
self,
spec_id: Optional[int] = None,
- ) -> None:
+ ) -> RewriteManifestsResult:
"""
Shorthand API for Rewriting manifests for the table.
@@ -1387,7 +1390,7 @@ def rewrite_manifests(
"""
with self.transaction() as tx:
- tx.rewrite_manifests(spec_id=spec_id)
+ return tx.rewrite_manifests(spec_id=spec_id)
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 2f21baeaa1..5cad13cbe1 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -24,7 +24,7 @@
from concurrent.futures import Future
from dataclasses import dataclass
from functools import cached_property
-from typing import TYPE_CHECKING, Callable, Dict, Generic, List, Optional, Set, Tuple
+from typing import TYPE_CHECKING, Any, Callable, Dict, Generic, List, Optional, Set, Tuple
from sortedcontainers import SortedList
@@ -543,6 +543,7 @@ class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
_target_size_bytes: int
rewritten_manifests: List[ManifestFile] = []
added_manifests: List[ManifestFile] = []
+ kept_manifests: List[ManifestFile] = []
def __init__(
self,
@@ -566,8 +567,38 @@ def __init__(
self._table = table
self._spec_id = spec_id or table.spec().spec_id
+ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
+ from pyiceberg.table import TableProperties
+
+ ssc = SnapshotSummaryCollector()
+ partition_summary_limit = int(
+ self._transaction.table_metadata.properties.get(
+ TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT
+ )
+ )
+ ssc.set_partition_summary_limit(partition_summary_limit)
+
+ props = {
+ "manifests-kept": str(len([])),
+ "manifests-created": str(len(self.added_manifests)),
+ "manifests-replaced": str(len(self.rewritten_manifests)),
+ "entries-processed": str(len([])),
+ }
+ previous_snapshot = (
+ self._transaction.table_metadata.snapshot_by_id(self._parent_snapshot_id)
+ if self._parent_snapshot_id is not None
+ else None
+ )
+
+ return update_snapshot_summaries(
+ summary=Summary(operation=self._operation, **ssc.build(), **props),
+ previous_summary=previous_snapshot.summary if previous_snapshot is not None else None,
+ truncate_full_table=False,
+ )
+
def rewrite_manifests(self) -> RewriteManifestsResult:
data_result = self._find_matching_manifests(ManifestContent.DATA)
+
self.rewritten_manifests.extend(data_result.rewritten_manifests)
self.added_manifests.extend(data_result.added_manifests)
@@ -604,8 +635,8 @@ def _find_matching_manifests(self, content: ManifestContent) -> RewriteManifests
return RewriteManifestsResult(rewritten_manifests=manifests, added_manifests=new_manifests)
- def _copy_manifest_file(self, manifest_file:ManifestFile, snapshot_id:int) -> ManifestFile:
- return ManifestFile(
+ def _copy_manifest_file(self, manifest_file: ManifestFile, snapshot_id: int) -> ManifestFile:
+ return ManifestFile(
manifest_path=manifest_file.manifest_path,
manifest_length=manifest_file.manifest_length,
partition_spec_id=manifest_file.partition_spec_id,
@@ -623,14 +654,14 @@ def _copy_manifest_file(self, manifest_file:ManifestFile, snapshot_id:int) -> Ma
key_metadata=manifest_file.key_metadata,
)
+ def __exit__(self, _: Any, value: Any, traceback: Any) -> None:
+ """Commit only if we have rewritten any manifests."""
+ if self.rewritten_manifests:
+ self.commit()
+
def _existing_manifests(self) -> List[ManifestFile]:
"""Determine if there are any existing manifest files."""
- new_manifests = [
- self._copy_manifest_file(manifest, self.snapshot_id)
- for manifest in self.added_manifests + self.rewritten_manifests
- ]
- return new_manifests
- # return []
+ return [self._copy_manifest_file(manifest, self.snapshot_id) for manifest in self.added_manifests]
def _deleted_entries(self) -> List[ManifestEntry]:
"""To determine if we need to record any deleted manifest entries.
@@ -640,277 +671,6 @@ def _deleted_entries(self) -> List[ManifestEntry]:
return []
-# class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
-# KEPT_MANIFESTS_COUNT = "manifests-kept"
-# CREATED_MANIFESTS_COUNT = "manifests-created"
-# REPLACED_MANIFESTS_COUNT = "manifests-replaced"
-# PROCESSED_ENTRY_COUNT = "entries-processed"
-# _target_size_bytes: int
-# _min_count_to_merge: int
-#
-# def __init__(
-# self,
-# table: Table,
-# transaction: Transaction,
-# io: FileIO,
-# commit_uuid: Optional[uuid.UUID] = None,
-# snapshot_properties: Dict[str, str] = EMPTY_DICT,
-# ) -> None:
-# from pyiceberg.table import TableProperties
-#
-# super().__init__(Operation.REPLACE, transaction, io, commit_uuid, snapshot_properties)
-# self._table = table
-# self.specs_by_id = self._table.spec() # ops.current().specs_by_id()
-# self.manifest_target_size_bytes = 8388608 # Default value
-# self.deleted_manifests: Set[ManifestFile] = set()
-# self.added_manifests: List[ManifestFile] = []
-# self.rewritten_added_manifests: List[ManifestFile] = []
-# self.kept_manifests: List[ManifestFile] = []
-# self.new_manifests: List[ManifestFile] = []
-# self.rewritten_manifests: Set[ManifestFile] = set()
-# self.snapshot_Id_Inheritance_enabled = property_as_bool(
-# self._transaction.table_metadata.properties,
-# TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED,
-# TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT,
-# ) # type: ignore
-# self.can_inherit_snapshot_id = table.metadata.format_version > 1 or self.snapshot_Id_Inheritance_enabled
-# # self.writers: Dict[Tuple[Any, int], 'WriterWrapper'] = {}
-# self.entry_count = 0
-# self.cluster_by_func: Optional[Callable[[DataFile], Any]] = None
-# self.predicate: Optional[Callable[[ManifestFile], bool]] = None
-# # self.summary_builder = SnapshotSummary.Builder()
-# # self.lock = threading.Lock()
-# self._target_size_bytes = property_as_int(
-# self._transaction.table_metadata.properties,
-# TableProperties.MANIFEST_TARGET_SIZE_BYTES,
-# TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
-# ) # type: ignore
-# self._min_count_to_merge = property_as_int(
-# self._transaction.table_metadata.properties,
-# TableProperties.MANIFEST_MIN_MERGE_COUNT,
-# TableProperties.MANIFEST_MIN_MERGE_COUNT_DEFAULT,
-# ) # type: ignore
-#
-# # def cluster_by(self, func: Callable[[DataFile], Any]]) -> RewriteManifests:
-# # self.cluster_by_func = func
-# # return self
-#
-# # def rewrite_if(self, predicate: Callable[[ManifestFile], bool]]) -> RewriteManifests:
-# # self.predicate = predicate
-# # return self
-#
-# # def delete_manifest(self, manifest: ManifestFile) -> "_RewriteManifests":
-# # self.deleted_manifests.add(manifest)
-# # return self
-# #
-# # def add_manifest(self, manifest: ManifestFile) -> "_RewriteManifests":
-# # if self.can_inherit_snapshot_id and manifest.added_snapshot_id is None:
-# # self.added_manifests.append(manifest)
-# # else:
-# # copied_manifest = self.copy_manifest(manifest)
-# # self.rewritten_added_manifests.append(copied_manifest)
-# # return self
-#
-# # def copy_manifest(self, manifest: ManifestFile) -> ManifestFile:
-# # """
-# # Copies a manifest file to a new location, updating its metadata (e.g., snapshot ID).
-# #
-# # Args:
-# # manifest: The manifest file to copy.
-# #
-# # Returns:
-# # A new ManifestFile object representing the copied manifest.
-# # """
-# # # Get the current table metadata
-# # current_metadata = self._table.metadata
-# #
-# # # Create an input file for the manifest to be copied
-# # input_file = self.ops.io().new_input_file(manifest.path)
-# #
-# # # Create an output file for the new manifest
-# # output_file = self.new_manifest_output_file()
-# #
-# # # Copy the manifest file, updating its metadata
-# # new_manifest = ManifestFile(
-# # format_version=current_metadata.format_version,
-# # spec_id=manifest.partition_spec_id,
-# # input_file=input_file,
-# # specs_by_id=self.specs_by_id,
-# # output_file=output_file,
-# # snapshot_id=self.snapshot_id(),
-# # summary_builder=self.summary_builder,
-# # )
-# #
-# # return new_manifest
-#
-# def _existing_manifests(self) -> List[ManifestFile]:
-# """To determine if there are any existing manifest files.
-#
-# A fast append will add another ManifestFile to the ManifestList.
-# All the existing manifest files are considered existing.
-# """
-# existing_manifests = []
-#
-# if self._parent_snapshot_id is not None:
-# previous_snapshot = self._transaction.table_metadata.snapshot_by_id(self._parent_snapshot_id)
-#
-# if previous_snapshot is None:
-# raise ValueError(f"Snapshot could not be found: {self._parent_snapshot_id}")
-#
-# for manifest in previous_snapshot.manifests(io=self._io):
-# if manifest.has_added_files() or manifest.has_existing_files() or manifest.added_snapshot_id == self._snapshot_id:
-# existing_manifests.append(manifest)
-#
-# return existing_manifests
-# def _deleted_entries(self) -> List[ManifestEntry]:
-# """To determine if we need to record any deleted manifest entries.
-#
-# In case of an append, nothing is deleted.
-# """
-# return []
-#
-#
-# def rewrite_manifests(self) -> List[ManifestFile]:
-# snapshot = self._table.current_snapshot()
-# if not snapshot:
-# raise ValueError("Cannot rewrite manifests without a current snapshot")
-# current_manifests = snapshot.manifests(self._io)
-# current_manifest_set = set(current_manifests)
-# self.validate_deleted_manifests(current_manifest_set, snapshot.snapshot_id)
-#
-# if self.requires_rewrite(current_manifest_set):
-# self.perform_rewrite(current_manifests)
-# else:
-# self.keep_active_manifests(current_manifests)
-#
-# self.validate_files_counts()
-#
-# new_manifests = [
-# self.with_snapshot_id(manifest)
-# for manifest in list(self.new_manifests) + self.added_manifests + self.rewritten_added_manifests
-# ]
-# return new_manifests + list(self.kept_manifests)
-#
-# def perform_rewrite(self, current_manifests: List[ManifestFile]) -> None:
-# # self.reset()
-# remaining_manifests = [m for m in current_manifests if m not in self.deleted_manifests]
-# data_manifest_merge_manager = _ManifestMergeManager(
-# target_size_bytes=self._target_size_bytes,
-# min_count_to_merge=self._min_count_to_merge,
-# merge_enabled=True,
-# snapshot_producer=self,
-# )
-#
-# new_manifests = data_manifest_merge_manager.merge_manifests(remaining_manifests)
-# self.new_manifests.extend(new_manifests)
-#
-# # def process_manifest(self, manifest: ManifestFile):
-# # if not self.contains_deletes(manifest) and self.matches_predicate(manifest):
-# # self.rewritten_manifests.add(manifest)
-# # try:
-# # reader = ManifestReader(manifest, self.ops.io(), self.ops.current().specs_by_id())
-# # for entry in reader.live_entries():
-# # self.append_entry(entry, self.cluster_by_func(entry.file), manifest.partition_spec_id)
-# # reader.close()
-# # except IOError as e:
-# # raise RuntimeIOException from e
-# # else:
-# # self.kept_manifests.put(manifest)
-#
-# # def append_entry(self, entry: ManifestEntry, key: Any, partition_spec_id: int):
-# # with self.lock:
-# # writer = self.get_writer(key, partition_spec_id)
-# # writer.add_entry(entry)
-# # self.entry_count += 1
-#
-# # def get_writer(self, key: Any, partition_spec_id: int) -> 'WriterWrapper':
-# # return self.writers.setdefault((key, partition_spec_id), WriterWrapper(self, self.specs_by_id[partition_spec_id]))
-#
-# def validate_deleted_manifests(self, current_manifests: Set[ManifestFile], current_snapshot_id: int) -> None:
-# for manifest in self.deleted_manifests:
-# if manifest not in current_manifests:
-# raise ValueError(
-# f"Deleted manifest {manifest.manifest_path} could not be found in the latest snapshot {current_snapshot_id}"
-# )
-#
-# def requires_rewrite(self, current_manifests: Set[ManifestFile]) -> bool:
-# # if self.cluster_by_func is None:
-# # return False
-# return len(self.rewritten_manifests) == 0 or any(
-# manifest not in current_manifests for manifest in self.rewritten_manifests
-# )
-#
-# def keep_active_manifests(self, current_manifests: List[ManifestFile]) -> None:
-# for manifest in current_manifests:
-# if manifest not in self.rewritten_manifests and manifest not in self.deleted_manifests:
-# self.kept_manifests.append(manifest)
-#
-# def validate_files_counts(self) -> None:
-# created_manifests = list(self.new_manifests) + self.added_manifests + self.rewritten_added_manifests
-# created_files_count = self.active_files_count(created_manifests)
-# replaced_manifests = list(self.rewritten_manifests) + list(self.deleted_manifests)
-# replaced_files_count = self.active_files_count(replaced_manifests)
-#
-# if created_files_count != replaced_files_count:
-# raise ValueError(
-# f"Replaced and created manifests must have the same number of active files: {created_files_count} (new), {replaced_files_count} (old)"
-# )
-#
-# def active_files_count(self, manifests: List[ManifestFile]) -> int:
-# count = 0
-# for manifest in manifests:
-# count += manifest.existing_files_count + manifest.added_files_count
-# return count
-#
-# # def reset(self) -> None:
-# # self.clean_uncommitted(self.new_manifests, set())
-# # self.entry_count = 0
-# # self.kept_manifests.clear()
-# # self.rewritten_manifests.clear()
-# # self.new_manifests.clear()
-# #
-# # def clean_uncommitted(self, manifests: List[ManifestFile], committed: Set[ManifestFile]) -> None:
-# # for manifest in manifests:
-# # if manifest not in committed:
-# # self.delete_file(manifest.manifest_path)
-# #
-# # def delete_file(self, path: str):
-# # # Mock implementation
-# # if os.path.exists(path):
-# # os.remove(path)
-#
-# def with_snapshot_id(self, manifest: ManifestFile) -> ManifestFile:
-# # Mock implementation
-# return ManifestFile(manifest.manifest_path, snapshot_id=0)
-#
-# def new_manifest_writer(self, spec: PartitionSpec) -> ManifestWriter:
-# # Mock implementation
-# return ManifestWriter(spec)
-#
-# # class WriterWrapper:
-# # def __init__(self, outer: 'BaseRewriteManifests', spec: PartitionSpec):
-# # self.outer = outer
-# # self.spec = spec
-# # self.writer: Optional[ManifestWriter] = None
-# # self.lock = threading.Lock()
-# #
-# # def add_entry(self, entry: ManifestEntry):
-# # with self.lock:
-# # if self.writer is None or self.writer.length() >= self.outer.manifest_target_size_bytes:
-# # self._close_writer()
-# # self.writer = self.outer.new_manifest_writer(self.spec)
-# # self.writer.existing(entry)
-# #
-# # def _close_writer(self):
-# # if self.writer:
-# # self.writer.close()
-# # self.outer.new_manifests.put(self.writer.to_manifest_file())
-# #
-# # def close(self):
-# # with self.lock:
-# # self._close_writer()
-
-
class _OverwriteFiles(_SnapshotProducer["_OverwriteFiles"]):
"""Overwrites data from the table. This will produce an OVERWRITE snapshot.
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index 1946ef542d..faf1285058 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -18,7 +18,6 @@
import pyarrow as pa
import pytest
-from pyspark.sql import SparkSession
from pyiceberg.catalog import Catalog
from utils import _create_table
@@ -94,23 +93,127 @@ def table_v1_v2_appended_with_null(session_catalog: Catalog, arrow_table_with_nu
assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
+# @pytest.mark.integration
+# def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+# identifier = "default.arrow_table_summaries"
+# tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
+# tbl.append(arrow_table_with_null)
+# tbl.append(arrow_table_with_null)
+#
+# # tbl.rewrite_manifests()
+#
+# # records1 = [ThreeColumnRecord(1, None, "AAAA")]
+# # write_records(spark, table_location, records1)
+# before_pandas = tbl.scan().to_pandas()
+# before_count = before_pandas.shape[0]
+# tbl.refresh()
+# manifests = tbl.inspect.manifests().to_pylist()
+# assert len(manifests) == 2, "Should have 2 manifests before rewrite"
+#
+# tbl.rewrite_manifests()
+# tbl.refresh()
+#
+# after_pandas = tbl.scan().to_pandas()
+# after_count = before_pandas.shape[0]
+# manifests = tbl.inspect.manifests().to_pylist()
+# assert len(manifests) == 1, "Should have 1 manifests before rewrite"
+#
+# snaps = tbl.inspect.snapshots().to_pandas()
+# print(snaps)
+
+
@pytest.mark.integration
-def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
- identifier = "default.arrow_table_summaries"
+def test_rewrite_manifests_empty_table(session_catalog: Catalog) -> None:
+ # Create an unpartitioned table
+ identifier = "default.test_rewrite_manifests_empty_table"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
+
+ assert tbl.current_snapshot() is None, "Table must be empty"
+
+ # Execute rewrite manifests action
+ tbl.rewrite_manifests()
+
+ tbl.refresh()
+ assert tbl.current_snapshot() is None, "Table must stay empty"
+
+
+@pytest.mark.integration
+def test_rewrite_small_manifests_non_partitioned_table(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_rewrite_small_manifests_non_partitioned_table"
tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
tbl.append(arrow_table_with_null)
tbl.append(arrow_table_with_null)
+ tbl.refresh()
- # tbl.rewrite_manifests()
+ manifests = tbl.inspect.manifests()
+ assert len(manifests) == 2, "Should have 2 manifests before rewrite"
- # records1 = [ThreeColumnRecord(1, None, "AAAA")]
- # write_records(spark, table_location, records1)
+ result = tbl.rewrite_manifests()
+
+ assert len(result.rewritten_manifests) == 2, "Action should rewrite 2 manifests"
+ assert len(result.added_manifests) == 1, "Action should add 1 manifest"
tbl.refresh()
- manifests = tbl.inspect.all_manifests().to_pylist()
- assert len(manifests) == 3, "Should have 3 manifests before rewrite"
- tbl.rewrite_manifests()
+ current_snapshot = tbl.current_snapshot()
+ if not current_snapshot:
+ raise AssertionError
+ new_manifests = current_snapshot.manifests(tbl.io)
+ assert len(new_manifests) == 1, "Should have 1 manifest after rewrite"
+ assert new_manifests[0].existing_files_count == 2, "Should have 4 files in the new manifest"
+ assert new_manifests[0].added_files_count == 0, "Should have no added files in the new manifest"
+ assert new_manifests[0].deleted_files_count == 0, "Should have no deleted files in the new manifest"
+
+ # Validate the records
+ expected_records_count = arrow_table_with_null.shape[0] * 2
+ result_df = tbl.scan().to_pandas()
+ actual_records_count = result_df.shape[0]
+ assert expected_records_count == actual_records_count, "Rows must match"
+
+
+def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ # Create and append data files
+ # records1 = [ThreeColumnRecord(1, None, "AAAA"), ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")]
+ # records2 = [ThreeColumnRecord(2, "CCCCCCCCCC", "CCCC"), ThreeColumnRecord(2, "DDDDDDDDDD", "DDDD")]
+ # records3 = [ThreeColumnRecord(3, "EEEEEEEEEE", "EEEE"), ThreeColumnRecord(3, "FFFFFFFFFF", "FFFF")]
+ # records4 = [ThreeColumnRecord(4, "GGGGGGGGGG", "GGGG"), ThreeColumnRecord(4, "HHHHHHHHHH", "HHHH")]
+ # self.table.newFastAppend().appendFile(DataFile.from_records(records1)).commit()
+ # self.table.newFastAppend().appendFile(DataFile.from_records(records2)).commit()
+ # self.table.newFastAppend().appendFile(DataFile.from_records(records3)).commit()
+ # self.table.newFastAppend().appendFile(DataFile.from_records(records4)).commit()
+
+ identifier = "default.test_rewrite_small_manifests_non_partitioned_table"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
+ tbl.append(arrow_table_with_null)
+ tbl.append(arrow_table_with_null)
+ tbl.append(arrow_table_with_null)
+ tbl.append(arrow_table_with_null)
+
+ tbl.refresh()
+ manifests = tbl.current_snapshot().manifests(tbl.io)
+ assert len(manifests) == 4, "Should have 4 manifests before rewrite"
+
+ # Perform the rewrite manifests action
+ # actions = SparkActions.get()
+ result = tbl.rewrite_manifests()
+
+ assert len(result.rewritten_manifests) == 4, "Action should rewrite 4 manifests"
+ assert len(result.added_manifests) == 2, "Action should add 2 manifests"
+
tbl.refresh()
- manifests = tbl.inspect.all_manifests().to_pylist()
- assert len(manifests) == 1, "Should have 1 manifests before rewrite"
+ new_manifests = tbl.current_snapshot().manifests(tbl.io)
+ assert len(new_manifests) == 2, "Should have 2 manifests after rewrite"
+
+ assert new_manifests[0].existing_files_count == 4
+ assert new_manifests[0].added_files_count == 0
+ assert new_manifests[0].deleted_files_count == 0
+ #
+ # assertnew_manifests[1].existingFilesCount(), 4)
+ # self.assertFalse(new_manifests[1].hasAddedFiles())
+ # self.assertFalse(new_manifests[1].hasDeletedFiles())
+ #
+ # # Validate the records
+ # expected_records = records1 + records2 + records3 + records4
+ # result_df = tbl.read()
+ # actual_records = result_df.collect()
+ # self.assertEqual(actual_records, expected_records, "Rows must match")
From 82d0d3a40c86bcf1f5f5ed03b367bfa13f0dd3f2 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Sun, 2 Mar 2025 09:42:06 +0200
Subject: [PATCH 007/112] add test
---
pyiceberg/manifest.py | 1 +
.../test_writes/test_rewrite_manifests.py | 86 +++++++++++++------
2 files changed, 59 insertions(+), 28 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 26f11792c0..2ab52f12d7 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -961,6 +961,7 @@ def existing(self, entry: ManifestEntry) -> ManifestWriter:
data_file=entry.data_file,
)
)
+ self._existing_files += 1
return self
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index faf1285058..922a29d30f 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -15,11 +15,14 @@
# specific language governing permissions and limitations
# under the License.
# pylint:disable=redefined-outer-name
+from typing import List
import pyarrow as pa
import pytest
from pyiceberg.catalog import Catalog
+from pyiceberg.manifest import ManifestFile
+from pyiceberg.table import TableProperties
from utils import _create_table
@@ -171,49 +174,76 @@ def test_rewrite_small_manifests_non_partitioned_table(session_catalog: Catalog,
assert expected_records_count == actual_records_count, "Rows must match"
-def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
- # Create and append data files
- # records1 = [ThreeColumnRecord(1, None, "AAAA"), ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")]
- # records2 = [ThreeColumnRecord(2, "CCCCCCCCCC", "CCCC"), ThreeColumnRecord(2, "DDDDDDDDDD", "DDDD")]
- # records3 = [ThreeColumnRecord(3, "EEEEEEEEEE", "EEEE"), ThreeColumnRecord(3, "FFFFFFFFFF", "FFFF")]
- # records4 = [ThreeColumnRecord(4, "GGGGGGGGGG", "GGGG"), ThreeColumnRecord(4, "HHHHHHHHHH", "HHHH")]
- # self.table.newFastAppend().appendFile(DataFile.from_records(records1)).commit()
- # self.table.newFastAppend().appendFile(DataFile.from_records(records2)).commit()
- # self.table.newFastAppend().appendFile(DataFile.from_records(records3)).commit()
- # self.table.newFastAppend().appendFile(DataFile.from_records(records4)).commit()
+def compute_manifest_entry_size_bytes(manifests: List[ManifestFile]) -> float:
+ total_size = 0
+ num_entries = 0
+
+ for manifest in manifests:
+ total_size += manifest.manifest_length
+ num_entries += manifest.added_files_count + manifest.existing_files_count + manifest.deleted_files_count
+
+ return total_size / num_entries if num_entries > 0 else 0
+
+
+def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog) -> None:
+ records1 = pa.Table.from_pydict({"c1": [1, 1], "c2": [None, "BBBBBBBBBB"], "c3": ["AAAA", "BBBB"]})
+
+ records2 = records2 = pa.Table.from_pydict({"c1": [2, 2], "c2": ["CCCCCCCCCC", "DDDDDDDDDD"], "c3": ["CCCC", "DDDD"]})
+
+ records3 = records3 = pa.Table.from_pydict({"c1": [3, 3], "c2": ["EEEEEEEEEE", "FFFFFFFFFF"], "c3": ["EEEE", "FFFF"]})
+
+ records4 = records4 = pa.Table.from_pydict({"c1": [4, 4], "c2": ["GGGGGGGGGG", "HHHHHHHHHG"], "c3": ["GGGG", "HHHH"]})
+
+ schema = pa.schema(
+ [
+ ("c1", pa.int64()),
+ ("c2", pa.string()),
+ ("c3", pa.string()),
+ ]
+ )
identifier = "default.test_rewrite_small_manifests_non_partitioned_table"
- tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
- tbl.append(arrow_table_with_null)
- tbl.append(arrow_table_with_null)
- tbl.append(arrow_table_with_null)
- tbl.append(arrow_table_with_null)
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, schema=schema)
+
+ tbl.append(records1)
+ tbl.append(records2)
+ tbl.append(records3)
+ tbl.append(records4)
+ tbl.refresh()
tbl.refresh()
manifests = tbl.current_snapshot().manifests(tbl.io)
assert len(manifests) == 4, "Should have 4 manifests before rewrite"
- # Perform the rewrite manifests action
- # actions = SparkActions.get()
+ # manifest_entry_size_bytes = compute_manifest_entry_size_bytes(manifests)
+ target_manifest_size_bytes = 5200 * 2 + 100
+ tbl = (
+ tbl.transaction()
+ .set_properties({TableProperties.MANIFEST_TARGET_SIZE_BYTES: str(target_manifest_size_bytes)})
+ .commit_transaction()
+ )
+
result = tbl.rewrite_manifests()
+ tbl.refresh()
assert len(result.rewritten_manifests) == 4, "Action should rewrite 4 manifests"
assert len(result.added_manifests) == 2, "Action should add 2 manifests"
- tbl.refresh()
new_manifests = tbl.current_snapshot().manifests(tbl.io)
assert len(new_manifests) == 2, "Should have 2 manifests after rewrite"
assert new_manifests[0].existing_files_count == 4
assert new_manifests[0].added_files_count == 0
assert new_manifests[0].deleted_files_count == 0
- #
- # assertnew_manifests[1].existingFilesCount(), 4)
- # self.assertFalse(new_manifests[1].hasAddedFiles())
- # self.assertFalse(new_manifests[1].hasDeletedFiles())
- #
- # # Validate the records
- # expected_records = records1 + records2 + records3 + records4
- # result_df = tbl.read()
- # actual_records = result_df.collect()
- # self.assertEqual(actual_records, expected_records, "Rows must match")
+
+ assert new_manifests[1].existing_files_count == 4
+ assert new_manifests[1].added_files_count == 0
+ assert new_manifests[1].deleted_files_count == 0
+
+ sorted_df = tbl.scan().to_pandas().sort_values(["c1", "c2"], ascending=[False, False])
+ expectedRecords = (
+ pa.concat_tables([records1, records2, records3, records4]).to_pandas().sort_values(["c1", "c2"], ascending=[False, False])
+ )
+ from pandas.testing import assert_frame_equal
+
+ assert_frame_equal(sorted_df.reset_index(drop=True), expectedRecords.reset_index(drop=True))
From 6913f0900c144436c1760d1e265172156d9c1df1 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Thu, 15 May 2025 22:24:57 +0300
Subject: [PATCH 008/112] fixes according to pr
---
pyiceberg/manifest.py | 1 -
pyiceberg/table/__init__.py | 3 -
pyiceberg/table/update/snapshot.py | 64 +++++++++++--------
.../test_writes/test_rewrite_manifests.py | 9 +--
4 files changed, 44 insertions(+), 33 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 2ab52f12d7..26f11792c0 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -961,7 +961,6 @@ def existing(self, entry: ManifestEntry) -> ManifestWriter:
data_file=entry.data_file,
)
)
- self._existing_files += 1
return self
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 5b0e1da10f..69bcbc2759 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -237,9 +237,6 @@ class TableProperties:
MIN_SNAPSHOTS_TO_KEEP = "history.expire.min-snapshots-to-keep"
MIN_SNAPSHOTS_TO_KEEP_DEFAULT = 1
- SNAPSHOT_ID_INHERITANCE_ENABLED = "compatibility.snapshot-id-inheritance.enabled"
- SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT = False
-
class Transaction:
_table: Table
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 5cad13cbe1..fac70aa49c 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -22,7 +22,7 @@
from abc import abstractmethod
from collections import defaultdict
from concurrent.futures import Future
-from dataclasses import dataclass
+from dataclasses import dataclass, field
from functools import cached_property
from typing import TYPE_CHECKING, Any, Callable, Dict, Generic, List, Optional, Set, Tuple
@@ -476,8 +476,8 @@ def _deleted_entries(self) -> List[ManifestEntry]:
@dataclass(init=False)
class RewriteManifestsResult:
- rewritten_manifests: List[ManifestFile]
- added_manifests: List[ManifestFile]
+ rewritten_manifests: List[ManifestFile] = field(default_factory=list)
+ added_manifests: List[ManifestFile] = field(default_factory=list)
def __init__(
self,
@@ -540,7 +540,11 @@ def _process_manifests(self, manifests: List[ManifestFile]) -> List[ManifestFile
class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
+ _table: Table
+ _spec_id: int
_target_size_bytes: int
+ _min_count_to_merge: int
+ _merge_enabled: bool
rewritten_manifests: List[ManifestFile] = []
added_manifests: List[ManifestFile] = []
kept_manifests: List[ManifestFile] = []
@@ -555,10 +559,15 @@ def __init__(
):
from pyiceberg.table import TableProperties
- _table: Table
- _spec: PartitionSpec
-
super().__init__(Operation.REPLACE, transaction, io, snapshot_properties=snapshot_properties)
+
+ snapshot = self._table.current_snapshot()
+ if self._spec_id and self._spec_id not in self._table.specs():
+ raise ValueError(f"Cannot find spec with id: {self._spec_id}")
+
+ if not snapshot:
+ raise ValueError("Cannot rewrite manifests without a current snapshot")
+
self._target_size_bytes = property_as_int(
self._transaction.table_metadata.properties,
TableProperties.MANIFEST_TARGET_SIZE_BYTES,
@@ -567,6 +576,17 @@ def __init__(
self._table = table
self._spec_id = spec_id or table.spec().spec_id
+ self._min_count_to_merge = property_as_int(
+ self._transaction.table_metadata.properties,
+ TableProperties.MANIFEST_MIN_MERGE_COUNT,
+ TableProperties.MANIFEST_MIN_MERGE_COUNT_DEFAULT,
+ ) # type: ignore
+ self._merge_enabled = property_as_bool(
+ self._transaction.table_metadata.properties,
+ TableProperties.MANIFEST_MERGE_ENABLED,
+ TableProperties.MANIFEST_MERGE_ENABLED_DEFAULT,
+ )
+
def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
from pyiceberg.table import TableProperties
@@ -579,10 +599,10 @@ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
ssc.set_partition_summary_limit(partition_summary_limit)
props = {
- "manifests-kept": str(len([])),
+ "manifests-kept": "0",
"manifests-created": str(len(self.added_manifests)),
"manifests-replaced": str(len(self.rewritten_manifests)),
- "entries-processed": str(len([])),
+ "entries-processed": "0",
}
previous_snapshot = (
self._transaction.table_metadata.snapshot_by_id(self._parent_snapshot_id)
@@ -597,28 +617,25 @@ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
)
def rewrite_manifests(self) -> RewriteManifestsResult:
- data_result = self._find_matching_manifests(ManifestContent.DATA)
+ snapshot = self._table.current_snapshot()
+ if not snapshot:
+ raise ValueError("Cannot rewrite manifests without a current snapshot")
+
+ data_result = self._find_matching_manifests(snapshot, ManifestContent.DATA)
self.rewritten_manifests.extend(data_result.rewritten_manifests)
self.added_manifests.extend(data_result.added_manifests)
- deletes_result = self._find_matching_manifests(ManifestContent.DELETES)
+ deletes_result = self._find_matching_manifests(snapshot, ManifestContent.DELETES)
self.rewritten_manifests.extend(deletes_result.rewritten_manifests)
self.added_manifests.extend(deletes_result.added_manifests)
- if not self.rewritten_manifests:
+ if len(self.rewritten_manifests) == 0:
return RewriteManifestsResult(rewritten_manifests=[], added_manifests=[])
return RewriteManifestsResult(rewritten_manifests=self.rewritten_manifests, added_manifests=self.added_manifests)
- def _find_matching_manifests(self, content: ManifestContent) -> RewriteManifestsResult:
- snapshot = self._table.current_snapshot()
- if self._spec_id and self._spec_id not in self._table.specs():
- raise ValueError(f"Cannot find spec with id: {self._spec_id}")
-
- if not snapshot:
- raise ValueError("Cannot rewrite manifests without a current snapshot")
-
+ def _find_matching_manifests(self, snapshot: Snapshot, content: ManifestContent) -> RewriteManifestsResult:
manifests = [
manifest
for manifest in snapshot.manifests(io=self._io)
@@ -627,8 +644,8 @@ def _find_matching_manifests(self, content: ManifestContent) -> RewriteManifests
data_manifest_merge_manager = _ManifestMergeManager(
target_size_bytes=self._target_size_bytes,
- min_count_to_merge=2,
- merge_enabled=True,
+ min_count_to_merge=self._min_count_to_merge,
+ merge_enabled=self._merge_enabled,
snapshot_producer=self,
)
new_manifests = data_manifest_merge_manager.merge_manifests(manifests=manifests)
@@ -664,10 +681,7 @@ def _existing_manifests(self) -> List[ManifestFile]:
return [self._copy_manifest_file(manifest, self.snapshot_id) for manifest in self.added_manifests]
def _deleted_entries(self) -> List[ManifestEntry]:
- """To determine if we need to record any deleted manifest entries.
-
- In case of an append, nothing is deleted.
- """
+ """To determine if we need to record any deleted manifest entries."""
return []
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index 922a29d30f..ab655963a0 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -180,8 +180,9 @@ def compute_manifest_entry_size_bytes(manifests: List[ManifestFile]) -> float:
for manifest in manifests:
total_size += manifest.manifest_length
- num_entries += manifest.added_files_count + manifest.existing_files_count + manifest.deleted_files_count
-
+ num_entries += (
+ (manifest.added_files_count or 0) + (manifest.existing_files_count or 0) + (manifest.deleted_files_count or 0)
+ )
return total_size / num_entries if num_entries > 0 else 0
@@ -212,7 +213,7 @@ def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog) ->
tbl.refresh()
tbl.refresh()
- manifests = tbl.current_snapshot().manifests(tbl.io)
+ manifests = tbl.current_snapshot().manifests(tbl.io) # type: ignore
assert len(manifests) == 4, "Should have 4 manifests before rewrite"
# manifest_entry_size_bytes = compute_manifest_entry_size_bytes(manifests)
@@ -229,7 +230,7 @@ def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog) ->
assert len(result.rewritten_manifests) == 4, "Action should rewrite 4 manifests"
assert len(result.added_manifests) == 2, "Action should add 2 manifests"
- new_manifests = tbl.current_snapshot().manifests(tbl.io)
+ new_manifests = tbl.current_snapshot().manifests(tbl.io) # type: ignore
assert len(new_manifests) == 2, "Should have 2 manifests after rewrite"
assert new_manifests[0].existing_files_count == 4
From 74e42e0976381589775e99dd7617354c5ca2f9d0 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Thu, 15 May 2025 22:52:06 +0300
Subject: [PATCH 009/112] more fixes
---
pyiceberg/table/__init__.py | 15 +--------------
pyiceberg/table/update/snapshot.py | 15 ++++-----------
tests/integration/test_inspect_table.py | 4 ++--
3 files changed, 7 insertions(+), 27 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 69bcbc2759..b4aa70db6f 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -438,6 +438,7 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> U
def rewrite_manifests(self) -> RewriteManifestsResult:
if self._table.current_snapshot() is None:
return RewriteManifestsResult(rewritten_manifests=[], added_manifests=[])
+
with self.update_snapshot().rewrite() as rewrite:
rewritten = rewrite.rewrite_manifests()
return rewritten
@@ -1375,20 +1376,6 @@ def add_files(
file_paths=file_paths, snapshot_properties=snapshot_properties, check_duplicate_files=check_duplicate_files
)
- def rewrite_manifests(
- self,
- spec_id: Optional[int] = None,
- ) -> RewriteManifestsResult:
- """
- Shorthand API for Rewriting manifests for the table.
-
- Args:
- spec_id: Spec id of the manifests to rewrite (defaults to current spec id)
-
- """
- with self.transaction() as tx:
- return tx.rewrite_manifests(spec_id=spec_id)
-
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index fac70aa49c..d786493ba4 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -474,19 +474,11 @@ def _deleted_entries(self) -> List[ManifestEntry]:
return []
-@dataclass(init=False)
+@dataclass(frozen=True)
class RewriteManifestsResult:
rewritten_manifests: List[ManifestFile] = field(default_factory=list)
added_manifests: List[ManifestFile] = field(default_factory=list)
- def __init__(
- self,
- rewritten_manifests: Optional[List[ManifestFile]],
- added_manifests: Optional[List[ManifestFile]],
- ) -> None:
- self.rewritten_manifests = rewritten_manifests or []
- self.added_manifests = added_manifests or []
-
class _MergeAppendFiles(_FastAppendFiles):
_target_size_bytes: int
@@ -653,14 +645,15 @@ def _find_matching_manifests(self, snapshot: Snapshot, content: ManifestContent)
return RewriteManifestsResult(rewritten_manifests=manifests, added_manifests=new_manifests)
def _copy_manifest_file(self, manifest_file: ManifestFile, snapshot_id: int) -> ManifestFile:
- return ManifestFile(
+ return ManifestFile.from_args(
+ _table_format_version=self._transaction.table_metadata.format_version,
manifest_path=manifest_file.manifest_path,
manifest_length=manifest_file.manifest_length,
partition_spec_id=manifest_file.partition_spec_id,
content=manifest_file.content,
sequence_number=manifest_file.sequence_number,
min_sequence_number=manifest_file.min_sequence_number,
- added_snapshot_id=snapshot_id,
+ added_snapshot_id=snapshot_id, # Using the new snapshot ID here
added_files_count=manifest_file.added_files_count,
existing_files_count=manifest_file.existing_files_count,
deleted_files_count=manifest_file.deleted_files_count,
diff --git a/tests/integration/test_inspect_table.py b/tests/integration/test_inspect_table.py
index 7c805bae13..875b1f69ad 100644
--- a/tests/integration/test_inspect_table.py
+++ b/tests/integration/test_inspect_table.py
@@ -1102,6 +1102,7 @@ def test_inspect_files_partitioned(spark: SparkSession, session_catalog: Catalog
)
assert_frame_equal(lhs, rhs, check_dtype=False)
+
def test_inspect_all_example(spark: SparkSession, session_catalog: Catalog, format_version: int) -> None:
from pandas.testing import assert_frame_equal
@@ -1140,6 +1141,5 @@ def test_inspect_all_example(spark: SparkSession, session_catalog: Catalog, form
tbl.refresh()
-
tbl.rewrite_manifests()
- print("efd")
\ No newline at end of file
+ print("efd")
From ba5ebc424cab4b170e917b80807befe8ef1c4ddf Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Sat, 17 May 2025 17:23:45 +0300
Subject: [PATCH 010/112] fixed comments to pr
---
pyiceberg/table/__init__.py | 4 +-
pyiceberg/table/update/snapshot.py | 16 +--
.../test_writes/test_rewrite_manifests.py | 98 +++++++++++++------
3 files changed, 76 insertions(+), 42 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index b4aa70db6f..1dc20f3d80 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -1237,9 +1237,9 @@ def name_mapping(self) -> Optional[NameMapping]:
"""Return the table's field-id NameMapping."""
return self.metadata.name_mapping()
- def rewrite_manifests(self) -> None:
+ def rewrite_manifests(self) -> RewriteManifestsResult:
with self.transaction() as tx:
- tx.rewrite_manifests()
+ return tx.rewrite_manifests()
def upsert(
self,
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index d786493ba4..87f0481f2e 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -553,6 +553,9 @@ def __init__(
super().__init__(Operation.REPLACE, transaction, io, snapshot_properties=snapshot_properties)
+ self._table = table
+ self._spec_id = spec_id or table.spec().spec_id
+
snapshot = self._table.current_snapshot()
if self._spec_id and self._spec_id not in self._table.specs():
raise ValueError(f"Cannot find spec with id: {self._spec_id}")
@@ -565,9 +568,6 @@ def __init__(
TableProperties.MANIFEST_TARGET_SIZE_BYTES,
TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
) # type: ignore
- self._table = table
- self._spec_id = spec_id or table.spec().spec_id
-
self._min_count_to_merge = property_as_int(
self._transaction.table_metadata.properties,
TableProperties.MANIFEST_MIN_MERGE_COUNT,
@@ -582,13 +582,13 @@ def __init__(
def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
from pyiceberg.table import TableProperties
- ssc = SnapshotSummaryCollector()
- partition_summary_limit = int(
- self._transaction.table_metadata.properties.get(
- TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT
+ ssc = SnapshotSummaryCollector(
+ int(
+ self._transaction.table_metadata.properties.get(
+ TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT
+ )
)
)
- ssc.set_partition_summary_limit(partition_summary_limit)
props = {
"manifests-kept": "0",
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index ab655963a0..302104e08f 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -96,33 +96,57 @@ def table_v1_v2_appended_with_null(session_catalog: Catalog, arrow_table_with_nu
assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
-# @pytest.mark.integration
-# def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
-# identifier = "default.arrow_table_summaries"
-# tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
-# tbl.append(arrow_table_with_null)
-# tbl.append(arrow_table_with_null)
-#
-# # tbl.rewrite_manifests()
-#
-# # records1 = [ThreeColumnRecord(1, None, "AAAA")]
-# # write_records(spark, table_location, records1)
-# before_pandas = tbl.scan().to_pandas()
-# before_count = before_pandas.shape[0]
-# tbl.refresh()
-# manifests = tbl.inspect.manifests().to_pylist()
-# assert len(manifests) == 2, "Should have 2 manifests before rewrite"
-#
-# tbl.rewrite_manifests()
-# tbl.refresh()
-#
-# after_pandas = tbl.scan().to_pandas()
-# after_count = before_pandas.shape[0]
-# manifests = tbl.inspect.manifests().to_pylist()
-# assert len(manifests) == 1, "Should have 1 manifests before rewrite"
-#
-# snaps = tbl.inspect.snapshots().to_pandas()
-# print(snaps)
+@pytest.mark.integration
+def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_rewrite_v1_v2_manifests"
+ # Create a v1 table and append data
+ tbl = _create_table(
+ session_catalog,
+ identifier,
+ {"format-version": "1"},
+ [arrow_table_with_null],
+ )
+ assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
+
+ # tbl.append(arrow_table_with_null)
+ # Upgrade to v2 and append more data
+ with tbl.transaction() as tx:
+ tx.upgrade_table_version(format_version=2)
+
+ tbl.append(arrow_table_with_null)
+ assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
+
+ with tbl.transaction() as tx: # type: ignore[unreachable]
+ tx.set_properties({"commit.manifest-merge.enabled": "true", "commit.manifest.min-count-to-merge": "2"})
+
+ # Get initial manifest state
+ manifests = tbl.inspect.manifests()
+ assert len(manifests) == 2, "Should have 2 manifests before rewrite"
+
+ # Execute rewrite manifests
+ result = tbl.rewrite_manifests()
+
+ assert len(result.rewritten_manifests) == 2, "Action should rewrite 2 manifests"
+ assert len(result.added_manifests) == 1, "Action should add 1 manifest"
+
+ tbl.refresh()
+
+ # Verify final state
+ current_snapshot = tbl.current_snapshot()
+ if not current_snapshot:
+ raise AssertionError("Expected a current snapshot")
+
+ new_manifests = current_snapshot.manifests(tbl.io)
+ assert len(new_manifests) == 1, "Should have 1 manifest after rewrite"
+ assert new_manifests[0].existing_files_count == 2, "Should have 2 existing files in the new manifest"
+ assert new_manifests[0].added_files_count == 0, "Should have no added files in the new manifest"
+ assert new_manifests[0].deleted_files_count == 0, "Should have no deleted files in the new manifest"
+
+ # Validate the data is intact
+ expected_records_count = arrow_table_with_null.shape[0] * 2
+ result_df = tbl.scan().to_pandas()
+ actual_records_count = result_df.shape[0]
+ assert expected_records_count == actual_records_count, "Record count must match"
@pytest.mark.integration
@@ -146,8 +170,11 @@ def test_rewrite_small_manifests_non_partitioned_table(session_catalog: Catalog,
tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
tbl.append(arrow_table_with_null)
tbl.append(arrow_table_with_null)
- tbl.refresh()
+ tbl.transaction().set_properties(
+ {"commit.manifest-merge.enabled": "true", "commit.manifest.min-count-to-merge": "2"}
+ ).commit_transaction()
+ tbl = tbl.refresh()
manifests = tbl.inspect.manifests()
assert len(manifests) == 2, "Should have 2 manifests before rewrite"
@@ -212,15 +239,22 @@ def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog) ->
tbl.append(records4)
tbl.refresh()
- tbl.refresh()
+ tbl = tbl.refresh()
manifests = tbl.current_snapshot().manifests(tbl.io) # type: ignore
assert len(manifests) == 4, "Should have 4 manifests before rewrite"
# manifest_entry_size_bytes = compute_manifest_entry_size_bytes(manifests)
target_manifest_size_bytes = 5200 * 2 + 100
+
tbl = (
tbl.transaction()
- .set_properties({TableProperties.MANIFEST_TARGET_SIZE_BYTES: str(target_manifest_size_bytes)})
+ .set_properties(
+ {
+ TableProperties.MANIFEST_TARGET_SIZE_BYTES: str(target_manifest_size_bytes),
+ "commit.manifest-merge.enabled": "true",
+ "commit.manifest.min-count-to-merge": "2",
+ }
+ )
.commit_transaction()
)
@@ -233,11 +267,11 @@ def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog) ->
new_manifests = tbl.current_snapshot().manifests(tbl.io) # type: ignore
assert len(new_manifests) == 2, "Should have 2 manifests after rewrite"
- assert new_manifests[0].existing_files_count == 4
+ assert new_manifests[0].existing_files_count == 2
assert new_manifests[0].added_files_count == 0
assert new_manifests[0].deleted_files_count == 0
- assert new_manifests[1].existing_files_count == 4
+ assert new_manifests[1].existing_files_count == 2
assert new_manifests[1].added_files_count == 0
assert new_manifests[1].deleted_files_count == 0
From d7dd666b33d7ebbd01a26d327e6cae0ae57b4c38 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Sat, 17 May 2025 17:35:23 +0300
Subject: [PATCH 011/112] validate seq number
---
.../integration/test_writes/test_rewrite_manifests.py | 10 +++++-----
1 file changed, 5 insertions(+), 5 deletions(-)
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index 302104e08f..a57144bd78 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -108,7 +108,6 @@ def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null
)
assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
- # tbl.append(arrow_table_with_null)
# Upgrade to v2 and append more data
with tbl.transaction() as tx:
tx.upgrade_table_version(format_version=2)
@@ -117,7 +116,7 @@ def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null
assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
with tbl.transaction() as tx: # type: ignore[unreachable]
- tx.set_properties({"commit.manifest-merge.enabled": "true", "commit.manifest.min-count-to-merge": "2"})
+ tx.set_properties({TableProperties.MANIFEST_MERGE_ENABLED: "true", TableProperties.MANIFEST_MIN_MERGE_COUNT: "2"})
# Get initial manifest state
manifests = tbl.inspect.manifests()
@@ -141,6 +140,7 @@ def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null
assert new_manifests[0].existing_files_count == 2, "Should have 2 existing files in the new manifest"
assert new_manifests[0].added_files_count == 0, "Should have no added files in the new manifest"
assert new_manifests[0].deleted_files_count == 0, "Should have no deleted files in the new manifest"
+ assert new_manifests[0].sequence_number is not None, "Should have a sequence number in the new manifest"
# Validate the data is intact
expected_records_count = arrow_table_with_null.shape[0] * 2
@@ -172,7 +172,7 @@ def test_rewrite_small_manifests_non_partitioned_table(session_catalog: Catalog,
tbl.append(arrow_table_with_null)
tbl.transaction().set_properties(
- {"commit.manifest-merge.enabled": "true", "commit.manifest.min-count-to-merge": "2"}
+ {TableProperties.MANIFEST_MERGE_ENABLED: "true", TableProperties.MANIFEST_MIN_MERGE_COUNT: "2"}
).commit_transaction()
tbl = tbl.refresh()
manifests = tbl.inspect.manifests()
@@ -251,8 +251,8 @@ def test_rewrite_small_manifests_partitioned_table(session_catalog: Catalog) ->
.set_properties(
{
TableProperties.MANIFEST_TARGET_SIZE_BYTES: str(target_manifest_size_bytes),
- "commit.manifest-merge.enabled": "true",
- "commit.manifest.min-count-to-merge": "2",
+ TableProperties.MANIFEST_MERGE_ENABLED: "true",
+ TableProperties.MANIFEST_MIN_MERGE_COUNT: "2",
}
)
.commit_transaction()
From 98db350d38e3b74f8d280b18b74b340e469dc448 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Sat, 17 May 2025 17:38:49 +0300
Subject: [PATCH 012/112] remove redundant test
---
.../test_writes/test_rewrite_manifests.py | 13 -------------
1 file changed, 13 deletions(-)
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index a57144bd78..b4fd6ec37f 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -82,19 +82,6 @@ def table_v2_appended_with_null(session_catalog: Catalog, arrow_table_with_null:
assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
-@pytest.fixture(scope="session", autouse=True)
-def table_v1_v2_appended_with_null(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
- identifier = "default.arrow_table_v1_v2_appended_with_null"
- tbl = _create_table(session_catalog, identifier, {"format-version": "1"}, [arrow_table_with_null])
- assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
-
- with tbl.transaction() as tx:
- tx.upgrade_table_version(format_version=2)
-
- tbl.append(arrow_table_with_null)
-
- assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
-
@pytest.mark.integration
def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
From fe4811dd9c028aa5cd0abedda02f8655ea945e5b Mon Sep 17 00:00:00 2001
From: Sung Yun <107272191+sungwy@users.noreply.github.com>
Date: Fri, 16 May 2025 07:25:40 -0400
Subject: [PATCH 013/112] Replace Deprecated (Current) OAuth2 Handling with
AuthManager Implementation `LegacyOAuth2AuthManager` (#1981)
# Rationale for this change
Replace existing Auth handling with `LegacyOAuth2AuthManager`. Tracking
issue: https://github.com/apache/iceberg-python/issues/1909
There will be follow up PRs to this PR that will address the following:
- introduce a mechanism for using a custom `AuthManager` implementation,
along with the ability to use a set of config parameters
- introduce a `OAuth2AuthManager` that more closely follows the OAuth2
protocol, and also uses a separate thread to proactively refreshes the
token, rather than reactively refreshing the token on
`UnAuthorizedError` or the deprecated `AuthorizationExpiredError`.
# Are these changes tested?
Yes, both through unit and integration tests
# Are there any user-facing changes?
Yes - previously, if `TOKEN` and `CREDENTIAL` are both defined,
`oauth/tokens` endpoint wouldn't be used to refresh the token with
client credentials when the `RestCatalog` was initialized. However,
`oauth/tokens` endpoint would be used on retries that handled 401 or 419
error.
This erratic behavior will now be updated as follows:
- if `CREDENTIAL` is defined, `oauth/tokens` endpoint will be used to
fetch the access token using the client credentials both when the
RestCatalog is initialized, and when the refresh_tokens call is made as
a reaction to 401 or 419 error.
- if both `CREDENTIAL` and `TOKEN` are defined, we will follow the above
behavior.
- if only `TOKEN` is defined, the initial token will be used instead
---
pyiceberg/catalog/rest/__init__.py | 189 ++++++++---------------------
pyiceberg/catalog/rest/auth.py | 111 ++++++++++++++++-
pyiceberg/catalog/rest/response.py | 111 +++++++++++++++++
tests/catalog/test_rest.py | 8 +-
4 files changed, 275 insertions(+), 144 deletions(-)
create mode 100644 pyiceberg/catalog/rest/response.py
diff --git a/pyiceberg/catalog/rest/__init__.py b/pyiceberg/catalog/rest/__init__.py
index 633ef6498b..4ad3395378 100644
--- a/pyiceberg/catalog/rest/__init__.py
+++ b/pyiceberg/catalog/rest/__init__.py
@@ -15,21 +15,18 @@
# specific language governing permissions and limitations
# under the License.
from enum import Enum
-from json import JSONDecodeError
from typing import (
TYPE_CHECKING,
Any,
Dict,
List,
- Literal,
Optional,
Set,
Tuple,
- Type,
Union,
)
-from pydantic import Field, ValidationError, field_validator
+from pydantic import Field, field_validator
from requests import HTTPError, Session
from tenacity import RetryCallState, retry, retry_if_exception_type, stop_after_attempt
@@ -41,22 +38,18 @@
Catalog,
PropertiesUpdateSummary,
)
+from pyiceberg.catalog.rest.auth import AuthManager, AuthManagerAdapter, AuthManagerFactory, LegacyOAuth2AuthManager
+from pyiceberg.catalog.rest.response import _handle_non_200_response
from pyiceberg.exceptions import (
AuthorizationExpiredError,
- BadRequestError,
CommitFailedException,
CommitStateUnknownException,
- ForbiddenError,
NamespaceAlreadyExistsError,
NamespaceNotEmptyError,
NoSuchIdentifierError,
NoSuchNamespaceError,
NoSuchTableError,
NoSuchViewError,
- OAuthError,
- RESTError,
- ServerError,
- ServiceUnavailableError,
TableAlreadyExistsError,
UnauthorizedError,
)
@@ -182,15 +175,6 @@ class RegisterTableRequest(IcebergBaseModel):
metadata_location: str = Field(..., alias="metadata-location")
-class TokenResponse(IcebergBaseModel):
- access_token: str = Field()
- token_type: str = Field()
- expires_in: Optional[int] = Field(default=None)
- issued_token_type: Optional[str] = Field(default=None)
- refresh_token: Optional[str] = Field(default=None)
- scope: Optional[str] = Field(default=None)
-
-
class ConfigResponse(IcebergBaseModel):
defaults: Properties = Field()
overrides: Properties = Field()
@@ -229,24 +213,6 @@ class ListViewsResponse(IcebergBaseModel):
identifiers: List[ListViewResponseEntry] = Field()
-class ErrorResponseMessage(IcebergBaseModel):
- message: str = Field()
- type: str = Field()
- code: int = Field()
-
-
-class ErrorResponse(IcebergBaseModel):
- error: ErrorResponseMessage = Field()
-
-
-class OAuthErrorResponse(IcebergBaseModel):
- error: Literal[
- "invalid_request", "invalid_client", "invalid_grant", "unauthorized_client", "unsupported_grant_type", "invalid_scope"
- ]
- error_description: Optional[str] = None
- error_uri: Optional[str] = None
-
-
class RestCatalog(Catalog):
uri: str
_session: Session
@@ -279,8 +245,7 @@ def _create_session(self) -> Session:
elif ssl_client_cert := ssl_client.get(CERT):
session.cert = ssl_client_cert
- self._refresh_token(session, self.properties.get(TOKEN))
-
+ session.auth = AuthManagerAdapter(self._create_legacy_oauth2_auth_manager(session))
# Set HTTP headers
self._config_headers(session)
@@ -290,6 +255,26 @@ def _create_session(self) -> Session:
return session
+ def _create_legacy_oauth2_auth_manager(self, session: Session) -> AuthManager:
+ """Create the LegacyOAuth2AuthManager by fetching required properties.
+
+ This will be removed in PyIceberg 1.0
+ """
+ client_credentials = self.properties.get(CREDENTIAL)
+ # We want to call `self.auth_url` only when we are using CREDENTIAL
+ # with the legacy OAUTH2 flow as it will raise a DeprecationWarning
+ auth_url = self.auth_url if client_credentials is not None else None
+
+ auth_config = {
+ "session": session,
+ "auth_url": auth_url,
+ "credential": client_credentials,
+ "initial_token": self.properties.get(TOKEN),
+ "optional_oauth_params": self._extract_optional_oauth_params(),
+ }
+
+ return AuthManagerFactory.create("legacyoauth2", auth_config)
+
def _check_valid_namespace_identifier(self, identifier: Union[str, Identifier]) -> Identifier:
"""Check if the identifier has at least one element."""
identifier_tuple = Catalog.identifier_to_tuple(identifier)
@@ -352,27 +337,6 @@ def _extract_optional_oauth_params(self) -> Dict[str, str]:
return optional_oauth_param
- def _fetch_access_token(self, session: Session, credential: str) -> str:
- if SEMICOLON in credential:
- client_id, client_secret = credential.split(SEMICOLON)
- else:
- client_id, client_secret = None, credential
-
- data = {GRANT_TYPE: CLIENT_CREDENTIALS, CLIENT_ID: client_id, CLIENT_SECRET: client_secret}
-
- optional_oauth_params = self._extract_optional_oauth_params()
- data.update(optional_oauth_params)
-
- response = session.post(
- url=self.auth_url, data=data, headers={**session.headers, "Content-type": "application/x-www-form-urlencoded"}
- )
- try:
- response.raise_for_status()
- except HTTPError as exc:
- self._handle_non_200_response(exc, {400: OAuthError, 401: OAuthError})
-
- return TokenResponse.model_validate_json(response.text).access_token
-
def _fetch_config(self) -> None:
params = {}
if warehouse_location := self.properties.get(WAREHOUSE_LOCATION):
@@ -383,7 +347,7 @@ def _fetch_config(self) -> None:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {})
+ _handle_non_200_response(exc, {})
config_response = ConfigResponse.model_validate_json(response.text)
config = config_response.defaults
@@ -413,58 +377,6 @@ def _split_identifier_for_json(self, identifier: Union[str, Identifier]) -> Dict
identifier_tuple = self._identifier_to_validated_tuple(identifier)
return {"namespace": identifier_tuple[:-1], "name": identifier_tuple[-1]}
- def _handle_non_200_response(self, exc: HTTPError, error_handler: Dict[int, Type[Exception]]) -> None:
- exception: Type[Exception]
-
- if exc.response is None:
- raise ValueError("Did not receive a response")
-
- code = exc.response.status_code
- if code in error_handler:
- exception = error_handler[code]
- elif code == 400:
- exception = BadRequestError
- elif code == 401:
- exception = UnauthorizedError
- elif code == 403:
- exception = ForbiddenError
- elif code == 422:
- exception = RESTError
- elif code == 419:
- exception = AuthorizationExpiredError
- elif code == 501:
- exception = NotImplementedError
- elif code == 503:
- exception = ServiceUnavailableError
- elif 500 <= code < 600:
- exception = ServerError
- else:
- exception = RESTError
-
- try:
- if exception == OAuthError:
- # The OAuthErrorResponse has a different format
- error = OAuthErrorResponse.model_validate_json(exc.response.text)
- response = str(error.error)
- if description := error.error_description:
- response += f": {description}"
- if uri := error.error_uri:
- response += f" ({uri})"
- else:
- error = ErrorResponse.model_validate_json(exc.response.text).error
- response = f"{error.type}: {error.message}"
- except JSONDecodeError:
- # In the case we don't have a proper response
- response = f"RESTError {exc.response.status_code}: Could not decode json payload: {exc.response.text}"
- except ValidationError as e:
- # In the case we don't have a proper response
- errs = ", ".join(err["msg"] for err in e.errors())
- response = (
- f"RESTError {exc.response.status_code}: Received unexpected JSON Payload: {exc.response.text}, errors: {errs}"
- )
-
- raise exception(response) from exc
-
def _init_sigv4(self, session: Session) -> None:
from urllib import parse
@@ -534,16 +446,13 @@ def _response_to_staged_table(self, identifier_tuple: Tuple[str, ...], table_res
catalog=self,
)
- def _refresh_token(self, session: Optional[Session] = None, initial_token: Optional[str] = None) -> None:
- session = session or self._session
- if initial_token is not None:
- self.properties[TOKEN] = initial_token
- elif CREDENTIAL in self.properties:
- self.properties[TOKEN] = self._fetch_access_token(session, self.properties[CREDENTIAL])
-
- # Set Auth token for subsequent calls in the session
- if token := self.properties.get(TOKEN):
- session.headers[AUTHORIZATION_HEADER] = f"{BEARER_PREFIX} {token}"
+ def _refresh_token(self) -> None:
+ # Reactive token refresh is atypical - we should proactively refresh tokens in a separate thread
+ # instead of retrying on Auth Exceptions. Keeping refresh behavior for the LegacyOAuth2AuthManager
+ # for backward compatibility
+ auth_manager = self._session.auth.auth_manager # type: ignore[union-attr]
+ if isinstance(auth_manager, LegacyOAuth2AuthManager):
+ auth_manager._refresh_token()
def _config_headers(self, session: Session) -> None:
header_properties = get_header_properties(self.properties)
@@ -588,7 +497,7 @@ def _create_table(
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {409: TableAlreadyExistsError})
+ _handle_non_200_response(exc, {409: TableAlreadyExistsError})
return TableResponse.model_validate_json(response.text)
@retry(**_RETRY_ARGS)
@@ -661,7 +570,7 @@ def register_table(self, identifier: Union[str, Identifier], metadata_location:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {409: TableAlreadyExistsError})
+ _handle_non_200_response(exc, {409: TableAlreadyExistsError})
table_response = TableResponse.model_validate_json(response.text)
return self._response_to_table(self.identifier_to_tuple(identifier), table_response)
@@ -674,7 +583,7 @@ def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchNamespaceError})
+ _handle_non_200_response(exc, {404: NoSuchNamespaceError})
return [(*table.namespace, table.name) for table in ListTablesResponse.model_validate_json(response.text).identifiers]
@retry(**_RETRY_ARGS)
@@ -692,7 +601,7 @@ def load_table(self, identifier: Union[str, Identifier]) -> Table:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchTableError})
+ _handle_non_200_response(exc, {404: NoSuchTableError})
table_response = TableResponse.model_validate_json(response.text)
return self._response_to_table(self.identifier_to_tuple(identifier), table_response)
@@ -705,7 +614,7 @@ def drop_table(self, identifier: Union[str, Identifier], purge_requested: bool =
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchTableError})
+ _handle_non_200_response(exc, {404: NoSuchTableError})
@retry(**_RETRY_ARGS)
def purge_table(self, identifier: Union[str, Identifier]) -> None:
@@ -721,7 +630,7 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchTableError, 409: TableAlreadyExistsError})
+ _handle_non_200_response(exc, {404: NoSuchTableError, 409: TableAlreadyExistsError})
return self.load_table(to_identifier)
@@ -744,7 +653,7 @@ def list_views(self, namespace: Union[str, Identifier]) -> List[Identifier]:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchNamespaceError})
+ _handle_non_200_response(exc, {404: NoSuchNamespaceError})
return [(*view.namespace, view.name) for view in ListViewsResponse.model_validate_json(response.text).identifiers]
@retry(**_RETRY_ARGS)
@@ -782,7 +691,7 @@ def commit_table(
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(
+ _handle_non_200_response(
exc,
{
409: CommitFailedException,
@@ -801,7 +710,7 @@ def create_namespace(self, namespace: Union[str, Identifier], properties: Proper
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {409: NamespaceAlreadyExistsError})
+ _handle_non_200_response(exc, {409: NamespaceAlreadyExistsError})
@retry(**_RETRY_ARGS)
def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
@@ -811,7 +720,7 @@ def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchNamespaceError, 409: NamespaceNotEmptyError})
+ _handle_non_200_response(exc, {404: NoSuchNamespaceError, 409: NamespaceNotEmptyError})
@retry(**_RETRY_ARGS)
def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
@@ -826,7 +735,7 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchNamespaceError})
+ _handle_non_200_response(exc, {404: NoSuchNamespaceError})
return ListNamespaceResponse.model_validate_json(response.text).namespaces
@@ -838,7 +747,7 @@ def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Proper
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchNamespaceError})
+ _handle_non_200_response(exc, {404: NoSuchNamespaceError})
return NamespaceResponse.model_validate_json(response.text).properties
@@ -853,7 +762,7 @@ def update_namespace_properties(
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchNamespaceError})
+ _handle_non_200_response(exc, {404: NoSuchNamespaceError})
parsed_response = UpdateNamespacePropertiesResponse.model_validate_json(response.text)
return PropertiesUpdateSummary(
removed=parsed_response.removed,
@@ -875,7 +784,7 @@ def namespace_exists(self, namespace: Union[str, Identifier]) -> bool:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {})
+ _handle_non_200_response(exc, {})
return False
@@ -901,7 +810,7 @@ def table_exists(self, identifier: Union[str, Identifier]) -> bool:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {})
+ _handle_non_200_response(exc, {})
return False
@@ -926,7 +835,7 @@ def view_exists(self, identifier: Union[str, Identifier]) -> bool:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {})
+ _handle_non_200_response(exc, {})
return False
@@ -938,4 +847,4 @@ def drop_view(self, identifier: Union[str]) -> None:
try:
response.raise_for_status()
except HTTPError as exc:
- self._handle_non_200_response(exc, {404: NoSuchViewError})
+ _handle_non_200_response(exc, {404: NoSuchViewError})
diff --git a/pyiceberg/catalog/rest/auth.py b/pyiceberg/catalog/rest/auth.py
index 041a8a4cd1..89395f1158 100644
--- a/pyiceberg/catalog/rest/auth.py
+++ b/pyiceberg/catalog/rest/auth.py
@@ -16,12 +16,18 @@
# under the License.
import base64
+import importlib
from abc import ABC, abstractmethod
-from typing import Optional
+from typing import Any, Dict, Optional, Type
-from requests import PreparedRequest
+from requests import HTTPError, PreparedRequest, Session
from requests.auth import AuthBase
+from pyiceberg.catalog.rest.response import TokenResponse, _handle_non_200_response
+from pyiceberg.exceptions import OAuthError
+
+COLON = ":"
+
class AuthManager(ABC):
"""
@@ -49,6 +55,60 @@ def auth_header(self) -> str:
return f"Basic {self._token}"
+class LegacyOAuth2AuthManager(AuthManager):
+ _session: Session
+ _auth_url: Optional[str]
+ _token: Optional[str]
+ _credential: Optional[str]
+ _optional_oauth_params: Optional[Dict[str, str]]
+
+ def __init__(
+ self,
+ session: Session,
+ auth_url: Optional[str] = None,
+ credential: Optional[str] = None,
+ initial_token: Optional[str] = None,
+ optional_oauth_params: Optional[Dict[str, str]] = None,
+ ):
+ self._session = session
+ self._auth_url = auth_url
+ self._token = initial_token
+ self._credential = credential
+ self._optional_oauth_params = optional_oauth_params
+ self._refresh_token()
+
+ def _fetch_access_token(self, credential: str) -> str:
+ if COLON in credential:
+ client_id, client_secret = credential.split(COLON)
+ else:
+ client_id, client_secret = None, credential
+
+ data = {"grant_type": "client_credentials", "client_id": client_id, "client_secret": client_secret}
+
+ if self._optional_oauth_params:
+ data.update(self._optional_oauth_params)
+
+ if self._auth_url is None:
+ raise ValueError("Cannot fetch access token from undefined auth_url")
+
+ response = self._session.post(
+ url=self._auth_url, data=data, headers={**self._session.headers, "Content-type": "application/x-www-form-urlencoded"}
+ )
+ try:
+ response.raise_for_status()
+ except HTTPError as exc:
+ _handle_non_200_response(exc, {400: OAuthError, 401: OAuthError})
+
+ return TokenResponse.model_validate_json(response.text).access_token
+
+ def _refresh_token(self) -> None:
+ if self._credential is not None:
+ self._token = self._fetch_access_token(self._credential)
+
+ def auth_header(self) -> str:
+ return f"Bearer {self._token}"
+
+
class AuthManagerAdapter(AuthBase):
"""A `requests.auth.AuthBase` adapter that integrates an `AuthManager` into a `requests.Session` to automatically attach the appropriate Authorization header to every request.
@@ -80,3 +140,50 @@ def __call__(self, request: PreparedRequest) -> PreparedRequest:
if auth_header := self.auth_manager.auth_header():
request.headers["Authorization"] = auth_header
return request
+
+
+class AuthManagerFactory:
+ _registry: Dict[str, Type["AuthManager"]] = {}
+
+ @classmethod
+ def register(cls, name: str, auth_manager_class: Type["AuthManager"]) -> None:
+ """
+ Register a string name to a known AuthManager class.
+
+ Args:
+ name (str): unique name like 'oauth2' to register the AuthManager with
+ auth_manager_class (Type["AuthManager"]): Implementation of AuthManager
+
+ Returns:
+ None
+ """
+ cls._registry[name] = auth_manager_class
+
+ @classmethod
+ def create(cls, class_or_name: str, config: Dict[str, Any]) -> AuthManager:
+ """
+ Create an AuthManager by name or fully-qualified class path.
+
+ Args:
+ class_or_name (str): Either a name like 'oauth2' or a full class path like 'my.module.CustomAuthManager'
+ config (Dict[str, Any]): Configuration passed to the AuthManager constructor
+
+ Returns:
+ AuthManager: An instantiated AuthManager subclass
+ """
+ if class_or_name in cls._registry:
+ manager_cls = cls._registry[class_or_name]
+ else:
+ try:
+ module_path, class_name = class_or_name.rsplit(".", 1)
+ module = importlib.import_module(module_path)
+ manager_cls = getattr(module, class_name)
+ except Exception as err:
+ raise ValueError(f"Could not load AuthManager class for '{class_or_name}'") from err
+
+ return manager_cls(**config)
+
+
+AuthManagerFactory.register("noop", NoopAuthManager)
+AuthManagerFactory.register("basic", BasicAuthManager)
+AuthManagerFactory.register("legacyoauth2", LegacyOAuth2AuthManager)
diff --git a/pyiceberg/catalog/rest/response.py b/pyiceberg/catalog/rest/response.py
new file mode 100644
index 0000000000..8f23af8c35
--- /dev/null
+++ b/pyiceberg/catalog/rest/response.py
@@ -0,0 +1,111 @@
+# 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.
+from json import JSONDecodeError
+from typing import Dict, Literal, Optional, Type
+
+from pydantic import Field, ValidationError
+from requests import HTTPError
+
+from pyiceberg.exceptions import (
+ AuthorizationExpiredError,
+ BadRequestError,
+ ForbiddenError,
+ OAuthError,
+ RESTError,
+ ServerError,
+ ServiceUnavailableError,
+ UnauthorizedError,
+)
+from pyiceberg.typedef import IcebergBaseModel
+
+
+class TokenResponse(IcebergBaseModel):
+ access_token: str = Field()
+ token_type: str = Field()
+ expires_in: Optional[int] = Field(default=None)
+ issued_token_type: Optional[str] = Field(default=None)
+ refresh_token: Optional[str] = Field(default=None)
+ scope: Optional[str] = Field(default=None)
+
+
+class ErrorResponseMessage(IcebergBaseModel):
+ message: str = Field()
+ type: str = Field()
+ code: int = Field()
+
+
+class ErrorResponse(IcebergBaseModel):
+ error: ErrorResponseMessage = Field()
+
+
+class OAuthErrorResponse(IcebergBaseModel):
+ error: Literal[
+ "invalid_request", "invalid_client", "invalid_grant", "unauthorized_client", "unsupported_grant_type", "invalid_scope"
+ ]
+ error_description: Optional[str] = None
+ error_uri: Optional[str] = None
+
+
+def _handle_non_200_response(exc: HTTPError, error_handler: Dict[int, Type[Exception]]) -> None:
+ exception: Type[Exception]
+
+ if exc.response is None:
+ raise ValueError("Did not receive a response")
+
+ code = exc.response.status_code
+ if code in error_handler:
+ exception = error_handler[code]
+ elif code == 400:
+ exception = BadRequestError
+ elif code == 401:
+ exception = UnauthorizedError
+ elif code == 403:
+ exception = ForbiddenError
+ elif code == 422:
+ exception = RESTError
+ elif code == 419:
+ exception = AuthorizationExpiredError
+ elif code == 501:
+ exception = NotImplementedError
+ elif code == 503:
+ exception = ServiceUnavailableError
+ elif 500 <= code < 600:
+ exception = ServerError
+ else:
+ exception = RESTError
+
+ try:
+ if exception == OAuthError:
+ # The OAuthErrorResponse has a different format
+ error = OAuthErrorResponse.model_validate_json(exc.response.text)
+ response = str(error.error)
+ if description := error.error_description:
+ response += f": {description}"
+ if uri := error.error_uri:
+ response += f" ({uri})"
+ else:
+ error = ErrorResponse.model_validate_json(exc.response.text).error
+ response = f"{error.type}: {error.message}"
+ except JSONDecodeError:
+ # In the case we don't have a proper response
+ response = f"RESTError {exc.response.status_code}: Could not decode json payload: {exc.response.text}"
+ except ValidationError as e:
+ # In the case we don't have a proper response
+ errs = ", ".join(err["msg"] for err in e.errors())
+ response = f"RESTError {exc.response.status_code}: Received unexpected JSON Payload: {exc.response.text}, errors: {errs}"
+
+ raise exception(response) from exc
diff --git a/tests/catalog/test_rest.py b/tests/catalog/test_rest.py
index dfc6a0e2fb..1ad6f57d36 100644
--- a/tests/catalog/test_rest.py
+++ b/tests/catalog/test_rest.py
@@ -620,6 +620,10 @@ def test_list_namespaces_token_expired_success_on_retries(rest_mock: Mocker, sta
status_code=200,
)
catalog = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN, credential=TEST_CREDENTIALS)
+ # LegacyOAuth2AuthManager is created twice through `_create_session()`
+ # which results in the token being refreshed twice when the RestCatalog is initialized.
+ assert tokens.call_count == 2
+
assert catalog.list_namespaces() == [
("default",),
("examples",),
@@ -627,7 +631,7 @@ def test_list_namespaces_token_expired_success_on_retries(rest_mock: Mocker, sta
("system",),
]
assert namespaces.call_count == 2
- assert tokens.call_count == 1
+ assert tokens.call_count == 3
assert catalog.list_namespaces() == [
("default",),
@@ -636,7 +640,7 @@ def test_list_namespaces_token_expired_success_on_retries(rest_mock: Mocker, sta
("system",),
]
assert namespaces.call_count == 3
- assert tokens.call_count == 1
+ assert tokens.call_count == 3
def test_create_namespace_200(rest_mock: Mocker) -> None:
From 4f9ce30a011f6ec063de2aae3797b80870956304 Mon Sep 17 00:00:00 2001
From: Quentin Lhoest <42851186+lhoestq@users.noreply.github.com>
Date: Fri, 16 May 2025 18:43:22 +0200
Subject: [PATCH 014/112] Add Hugging Face filesystem support to fsspec (#1997)
# Rationale for this change
Add support for the Hugging Face filesystem in `fsspec`, which uses
`hf://` paths.
This allows to import [HF datasets](https://huggingface.co/datasets).
Authentication is done using the `"hf.token"` property.
# Are these changes tested?
I tried locally but haven't added tests in test_fsspec.py (lmk if it's a
requirement)
# Are there any user-facing changes?
No changes, it simply adds support for `hf://` URLs
---
mkdocs/docs/configuration.md | 12 ++++++
poetry.lock | 76 +++++++++++++++++++++++++++---------
pyiceberg/io/__init__.py | 3 ++
pyiceberg/io/fsspec.py | 12 ++++++
pyproject.toml | 6 +++
5 files changed, 90 insertions(+), 19 deletions(-)
diff --git a/mkdocs/docs/configuration.md b/mkdocs/docs/configuration.md
index dc51f81434..c91ef42e60 100644
--- a/mkdocs/docs/configuration.md
+++ b/mkdocs/docs/configuration.md
@@ -95,6 +95,7 @@ Iceberg works with the concept of a FileIO which is a pluggable module for readi
- **hdfs**: `PyArrowFileIO`
- **abfs**, **abfss**: `FsspecFileIO`
- **oss**: `PyArrowFileIO`
+- **hf**: `FsspecFileIO`
You can also set the FileIO explicitly:
@@ -193,6 +194,17 @@ PyIceberg uses [S3FileSystem](https://arrow.apache.org/docs/python/generated/pya
+### Hugging Face
+
+
+
+| Key | Example | Description |
+| ----------- | ------------------------ | --------------------------------------------------------- |
+| hf.endpoint | | Configure the endpoint for Hugging Face |
+| hf.token | hf_xxx | The Hugging Face token to access HF Datasets repositories |
+
+
+
### PyArrow
diff --git a/poetry.lock b/poetry.lock
index ec2872048b..4b08064b92 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -58,7 +58,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -71,7 +71,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e2bc827c01f75803de77b134afdbf74fa74b62970eafdf190f3244931d7a5c0d"},
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e365034c5cf6cf74f57420b57682ea79e19eb29033399dd3f40de4d0171998fa"},
@@ -196,7 +196,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -248,7 +248,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -265,7 +265,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -467,7 +467,7 @@ files = [
{file = "boto3-1.37.3-py3-none-any.whl", hash = "sha256:2063b40af99fd02f6228ff52397b552ff3353831edaf8d25cc04801827ab9794"},
{file = "boto3-1.37.3.tar.gz", hash = "sha256:21f3ce0ef111297e63a6eb998a25197b8c10982970c320d4c6e8db08be2157be"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.3,<1.38.0"
@@ -488,7 +488,7 @@ files = [
{file = "botocore-1.37.3-py3-none-any.whl", hash = "sha256:d01bd3bf4c80e61fa88d636ad9f5c9f60a551d71549b481386c6b4efe0bb2b2e"},
{file = "botocore-1.37.3.tar.gz", hash = "sha256:fe8403eb55a88faf9b0f9da6615e5bee7be056d75e17af66c3c8f0a3b0648da4"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -1387,7 +1387,7 @@ files = [
{file = "filelock-3.18.0-py3-none-any.whl", hash = "sha256:c401f4f8377c4464e6db25fff06205fd89bdd83b65eb0488ed1b160f780e21de"},
{file = "filelock-3.18.0.tar.gz", hash = "sha256:adbc88eabb99d2fec8c9c1b229b171f18afa655400173ddc653d5d01501fb9f2"},
]
-markers = {main = "extra == \"ray\""}
+markers = {main = "extra == \"ray\" or extra == \"hf\""}
[package.extras]
docs = ["furo (>=2024.8.6)", "sphinx (>=8.1.3)", "sphinx-autodoc-typehints (>=3)"]
@@ -1442,7 +1442,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5b6a66c18b5b9dd261ca98dffcb826a525334b2f29e7caa54e182255c5f6a65a"},
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d1b3eb7b05ea246510b43a7e53ed1653e55c2121019a97e60cad7efb881a97bb"},
@@ -1984,6 +1984,43 @@ files = [
[package.dependencies]
colorama = ">=0.4"
+[[package]]
+name = "huggingface-hub"
+version = "0.31.2"
+description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
+optional = true
+python-versions = ">=3.8.0"
+groups = ["main"]
+markers = "extra == \"hf\""
+files = [
+ {file = "huggingface_hub-0.31.2-py3-none-any.whl", hash = "sha256:8138cd52aa2326b4429bb00a4a1ba8538346b7b8a808cdce30acb6f1f1bdaeec"},
+ {file = "huggingface_hub-0.31.2.tar.gz", hash = "sha256:7053561376ed7f6ffdaecf09cc54d70dc784ac6315fa4bb9b93e19662b029675"},
+]
+
+[package.dependencies]
+filelock = "*"
+fsspec = ">=2023.5.0"
+packaging = ">=20.9"
+pyyaml = ">=5.1"
+requests = "*"
+tqdm = ">=4.42.1"
+typing-extensions = ">=3.7.4.3"
+
+[package.extras]
+all = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "fastapi", "gradio (>=4.0.0)", "jedi", "libcst (==1.4.0)", "mypy (==1.5.1)", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "ruff (>=0.9.0)", "soundfile", "types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)", "urllib3 (<2.0)"]
+cli = ["InquirerPy (==0.3.4)"]
+dev = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "fastapi", "gradio (>=4.0.0)", "jedi", "libcst (==1.4.0)", "mypy (==1.5.1)", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "ruff (>=0.9.0)", "soundfile", "types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)", "urllib3 (<2.0)"]
+fastai = ["fastai (>=2.4)", "fastcore (>=1.3.27)", "toml"]
+hf-transfer = ["hf-transfer (>=0.1.4)"]
+hf-xet = ["hf-xet (>=1.1.1,<2.0.0)"]
+inference = ["aiohttp"]
+quality = ["libcst (==1.4.0)", "mypy (==1.5.1)", "ruff (>=0.9.0)"]
+tensorflow = ["graphviz", "pydot", "tensorflow"]
+tensorflow-testing = ["keras (<3.0)", "tensorflow"]
+testing = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "fastapi", "gradio (>=4.0.0)", "jedi", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "soundfile", "urllib3 (<2.0)"]
+torch = ["safetensors[torch]", "torch"]
+typing = ["types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)"]
+
[[package]]
name = "identify"
version = "2.6.9"
@@ -2158,7 +2195,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -2976,7 +3013,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:b9f6392d98c0bd70676ae41474e2eecf4c7150cb419237a41f8f96043fcb81d1"},
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3501621d5e86f1a88521ea65d5cad0a0834c77b26f193747615b7c911e5422d2"},
@@ -3324,7 +3361,7 @@ files = [
{file = "packaging-24.2-py3-none-any.whl", hash = "sha256:09abb1bccd265c01f4a3aa3f7a7db064b36514d2cba19a2f694fe6150451a759"},
{file = "packaging-24.2.tar.gz", hash = "sha256:c228a6dc5e932d346bc5739379109d49e8853dd8223571c7c5b55260edc0b97f"},
]
-markers = {main = "extra == \"ray\""}
+markers = {main = "extra == \"ray\" or extra == \"hf\""}
[[package]]
name = "paginate"
@@ -3570,7 +3607,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:f27785888d2fdd918bc36de8b8739f2d6c791399552333721b58193f68ea3e98"},
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4e89cde74154c7b5957f87a355bb9c8ec929c167b59c83d90654ea36aeb6180"},
@@ -3897,7 +3934,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
+markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -4502,7 +4539,7 @@ files = [
{file = "PyYAML-6.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:39693e1f8320ae4f43943590b49779ffb98acb81f788220ea932a6b6c51004d8"},
{file = "pyyaml-6.0.2.tar.gz", hash = "sha256:d584d9ec91ad65861cc08d42e834324ef890a082e591037abe114850ff7bbc3e"},
]
-markers = {main = "extra == \"ray\""}
+markers = {main = "extra == \"ray\" or extra == \"hf\""}
[[package]]
name = "pyyaml-env-tag"
@@ -5007,7 +5044,7 @@ files = [
{file = "s3transfer-0.11.3-py3-none-any.whl", hash = "sha256:ca855bdeb885174b5ffa95b9913622459d4ad8e331fc98eb01e6d5eb6a30655d"},
{file = "s3transfer-0.11.3.tar.gz", hash = "sha256:edae4977e3a122445660c7c114bba949f9d191bae3b34a096f18a1c8c354527a"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.36.0,<2.0a.0"
@@ -5443,7 +5480,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"daft\""
+markers = "extra == \"hf\" or extra == \"daft\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -5743,7 +5780,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7df647e8edd71f000a5208fe6ff8c382a1de8edfbccdbbfe649d263de07d8c34"},
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c69697d3adff5aa4f874b19c0e4ed65180ceed6318ec856ebc423aa5850d84f7"},
@@ -5976,6 +6013,7 @@ duckdb = ["duckdb", "pyarrow"]
dynamodb = ["boto3"]
gcsfs = ["gcsfs"]
glue = ["boto3", "mypy-boto3-glue"]
+hf = ["huggingface-hub"]
hive = ["thrift"]
hive-kerberos = ["kerberos", "thrift", "thrift-sasl"]
pandas = ["pandas", "pyarrow"]
@@ -5993,4 +6031,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "f3d2267ce4f380399dc767a6483a1f198127856883e394d8befbd8a871bbeab9"
+content-hash = "c0afe28336cc5f75e725f3e79febe3058c1a453a61745647b9a81b49a2e2bf71"
diff --git a/pyiceberg/io/__init__.py b/pyiceberg/io/__init__.py
index 475ae6176b..5e5c7e2c7c 100644
--- a/pyiceberg/io/__init__.py
+++ b/pyiceberg/io/__init__.py
@@ -92,6 +92,8 @@
GCS_SERVICE_HOST = "gcs.service.host"
GCS_DEFAULT_LOCATION = "gcs.default-bucket-location"
GCS_VERSION_AWARE = "gcs.version-aware"
+HF_ENDPOINT = "hf.endpoint"
+HF_TOKEN = "hf.token"
PYARROW_USE_LARGE_TYPES_ON_READ = "pyarrow.use-large-types-on-read"
@@ -306,6 +308,7 @@ def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
"viewfs": [ARROW_FILE_IO],
"abfs": [FSSPEC_FILE_IO],
"abfss": [FSSPEC_FILE_IO],
+ "hf": [FSSPEC_FILE_IO],
}
diff --git a/pyiceberg/io/fsspec.py b/pyiceberg/io/fsspec.py
index aced17d64b..c0b019dfc5 100644
--- a/pyiceberg/io/fsspec.py
+++ b/pyiceberg/io/fsspec.py
@@ -59,6 +59,8 @@
GCS_SESSION_KWARGS,
GCS_TOKEN,
GCS_VERSION_AWARE,
+ HF_ENDPOINT,
+ HF_TOKEN,
S3_ACCESS_KEY_ID,
S3_CONNECT_TIMEOUT,
S3_ENDPOINT,
@@ -209,6 +211,15 @@ def _adls(properties: Properties) -> AbstractFileSystem:
)
+def _hf(properties: Properties) -> AbstractFileSystem:
+ from huggingface_hub import HfFileSystem
+
+ return HfFileSystem(
+ endpoint=properties.get(HF_ENDPOINT),
+ token=properties.get(HF_TOKEN),
+ )
+
+
SCHEME_TO_FS = {
"": _file,
"file": _file,
@@ -219,6 +230,7 @@ def _adls(properties: Properties) -> AbstractFileSystem:
"abfss": _adls,
"gs": _gs,
"gcs": _gs,
+ "hf": _hf,
}
diff --git a/pyproject.toml b/pyproject.toml
index 3116214f71..527f59c86e 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -76,6 +76,7 @@ boto3 = { version = ">=1.24.59", optional = true }
s3fs = { version = ">=2023.1.0", optional = true }
adlfs = { version = ">=2023.1.0", optional = true }
gcsfs = { version = ">=2023.1.0", optional = true }
+huggingface-hub = { version = ">=0.24.0", optional = true }
psycopg2-binary = { version = ">=2.9.6", optional = true }
sqlalchemy = { version = "^2.0.18", optional = true }
getdaft = { version = ">=0.2.12", optional = true }
@@ -306,6 +307,7 @@ sql-postgres = ["sqlalchemy", "psycopg2-binary"]
sql-sqlite = ["sqlalchemy"]
gcsfs = ["gcsfs"]
rest-sigv4 = ["boto3"]
+hf = ["huggingface-hub"]
pyiceberg-core = ["pyiceberg-core"]
[tool.pytest.ini_options]
@@ -427,6 +429,10 @@ ignore_missing_imports = true
module = "gcsfs.*"
ignore_missing_imports = true
+[[tool.mypy.overrides]]
+module = "huggingface_hub.*"
+ignore_missing_imports = true
+
[[tool.mypy.overrides]]
module = "packaging.*"
ignore_missing_imports = true
From e10e81d7193dd6cfde685b289c69f99fbcc4060e Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Sun, 18 May 2025 08:03:34 +0200
Subject: [PATCH 015/112] Build: Bump flask-cors from 5.0.1 to 6.0.0 (#2012)
---
poetry.lock | 36 ++++++++++++++++++------------------
1 file changed, 18 insertions(+), 18 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 4b08064b92..1ffd50b9a5 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -58,7 +58,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -71,7 +71,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e2bc827c01f75803de77b134afdbf74fa74b62970eafdf190f3244931d7a5c0d"},
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e365034c5cf6cf74f57420b57682ea79e19eb29033399dd3f40de4d0171998fa"},
@@ -196,7 +196,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -248,7 +248,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -265,7 +265,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -467,7 +467,7 @@ files = [
{file = "boto3-1.37.3-py3-none-any.whl", hash = "sha256:2063b40af99fd02f6228ff52397b552ff3353831edaf8d25cc04801827ab9794"},
{file = "boto3-1.37.3.tar.gz", hash = "sha256:21f3ce0ef111297e63a6eb998a25197b8c10982970c320d4c6e8db08be2157be"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.3,<1.38.0"
@@ -488,7 +488,7 @@ files = [
{file = "botocore-1.37.3-py3-none-any.whl", hash = "sha256:d01bd3bf4c80e61fa88d636ad9f5c9f60a551d71549b481386c6b4efe0bb2b2e"},
{file = "botocore-1.37.3.tar.gz", hash = "sha256:fe8403eb55a88faf9b0f9da6615e5bee7be056d75e17af66c3c8f0a3b0648da4"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -1421,14 +1421,14 @@ dotenv = ["python-dotenv"]
[[package]]
name = "flask-cors"
-version = "5.0.1"
+version = "6.0.0"
description = "A Flask extension simplifying CORS support"
optional = false
python-versions = "<4.0,>=3.9"
groups = ["dev"]
files = [
- {file = "flask_cors-5.0.1-py3-none-any.whl", hash = "sha256:fa5cb364ead54bbf401a26dbf03030c6b18fb2fcaf70408096a572b409586b0c"},
- {file = "flask_cors-5.0.1.tar.gz", hash = "sha256:6ccb38d16d6b72bbc156c1c3f192bc435bfcc3c2bc864b2df1eb9b2d97b2403c"},
+ {file = "flask_cors-6.0.0-py3-none-any.whl", hash = "sha256:6332073356452343a8ccddbfec7befdc3fdd040141fe776ec9b94c262f058657"},
+ {file = "flask_cors-6.0.0.tar.gz", hash = "sha256:4592c1570246bf7beee96b74bc0adbbfcb1b0318f6ba05c412e8909eceec3393"},
]
[package.dependencies]
@@ -1442,7 +1442,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5b6a66c18b5b9dd261ca98dffcb826a525334b2f29e7caa54e182255c5f6a65a"},
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d1b3eb7b05ea246510b43a7e53ed1653e55c2121019a97e60cad7efb881a97bb"},
@@ -2195,7 +2195,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -3013,7 +3013,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:b9f6392d98c0bd70676ae41474e2eecf4c7150cb419237a41f8f96043fcb81d1"},
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3501621d5e86f1a88521ea65d5cad0a0834c77b26f193747615b7c911e5422d2"},
@@ -3607,7 +3607,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:f27785888d2fdd918bc36de8b8739f2d6c791399552333721b58193f68ea3e98"},
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4e89cde74154c7b5957f87a355bb9c8ec929c167b59c83d90654ea36aeb6180"},
@@ -3934,7 +3934,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
+markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -5044,7 +5044,7 @@ files = [
{file = "s3transfer-0.11.3-py3-none-any.whl", hash = "sha256:ca855bdeb885174b5ffa95b9913622459d4ad8e331fc98eb01e6d5eb6a30655d"},
{file = "s3transfer-0.11.3.tar.gz", hash = "sha256:edae4977e3a122445660c7c114bba949f9d191bae3b34a096f18a1c8c354527a"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.36.0,<2.0a.0"
@@ -5480,7 +5480,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"hf\" or extra == \"daft\""
+markers = "extra == \"daft\" or extra == \"hf\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -5780,7 +5780,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7df647e8edd71f000a5208fe6ff8c382a1de8edfbccdbbfe649d263de07d8c34"},
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c69697d3adff5aa4f874b19c0e4ed65180ceed6318ec856ebc423aa5850d84f7"},
From 981d3854ae4b505d9539b5c47042d5003c45fccb Mon Sep 17 00:00:00 2001
From: Victor
Date: Mon, 19 May 2025 22:00:45 +0200
Subject: [PATCH 016/112] Add ADLFS property `adls.account-host` (#2016)
Closes #2015
# Rationale for this change
Enable the input of the ADLFS option `account_host` via the properties
option `adls.account-host`
# Are these changes tested?
Yes
# Are there any user-facing changes?
The option is now available :)
---
mkdocs/docs/configuration.md | 1 +
pyiceberg/io/__init__.py | 3 ++-
pyiceberg/io/fsspec.py | 6 ++++--
3 files changed, 7 insertions(+), 3 deletions(-)
diff --git a/mkdocs/docs/configuration.md b/mkdocs/docs/configuration.md
index c91ef42e60..4cc38db5dc 100644
--- a/mkdocs/docs/configuration.md
+++ b/mkdocs/docs/configuration.md
@@ -155,6 +155,7 @@ For the FileIO there are several configuration options available:
| adls.tenant-id | ad667be4-b811-11ed-afa1-0242ac120002 | The tenant-id |
| adls.client-id | ad667be4-b811-11ed-afa1-0242ac120002 | The client-id |
| adls.client-secret | oCA3R6P\*ka#oa1Sms2J74z... | The client-secret |
+| adls.account-host | accountname1.blob.core.windows.net | The storage account host. See [AzureBlobFileSystem](https://github.com/fsspec/adlfs/blob/adb9c53b74a0d420625b86dd00fbe615b43201d2/adlfs/spec.py#L125) for reference |
diff --git a/pyiceberg/io/__init__.py b/pyiceberg/io/__init__.py
index 5e5c7e2c7c..b6fa934fdd 100644
--- a/pyiceberg/io/__init__.py
+++ b/pyiceberg/io/__init__.py
@@ -80,7 +80,8 @@
ADLS_SAS_TOKEN = "adls.sas-token"
ADLS_TENANT_ID = "adls.tenant-id"
ADLS_CLIENT_ID = "adls.client-id"
-ADLS_ClIENT_SECRET = "adls.client-secret"
+ADLS_CLIENT_SECRET = "adls.client-secret"
+ADLS_ACCOUNT_HOST = "adls.account-host"
GCS_TOKEN = "gcs.oauth2.token"
GCS_TOKEN_EXPIRES_AT_MS = "gcs.oauth2.token-expires-at"
GCS_PROJECT_ID = "gcs.project-id"
diff --git a/pyiceberg/io/fsspec.py b/pyiceberg/io/fsspec.py
index c0b019dfc5..1cba30a038 100644
--- a/pyiceberg/io/fsspec.py
+++ b/pyiceberg/io/fsspec.py
@@ -39,9 +39,11 @@
from pyiceberg.catalog import TOKEN
from pyiceberg.exceptions import SignError
from pyiceberg.io import (
+ ADLS_ACCOUNT_HOST,
ADLS_ACCOUNT_KEY,
ADLS_ACCOUNT_NAME,
ADLS_CLIENT_ID,
+ ADLS_CLIENT_SECRET,
ADLS_CONNECTION_STRING,
ADLS_SAS_TOKEN,
ADLS_TENANT_ID,
@@ -73,7 +75,6 @@
S3_SIGNER_ENDPOINT,
S3_SIGNER_ENDPOINT_DEFAULT,
S3_SIGNER_URI,
- ADLS_ClIENT_SECRET,
FileIO,
InputFile,
InputStream,
@@ -207,7 +208,8 @@ def _adls(properties: Properties) -> AbstractFileSystem:
sas_token=properties.get(ADLS_SAS_TOKEN),
tenant_id=properties.get(ADLS_TENANT_ID),
client_id=properties.get(ADLS_CLIENT_ID),
- client_secret=properties.get(ADLS_ClIENT_SECRET),
+ client_secret=properties.get(ADLS_CLIENT_SECRET),
+ account_host=properties.get(ADLS_ACCOUNT_HOST),
)
From b3f7d18f25663036d47097450ee047d31ffa206c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 21:10:37 +0200
Subject: [PATCH 017/112] Build: Bump setuptools from 78.1.0 to 78.1.1 (#2027)
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 1ffd50b9a5..b4460924cc 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -5054,14 +5054,14 @@ crt = ["botocore[crt] (>=1.36.0,<2.0a.0)"]
[[package]]
name = "setuptools"
-version = "78.1.0"
+version = "78.1.1"
description = "Easily download, build, install, upgrade, and uninstall Python packages"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "setuptools-78.1.0-py3-none-any.whl", hash = "sha256:3e386e96793c8702ae83d17b853fb93d3e09ef82ec62722e61da5cd22376dcd8"},
- {file = "setuptools-78.1.0.tar.gz", hash = "sha256:18fd474d4a82a5f83dac888df697af65afa82dec7323d09c3e37d1f14288da54"},
+ {file = "setuptools-78.1.1-py3-none-any.whl", hash = "sha256:c3a9c4211ff4c309edb8b8c4f1cbfa7ae324c4ba9f91ff254e3d305b9fd54561"},
+ {file = "setuptools-78.1.1.tar.gz", hash = "sha256:fcc17fd9cd898242f6b4adfaca46137a9edef687f43e6f78469692a5e70d851d"},
]
[package.extras]
From 62cf1a7705224caaaf74e78766b62da5dc943a24 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 21:13:30 +0200
Subject: [PATCH 018/112] Build: Bump fastavro from 1.10.0 to 1.11.1 (#2026)
---
poetry.lock | 71 +++++++++++++++++++++++++++-----------------------
pyproject.toml | 2 +-
2 files changed, 39 insertions(+), 34 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index b4460924cc..f2d0d8860b 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1331,43 +1331,48 @@ test = ["pytest (>=6)"]
[[package]]
name = "fastavro"
-version = "1.10.0"
+version = "1.11.1"
description = "Fast read/write of AVRO files"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "fastavro-1.10.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:1a9fe0672d2caf0fe54e3be659b13de3cad25a267f2073d6f4b9f8862acc31eb"},
- {file = "fastavro-1.10.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:86dd0410770e0c99363788f0584523709d85e57bb457372ec5c285a482c17fe6"},
- {file = "fastavro-1.10.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:190e80dc7d77d03a6a8597a026146b32a0bbe45e3487ab4904dc8c1bebecb26d"},
- {file = "fastavro-1.10.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:bf570d63be9155c3fdc415f60a49c171548334b70fff0679a184b69c29b6bc61"},
- {file = "fastavro-1.10.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:e07abb6798e95dccecaec316265e35a018b523d1f3944ad396d0a93cb95e0a08"},
- {file = "fastavro-1.10.0-cp310-cp310-win_amd64.whl", hash = "sha256:37203097ed11d0b8fd3c004904748777d730cafd26e278167ea602eebdef8eb2"},
- {file = "fastavro-1.10.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:d183c075f527ab695a27ae75f210d4a86bce660cda2f85ae84d5606efc15ef50"},
- {file = "fastavro-1.10.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d7a95a2c0639bffd7c079b59e9a796bfc3a9acd78acff7088f7c54ade24e4a77"},
- {file = "fastavro-1.10.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0a678153b5da1b024a32ec3f611b2e7afd24deac588cb51dd1b0019935191a6d"},
- {file = "fastavro-1.10.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:67a597a5cfea4dddcf8b49eaf8c2b5ffee7fda15b578849185bc690ec0cd0d8f"},
- {file = "fastavro-1.10.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:1fd689724760b17f69565d8a4e7785ed79becd451d1c99263c40cb2d6491f1d4"},
- {file = "fastavro-1.10.0-cp311-cp311-win_amd64.whl", hash = "sha256:4f949d463f9ac4221128a51e4e34e2562f401e5925adcadfd28637a73df6c2d8"},
- {file = "fastavro-1.10.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:cfe57cb0d72f304bd0dcc5a3208ca6a7363a9ae76f3073307d095c9d053b29d4"},
- {file = "fastavro-1.10.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:74e517440c824cb65fb29d3e3903a9406f4d7c75490cef47e55c4c82cdc66270"},
- {file = "fastavro-1.10.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:203c17d44cadde76e8eecb30f2d1b4f33eb478877552d71f049265dc6f2ecd10"},
- {file = "fastavro-1.10.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6575be7f2b5f94023b5a4e766b0251924945ad55e9a96672dc523656d17fe251"},
- {file = "fastavro-1.10.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:fe471deb675ed2f01ee2aac958fbf8ebb13ea00fa4ce7f87e57710a0bc592208"},
- {file = "fastavro-1.10.0-cp312-cp312-win_amd64.whl", hash = "sha256:567ff515f2a5d26d9674b31c95477f3e6022ec206124c62169bc2ffaf0889089"},
- {file = "fastavro-1.10.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:82263af0adfddb39c85f9517d736e1e940fe506dfcc35bc9ab9f85e0fa9236d8"},
- {file = "fastavro-1.10.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:566c193109ff0ff84f1072a165b7106c4f96050078a4e6ac7391f81ca1ef3efa"},
- {file = "fastavro-1.10.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e400d2e55d068404d9fea7c5021f8b999c6f9d9afa1d1f3652ec92c105ffcbdd"},
- {file = "fastavro-1.10.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:9b8227497f71565270f9249fc9af32a93644ca683a0167cfe66d203845c3a038"},
- {file = "fastavro-1.10.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8e62d04c65461b30ac6d314e4197ad666371e97ae8cb2c16f971d802f6c7f514"},
- {file = "fastavro-1.10.0-cp313-cp313-win_amd64.whl", hash = "sha256:86baf8c9740ab570d0d4d18517da71626fe9be4d1142bea684db52bd5adb078f"},
- {file = "fastavro-1.10.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:5bccbb6f8e9e5b834cca964f0e6ebc27ebe65319d3940b0b397751a470f45612"},
- {file = "fastavro-1.10.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b0132f6b0b53f61a0a508a577f64beb5de1a5e068a9b4c0e1df6e3b66568eec4"},
- {file = "fastavro-1.10.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ca37a363b711202c6071a6d4787e68e15fa3ab108261058c4aae853c582339af"},
- {file = "fastavro-1.10.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:cf38cecdd67ca9bd92e6e9ba34a30db6343e7a3bedf171753ee78f8bd9f8a670"},
- {file = "fastavro-1.10.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:f4dd10e0ed42982122d20cdf1a88aa50ee09e5a9cd9b39abdffb1aa4f5b76435"},
- {file = "fastavro-1.10.0-cp39-cp39-win_amd64.whl", hash = "sha256:aaef147dc14dd2d7823246178fd06fc5e477460e070dc6d9e07dd8193a6bc93c"},
- {file = "fastavro-1.10.0.tar.gz", hash = "sha256:47bf41ac6d52cdfe4a3da88c75a802321321b37b663a900d12765101a5d6886f"},
+ {file = "fastavro-1.11.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:603aa1c1d1be21fb4bcb63e1efb0711a9ddb337de81391c32dac95c6e0dacfcc"},
+ {file = "fastavro-1.11.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:45653b312d4ce297e2bd802ea3ffd17ecbe718e5e8b6e2ae04cd72cb50bb99d5"},
+ {file = "fastavro-1.11.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:998a53fc552e6bee9acda32af258f02557313c85fb5b48becba5b71ec82f421e"},
+ {file = "fastavro-1.11.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:9f878c9ad819467120cb066f1c73496c42eb24ecdd7c992ec996f465ef4cedad"},
+ {file = "fastavro-1.11.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:da9e4c231ac4951092c2230ca423d8a3f2966718f072ac1e2c5d2d44c70b2a50"},
+ {file = "fastavro-1.11.1-cp310-cp310-win_amd64.whl", hash = "sha256:7423bfad3199567eeee7ad6816402c7c0ee1658b959e8c10540cfbc60ce96c2a"},
+ {file = "fastavro-1.11.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:3573340e4564e8962e22f814ac937ffe0d4be5eabbd2250f77738dc47e3c8fe9"},
+ {file = "fastavro-1.11.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7291cf47735b8bd6ff5d9b33120e6e0974f52fd5dff90cd24151b22018e7fd29"},
+ {file = "fastavro-1.11.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bf3bb065d657d5bac8b2cb39945194aa086a9b3354f2da7f89c30e4dc20e08e2"},
+ {file = "fastavro-1.11.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8758317c85296b848698132efb13bc44a4fbd6017431cc0f26eaeb0d6fa13d35"},
+ {file = "fastavro-1.11.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ad99d57228f83bf3e2214d183fbf6e2fda97fd649b2bdaf8e9110c36cbb02624"},
+ {file = "fastavro-1.11.1-cp311-cp311-win_amd64.whl", hash = "sha256:9134090178bdbf9eefd467717ced3dc151e27a7e7bfc728260ce512697efe5a4"},
+ {file = "fastavro-1.11.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:e8bc238f2637cd5d15238adbe8fb8c58d2e6f1870e0fb28d89508584670bae4b"},
+ {file = "fastavro-1.11.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b403933081c83fc4d8a012ee64b86e560a024b1280e3711ee74f2abc904886e8"},
+ {file = "fastavro-1.11.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3f6ecb4b5f77aa756d973b7dd1c2fb4e4c95b4832a3c98b059aa96c61870c709"},
+ {file = "fastavro-1.11.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:059893df63ef823b0231b485c9d43016c7e32850cae7bf69f4e9d46dd41c28f2"},
+ {file = "fastavro-1.11.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:5120ffc9a200699218e01777e695a2f08afb3547ba818184198c757dc39417bd"},
+ {file = "fastavro-1.11.1-cp312-cp312-win_amd64.whl", hash = "sha256:7bb9d0d2233f33a52908b6ea9b376fe0baf1144bdfdfb3c6ad326e200a8b56b0"},
+ {file = "fastavro-1.11.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:f963b8ddaf179660e814ab420850c1b4ea33e2ad2de8011549d958b21f77f20a"},
+ {file = "fastavro-1.11.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0253e5b6a3c9b62fae9fc3abd8184c5b64a833322b6af7d666d3db266ad879b5"},
+ {file = "fastavro-1.11.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ca637b150e1f4c0e8e564fad40a16bd922bcb7ffd1a6e4836e6084f2c4f4e8db"},
+ {file = "fastavro-1.11.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:76af1709031621828ca6ce7f027f7711fa33ac23e8269e7a5733996ff8d318da"},
+ {file = "fastavro-1.11.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8224e6d8d9864d4e55dafbe88920d6a1b8c19cc3006acfac6aa4f494a6af3450"},
+ {file = "fastavro-1.11.1-cp313-cp313-win_amd64.whl", hash = "sha256:cde7ed91b52ff21f0f9f157329760ba7251508ca3e9618af3ffdac986d9faaa2"},
+ {file = "fastavro-1.11.1-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:e5ed1325c1c414dd954e7a2c5074daefe1eceb672b8c727aa030ba327aa00693"},
+ {file = "fastavro-1.11.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8cd3c95baeec37188899824faf44a5ee94dfc4d8667b05b2f867070c7eb174c4"},
+ {file = "fastavro-1.11.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e0babcd81acceb4c60110af9efa25d890dbb68f7de880f806dadeb1e70fe413"},
+ {file = "fastavro-1.11.1-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:b2c0cb8063c7208b53b6867983dc6ae7cc80b91116b51d435d2610a5db2fc52f"},
+ {file = "fastavro-1.11.1-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:1bc2824e9969c04ab6263d269a1e0e5d40b9bd16ade6b70c29d6ffbc4f3cc102"},
+ {file = "fastavro-1.11.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:8b579bab31ff87fcb5ef9f6f13baaf99f189b92ed287af60348777583628c327"},
+ {file = "fastavro-1.11.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c646f07c7827fea7425b6936a27f67f356a2a80ac19e6100ed6d3bb0610cc3d"},
+ {file = "fastavro-1.11.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2915324e1edb0e06f0be0c18279c60f4cff49f6fe01626594707eb75cd9952fc"},
+ {file = "fastavro-1.11.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:8fd87ee1e9101b45172fb3cff21b56ce08270d9474eec1d436393677daa95938"},
+ {file = "fastavro-1.11.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:88876568ef387996fbfc6b193a5b9830de3c0497af7d07e5c839a70b86bb47e7"},
+ {file = "fastavro-1.11.1-cp39-cp39-win_amd64.whl", hash = "sha256:cebb7433b860d9b13090d0e53f6db075e4e2042aeb2c577f515e73d2b9c98075"},
+ {file = "fastavro-1.11.1.tar.gz", hash = "sha256:bf6acde5ee633a29fb8dfd6dfea13b164722bc3adc05a0e055df080549c1c2f8"},
]
[package.extras]
@@ -6031,4 +6036,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "c0afe28336cc5f75e725f3e79febe3058c1a453a61745647b9a81b49a2e2bf71"
+content-hash = "2fbf25e57ee037313612299fad870d31d1e1244f23810e34e04076c55f016031"
diff --git a/pyproject.toml b/pyproject.toml
index 527f59c86e..ba9b67d3df 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -91,7 +91,7 @@ pytest = "7.4.4"
pytest-checkdocs = "2.13.0"
pytest-lazy-fixture = "0.6.3"
pre-commit = "4.2.0"
-fastavro = "1.10.0"
+fastavro = "1.11.1"
coverage = { version = "^7.4.2", extras = ["toml"] }
requests-mock = "1.12.1"
moto = { version = "^5.0.2", extras = ["server"] }
From 9eec12acdfacdd14717f8aba14654bdda39742ae Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 21:13:52 +0200
Subject: [PATCH 019/112] Build: Bump sqlalchemy from 2.0.40 to 2.0.41 (#2025)
---
poetry.lock | 116 ++++++++++++++++++++++++++--------------------------
1 file changed, 58 insertions(+), 58 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index f2d0d8860b..cce665341f 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -5253,70 +5253,70 @@ test = ["pytest"]
[[package]]
name = "sqlalchemy"
-version = "2.0.40"
+version = "2.0.41"
description = "Database Abstraction Library"
optional = true
python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"sql-postgres\" or extra == \"sql-sqlite\""
files = [
- {file = "SQLAlchemy-2.0.40-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:ae9597cab738e7cc823f04a704fb754a9249f0b6695a6aeb63b74055cd417a96"},
- {file = "SQLAlchemy-2.0.40-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:37a5c21ab099a83d669ebb251fddf8f5cee4d75ea40a5a1653d9c43d60e20867"},
- {file = "SQLAlchemy-2.0.40-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bece9527f5a98466d67fb5d34dc560c4da964240d8b09024bb21c1246545e04e"},
- {file = "SQLAlchemy-2.0.40-cp37-cp37m-musllinux_1_2_aarch64.whl", hash = "sha256:8bb131ffd2165fae48162c7bbd0d97c84ab961deea9b8bab16366543deeab625"},
- {file = "SQLAlchemy-2.0.40-cp37-cp37m-musllinux_1_2_x86_64.whl", hash = "sha256:9408fd453d5f8990405cc9def9af46bfbe3183e6110401b407c2d073c3388f47"},
- {file = "SQLAlchemy-2.0.40-cp37-cp37m-win32.whl", hash = "sha256:00a494ea6f42a44c326477b5bee4e0fc75f6a80c01570a32b57e89cf0fbef85a"},
- {file = "SQLAlchemy-2.0.40-cp37-cp37m-win_amd64.whl", hash = "sha256:c7b927155112ac858357ccf9d255dd8c044fd9ad2dc6ce4c4149527c901fa4c3"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:f1ea21bef99c703f44444ad29c2c1b6bd55d202750b6de8e06a955380f4725d7"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:afe63b208153f3a7a2d1a5b9df452b0673082588933e54e7c8aac457cf35e758"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a8aae085ea549a1eddbc9298b113cffb75e514eadbb542133dd2b99b5fb3b6af"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5ea9181284754d37db15156eb7be09c86e16e50fbe77610e9e7bee09291771a1"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:5434223b795be5c5ef8244e5ac98056e290d3a99bdcc539b916e282b160dda00"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:15d08d5ef1b779af6a0909b97be6c1fd4298057504eb6461be88bd1696cb438e"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-win32.whl", hash = "sha256:cd2f75598ae70bcfca9117d9e51a3b06fe29edd972fdd7fd57cc97b4dbf3b08a"},
- {file = "sqlalchemy-2.0.40-cp310-cp310-win_amd64.whl", hash = "sha256:2cbafc8d39ff1abdfdda96435f38fab141892dc759a2165947d1a8fffa7ef596"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f6bacab7514de6146a1976bc56e1545bee247242fab030b89e5f70336fc0003e"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5654d1ac34e922b6c5711631f2da497d3a7bffd6f9f87ac23b35feea56098011"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:35904d63412db21088739510216e9349e335f142ce4a04b69e2528020ee19ed4"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c7a80ed86d6aaacb8160a1caef6680d4ddd03c944d985aecee940d168c411d1"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:519624685a51525ddaa7d8ba8265a1540442a2ec71476f0e75241eb8263d6f51"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:2ee5f9999a5b0e9689bed96e60ee53c3384f1a05c2dd8068cc2e8361b0df5b7a"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-win32.whl", hash = "sha256:c0cae71e20e3c02c52f6b9e9722bca70e4a90a466d59477822739dc31ac18b4b"},
- {file = "sqlalchemy-2.0.40-cp311-cp311-win_amd64.whl", hash = "sha256:574aea2c54d8f1dd1699449f332c7d9b71c339e04ae50163a3eb5ce4c4325ee4"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:9d3b31d0a1c44b74d3ae27a3de422dfccd2b8f0b75e51ecb2faa2bf65ab1ba0d"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:37f7a0f506cf78c80450ed1e816978643d3969f99c4ac6b01104a6fe95c5490a"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0bb933a650323e476a2e4fbef8997a10d0003d4da996aad3fd7873e962fdde4d"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6959738971b4745eea16f818a2cd086fb35081383b078272c35ece2b07012716"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:110179728e442dae85dd39591beb74072ae4ad55a44eda2acc6ec98ead80d5f2"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e8040680eaacdce4d635f12c55c714f3d4c7f57da2bc47a01229d115bd319191"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-win32.whl", hash = "sha256:650490653b110905c10adac69408380688cefc1f536a137d0d69aca1069dc1d1"},
- {file = "sqlalchemy-2.0.40-cp312-cp312-win_amd64.whl", hash = "sha256:2be94d75ee06548d2fc591a3513422b873490efb124048f50556369a834853b0"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:915866fd50dd868fdcc18d61d8258db1bf9ed7fbd6dfec960ba43365952f3b01"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:4a4c5a2905a9ccdc67a8963e24abd2f7afcd4348829412483695c59e0af9a705"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:55028d7a3ebdf7ace492fab9895cbc5270153f75442a0472d8516e03159ab364"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6cfedff6878b0e0d1d0a50666a817ecd85051d12d56b43d9d425455e608b5ba0"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:bb19e30fdae77d357ce92192a3504579abe48a66877f476880238a962e5b96db"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:16d325ea898f74b26ffcd1cf8c593b0beed8714f0317df2bed0d8d1de05a8f26"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-win32.whl", hash = "sha256:a669cbe5be3c63f75bcbee0b266779706f1a54bcb1000f302685b87d1b8c1500"},
- {file = "sqlalchemy-2.0.40-cp313-cp313-win_amd64.whl", hash = "sha256:641ee2e0834812d657862f3a7de95e0048bdcb6c55496f39c6fa3d435f6ac6ad"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:50f5885bbed261fc97e2e66c5156244f9704083a674b8d17f24c72217d29baf5"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:cf0e99cdb600eabcd1d65cdba0d3c91418fee21c4aa1d28db47d095b1064a7d8"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fe147fcd85aaed53ce90645c91ed5fca0cc88a797314c70dfd9d35925bd5d106"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:baf7cee56bd552385c1ee39af360772fbfc2f43be005c78d1140204ad6148438"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:4aeb939bcac234b88e2d25d5381655e8353fe06b4e50b1c55ecffe56951d18c2"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:c268b5100cfeaa222c40f55e169d484efa1384b44bf9ca415eae6d556f02cb08"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-win32.whl", hash = "sha256:46628ebcec4f23a1584fb52f2abe12ddb00f3bb3b7b337618b80fc1b51177aff"},
- {file = "sqlalchemy-2.0.40-cp38-cp38-win_amd64.whl", hash = "sha256:7e0505719939e52a7b0c65d20e84a6044eb3712bb6f239c6b1db77ba8e173a37"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:c884de19528e0fcd9dc34ee94c810581dd6e74aef75437ff17e696c2bfefae3e"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:1abb387710283fc5983d8a1209d9696a4eae9db8d7ac94b402981fe2fe2e39ad"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5cfa124eda500ba4b0d3afc3e91ea27ed4754e727c7f025f293a22f512bcd4c9"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8b6b28d303b9d57c17a5164eb1fd2d5119bb6ff4413d5894e74873280483eeb5"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:b5a5bbe29c10c5bfd63893747a1bf6f8049df607638c786252cb9243b86b6706"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:f0fda83e113bb0fb27dc003685f32a5dcb99c9c4f41f4fa0838ac35265c23b5c"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-win32.whl", hash = "sha256:957f8d85d5e834397ef78a6109550aeb0d27a53b5032f7a57f2451e1adc37e98"},
- {file = "sqlalchemy-2.0.40-cp39-cp39-win_amd64.whl", hash = "sha256:1ffdf9c91428e59744f8e6f98190516f8e1d05eec90e936eb08b257332c5e870"},
- {file = "sqlalchemy-2.0.40-py3-none-any.whl", hash = "sha256:32587e2e1e359276957e6fe5dad089758bc042a971a8a09ae8ecf7a8fe23d07a"},
- {file = "sqlalchemy-2.0.40.tar.gz", hash = "sha256:d827099289c64589418ebbcaead0145cd19f4e3e8a93919a0100247af245fa00"},
+ {file = "SQLAlchemy-2.0.41-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:6854175807af57bdb6425e47adbce7d20a4d79bbfd6f6d6519cd10bb7109a7f8"},
+ {file = "SQLAlchemy-2.0.41-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:05132c906066142103b83d9c250b60508af556982a385d96c4eaa9fb9720ac2b"},
+ {file = "SQLAlchemy-2.0.41-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8b4af17bda11e907c51d10686eda89049f9ce5669b08fbe71a29747f1e876036"},
+ {file = "SQLAlchemy-2.0.41-cp37-cp37m-musllinux_1_2_aarch64.whl", hash = "sha256:c0b0e5e1b5d9f3586601048dd68f392dc0cc99a59bb5faf18aab057ce00d00b2"},
+ {file = "SQLAlchemy-2.0.41-cp37-cp37m-musllinux_1_2_x86_64.whl", hash = "sha256:0b3dbf1e7e9bc95f4bac5e2fb6d3fb2f083254c3fdd20a1789af965caf2d2348"},
+ {file = "SQLAlchemy-2.0.41-cp37-cp37m-win32.whl", hash = "sha256:1e3f196a0c59b0cae9a0cd332eb1a4bda4696e863f4f1cf84ab0347992c548c2"},
+ {file = "SQLAlchemy-2.0.41-cp37-cp37m-win_amd64.whl", hash = "sha256:6ab60a5089a8f02009f127806f777fca82581c49e127f08413a66056bd9166dd"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b1f09b6821406ea1f94053f346f28f8215e293344209129a9c0fcc3578598d7b"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1936af879e3db023601196a1684d28e12f19ccf93af01bf3280a3262c4b6b4e5"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b2ac41acfc8d965fb0c464eb8f44995770239668956dc4cdf502d1b1ffe0d747"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:81c24e0c0fde47a9723c81d5806569cddef103aebbf79dbc9fcbb617153dea30"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:23a8825495d8b195c4aa9ff1c430c28f2c821e8c5e2d98089228af887e5d7e29"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:60c578c45c949f909a4026b7807044e7e564adf793537fc762b2489d522f3d11"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-win32.whl", hash = "sha256:118c16cd3f1b00c76d69343e38602006c9cfb9998fa4f798606d28d63f23beda"},
+ {file = "sqlalchemy-2.0.41-cp310-cp310-win_amd64.whl", hash = "sha256:7492967c3386df69f80cf67efd665c0f667cee67032090fe01d7d74b0e19bb08"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:6375cd674fe82d7aa9816d1cb96ec592bac1726c11e0cafbf40eeee9a4516b5f"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9f8c9fdd15a55d9465e590a402f42082705d66b05afc3ffd2d2eb3c6ba919560"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:32f9dc8c44acdee06c8fc6440db9eae8b4af8b01e4b1aee7bdd7241c22edff4f"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:90c11ceb9a1f482c752a71f203a81858625d8df5746d787a4786bca4ffdf71c6"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:911cc493ebd60de5f285bcae0491a60b4f2a9f0f5c270edd1c4dbaef7a38fc04"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:03968a349db483936c249f4d9cd14ff2c296adfa1290b660ba6516f973139582"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-win32.whl", hash = "sha256:293cd444d82b18da48c9f71cd7005844dbbd06ca19be1ccf6779154439eec0b8"},
+ {file = "sqlalchemy-2.0.41-cp311-cp311-win_amd64.whl", hash = "sha256:3d3549fc3e40667ec7199033a4e40a2f669898a00a7b18a931d3efb4c7900504"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:81f413674d85cfd0dfcd6512e10e0f33c19c21860342a4890c3a2b59479929f9"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:598d9ebc1e796431bbd068e41e4de4dc34312b7aa3292571bb3674a0cb415dd1"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a104c5694dfd2d864a6f91b0956eb5d5883234119cb40010115fd45a16da5e70"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6145afea51ff0af7f2564a05fa95eb46f542919e6523729663a5d285ecb3cf5e"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:b46fa6eae1cd1c20e6e6f44e19984d438b6b2d8616d21d783d150df714f44078"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:41836fe661cc98abfae476e14ba1906220f92c4e528771a8a3ae6a151242d2ae"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-win32.whl", hash = "sha256:a8808d5cf866c781150d36a3c8eb3adccfa41a8105d031bf27e92c251e3969d6"},
+ {file = "sqlalchemy-2.0.41-cp312-cp312-win_amd64.whl", hash = "sha256:5b14e97886199c1f52c14629c11d90c11fbb09e9334fa7bb5f6d068d9ced0ce0"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:4eeb195cdedaf17aab6b247894ff2734dcead6c08f748e617bfe05bd5a218443"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:d4ae769b9c1c7757e4ccce94b0641bc203bbdf43ba7a2413ab2523d8d047d8dc"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a62448526dd9ed3e3beedc93df9bb6b55a436ed1474db31a2af13b313a70a7e1"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dc56c9788617b8964ad02e8fcfeed4001c1f8ba91a9e1f31483c0dffb207002a"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:c153265408d18de4cc5ded1941dcd8315894572cddd3c58df5d5b5705b3fa28d"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:4f67766965996e63bb46cfbf2ce5355fc32d9dd3b8ad7e536a920ff9ee422e23"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-win32.whl", hash = "sha256:bfc9064f6658a3d1cadeaa0ba07570b83ce6801a1314985bf98ec9b95d74e15f"},
+ {file = "sqlalchemy-2.0.41-cp313-cp313-win_amd64.whl", hash = "sha256:82ca366a844eb551daff9d2e6e7a9e5e76d2612c8564f58db6c19a726869c1df"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:90144d3b0c8b139408da50196c5cad2a6909b51b23df1f0538411cd23ffa45d3"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:023b3ee6169969beea3bb72312e44d8b7c27c75b347942d943cf49397b7edeb5"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:725875a63abf7c399d4548e686debb65cdc2549e1825437096a0af1f7e374814"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:81965cc20848ab06583506ef54e37cf15c83c7e619df2ad16807c03100745dea"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:dd5ec3aa6ae6e4d5b5de9357d2133c07be1aff6405b136dad753a16afb6717dd"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:ff8e80c4c4932c10493ff97028decfdb622de69cae87e0f127a7ebe32b4069c6"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-win32.whl", hash = "sha256:4d44522480e0bf34c3d63167b8cfa7289c1c54264c2950cc5fc26e7850967e45"},
+ {file = "sqlalchemy-2.0.41-cp38-cp38-win_amd64.whl", hash = "sha256:81eedafa609917040d39aa9332e25881a8e7a0862495fcdf2023a9667209deda"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9a420a91913092d1e20c86a2f5f1fc85c1a8924dbcaf5e0586df8aceb09c9cc2"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:906e6b0d7d452e9a98e5ab8507c0da791856b2380fdee61b765632bb8698026f"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a373a400f3e9bac95ba2a06372c4fd1412a7cee53c37fc6c05f829bf672b8769"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:087b6b52de812741c27231b5a3586384d60c353fbd0e2f81405a814b5591dc8b"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:34ea30ab3ec98355235972dadc497bb659cc75f8292b760394824fab9cf39826"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:8280856dd7c6a68ab3a164b4a4b1c51f7691f6d04af4d4ca23d6ecf2261b7923"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-win32.whl", hash = "sha256:b50eab9994d64f4a823ff99a0ed28a6903224ddbe7fef56a6dd865eec9243440"},
+ {file = "sqlalchemy-2.0.41-cp39-cp39-win_amd64.whl", hash = "sha256:5e22575d169529ac3e0a120cf050ec9daa94b6a9597993d1702884f6954a7d71"},
+ {file = "sqlalchemy-2.0.41-py3-none-any.whl", hash = "sha256:57df5dc6fdb5ed1a88a1ed2195fd31927e705cad62dedd86b46972752a80f576"},
+ {file = "sqlalchemy-2.0.41.tar.gz", hash = "sha256:edba70118c4be3c2b1f90754d308d0b79c6fe2c0fdc52d8ddf603916f83f4db9"},
]
[package.dependencies]
From 77f24003c50dbc45fda3db37d76a8bab935ad15f Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 21:14:19 +0200
Subject: [PATCH 020/112] Build: Bump getdaft from 0.4.14 to 0.4.15 (#2023)
---
poetry.lock | 14 +++++++-------
1 file changed, 7 insertions(+), 7 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index cce665341f..7bed537138 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1611,19 +1611,19 @@ gcsfuse = ["fusepy"]
[[package]]
name = "getdaft"
-version = "0.4.14"
+version = "0.4.15"
description = "Distributed Dataframes for Multimodal Data"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"daft\""
files = [
- {file = "getdaft-0.4.14-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:0db2b53d6d7a22349a82c44baea1a64e431207f5df1a1c72e1b50d6036a7221c"},
- {file = "getdaft-0.4.14-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:f1063e4ca2a4aa956eeedd0fae06e129cb6047249a0f903beeb16b94a7ef46ed"},
- {file = "getdaft-0.4.14-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:9570b409adcb7a866d24ae785d73c2b60cc6febf880d7b2e789a22c6bbe9372c"},
- {file = "getdaft-0.4.14-cp39-abi3-manylinux_2_24_x86_64.whl", hash = "sha256:9c2f8d53b4c89fb793eb9f1f170b3d7d579651e0e7991ee60a65e728c182001c"},
- {file = "getdaft-0.4.14-cp39-abi3-win_amd64.whl", hash = "sha256:cfbd0e758d76bb76a4960295817c805fbfba96de27a5cc621cd0df143051e495"},
- {file = "getdaft-0.4.14.tar.gz", hash = "sha256:4309fc33b0ac4fd1b02cdd718a583c51a46ef1c79ecacf4ae4fa18e9d3c0b211"},
+ {file = "getdaft-0.4.15-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:e77e7ca398cd87c60f2acb4a7b094859fe48dc3577b103bc86d7a69a79641cc8"},
+ {file = "getdaft-0.4.15-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:ef1e63ff3197fc677375eb538d8965900fe0dd78288e5c600503e8745f6149cc"},
+ {file = "getdaft-0.4.15-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:e4b24ef9bba477dcc86e0ae1c60c3b9a42ac769684b13bf5bddff33b55ba0bcb"},
+ {file = "getdaft-0.4.15-cp39-abi3-manylinux_2_24_x86_64.whl", hash = "sha256:d61b565b669ba80d16fa234e8aade0ccb3d6a2a728a85c3d55e800137d2e7c74"},
+ {file = "getdaft-0.4.15-cp39-abi3-win_amd64.whl", hash = "sha256:972dcdf995d37650854d7ab31d03b9bc99492ce4c3d69199490ffe01a9766272"},
+ {file = "getdaft-0.4.15.tar.gz", hash = "sha256:2d6f6e8945a0c49ec65129ac3b7db2b070b575c55470f11322d0d01a5af668ab"},
]
[package.dependencies]
From a20da109b2672fbcb0496f5475420b6ec3bea5f0 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 21:14:42 +0200
Subject: [PATCH 021/112] Build: Bump mypy-boto3-glue from 1.38.12 to 1.38.18
(#2021)
---
poetry.lock | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 7bed537138..8ae177ed91 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -3119,15 +3119,15 @@ typing-extensions = {version = ">=4.1.0", markers = "python_version < \"3.11\""}
[[package]]
name = "mypy-boto3-glue"
-version = "1.38.12"
-description = "Type annotations for boto3 Glue 1.38.12 service generated with mypy-boto3-builder 8.11.0"
+version = "1.38.18"
+description = "Type annotations for boto3 Glue 1.38.18 service generated with mypy-boto3-builder 8.11.0"
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"glue\""
files = [
- {file = "mypy_boto3_glue-1.38.12-py3-none-any.whl", hash = "sha256:2fd93c0c941f5a603e6fba7c60d277053bf68a72df8e6e6532addef61267f892"},
- {file = "mypy_boto3_glue-1.38.12.tar.gz", hash = "sha256:d0c709de2f23b6a9277ec03bb0df4fefe20e32a6c748531049c17d8349bc5a09"},
+ {file = "mypy_boto3_glue-1.38.18-py3-none-any.whl", hash = "sha256:0ae0a50972f0850ed7c2db3b566cc854c173f793f95c0918e937c413b6dc83a6"},
+ {file = "mypy_boto3_glue-1.38.18.tar.gz", hash = "sha256:8aa08d222ae7d361a5d28416d34c4ac70a7edf2385ac65e992c9081f80b99e3b"},
]
[package.dependencies]
From f42e0ca3f120f776e4c45d2d6e8ff6f1f0a6aebe Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 21:15:18 +0200
Subject: [PATCH 022/112] Build: Bump huggingface-hub from 0.31.2 to 0.31.4
(#2020)
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 8ae177ed91..359da4a611 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1991,15 +1991,15 @@ colorama = ">=0.4"
[[package]]
name = "huggingface-hub"
-version = "0.31.2"
+version = "0.31.4"
description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
optional = true
python-versions = ">=3.8.0"
groups = ["main"]
markers = "extra == \"hf\""
files = [
- {file = "huggingface_hub-0.31.2-py3-none-any.whl", hash = "sha256:8138cd52aa2326b4429bb00a4a1ba8538346b7b8a808cdce30acb6f1f1bdaeec"},
- {file = "huggingface_hub-0.31.2.tar.gz", hash = "sha256:7053561376ed7f6ffdaecf09cc54d70dc784ac6315fa4bb9b93e19662b029675"},
+ {file = "huggingface_hub-0.31.4-py3-none-any.whl", hash = "sha256:4f70704760296cc69b612916056e9845f5490a33782b924fc531767967acc15d"},
+ {file = "huggingface_hub-0.31.4.tar.gz", hash = "sha256:5a7bc710b9f9c028aee5b1476867b4ec5c1b92f043cb364d5fdc54354757e4ce"},
]
[package.dependencies]
From f5cc239d93e5045f6f2b69bcc408b5ebfb394dfb Mon Sep 17 00:00:00 2001
From: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
Date: Tue, 20 May 2025 15:39:22 -0400
Subject: [PATCH 023/112] feat: `validate_deleted_data_files` (#1938)
Closes #1928
# Rationale for this change
Add `validate_deleted_data_files` which depends on #1935
# Are these changes tested?
Added a test!
# References
Java `deletedDataFiles` impl:
https://github.com/apache/iceberg/blob/3a29199e73f2e9ae0f8f92a1a0732a338c66aa0d/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java#L678
Java `ManifestGroup.entries` impl:
https://github.com/apache/iceberg/blob/3a29199e73f2e9ae0f8f92a1a0732a338c66aa0d/core/src/main/java/org/apache/iceberg/ManifestGroup.java#L242
---------
Co-authored-by: Fokko Driesprong
---
pyiceberg/table/update/validate.py | 83 +++++++++++++++++++++++++++-
tests/table/test_validate.py | 87 ++++++++++++++++++++++++++++--
2 files changed, 166 insertions(+), 4 deletions(-)
diff --git a/pyiceberg/table/update/validate.py b/pyiceberg/table/update/validate.py
index 93cf12d669..55c34676e3 100644
--- a/pyiceberg/table/update/validate.py
+++ b/pyiceberg/table/update/validate.py
@@ -14,11 +14,17 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
+from typing import Iterator, Optional
from pyiceberg.exceptions import ValidationException
-from pyiceberg.manifest import ManifestContent, ManifestFile
+from pyiceberg.expressions import BooleanExpression
+from pyiceberg.expressions.visitors import ROWS_CANNOT_MATCH, _InclusiveMetricsEvaluator
+from pyiceberg.manifest import ManifestContent, ManifestEntry, ManifestEntryStatus, ManifestFile
from pyiceberg.table import Table
from pyiceberg.table.snapshots import Operation, Snapshot, ancestors_between
+from pyiceberg.typedef import Record
+
+VALIDATE_DATA_FILES_EXIST_OPERATIONS = {Operation.OVERWRITE, Operation.REPLACE, Operation.DELETE}
def validation_history(
@@ -69,3 +75,78 @@ def validation_history(
raise ValidationException("No matching snapshot found.")
return manifests_files, snapshots
+
+
+def _deleted_data_files(
+ table: Table,
+ starting_snapshot: Snapshot,
+ data_filter: Optional[BooleanExpression],
+ partition_set: Optional[dict[int, set[Record]]],
+ parent_snapshot: Optional[Snapshot],
+) -> Iterator[ManifestEntry]:
+ """Find deleted data files matching a filter since a starting snapshot.
+
+ Args:
+ table: Table to validate
+ starting_snapshot: Snapshot current at the start of the operation
+ data_filter: Expression used to find deleted data files
+ partition_set: dict of {spec_id: set[partition]} to filter on
+ parent_snapshot: Ending snapshot on the branch being validated
+
+ Returns:
+ List of conflicting manifest-entries
+ """
+ # if there is no current table state, no files have been deleted
+ if parent_snapshot is None:
+ return
+
+ manifests, snapshot_ids = validation_history(
+ table,
+ parent_snapshot,
+ starting_snapshot,
+ VALIDATE_DATA_FILES_EXIST_OPERATIONS,
+ ManifestContent.DATA,
+ )
+
+ if data_filter is not None:
+ evaluator = _InclusiveMetricsEvaluator(table.schema(), data_filter).eval
+
+ for manifest in manifests:
+ for entry in manifest.fetch_manifest_entry(table.io, discard_deleted=False):
+ if entry.snapshot_id not in snapshot_ids:
+ continue
+
+ if entry.status != ManifestEntryStatus.DELETED:
+ continue
+
+ if data_filter is not None and evaluator(entry.data_file) is ROWS_CANNOT_MATCH:
+ continue
+
+ if partition_set is not None:
+ spec_id = entry.data_file.spec_id
+ partition = entry.data_file.partition
+ if spec_id not in partition_set or partition not in partition_set[spec_id]:
+ continue
+
+ yield entry
+
+
+def _validate_deleted_data_files(
+ table: Table,
+ starting_snapshot: Snapshot,
+ data_filter: Optional[BooleanExpression],
+ parent_snapshot: Snapshot,
+) -> None:
+ """Validate that no files matching a filter have been deleted from the table since a starting snapshot.
+
+ Args:
+ table: Table to validate
+ starting_snapshot: Snapshot current at the start of the operation
+ data_filter: Expression used to find deleted data files
+ parent_snapshot: Ending snapshot on the branch being validated
+
+ """
+ conflicting_entries = _deleted_data_files(table, starting_snapshot, data_filter, None, parent_snapshot)
+ if any(conflicting_entries):
+ conflicting_snapshots = {entry.snapshot_id for entry in conflicting_entries}
+ raise ValidationException(f"Deleted data files were found matching the filter for snapshots {conflicting_snapshots}!")
diff --git a/tests/table/test_validate.py b/tests/table/test_validate.py
index ca7f83badd..74a0b59566 100644
--- a/tests/table/test_validate.py
+++ b/tests/table/test_validate.py
@@ -22,10 +22,10 @@
from pyiceberg.exceptions import ValidationException
from pyiceberg.io import FileIO
-from pyiceberg.manifest import ManifestContent, ManifestFile
+from pyiceberg.manifest import ManifestContent, ManifestEntry, ManifestEntryStatus, ManifestFile
from pyiceberg.table import Table
-from pyiceberg.table.snapshots import Operation, Snapshot
-from pyiceberg.table.update.validate import validation_history
+from pyiceberg.table.snapshots import Operation, Snapshot, Summary
+from pyiceberg.table.update.validate import _deleted_data_files, _validate_deleted_data_files, validation_history
@pytest.fixture
@@ -136,3 +136,84 @@ def mock_read_manifest_side_effect(self: Snapshot, io: FileIO) -> list[ManifestF
{Operation.APPEND},
ManifestContent.DATA,
)
+
+
+def test_deleted_data_files(
+ table_v2_with_extensive_snapshots_and_manifests: tuple[Table, dict[int, list[ManifestFile]]],
+) -> None:
+ table, mock_manifests = table_v2_with_extensive_snapshots_and_manifests
+
+ oldest_snapshot = table.snapshots()[0]
+ newest_snapshot = cast(Snapshot, table.current_snapshot())
+
+ def mock_read_manifest_side_effect(self: Snapshot, io: FileIO) -> list[ManifestFile]:
+ """Mock the manifests method to use the snapshot_id for lookup."""
+ snapshot_id = self.snapshot_id
+ if snapshot_id in mock_manifests:
+ return mock_manifests[snapshot_id]
+ return []
+
+ # every snapshot is an append, so we should get nothing!
+ with patch("pyiceberg.table.snapshots.Snapshot.manifests", new=mock_read_manifest_side_effect):
+ result = list(
+ _deleted_data_files(
+ table=table,
+ starting_snapshot=newest_snapshot,
+ data_filter=None,
+ parent_snapshot=oldest_snapshot,
+ partition_set=None,
+ )
+ )
+
+ assert result == []
+
+ # modify second to last snapshot to be a delete
+ snapshots = table.snapshots()
+ altered_snapshot = snapshots[-2]
+ altered_snapshot = altered_snapshot.model_copy(update={"summary": Summary(operation=Operation.DELETE)})
+ snapshots[-2] = altered_snapshot
+
+ table.metadata = table.metadata.model_copy(
+ update={"snapshots": snapshots},
+ )
+
+ my_entry = ManifestEntry.from_args(
+ status=ManifestEntryStatus.DELETED,
+ snapshot_id=altered_snapshot.snapshot_id,
+ )
+
+ with (
+ patch("pyiceberg.table.snapshots.Snapshot.manifests", new=mock_read_manifest_side_effect),
+ patch("pyiceberg.manifest.ManifestFile.fetch_manifest_entry", return_value=[my_entry]),
+ ):
+ result = list(
+ _deleted_data_files(
+ table=table,
+ starting_snapshot=newest_snapshot,
+ data_filter=None,
+ parent_snapshot=oldest_snapshot,
+ partition_set=None,
+ )
+ )
+
+ assert result == [my_entry]
+
+
+def test_validate_deleted_data_files_raises_on_conflict(
+ table_v2_with_extensive_snapshots_and_manifests: tuple[Table, dict[int, list[ManifestFile]]],
+) -> None:
+ table, _ = table_v2_with_extensive_snapshots_and_manifests
+ oldest_snapshot = table.snapshots()[0]
+ newest_snapshot = cast(Snapshot, table.current_snapshot())
+
+ class DummyEntry:
+ snapshot_id = 123
+
+ with patch("pyiceberg.table.update.validate._deleted_data_files", return_value=[DummyEntry()]):
+ with pytest.raises(ValidationException):
+ _validate_deleted_data_files(
+ table=table,
+ starting_snapshot=newest_snapshot,
+ data_filter=None,
+ parent_snapshot=oldest_snapshot,
+ )
From 3419faa6b94d4763aa52705e91c5ca7f592f3305 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 21:50:46 +0200
Subject: [PATCH 024/112] Build: Bump mkdocs-material from 9.6.13 to 9.6.14
(#2024)
---
poetry.lock | 8 ++++----
pyproject.toml | 2 +-
2 files changed, 5 insertions(+), 5 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 359da4a611..1201a26214 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -2620,14 +2620,14 @@ mkdocs = ">=1.4.1"
[[package]]
name = "mkdocs-material"
-version = "9.6.13"
+version = "9.6.14"
description = "Documentation that simply works"
optional = false
python-versions = ">=3.8"
groups = ["docs"]
files = [
- {file = "mkdocs_material-9.6.13-py3-none-any.whl", hash = "sha256:3730730314e065f422cc04eacbc8c6084530de90f4654a1482472283a38e30d3"},
- {file = "mkdocs_material-9.6.13.tar.gz", hash = "sha256:7bde7ebf33cfd687c1c86c08ed8f6470d9a5ba737bd89e7b3e5d9f94f8c72c16"},
+ {file = "mkdocs_material-9.6.14-py3-none-any.whl", hash = "sha256:3b9cee6d3688551bf7a8e8f41afda97a3c39a12f0325436d76c86706114b721b"},
+ {file = "mkdocs_material-9.6.14.tar.gz", hash = "sha256:39d795e90dce6b531387c255bd07e866e027828b7346d3eba5ac3de265053754"},
]
[package.dependencies]
@@ -6036,4 +6036,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "2fbf25e57ee037313612299fad870d31d1e1244f23810e34e04076c55f016031"
+content-hash = "b6cf3047d966550b8c57aa92cff5ba153581d1a06a410bf33b9b9149139b2485"
diff --git a/pyproject.toml b/pyproject.toml
index ba9b67d3df..c6016728d0 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -113,7 +113,7 @@ mkdocstrings-python = "1.16.10"
mkdocs-literate-nav = "0.6.2"
mkdocs-autorefs = "1.4.1"
mkdocs-gen-files = "0.5.0"
-mkdocs-material = "9.6.13"
+mkdocs-material = "9.6.14"
mkdocs-material-extensions = "1.3.1"
mkdocs-section-index = "0.3.10"
From 28618810e96bcee6f9164a24b7b81509e634cfdd Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 May 2025 22:20:05 +0200
Subject: [PATCH 025/112] Build: Bump cython from 3.0.12 to 3.1.1 (#2022)
---
poetry.lock | 132 ++++++++++++++++++++++++-------------------------
pyproject.toml | 2 +-
2 files changed, 66 insertions(+), 68 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 1201a26214..aa091215e9 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1046,76 +1046,74 @@ test-randomorder = ["pytest-randomly"]
[[package]]
name = "cython"
-version = "3.0.12"
+version = "3.1.1"
description = "The Cython compiler for writing C extensions in the Python language."
optional = false
-python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,>=2.7"
+python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "Cython-3.0.12-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:ba67eee9413b66dd9fbacd33f0bc2e028a2a120991d77b5fd4b19d0b1e4039b9"},
- {file = "Cython-3.0.12-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bee2717e5b5f7d966d0c6e27d2efe3698c357aa4d61bb3201997c7a4f9fe485a"},
- {file = "Cython-3.0.12-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7cffc3464f641c8d0dda942c7c53015291beea11ec4d32421bed2f13b386b819"},
- {file = "Cython-3.0.12-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:d3a8f81980ffbd74e52f9186d8f1654e347d0c44bfea6b5997028977f481a179"},
- {file = "Cython-3.0.12-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:8d32856716c369d01f2385ad9177cdd1a11079ac89ea0932dc4882de1aa19174"},
- {file = "Cython-3.0.12-cp310-cp310-win32.whl", hash = "sha256:712c3f31adec140dc60d064a7f84741f50e2c25a8edd7ae746d5eb4d3ef7072a"},
- {file = "Cython-3.0.12-cp310-cp310-win_amd64.whl", hash = "sha256:d6945694c5b9170cfbd5f2c0d00ef7487a2de7aba83713a64ee4ebce7fad9e05"},
- {file = "Cython-3.0.12-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:feb86122a823937cc06e4c029d80ff69f082ebb0b959ab52a5af6cdd271c5dc3"},
- {file = "Cython-3.0.12-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dfdbea486e702c328338314adb8e80f5f9741f06a0ae83aaec7463bc166d12e8"},
- {file = "Cython-3.0.12-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:563de1728c8e48869d2380a1b76bbc1b1b1d01aba948480d68c1d05e52d20c92"},
- {file = "Cython-3.0.12-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:398d4576c1e1f6316282aa0b4a55139254fbed965cba7813e6d9900d3092b128"},
- {file = "Cython-3.0.12-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:1e5eadef80143026944ea8f9904715a008f5108d1d644a89f63094cc37351e73"},
- {file = "Cython-3.0.12-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:5a93cbda00a5451175b97dea5a9440a3fcee9e54b4cba7a7dbcba9a764b22aec"},
- {file = "Cython-3.0.12-cp311-cp311-win32.whl", hash = "sha256:3109e1d44425a2639e9a677b66cd7711721a5b606b65867cb2d8ef7a97e2237b"},
- {file = "Cython-3.0.12-cp311-cp311-win_amd64.whl", hash = "sha256:d4b70fc339adba1e2111b074ee6119fe9fd6072c957d8597bce9a0dd1c3c6784"},
- {file = "Cython-3.0.12-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:fe030d4a00afb2844f5f70896b7f2a1a0d7da09bf3aa3d884cbe5f73fff5d310"},
- {file = "Cython-3.0.12-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a7fec4f052b8fe173fe70eae75091389955b9a23d5cec3d576d21c5913b49d47"},
- {file = "Cython-3.0.12-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0faa5e39e5c8cdf6f9c3b1c3f24972826e45911e7f5b99cf99453fca5432f45e"},
- {file = "Cython-3.0.12-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2d53de996ed340e9ab0fc85a88aaa8932f2591a2746e1ab1c06e262bd4ec4be7"},
- {file = "Cython-3.0.12-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:ea3a0e19ab77266c738aa110684a753a04da4e709472cadeff487133354d6ab8"},
- {file = "Cython-3.0.12-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c151082884be468f2f405645858a857298ac7f7592729e5b54788b5c572717ba"},
- {file = "Cython-3.0.12-cp312-cp312-win32.whl", hash = "sha256:3083465749911ac3b2ce001b6bf17f404ac9dd35d8b08469d19dc7e717f5877a"},
- {file = "Cython-3.0.12-cp312-cp312-win_amd64.whl", hash = "sha256:c0b91c7ebace030dd558ea28730de8c580680b50768e5af66db2904a3716c3e3"},
- {file = "Cython-3.0.12-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:4ee6f1ea1bead8e6cbc4e64571505b5d8dbdb3b58e679d31f3a84160cebf1a1a"},
- {file = "Cython-3.0.12-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:57aefa6d3341109e46ec1a13e3a763aaa2cbeb14e82af2485b318194be1d9170"},
- {file = "Cython-3.0.12-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:879ae9023958d63c0675015369384642d0afb9c9d1f3473df9186c42f7a9d265"},
- {file = "Cython-3.0.12-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:36fcd584dae547de6f095500a380f4a0cce72b7a7e409e9ff03cb9beed6ac7a1"},
- {file = "Cython-3.0.12-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:62b79dcc0de49efe9e84b9d0e2ae0a6fc9b14691a65565da727aa2e2e63c6a28"},
- {file = "Cython-3.0.12-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:4aa255781b093a8401109d8f2104bbb2e52de7639d5896aefafddc85c30e0894"},
- {file = "Cython-3.0.12-cp313-cp313-win32.whl", hash = "sha256:77d48f2d4bab9fe1236eb753d18f03e8b2619af5b6f05d51df0532a92dfb38ab"},
- {file = "Cython-3.0.12-cp313-cp313-win_amd64.whl", hash = "sha256:86c304b20bd57c727c7357e90d5ba1a2b6f1c45492de2373814d7745ef2e63b4"},
- {file = "Cython-3.0.12-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:ff5c0b6a65b08117d0534941d404833d516dac422eee88c6b4fd55feb409a5ed"},
- {file = "Cython-3.0.12-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:680f1d6ed4436ae94805db264d6155ed076d2835d84f20dcb31a7a3ad7f8668c"},
- {file = "Cython-3.0.12-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ebc24609613fa06d0d896309f7164ba168f7e8d71c1e490ed2a08d23351c3f41"},
- {file = "Cython-3.0.12-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c1879c073e2b34924ce9b7ca64c212705dcc416af4337c45f371242b2e5f6d32"},
- {file = "Cython-3.0.12-cp36-cp36m-musllinux_1_2_aarch64.whl", hash = "sha256:bfb75123dd4ff767baa37d7036da0de2dfb6781ff256eef69b11b88b9a0691d1"},
- {file = "Cython-3.0.12-cp36-cp36m-musllinux_1_2_x86_64.whl", hash = "sha256:f39640f8df0400cde6882e23c734f15bb8196de0a008ae5dc6c8d1ec5957d7c8"},
- {file = "Cython-3.0.12-cp36-cp36m-win32.whl", hash = "sha256:8c9efe9a0895abee3cadfdad4130b30f7b5e57f6e6a51ef2a44f9fc66a913880"},
- {file = "Cython-3.0.12-cp36-cp36m-win_amd64.whl", hash = "sha256:63d840f2975e44d74512f8f34f1f7cb8121c9428e26a3f6116ff273deb5e60a2"},
- {file = "Cython-3.0.12-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:75c5acd40b97cff16fadcf6901a91586cbca5dcdba81f738efaf1f4c6bc8dccb"},
- {file = "Cython-3.0.12-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e62564457851db1c40399bd95a5346b9bb99e17a819bf583b362f418d8f3457a"},
- {file = "Cython-3.0.12-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3ccd1228cc203b1f1b8a3d403f5a20ad1c40e5879b3fbf5851ce09d948982f2c"},
- {file = "Cython-3.0.12-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:25529ee948f44d9a165ff960c49d4903267c20b5edf2df79b45924802e4cca6e"},
- {file = "Cython-3.0.12-cp37-cp37m-musllinux_1_2_aarch64.whl", hash = "sha256:90cf599372c5a22120609f7d3a963f17814799335d56dd0dcf8fe615980a8ae1"},
- {file = "Cython-3.0.12-cp37-cp37m-musllinux_1_2_x86_64.whl", hash = "sha256:9f8c48748a9c94ea5d59c26ab49ad0fad514d36f894985879cf3c3ca0e600bf4"},
- {file = "Cython-3.0.12-cp37-cp37m-win32.whl", hash = "sha256:3e4fa855d98bc7bd6a2049e0c7dc0dcf595e2e7f571a26e808f3efd84d2db374"},
- {file = "Cython-3.0.12-cp37-cp37m-win_amd64.whl", hash = "sha256:120681093772bf3600caddb296a65b352a0d3556e962b9b147efcfb8e8c9801b"},
- {file = "Cython-3.0.12-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:731d719423e041242c9303c80cae4327467299b90ffe62d4cc407e11e9ea3160"},
- {file = "Cython-3.0.12-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c3238a29f37999e27494d120983eca90d14896b2887a0bd858a381204549137a"},
- {file = "Cython-3.0.12-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b588c0a089a9f4dd316d2f9275230bad4a7271e5af04e1dc41d2707c816be44b"},
- {file = "Cython-3.0.12-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8ab9f5198af74eb16502cc143cdde9ca1cbbf66ea2912e67440dd18a36e3b5fa"},
- {file = "Cython-3.0.12-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:8ee841c0e114efa1e849c281ac9b8df8aa189af10b4a103b1c5fd71cbb799679"},
- {file = "Cython-3.0.12-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:43c48b5789398b228ea97499f5b864843ba9b1ab837562a9227c6f58d16ede8b"},
- {file = "Cython-3.0.12-cp38-cp38-win32.whl", hash = "sha256:5e5f17c48a4f41557fbcc7ee660ccfebe4536a34c557f553b6893c1b3c83df2d"},
- {file = "Cython-3.0.12-cp38-cp38-win_amd64.whl", hash = "sha256:309c081057930bb79dc9ea3061a1af5086c679c968206e9c9c2ec90ab7cb471a"},
- {file = "Cython-3.0.12-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:54115fcc126840926ff3b53cfd2152eae17b3522ae7f74888f8a41413bd32f25"},
- {file = "Cython-3.0.12-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:629db614b9c364596d7c975fa3fb3978e8c5349524353dbe11429896a783fc1e"},
- {file = "Cython-3.0.12-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:af081838b0f9e12a83ec4c3809a00a64c817f489f7c512b0e3ecaf5f90a2a816"},
- {file = "Cython-3.0.12-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:34ce459808f7d8d5d4007bc5486fe50532529096b43957af6cbffcb4d9cc5c8d"},
- {file = "Cython-3.0.12-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:d6c6cd6a75c8393e6805d17f7126b96a894f310a1a9ea91c47d141fb9341bfa8"},
- {file = "Cython-3.0.12-cp39-cp39-win32.whl", hash = "sha256:a4032e48d4734d2df68235d21920c715c451ac9de15fa14c71b378e8986b83be"},
- {file = "Cython-3.0.12-cp39-cp39-win_amd64.whl", hash = "sha256:dcdc3e5d4ce0e7a4af6903ed580833015641e968d18d528d8371e2435a34132c"},
- {file = "Cython-3.0.12-py2.py3-none-any.whl", hash = "sha256:0038c9bae46c459669390e53a1ec115f8096b2e4647ae007ff1bf4e6dee92806"},
- {file = "cython-3.0.12.tar.gz", hash = "sha256:b988bb297ce76c671e28c97d017b95411010f7c77fa6623dd0bb47eed1aee1bc"},
+ {file = "cython-3.1.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0de7adff5b42d2556d073e9f321c2faa639a17fb195ec1de130327f60ec209d8"},
+ {file = "cython-3.1.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9b61b99205308c96b1162de59bd67ecadcad3d166a4a1f03a3d9e826c39cd375"},
+ {file = "cython-3.1.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d14186bd96783d13b8fd0e5b289f2e137a8a25479638b73a1c7e4a99a8d70753"},
+ {file = "cython-3.1.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e3ccec55e2a534a712db14c6617b66f65ad149c014fad518fc3920f6edde770"},
+ {file = "cython-3.1.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a585796939b09b3205b1980e4a55e745c0251e45a5c637afbcac3c6cc9ad6f90"},
+ {file = "cython-3.1.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3fa4bd840de63509c74867b4b092541720a01db1e07351206011c34e0777dc96"},
+ {file = "cython-3.1.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b68f1bc80387554eb43f2b62795c173bed9e37201f39dc5084ac437c90a79c9f"},
+ {file = "cython-3.1.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:e851ab66a31794e40df1bc6f649cdc56c998c637f5a1b9410c97a90f6b6cb855"},
+ {file = "cython-3.1.1-cp310-cp310-win32.whl", hash = "sha256:64915259276482fa23417b284d1fdc7e3a618ee2f819bb6ea7f974c075633df6"},
+ {file = "cython-3.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:dee554f0a589377bdaea0eb70e212bf3f35dc6a51a2aa86c9351345e21fd2f07"},
+ {file = "cython-3.1.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c360823e1063784efc2335617e0f28573d7a594c5a8a05d85e850a9621cccb1f"},
+ {file = "cython-3.1.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:12e00b88147b03c148a95365f89dc1c45a0fc52f9c35aa75ff770ef65b615839"},
+ {file = "cython-3.1.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ab644415458d782c16ba7252de9cec1e3125371641cafea2e53a8c1cf85dd58d"},
+ {file = "cython-3.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c5cb6c054daadaf01a88c8f49f3edd9e829c9b76a82cbb4269e3f9878254540b"},
+ {file = "cython-3.1.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:af8f62cc9339b75fe8434325083e6a7cae88c9c21efd74bbb6ba4e3623219469"},
+ {file = "cython-3.1.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:689c1aad373556bd2ab1aa1c2dad8939a2891465a1fbd2cbbdd42b488fb40ec8"},
+ {file = "cython-3.1.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:953046c190fa9ab9a09a546a909b847cdbb4c1fe34e9bfa4a15b6ee1585a86aa"},
+ {file = "cython-3.1.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:755a991601b27dd3555310d0f95b19a05e622a80d7b4e7a91fa6f5f3ef3f3b80"},
+ {file = "cython-3.1.1-cp311-cp311-win32.whl", hash = "sha256:83b2af5c327f7da4f08afc34fddfaf6d24fa0c000b6b70a527c8125e493b6080"},
+ {file = "cython-3.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:141ffd6279411c562f6b707adc56b63e965a4fd7f21db83f5d4fcbd8c50ac546"},
+ {file = "cython-3.1.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:9d7dc0e4d0cd491fac679a61e9ede348c64ca449f99a284f9a01851aa1dbc7f6"},
+ {file = "cython-3.1.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:fd689910002adfac8734f237cdea1573e38345f27ed7fd445482813b65a29457"},
+ {file = "cython-3.1.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:10f0434916994fe213ea7749268b88d77e3ebcbd1b99542cf64bb7d180f45470"},
+ {file = "cython-3.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:873aac4ac0b0fb197557c0ac15458b780b9221daa4a716881cbd1a9016c8459f"},
+ {file = "cython-3.1.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:23b886a6c8a50b1101ccef2f2f3dc9c699b77633ef5bb5007090226c2ad3f9c2"},
+ {file = "cython-3.1.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:dff0e7dd53a0ca35b64cda843253d5cac944db26663dc097b3a1adf2c49514ad"},
+ {file = "cython-3.1.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:0f7954b0b4b3302655d3caa6924261de5907a4e129bc22ace52fe9ae0cd5a758"},
+ {file = "cython-3.1.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:dfa500fd7ae95ca152a5f8062b870532fa3e27efcef6d00612e1f28b9f72615f"},
+ {file = "cython-3.1.1-cp312-cp312-win32.whl", hash = "sha256:cd748fab8e4426dbcb2e0fa2979558333934d24365e0de5672fbabfe337d880c"},
+ {file = "cython-3.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:307f216ed319ea07644f2ef9974406c830f01bc8e677e2147e9bfcdf9e3ca8ad"},
+ {file = "cython-3.1.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:cb5661941707bd41ec7a9c273d698113ac50392444f785088e9d9706c6a5937b"},
+ {file = "cython-3.1.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:28b174f41718a7041cfbe0f48913020875ff1aaa4793942b2451ac6d2baf3f07"},
+ {file = "cython-3.1.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c740a10cd0f50321d048c8ca318eefb4c42b8bffef982dcd89c946d374192702"},
+ {file = "cython-3.1.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7da069ca769903c5dee56c5f7ab47b2b7b91030eee48912630db5f4f3ec5954a"},
+ {file = "cython-3.1.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:24c640c0746d984789fe2787a098f06cda456ef2dd78b90164d17884b350839a"},
+ {file = "cython-3.1.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:426d78565eb91d3366569b20e92b8f14bffef5f57b2acd05b60bbb9ce5c056a1"},
+ {file = "cython-3.1.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:b181158b5761bdaf40f6854f016ab7ddff64d3db4fca55cb3ca0f73813dd76d6"},
+ {file = "cython-3.1.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:7489559e6c5ecbba49d535c2e03cf77c2594a3190b6aca7da5b508ba1664a89a"},
+ {file = "cython-3.1.1-cp313-cp313-win32.whl", hash = "sha256:263cb0e497910fb5e0a361ad1393b6d728b092178afecc56e8a786f3739960c3"},
+ {file = "cython-3.1.1-cp313-cp313-win_amd64.whl", hash = "sha256:e000f0533eedf3d6dfbe30bb3c58a054c58f0a7778390342fa577a0dc47adab3"},
+ {file = "cython-3.1.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:cdf53dc4b2a13bd072d6c2c18ac073dbf0f798555bc27ba4f7546a275eb16a0f"},
+ {file = "cython-3.1.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:ce82070ccf92c3599d331b9eaaefd9d4562976fb86a8d6bccf05c4a0b8389f2a"},
+ {file = "cython-3.1.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:020089f9c9f10269181f17660a2cada7d4577bd8eea24b7d2b14e6b64b6996be"},
+ {file = "cython-3.1.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:402f86c00b08f875cd0990f0c4dc52eb3e0bc5d630066cdf3c798631976f1937"},
+ {file = "cython-3.1.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:54a8934cb3bf13b1f8f6cbdae8e382e25a26e67de08ea6ebfd0a467131b67227"},
+ {file = "cython-3.1.1-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:6ea77ad1e649cec38f8622ba28dcdfbe7bf519bc132abbcf5df759b3975b5a73"},
+ {file = "cython-3.1.1-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:7e5cad896af896482240979b996bf4136b0d18dc40c56c72c5641bf0ea085dfb"},
+ {file = "cython-3.1.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:16d9870654946375b28280371d370d541641d1071da123d0d64d2c7ebba0cc56"},
+ {file = "cython-3.1.1-cp38-cp38-win32.whl", hash = "sha256:8aaa29e763adf3496ab9d371e3caed8da5d3ce5ff8fb57433e2a2f2b5036e5c8"},
+ {file = "cython-3.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:011cdcbf7725f0cfc1abc55ec83d326e788050711272131daf3cc24a19c34bb2"},
+ {file = "cython-3.1.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:40f50b07c479eaf33981d81cad274c68cf9fb81dbe79cbf991f59491c88a4705"},
+ {file = "cython-3.1.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:a92f6bd395eadea6eed722a8188d3bdd49db1c9fa3c38710456d6148ab71bad7"},
+ {file = "cython-3.1.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:268420b92307ae6c5a16e3cf0e2ba1ae3c861650e992893922a0ce08db07cfdb"},
+ {file = "cython-3.1.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a19188ecd385cdc649e3fec370f38d5fd7f1651aeed0b3fb403180f38fc88e8a"},
+ {file = "cython-3.1.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7fff6526bb6f4eea615663117b86de6ede0d17c477b600d3d8302be3502bd3c3"},
+ {file = "cython-3.1.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:3192a61c2a532d3faccdff508bc8427de9530b587888218bfc0226eb33a84e11"},
+ {file = "cython-3.1.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:56c6768a6f601f93daab7c2487f9f110548a896a91e00a6e119445ada2575323"},
+ {file = "cython-3.1.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:50ad80e2f438e9127a87c10927e6ac16a987df39c248b19ab2cd31330129be3c"},
+ {file = "cython-3.1.1-cp39-cp39-win32.whl", hash = "sha256:b194a65a0fd91f305d2d1e7010f44111774a28533e1e44dd2a76e7de81a219b9"},
+ {file = "cython-3.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:c8b8be01fd40b3e38a76c60a524f956548a3a7566e5530a833a48a695f3d6c12"},
+ {file = "cython-3.1.1-py3-none-any.whl", hash = "sha256:07621e044f332d18139df2ccfcc930151fd323c2f61a58c82f304cffc9eb5280"},
+ {file = "cython-3.1.1.tar.gz", hash = "sha256:505ccd413669d5132a53834d792c707974248088c4f60c497deb1b416e366397"},
]
[[package]]
@@ -6036,4 +6034,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "b6cf3047d966550b8c57aa92cff5ba153581d1a06a410bf33b9b9149139b2485"
+content-hash = "63ecf3d6af1613a8b7dd034d4db07f2a75730558674edb7cb960adc4fb7e9480"
diff --git a/pyproject.toml b/pyproject.toml
index c6016728d0..b3cf089db3 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -98,7 +98,7 @@ moto = { version = "^5.0.2", extras = ["server"] }
typing-extensions = "4.13.2"
pytest-mock = "3.14.0"
pyspark = "3.5.5"
-cython = "3.0.12"
+cython = "3.1.1"
deptry = ">=0.14,<0.24"
datafusion = ">=44,<47"
docutils = "!=0.21.post1" # https://github.com/python-poetry/poetry/issues/9248#issuecomment-2026240520
From 0ddf3b755b38c7e9163cbfca01ee61b31290076a Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Tue, 27 May 2025 22:19:09 +0200
Subject: [PATCH 026/112] Fix the CI (#2049)
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
dev/docker-compose-integration.yml | 2 +-
dev/docker-compose.yml | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/dev/docker-compose-integration.yml b/dev/docker-compose-integration.yml
index cdae1f6695..500a042e16 100644
--- a/dev/docker-compose-integration.yml
+++ b/dev/docker-compose-integration.yml
@@ -81,7 +81,7 @@ services:
- AWS_REGION=us-east-1
entrypoint: >
/bin/sh -c "
- until (/usr/bin/mc config host add minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done;
+ until (/usr/bin/mc alias set minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done;
/usr/bin/mc mb minio/warehouse;
/usr/bin/mc policy set public minio/warehouse;
tail -f /dev/null
diff --git a/dev/docker-compose.yml b/dev/docker-compose.yml
index 5c2c800e5c..609ac8d51f 100644
--- a/dev/docker-compose.yml
+++ b/dev/docker-compose.yml
@@ -38,7 +38,7 @@ services:
- AWS_REGION=us-east-1
entrypoint: >
/bin/sh -c "
- until (/usr/bin/mc config host add minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done;
+ until (/usr/bin/mc alias set minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done;
/usr/bin/mc rm -r --force minio/warehouse;
/usr/bin/mc mb minio/warehouse;
/usr/bin/mc policy set public minio/warehouse;
From 741eae5ae53f583711181feede33d144ee0f31b4 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:01:08 +0200
Subject: [PATCH 027/112] Build: Bump mypy-boto3-glue from 1.38.18 to 1.38.22
(#2038)
---
poetry.lock | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index aa091215e9..ce12d07d19 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -3117,15 +3117,15 @@ typing-extensions = {version = ">=4.1.0", markers = "python_version < \"3.11\""}
[[package]]
name = "mypy-boto3-glue"
-version = "1.38.18"
-description = "Type annotations for boto3 Glue 1.38.18 service generated with mypy-boto3-builder 8.11.0"
+version = "1.38.22"
+description = "Type annotations for boto3 Glue 1.38.22 service generated with mypy-boto3-builder 8.11.0"
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"glue\""
files = [
- {file = "mypy_boto3_glue-1.38.18-py3-none-any.whl", hash = "sha256:0ae0a50972f0850ed7c2db3b566cc854c173f793f95c0918e937c413b6dc83a6"},
- {file = "mypy_boto3_glue-1.38.18.tar.gz", hash = "sha256:8aa08d222ae7d361a5d28416d34c4ac70a7edf2385ac65e992c9081f80b99e3b"},
+ {file = "mypy_boto3_glue-1.38.22-py3-none-any.whl", hash = "sha256:4fe34c858cbee41e8ad30382305c01b0dd9c1da4c84f894860b9249ddabb4a58"},
+ {file = "mypy_boto3_glue-1.38.22.tar.gz", hash = "sha256:a9c529fafaaa9845d39c3204b3fb6cbbb633fa747faf6a084a2b2a381ef12a2b"},
]
[package.dependencies]
From 899d364d99d39508880eb10e82d787bc0eee3dae Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:01:28 +0200
Subject: [PATCH 028/112] Build: Bump moto from 5.1.4 to 5.1.5 (#2039)
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index ce12d07d19..b4858c571b 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -2818,14 +2818,14 @@ type = ["mypy (==1.14.1)"]
[[package]]
name = "moto"
-version = "5.1.4"
+version = "5.1.5"
description = "A library that allows you to easily mock out tests based on AWS infrastructure"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "moto-5.1.4-py3-none-any.whl", hash = "sha256:9a19d7a64c3f03824389cfbd478b64c82bd4d8da21b242a34259360d66cd108b"},
- {file = "moto-5.1.4.tar.gz", hash = "sha256:b339c3514f2986ebefa465671b688bdbf51796705702214b1bad46490b68507a"},
+ {file = "moto-5.1.5-py3-none-any.whl", hash = "sha256:866ae85eb5efe11a78f991127531878fd7f49177eb4a6680f47060430eb8932d"},
+ {file = "moto-5.1.5.tar.gz", hash = "sha256:42b362ea9a16181e8e7b615ac212c294b882f020e9ae02f01230f167926df84e"},
]
[package.dependencies]
From 02d934f1ff4b0e665764dc06c46fc02368026bd9 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:01:47 +0200
Subject: [PATCH 029/112] Build: Bump pytest-mock from 3.14.0 to 3.14.1 (#2040)
---
poetry.lock | 8 ++++----
pyproject.toml | 2 +-
2 files changed, 5 insertions(+), 5 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index b4858c571b..d6a4473490 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -4393,14 +4393,14 @@ pytest = ">=3.2.5"
[[package]]
name = "pytest-mock"
-version = "3.14.0"
+version = "3.14.1"
description = "Thin-wrapper around the mock package for easier use with pytest"
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "pytest-mock-3.14.0.tar.gz", hash = "sha256:2719255a1efeceadbc056d6bf3df3d1c5015530fb40cf347c0f9afac88410bd0"},
- {file = "pytest_mock-3.14.0-py3-none-any.whl", hash = "sha256:0b72c38033392a5f4621342fe11e9219ac11ec9d375f8e2a0c164539e0d70f6f"},
+ {file = "pytest_mock-3.14.1-py3-none-any.whl", hash = "sha256:178aefcd11307d874b4cd3100344e7e2d888d9791a6a1d9bfe90fbc1b74fd1d0"},
+ {file = "pytest_mock-3.14.1.tar.gz", hash = "sha256:159e9edac4c451ce77a5cdb9fc5d1100708d2dd4ba3c3df572f14097351af80e"},
]
[package.dependencies]
@@ -6034,4 +6034,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "63ecf3d6af1613a8b7dd034d4db07f2a75730558674edb7cb960adc4fb7e9480"
+content-hash = "0d82f238d560b31f8c205ec3e896a2c01928e975e14e85275fcab0bfafe48b5a"
diff --git a/pyproject.toml b/pyproject.toml
index b3cf089db3..a92c012830 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -96,7 +96,7 @@ coverage = { version = "^7.4.2", extras = ["toml"] }
requests-mock = "1.12.1"
moto = { version = "^5.0.2", extras = ["server"] }
typing-extensions = "4.13.2"
-pytest-mock = "3.14.0"
+pytest-mock = "3.14.1"
pyspark = "3.5.5"
cython = "3.1.1"
deptry = ">=0.14,<0.24"
From 84c0c98fa663a2541cb0480cc702aa3b1cbab68d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:02:04 +0200
Subject: [PATCH 030/112] Build: Bump getdaft from 0.4.15 to 0.4.16 (#2042)
---
poetry.lock | 17 +++++++++--------
1 file changed, 9 insertions(+), 8 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index d6a4473490..365cb3cb92 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1609,19 +1609,19 @@ gcsfuse = ["fusepy"]
[[package]]
name = "getdaft"
-version = "0.4.15"
+version = "0.4.16"
description = "Distributed Dataframes for Multimodal Data"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"daft\""
files = [
- {file = "getdaft-0.4.15-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:e77e7ca398cd87c60f2acb4a7b094859fe48dc3577b103bc86d7a69a79641cc8"},
- {file = "getdaft-0.4.15-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:ef1e63ff3197fc677375eb538d8965900fe0dd78288e5c600503e8745f6149cc"},
- {file = "getdaft-0.4.15-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:e4b24ef9bba477dcc86e0ae1c60c3b9a42ac769684b13bf5bddff33b55ba0bcb"},
- {file = "getdaft-0.4.15-cp39-abi3-manylinux_2_24_x86_64.whl", hash = "sha256:d61b565b669ba80d16fa234e8aade0ccb3d6a2a728a85c3d55e800137d2e7c74"},
- {file = "getdaft-0.4.15-cp39-abi3-win_amd64.whl", hash = "sha256:972dcdf995d37650854d7ab31d03b9bc99492ce4c3d69199490ffe01a9766272"},
- {file = "getdaft-0.4.15.tar.gz", hash = "sha256:2d6f6e8945a0c49ec65129ac3b7db2b070b575c55470f11322d0d01a5af668ab"},
+ {file = "getdaft-0.4.16-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:2ae85aa3f7ff95afa366e30b2645db05dc77fb1cf1b31aca4617062ed7bfd808"},
+ {file = "getdaft-0.4.16-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:410a3fa24404ce302fd7278184fcda901393095a03b3ff35e7fb2ac06207c383"},
+ {file = "getdaft-0.4.16-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:45f816ce0dc897c4a69a41f5b8bc241192dadfe63abcc6dd0d6a61c85bf0375f"},
+ {file = "getdaft-0.4.16-cp39-abi3-manylinux_2_24_x86_64.whl", hash = "sha256:8ab96334e5447c60d07efe33f59b11f12377aaf5d44d686b76b6733ede28421c"},
+ {file = "getdaft-0.4.16-cp39-abi3-win_amd64.whl", hash = "sha256:9834eefabb5930729e8d0122e2c0af7c908d0f222e0ef2f5b091a40598da0e78"},
+ {file = "getdaft-0.4.16.tar.gz", hash = "sha256:3f407b710000f640e9c790b886615fe677a32b9dcbde3085fe026152f2233785"},
]
[package.dependencies]
@@ -1632,7 +1632,7 @@ tqdm = "*"
typing-extensions = {version = ">=4.0.0", markers = "python_full_version < \"3.10.0\""}
[package.extras]
-all = ["daft[aws,azure,deltalake,gcp,iceberg,numpy,pandas,ray,sql,unity]"]
+all = ["daft[aws,azure,deltalake,gcp,iceberg,numpy,pandas,ray,spark,sql,unity]"]
aws = ["boto3"]
deltalake = ["deltalake", "packaging"]
hudi = ["pyarrow (>=8.0.0)"]
@@ -1641,6 +1641,7 @@ lance = ["pylance"]
numpy = ["numpy"]
pandas = ["pandas"]
ray = ["packaging", "ray[client,data] (>=2.0.0) ; platform_system != \"Windows\"", "ray[client,data] (>=2.10.0) ; platform_system == \"Windows\""]
+spark = ["googleapis-common-protos (==1.56.4)", "grpcio (>=1.48,<1.57)", "grpcio-status (>=1.48,<1.57)", "numpy (>=1.15)", "pandas (>=1.0.5)", "py4j (>=0.10.9.7)", "pyspark"]
sql = ["connectorx", "sqlalchemy", "sqlglot"]
unity = ["httpx (<=0.27.2)", "unitycatalog"]
From 56d5d1a9a83c0ed7cf3d9e0e8478b9fd94cb9cfe Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:02:21 +0200
Subject: [PATCH 031/112] Build: Bump pydantic from 2.11.4 to 2.11.5 (#2045)
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 365cb3cb92..745dece881 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -3987,14 +3987,14 @@ markers = {main = "(extra == \"zstandard\" or extra == \"adlfs\") and platform_p
[[package]]
name = "pydantic"
-version = "2.11.4"
+version = "2.11.5"
description = "Data validation using Python type hints"
optional = false
python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "pydantic-2.11.4-py3-none-any.whl", hash = "sha256:d9615eaa9ac5a063471da949c8fc16376a84afb5024688b3ff885693506764eb"},
- {file = "pydantic-2.11.4.tar.gz", hash = "sha256:32738d19d63a226a52eed76645a98ee07c1f410ee41d93b4afbfa85ed8111c2d"},
+ {file = "pydantic-2.11.5-py3-none-any.whl", hash = "sha256:f9c26ba06f9747749ca1e5c94d6a85cb84254577553c8785576fd38fa64dc0f7"},
+ {file = "pydantic-2.11.5.tar.gz", hash = "sha256:7f853db3d0ce78ce8bbb148c401c2cdd6431b3473c0cdff2755c7690952a7b7a"},
]
[package.dependencies]
From 8be200d17fb6dffa6e77143f726be2897da07c6b Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:03:07 +0200
Subject: [PATCH 032/112] Build: Bump coverage from 7.8.0 to 7.8.2 (#2047)
---
poetry.lock | 132 +++++++++++++++++++++++++++-------------------------
1 file changed, 68 insertions(+), 64 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 745dece881..6fc0ba9a9d 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -804,75 +804,79 @@ markers = {main = "platform_system == \"Windows\"", dev = "platform_system == \"
[[package]]
name = "coverage"
-version = "7.8.0"
+version = "7.8.2"
description = "Code coverage measurement for Python"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "coverage-7.8.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2931f66991175369859b5fd58529cd4b73582461877ecfd859b6549869287ffe"},
- {file = "coverage-7.8.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:52a523153c568d2c0ef8826f6cc23031dc86cffb8c6aeab92c4ff776e7951b28"},
- {file = "coverage-7.8.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5c8a5c139aae4c35cbd7cadca1df02ea8cf28a911534fc1b0456acb0b14234f3"},
- {file = "coverage-7.8.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5a26c0c795c3e0b63ec7da6efded5f0bc856d7c0b24b2ac84b4d1d7bc578d676"},
- {file = "coverage-7.8.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:821f7bcbaa84318287115d54becb1915eece6918136c6f91045bb84e2f88739d"},
- {file = "coverage-7.8.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:a321c61477ff8ee705b8a5fed370b5710c56b3a52d17b983d9215861e37b642a"},
- {file = "coverage-7.8.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:ed2144b8a78f9d94d9515963ed273d620e07846acd5d4b0a642d4849e8d91a0c"},
- {file = "coverage-7.8.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:042e7841a26498fff7a37d6fda770d17519982f5b7d8bf5278d140b67b61095f"},
- {file = "coverage-7.8.0-cp310-cp310-win32.whl", hash = "sha256:f9983d01d7705b2d1f7a95e10bbe4091fabc03a46881a256c2787637b087003f"},
- {file = "coverage-7.8.0-cp310-cp310-win_amd64.whl", hash = "sha256:5a570cd9bd20b85d1a0d7b009aaf6c110b52b5755c17be6962f8ccd65d1dbd23"},
- {file = "coverage-7.8.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:e7ac22a0bb2c7c49f441f7a6d46c9c80d96e56f5a8bc6972529ed43c8b694e27"},
- {file = "coverage-7.8.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:bf13d564d310c156d1c8e53877baf2993fb3073b2fc9f69790ca6a732eb4bfea"},
- {file = "coverage-7.8.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5761c70c017c1b0d21b0815a920ffb94a670c8d5d409d9b38857874c21f70d7"},
- {file = "coverage-7.8.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e5ff52d790c7e1628241ffbcaeb33e07d14b007b6eb00a19320c7b8a7024c040"},
- {file = "coverage-7.8.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d39fc4817fd67b3915256af5dda75fd4ee10621a3d484524487e33416c6f3543"},
- {file = "coverage-7.8.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:b44674870709017e4b4036e3d0d6c17f06a0e6d4436422e0ad29b882c40697d2"},
- {file = "coverage-7.8.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:8f99eb72bf27cbb167b636eb1726f590c00e1ad375002230607a844d9e9a2318"},
- {file = "coverage-7.8.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:b571bf5341ba8c6bc02e0baeaf3b061ab993bf372d982ae509807e7f112554e9"},
- {file = "coverage-7.8.0-cp311-cp311-win32.whl", hash = "sha256:e75a2ad7b647fd8046d58c3132d7eaf31b12d8a53c0e4b21fa9c4d23d6ee6d3c"},
- {file = "coverage-7.8.0-cp311-cp311-win_amd64.whl", hash = "sha256:3043ba1c88b2139126fc72cb48574b90e2e0546d4c78b5299317f61b7f718b78"},
- {file = "coverage-7.8.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:bbb5cc845a0292e0c520656d19d7ce40e18d0e19b22cb3e0409135a575bf79fc"},
- {file = "coverage-7.8.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:4dfd9a93db9e78666d178d4f08a5408aa3f2474ad4d0e0378ed5f2ef71640cb6"},
- {file = "coverage-7.8.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f017a61399f13aa6d1039f75cd467be388d157cd81f1a119b9d9a68ba6f2830d"},
- {file = "coverage-7.8.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0915742f4c82208ebf47a2b154a5334155ed9ef9fe6190674b8a46c2fb89cb05"},
- {file = "coverage-7.8.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8a40fcf208e021eb14b0fac6bdb045c0e0cab53105f93ba0d03fd934c956143a"},
- {file = "coverage-7.8.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:a1f406a8e0995d654b2ad87c62caf6befa767885301f3b8f6f73e6f3c31ec3a6"},
- {file = "coverage-7.8.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:77af0f6447a582fdc7de5e06fa3757a3ef87769fbb0fdbdeba78c23049140a47"},
- {file = "coverage-7.8.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:f2d32f95922927186c6dbc8bc60df0d186b6edb828d299ab10898ef3f40052fe"},
- {file = "coverage-7.8.0-cp312-cp312-win32.whl", hash = "sha256:769773614e676f9d8e8a0980dd7740f09a6ea386d0f383db6821df07d0f08545"},
- {file = "coverage-7.8.0-cp312-cp312-win_amd64.whl", hash = "sha256:e5d2b9be5b0693cf21eb4ce0ec8d211efb43966f6657807f6859aab3814f946b"},
- {file = "coverage-7.8.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:5ac46d0c2dd5820ce93943a501ac5f6548ea81594777ca585bf002aa8854cacd"},
- {file = "coverage-7.8.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:771eb7587a0563ca5bb6f622b9ed7f9d07bd08900f7589b4febff05f469bea00"},
- {file = "coverage-7.8.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:42421e04069fb2cbcbca5a696c4050b84a43b05392679d4068acbe65449b5c64"},
- {file = "coverage-7.8.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:554fec1199d93ab30adaa751db68acec2b41c5602ac944bb19187cb9a41a8067"},
- {file = "coverage-7.8.0-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5aaeb00761f985007b38cf463b1d160a14a22c34eb3f6a39d9ad6fc27cb73008"},
- {file = "coverage-7.8.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:581a40c7b94921fffd6457ffe532259813fc68eb2bdda60fa8cc343414ce3733"},
- {file = "coverage-7.8.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:f319bae0321bc838e205bf9e5bc28f0a3165f30c203b610f17ab5552cff90323"},
- {file = "coverage-7.8.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:04bfec25a8ef1c5f41f5e7e5c842f6b615599ca8ba8391ec33a9290d9d2db3a3"},
- {file = "coverage-7.8.0-cp313-cp313-win32.whl", hash = "sha256:dd19608788b50eed889e13a5d71d832edc34fc9dfce606f66e8f9f917eef910d"},
- {file = "coverage-7.8.0-cp313-cp313-win_amd64.whl", hash = "sha256:a9abbccd778d98e9c7e85038e35e91e67f5b520776781d9a1e2ee9d400869487"},
- {file = "coverage-7.8.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:18c5ae6d061ad5b3e7eef4363fb27a0576012a7447af48be6c75b88494c6cf25"},
- {file = "coverage-7.8.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:95aa6ae391a22bbbce1b77ddac846c98c5473de0372ba5c463480043a07bff42"},
- {file = "coverage-7.8.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e013b07ba1c748dacc2a80e69a46286ff145935f260eb8c72df7185bf048f502"},
- {file = "coverage-7.8.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d766a4f0e5aa1ba056ec3496243150698dc0481902e2b8559314368717be82b1"},
- {file = "coverage-7.8.0-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ad80e6b4a0c3cb6f10f29ae4c60e991f424e6b14219d46f1e7d442b938ee68a4"},
- {file = "coverage-7.8.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:b87eb6fc9e1bb8f98892a2458781348fa37e6925f35bb6ceb9d4afd54ba36c73"},
- {file = "coverage-7.8.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:d1ba00ae33be84066cfbe7361d4e04dec78445b2b88bdb734d0d1cbab916025a"},
- {file = "coverage-7.8.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:f3c38e4e5ccbdc9198aecc766cedbb134b2d89bf64533973678dfcf07effd883"},
- {file = "coverage-7.8.0-cp313-cp313t-win32.whl", hash = "sha256:379fe315e206b14e21db5240f89dc0774bdd3e25c3c58c2c733c99eca96f1ada"},
- {file = "coverage-7.8.0-cp313-cp313t-win_amd64.whl", hash = "sha256:2e4b6b87bb0c846a9315e3ab4be2d52fac905100565f4b92f02c445c8799e257"},
- {file = "coverage-7.8.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:fa260de59dfb143af06dcf30c2be0b200bed2a73737a8a59248fcb9fa601ef0f"},
- {file = "coverage-7.8.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:96121edfa4c2dfdda409877ea8608dd01de816a4dc4a0523356067b305e4e17a"},
- {file = "coverage-7.8.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6b8af63b9afa1031c0ef05b217faa598f3069148eeee6bb24b79da9012423b82"},
- {file = "coverage-7.8.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:89b1f4af0d4afe495cd4787a68e00f30f1d15939f550e869de90a86efa7e0814"},
- {file = "coverage-7.8.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:94ec0be97723ae72d63d3aa41961a0b9a6f5a53ff599813c324548d18e3b9e8c"},
- {file = "coverage-7.8.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:8a1d96e780bdb2d0cbb297325711701f7c0b6f89199a57f2049e90064c29f6bd"},
- {file = "coverage-7.8.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:f1d8a2a57b47142b10374902777e798784abf400a004b14f1b0b9eaf1e528ba4"},
- {file = "coverage-7.8.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:cf60dd2696b457b710dd40bf17ad269d5f5457b96442f7f85722bdb16fa6c899"},
- {file = "coverage-7.8.0-cp39-cp39-win32.whl", hash = "sha256:be945402e03de47ba1872cd5236395e0f4ad635526185a930735f66710e1bd3f"},
- {file = "coverage-7.8.0-cp39-cp39-win_amd64.whl", hash = "sha256:90e7fbc6216ecaffa5a880cdc9c77b7418c1dcb166166b78dbc630d07f278cc3"},
- {file = "coverage-7.8.0-pp39.pp310.pp311-none-any.whl", hash = "sha256:b8194fb8e50d556d5849753de991d390c5a1edeeba50f68e3a9253fbd8bf8ccd"},
- {file = "coverage-7.8.0-py3-none-any.whl", hash = "sha256:dbf364b4c5e7bae9250528167dfe40219b62e2d573c854d74be213e1e52069f7"},
- {file = "coverage-7.8.0.tar.gz", hash = "sha256:7a3d62b3b03b4b6fd41a085f3574874cf946cb4604d2b4d3e8dca8cd570ca501"},
+ {file = "coverage-7.8.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bd8ec21e1443fd7a447881332f7ce9d35b8fbd2849e761bb290b584535636b0a"},
+ {file = "coverage-7.8.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:4c26c2396674816deaeae7ded0e2b42c26537280f8fe313335858ffff35019be"},
+ {file = "coverage-7.8.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1aec326ed237e5880bfe69ad41616d333712c7937bcefc1343145e972938f9b3"},
+ {file = "coverage-7.8.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5e818796f71702d7a13e50c70de2a1924f729228580bcba1607cccf32eea46e6"},
+ {file = "coverage-7.8.2-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:546e537d9e24efc765c9c891328f30f826e3e4808e31f5d0f87c4ba12bbd1622"},
+ {file = "coverage-7.8.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:ab9b09a2349f58e73f8ebc06fac546dd623e23b063e5398343c5270072e3201c"},
+ {file = "coverage-7.8.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:fd51355ab8a372d89fb0e6a31719e825cf8df8b6724bee942fb5b92c3f016ba3"},
+ {file = "coverage-7.8.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:0774df1e093acb6c9e4d58bce7f86656aeed6c132a16e2337692c12786b32404"},
+ {file = "coverage-7.8.2-cp310-cp310-win32.whl", hash = "sha256:00f2e2f2e37f47e5f54423aeefd6c32a7dbcedc033fcd3928a4f4948e8b96af7"},
+ {file = "coverage-7.8.2-cp310-cp310-win_amd64.whl", hash = "sha256:145b07bea229821d51811bf15eeab346c236d523838eda395ea969d120d13347"},
+ {file = "coverage-7.8.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b99058eef42e6a8dcd135afb068b3d53aff3921ce699e127602efff9956457a9"},
+ {file = "coverage-7.8.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5feb7f2c3e6ea94d3b877def0270dff0947b8d8c04cfa34a17be0a4dc1836879"},
+ {file = "coverage-7.8.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:670a13249b957bb9050fab12d86acef7bf8f6a879b9d1a883799276e0d4c674a"},
+ {file = "coverage-7.8.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0bdc8bf760459a4a4187b452213e04d039990211f98644c7292adf1e471162b5"},
+ {file = "coverage-7.8.2-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:07a989c867986c2a75f158f03fdb413128aad29aca9d4dbce5fc755672d96f11"},
+ {file = "coverage-7.8.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:2db10dedeb619a771ef0e2949ccba7b75e33905de959c2643a4607bef2f3fb3a"},
+ {file = "coverage-7.8.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e6ea7dba4e92926b7b5f0990634b78ea02f208d04af520c73a7c876d5a8d36cb"},
+ {file = "coverage-7.8.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ef2f22795a7aca99fc3c84393a55a53dd18ab8c93fb431004e4d8f0774150f54"},
+ {file = "coverage-7.8.2-cp311-cp311-win32.whl", hash = "sha256:641988828bc18a6368fe72355df5f1703e44411adbe49bba5644b941ce6f2e3a"},
+ {file = "coverage-7.8.2-cp311-cp311-win_amd64.whl", hash = "sha256:8ab4a51cb39dc1933ba627e0875046d150e88478dbe22ce145a68393e9652975"},
+ {file = "coverage-7.8.2-cp311-cp311-win_arm64.whl", hash = "sha256:8966a821e2083c74d88cca5b7dcccc0a3a888a596a04c0b9668a891de3a0cc53"},
+ {file = "coverage-7.8.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:e2f6fe3654468d061942591aef56686131335b7a8325684eda85dacdf311356c"},
+ {file = "coverage-7.8.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:76090fab50610798cc05241bf83b603477c40ee87acd358b66196ab0ca44ffa1"},
+ {file = "coverage-7.8.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2bd0a0a5054be160777a7920b731a0570284db5142abaaf81bcbb282b8d99279"},
+ {file = "coverage-7.8.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:da23ce9a3d356d0affe9c7036030b5c8f14556bd970c9b224f9c8205505e3b99"},
+ {file = "coverage-7.8.2-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c9392773cffeb8d7e042a7b15b82a414011e9d2b5fdbbd3f7e6a6b17d5e21b20"},
+ {file = "coverage-7.8.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:876cbfd0b09ce09d81585d266c07a32657beb3eaec896f39484b631555be0fe2"},
+ {file = "coverage-7.8.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:3da9b771c98977a13fbc3830f6caa85cae6c9c83911d24cb2d218e9394259c57"},
+ {file = "coverage-7.8.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:9a990f6510b3292686713bfef26d0049cd63b9c7bb17e0864f133cbfd2e6167f"},
+ {file = "coverage-7.8.2-cp312-cp312-win32.whl", hash = "sha256:bf8111cddd0f2b54d34e96613e7fbdd59a673f0cf5574b61134ae75b6f5a33b8"},
+ {file = "coverage-7.8.2-cp312-cp312-win_amd64.whl", hash = "sha256:86a323a275e9e44cdf228af9b71c5030861d4d2610886ab920d9945672a81223"},
+ {file = "coverage-7.8.2-cp312-cp312-win_arm64.whl", hash = "sha256:820157de3a589e992689ffcda8639fbabb313b323d26388d02e154164c57b07f"},
+ {file = "coverage-7.8.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:ea561010914ec1c26ab4188aef8b1567272ef6de096312716f90e5baa79ef8ca"},
+ {file = "coverage-7.8.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:cb86337a4fcdd0e598ff2caeb513ac604d2f3da6d53df2c8e368e07ee38e277d"},
+ {file = "coverage-7.8.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:26a4636ddb666971345541b59899e969f3b301143dd86b0ddbb570bd591f1e85"},
+ {file = "coverage-7.8.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5040536cf9b13fb033f76bcb5e1e5cb3b57c4807fef37db9e0ed129c6a094257"},
+ {file = "coverage-7.8.2-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dc67994df9bcd7e0150a47ef41278b9e0a0ea187caba72414b71dc590b99a108"},
+ {file = "coverage-7.8.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:6e6c86888fd076d9e0fe848af0a2142bf606044dc5ceee0aa9eddb56e26895a0"},
+ {file = "coverage-7.8.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:684ca9f58119b8e26bef860db33524ae0365601492e86ba0b71d513f525e7050"},
+ {file = "coverage-7.8.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8165584ddedb49204c4e18da083913bdf6a982bfb558632a79bdaadcdafd0d48"},
+ {file = "coverage-7.8.2-cp313-cp313-win32.whl", hash = "sha256:34759ee2c65362163699cc917bdb2a54114dd06d19bab860725f94ef45a3d9b7"},
+ {file = "coverage-7.8.2-cp313-cp313-win_amd64.whl", hash = "sha256:2f9bc608fbafaee40eb60a9a53dbfb90f53cc66d3d32c2849dc27cf5638a21e3"},
+ {file = "coverage-7.8.2-cp313-cp313-win_arm64.whl", hash = "sha256:9fe449ee461a3b0c7105690419d0b0aba1232f4ff6d120a9e241e58a556733f7"},
+ {file = "coverage-7.8.2-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:8369a7c8ef66bded2b6484053749ff220dbf83cba84f3398c84c51a6f748a008"},
+ {file = "coverage-7.8.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:159b81df53a5fcbc7d45dae3adad554fdbde9829a994e15227b3f9d816d00b36"},
+ {file = "coverage-7.8.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e6fcbbd35a96192d042c691c9e0c49ef54bd7ed865846a3c9d624c30bb67ce46"},
+ {file = "coverage-7.8.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:05364b9cc82f138cc86128dc4e2e1251c2981a2218bfcd556fe6b0fbaa3501be"},
+ {file = "coverage-7.8.2-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:46d532db4e5ff3979ce47d18e2fe8ecad283eeb7367726da0e5ef88e4fe64740"},
+ {file = "coverage-7.8.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:4000a31c34932e7e4fa0381a3d6deb43dc0c8f458e3e7ea6502e6238e10be625"},
+ {file = "coverage-7.8.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:43ff5033d657cd51f83015c3b7a443287250dc14e69910577c3e03bd2e06f27b"},
+ {file = "coverage-7.8.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:94316e13f0981cbbba132c1f9f365cac1d26716aaac130866ca812006f662199"},
+ {file = "coverage-7.8.2-cp313-cp313t-win32.whl", hash = "sha256:3f5673888d3676d0a745c3d0e16da338c5eea300cb1f4ada9c872981265e76d8"},
+ {file = "coverage-7.8.2-cp313-cp313t-win_amd64.whl", hash = "sha256:2c08b05ee8d7861e45dc5a2cc4195c8c66dca5ac613144eb6ebeaff2d502e73d"},
+ {file = "coverage-7.8.2-cp313-cp313t-win_arm64.whl", hash = "sha256:1e1448bb72b387755e1ff3ef1268a06617afd94188164960dba8d0245a46004b"},
+ {file = "coverage-7.8.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:496948261eaac5ac9cf43f5d0a9f6eb7a6d4cb3bedb2c5d294138142f5c18f2a"},
+ {file = "coverage-7.8.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:eacd2de0d30871eff893bab0b67840a96445edcb3c8fd915e6b11ac4b2f3fa6d"},
+ {file = "coverage-7.8.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b039ffddc99ad65d5078ef300e0c7eed08c270dc26570440e3ef18beb816c1ca"},
+ {file = "coverage-7.8.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0e49824808d4375ede9dd84e9961a59c47f9113039f1a525e6be170aa4f5c34d"},
+ {file = "coverage-7.8.2-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b069938961dfad881dc2f8d02b47645cd2f455d3809ba92a8a687bf513839787"},
+ {file = "coverage-7.8.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:de77c3ba8bb686d1c411e78ee1b97e6e0b963fb98b1637658dd9ad2c875cf9d7"},
+ {file = "coverage-7.8.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:1676628065a498943bd3f64f099bb573e08cf1bc6088bbe33cf4424e0876f4b3"},
+ {file = "coverage-7.8.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:8e1a26e7e50076e35f7afafde570ca2b4d7900a491174ca357d29dece5aacee7"},
+ {file = "coverage-7.8.2-cp39-cp39-win32.whl", hash = "sha256:6782a12bf76fa61ad9350d5a6ef5f3f020b57f5e6305cbc663803f2ebd0f270a"},
+ {file = "coverage-7.8.2-cp39-cp39-win_amd64.whl", hash = "sha256:1efa4166ba75ccefd647f2d78b64f53f14fb82622bc94c5a5cb0a622f50f1c9e"},
+ {file = "coverage-7.8.2-pp39.pp310.pp311-none-any.whl", hash = "sha256:ec455eedf3ba0bbdf8f5a570012617eb305c63cb9f03428d39bf544cb2b94837"},
+ {file = "coverage-7.8.2-py3-none-any.whl", hash = "sha256:726f32ee3713f7359696331a18daf0c3b3a70bb0ae71141b9d3c52be7c595e32"},
+ {file = "coverage-7.8.2.tar.gz", hash = "sha256:a886d531373a1f6ff9fad2a2ba4a045b68467b779ae729ee0b3b10ac20033b27"},
]
[package.dependencies]
From d65f7e6d7908d908d109818009a12911f24af5cc Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:03:24 +0200
Subject: [PATCH 033/112] Build: Bump huggingface-hub from 0.31.4 to 0.32.1
(#2046)
---
poetry.lock | 41 +++++++++++++++++++++++++++++++++--------
1 file changed, 33 insertions(+), 8 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 6fc0ba9a9d..a776424b0d 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1992,22 +1992,45 @@ files = [
[package.dependencies]
colorama = ">=0.4"
+[[package]]
+name = "hf-xet"
+version = "1.1.2"
+description = "Fast transfer of large files with the Hugging Face Hub."
+optional = true
+python-versions = ">=3.8"
+groups = ["main"]
+markers = "extra == \"hf\" and (platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"arm64\" or platform_machine == \"aarch64\")"
+files = [
+ {file = "hf_xet-1.1.2-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:dfd1873fd648488c70735cb60f7728512bca0e459e61fcd107069143cd798469"},
+ {file = "hf_xet-1.1.2-cp37-abi3-macosx_11_0_arm64.whl", hash = "sha256:29b584983b2d977c44157d9241dcf0fd50acde0b7bff8897fe4386912330090d"},
+ {file = "hf_xet-1.1.2-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b29ac84298147fe9164cc55ad994ba47399f90b5d045b0b803b99cf5f06d8ec"},
+ {file = "hf_xet-1.1.2-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:d921ba32615676e436a0d15e162331abc9ed43d440916b1d836dc27ce1546173"},
+ {file = "hf_xet-1.1.2-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:d9b03c34e13c44893ab6e8fea18ee8d2a6878c15328dd3aabedbdd83ee9f2ed3"},
+ {file = "hf_xet-1.1.2-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:01b18608955b3d826307d37da8bd38b28a46cd2d9908b3a3655d1363274f941a"},
+ {file = "hf_xet-1.1.2-cp37-abi3-win_amd64.whl", hash = "sha256:3562902c81299b09f3582ddfb324400c6a901a2f3bc854f83556495755f4954c"},
+ {file = "hf_xet-1.1.2.tar.gz", hash = "sha256:3712d6d4819d3976a1c18e36db9f503e296283f9363af818f50703506ed63da3"},
+]
+
+[package.extras]
+tests = ["pytest"]
+
[[package]]
name = "huggingface-hub"
-version = "0.31.4"
+version = "0.32.2"
description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
optional = true
python-versions = ">=3.8.0"
groups = ["main"]
markers = "extra == \"hf\""
files = [
- {file = "huggingface_hub-0.31.4-py3-none-any.whl", hash = "sha256:4f70704760296cc69b612916056e9845f5490a33782b924fc531767967acc15d"},
- {file = "huggingface_hub-0.31.4.tar.gz", hash = "sha256:5a7bc710b9f9c028aee5b1476867b4ec5c1b92f043cb364d5fdc54354757e4ce"},
+ {file = "huggingface_hub-0.32.2-py3-none-any.whl", hash = "sha256:f8fcf14603237eadf96dbe577d30b330f8c27b4a0a31e8f6c94fdc25e021fdb8"},
+ {file = "huggingface_hub-0.32.2.tar.gz", hash = "sha256:64a288b1eadad6b60bbfd50f0e52fd6cfa2ef77ab13c3e8a834a038ae929de54"},
]
[package.dependencies]
filelock = "*"
fsspec = ">=2023.5.0"
+hf-xet = {version = ">=1.1.2,<2.0.0", markers = "platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"arm64\" or platform_machine == \"aarch64\""}
packaging = ">=20.9"
pyyaml = ">=5.1"
requests = "*"
@@ -2015,17 +2038,19 @@ tqdm = ">=4.42.1"
typing-extensions = ">=3.7.4.3"
[package.extras]
-all = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "fastapi", "gradio (>=4.0.0)", "jedi", "libcst (==1.4.0)", "mypy (==1.5.1)", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "ruff (>=0.9.0)", "soundfile", "types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)", "urllib3 (<2.0)"]
+all = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "authlib (>=1.3.2)", "fastapi", "gradio (>=4.0.0)", "httpx", "itsdangerous", "jedi", "libcst (==1.4.0)", "mypy (==1.15.0) ; python_version >= \"3.9\"", "mypy (>=1.14.1,<1.15.0) ; python_version == \"3.8\"", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "ruff (>=0.9.0)", "soundfile", "types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)", "urllib3 (<2.0)"]
cli = ["InquirerPy (==0.3.4)"]
-dev = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "fastapi", "gradio (>=4.0.0)", "jedi", "libcst (==1.4.0)", "mypy (==1.5.1)", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "ruff (>=0.9.0)", "soundfile", "types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)", "urllib3 (<2.0)"]
+dev = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "authlib (>=1.3.2)", "fastapi", "gradio (>=4.0.0)", "httpx", "itsdangerous", "jedi", "libcst (==1.4.0)", "mypy (==1.15.0) ; python_version >= \"3.9\"", "mypy (>=1.14.1,<1.15.0) ; python_version == \"3.8\"", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "ruff (>=0.9.0)", "soundfile", "types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)", "urllib3 (<2.0)"]
fastai = ["fastai (>=2.4)", "fastcore (>=1.3.27)", "toml"]
hf-transfer = ["hf-transfer (>=0.1.4)"]
-hf-xet = ["hf-xet (>=1.1.1,<2.0.0)"]
+hf-xet = ["hf-xet (>=1.1.2,<2.0.0)"]
inference = ["aiohttp"]
-quality = ["libcst (==1.4.0)", "mypy (==1.5.1)", "ruff (>=0.9.0)"]
+mcp = ["aiohttp", "mcp (>=1.8.0)", "typer"]
+oauth = ["authlib (>=1.3.2)", "fastapi", "httpx", "itsdangerous"]
+quality = ["libcst (==1.4.0)", "mypy (==1.15.0) ; python_version >= \"3.9\"", "mypy (>=1.14.1,<1.15.0) ; python_version == \"3.8\"", "ruff (>=0.9.0)"]
tensorflow = ["graphviz", "pydot", "tensorflow"]
tensorflow-testing = ["keras (<3.0)", "tensorflow"]
-testing = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "fastapi", "gradio (>=4.0.0)", "jedi", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "soundfile", "urllib3 (<2.0)"]
+testing = ["InquirerPy (==0.3.4)", "Jinja2", "Pillow", "aiohttp", "authlib (>=1.3.2)", "fastapi", "gradio (>=4.0.0)", "httpx", "itsdangerous", "jedi", "numpy", "pytest (>=8.1.1,<8.2.2)", "pytest-asyncio", "pytest-cov", "pytest-env", "pytest-mock", "pytest-rerunfailures", "pytest-vcr", "pytest-xdist", "soundfile", "urllib3 (<2.0)"]
torch = ["safetensors[torch]", "torch"]
typing = ["types-PyYAML", "types-requests", "types-simplejson", "types-toml", "types-tqdm", "types-urllib3", "typing-extensions (>=4.8.0)"]
From fd559a2cc25e06d56c4d582ed75ab81be3c7ec19 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:03:45 +0200
Subject: [PATCH 034/112] Build: Bump thrift from 0.21.0 to 0.22.0 (#2043)
---
poetry.lock | 7 ++-----
1 file changed, 2 insertions(+), 5 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index a776424b0d..a1e8099a53 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -5427,19 +5427,16 @@ test = ["pytest", "tornado (>=4.5)", "typeguard"]
[[package]]
name = "thrift"
-version = "0.21.0"
+version = "0.22.0"
description = "Python bindings for the Apache Thrift RPC system"
optional = true
python-versions = "*"
groups = ["main"]
markers = "extra == \"hive-kerberos\" or extra == \"hive\""
files = [
- {file = "thrift-0.21.0.tar.gz", hash = "sha256:5e6f7c50f936ebfa23e924229afc95eb219f8c8e5a83202dd4a391244803e402"},
+ {file = "thrift-0.22.0.tar.gz", hash = "sha256:42e8276afbd5f54fe1d364858b6877bc5e5a4a5ed69f6a005b94ca4918fe1466"},
]
-[package.dependencies]
-six = ">=1.7.2"
-
[package.extras]
all = ["tornado (>=4.0)", "twisted"]
tornado = ["tornado (>=4.0)"]
From d4b5f9b8708787e2ef5378ace6f87e07f467e119 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 May 2025 23:36:04 -0400
Subject: [PATCH 035/112] Build: Bump mkdocstrings-python from 1.16.10 to
1.16.11 (#2044)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [mkdocstrings-python](https://github.com/mkdocstrings/python) from
1.16.10 to 1.16.11.
Release notes
Sourced from mkdocstrings-python's
releases.
1.16.11
Compare
with 1.16.10
Bug Fixes
Changelog
Sourced from mkdocstrings-python's
changelog.
Compare
with 1.16.10
Bug Fixes
Commits
5d2ba0a
chore: Prepare release 1.16.11
7f95686
fix: Fix highlighting for signature with known special names like
__init__
0a35b20
fix: Use default font-size for parameter headings
ba66969
fix: Prevent uppercasing H5 titles (by Material for MkDocs)
096960a
fix: Use configured heading even when signature is not separated
d4e618a
fix: Render attribute names without full path in ToC
bb36fa1
chore: Template upgrade
- See full diff in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 8 ++++----
pyproject.toml | 2 +-
2 files changed, 5 insertions(+), 5 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index a1e8099a53..8fb3c24e9e 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -2731,14 +2731,14 @@ python-legacy = ["mkdocstrings-python-legacy (>=0.2.1)"]
[[package]]
name = "mkdocstrings-python"
-version = "1.16.10"
+version = "1.16.11"
description = "A Python handler for mkdocstrings."
optional = false
python-versions = ">=3.9"
groups = ["docs"]
files = [
- {file = "mkdocstrings_python-1.16.10-py3-none-any.whl", hash = "sha256:63bb9f01f8848a644bdb6289e86dc38ceddeaa63ecc2e291e3b2ca52702a6643"},
- {file = "mkdocstrings_python-1.16.10.tar.gz", hash = "sha256:f9eedfd98effb612ab4d0ed6dd2b73aff6eba5215e0a65cea6d877717f75502e"},
+ {file = "mkdocstrings_python-1.16.11-py3-none-any.whl", hash = "sha256:25d96cc9c1f9c272ea1bd8222c900b5f852bf46c984003e9c7c56eaa4696190f"},
+ {file = "mkdocstrings_python-1.16.11.tar.gz", hash = "sha256:935f95efa887f99178e4a7becaaa1286fb35adafffd669b04fd611d97c00e5ce"},
]
[package.dependencies]
@@ -6061,4 +6061,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "0d82f238d560b31f8c205ec3e896a2c01928e975e14e85275fcab0bfafe48b5a"
+content-hash = "f4a9a7f5647a6fc51ec90ff2351020a21da67a953ba9f9bf2d2c04c3b7e06ce5"
diff --git a/pyproject.toml b/pyproject.toml
index a92c012830..9dfb0d361f 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -109,7 +109,7 @@ mkdocs = "1.6.1"
griffe = "1.7.3"
jinja2 = "3.1.6"
mkdocstrings = "0.29.1"
-mkdocstrings-python = "1.16.10"
+mkdocstrings-python = "1.16.11"
mkdocs-literate-nav = "0.6.2"
mkdocs-autorefs = "1.4.1"
mkdocs-gen-files = "0.5.0"
From 16ae0f5acb05b8c0bcc6dbbf57f5d45ac430a06b Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 28 May 2025 05:43:44 +0200
Subject: [PATCH 036/112] Build: Bump mkdocs-autorefs from 1.4.1 to 1.4.2
(#2048)
---
poetry.lock | 8 ++++----
pyproject.toml | 2 +-
2 files changed, 5 insertions(+), 5 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 8fb3c24e9e..5a75986e75 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -2583,14 +2583,14 @@ min-versions = ["babel (==2.9.0)", "click (==7.0)", "colorama (==0.4) ; platform
[[package]]
name = "mkdocs-autorefs"
-version = "1.4.1"
+version = "1.4.2"
description = "Automatically link across pages in MkDocs."
optional = false
python-versions = ">=3.9"
groups = ["docs"]
files = [
- {file = "mkdocs_autorefs-1.4.1-py3-none-any.whl", hash = "sha256:9793c5ac06a6ebbe52ec0f8439256e66187badf4b5334b5fde0b128ec134df4f"},
- {file = "mkdocs_autorefs-1.4.1.tar.gz", hash = "sha256:4b5b6235a4becb2b10425c2fa191737e415b37aa3418919db33e5d774c9db079"},
+ {file = "mkdocs_autorefs-1.4.2-py3-none-any.whl", hash = "sha256:83d6d777b66ec3c372a1aad4ae0cf77c243ba5bcda5bf0c6b8a2c5e7a3d89f13"},
+ {file = "mkdocs_autorefs-1.4.2.tar.gz", hash = "sha256:e2ebe1abd2b67d597ed19378c0fff84d73d1dbce411fce7a7cc6f161888b6749"},
]
[package.dependencies]
@@ -6061,4 +6061,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "f4a9a7f5647a6fc51ec90ff2351020a21da67a953ba9f9bf2d2c04c3b7e06ce5"
+content-hash = "aa1196f4d24001f4d49d6a927568fea6361161c24a65b6ec9bb0b6380aabf6b8"
diff --git a/pyproject.toml b/pyproject.toml
index 9dfb0d361f..6589a3f9aa 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -111,7 +111,7 @@ jinja2 = "3.1.6"
mkdocstrings = "0.29.1"
mkdocstrings-python = "1.16.11"
mkdocs-literate-nav = "0.6.2"
-mkdocs-autorefs = "1.4.1"
+mkdocs-autorefs = "1.4.2"
mkdocs-gen-files = "0.5.0"
mkdocs-material = "9.6.14"
mkdocs-material-extensions = "1.3.1"
From b410ca2975f7bf8ced7460a6f69d676e03446eb9 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 28 May 2025 00:02:13 -0400
Subject: [PATCH 037/112] Build: Bump cachetools from 5.5.2 to 6.0.0 (#2041)
Bumps [cachetools](https://github.com/tkem/cachetools) from 5.5.2 to
6.0.0.
Changelog
Sourced from cachetools's
changelog.
v6.0.0 (2025-05-23)
-
Require Python 3.9 or later (breaking change).
-
Remove MRUCache and the @func.mru_cache
decorator (breaking
change).
-
Add an optional condition parameter to the
@cached and
@cachedmethod decorators, which, when used with a
threading.Condition instance, should improve cache
stampede <https://en.wikipedia.org/wiki/Cache_stampede>_
issues in massively
parallel environments. Note that this will inflict some performance
penalty, and therefore has to be enabled explicitly.
-
Convert the cachetools.func decorators to use a
threading.Condition instance to deal with cache
stampede <https://en.wikipedia.org/wiki/Cache_stampede>_
issues. Note that
this may result in a noticable performance degradation,
depending
on your actual use case.
-
Deprecate support for cache(self) returning
None to suppress
caching with the @cachedmethod decorator.
-
Improve documentation.
-
Update CI environment.
Commits
e497575
Release v6.0.0.
0a8f391
Update related projects section.
3afef37
Pre-release v6.0.0b4.
34815dd
Update documentation for v6.0.0.
8854b50
Add intersphinx extension.
0af4a07
Bump codecov/codecov-action from 5.4.0 to 5.4.2
ad42371
Merge pull request #349
from tkem/dependabot/github_actions/actions/setup-pyt...
987c076
Bump actions/setup-python from 5.5.0 to 5.6.0
27d0dce
Add cache_condition wrapper attribute (and refactor a bit).
37b5c12
Use class method attributes in tests.
- Additional commits viewable in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 186 ++++++++++++++-----------------------------------
pyproject.toml | 2 +-
2 files changed, 52 insertions(+), 136 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 5a75986e75..8aa1c21011 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -530,14 +530,14 @@ virtualenv = ["virtualenv (>=20.0.35)"]
[[package]]
name = "cachetools"
-version = "5.5.2"
+version = "6.0.0"
description = "Extensible memoizing collections and decorators"
optional = false
-python-versions = ">=3.7"
+python-versions = ">=3.9"
groups = ["main"]
files = [
- {file = "cachetools-5.5.2-py3-none-any.whl", hash = "sha256:d26a22bcc62eb95c3beabd9f1ee5e820d3d2704fe2967cbe350e20c8ffcd3f0a"},
- {file = "cachetools-5.5.2.tar.gz", hash = "sha256:1a661caa9175d26759571b2e19580f9d6393969e5dfca11fdb1f947a23e640d4"},
+ {file = "cachetools-6.0.0-py3-none-any.whl", hash = "sha256:82e73ba88f7b30228b5507dce1a1f878498fc669d972aef2dde4f3a3c24f103e"},
+ {file = "cachetools-6.0.0.tar.gz", hash = "sha256:f225782b84438f828328fc2ad74346522f27e5b1440f4e9fd18b20ebfd1aa2cf"},
]
[[package]]
@@ -1669,74 +1669,65 @@ dev = ["flake8", "markdown", "twine", "wheel"]
[[package]]
name = "google-api-core"
-version = "2.24.2"
+version = "1.16.0"
description = "Google API client core library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_api_core-2.24.2-py3-none-any.whl", hash = "sha256:810a63ac95f3c441b7c0e43d344e372887f62ce9071ba972eacf32672e072de9"},
- {file = "google_api_core-2.24.2.tar.gz", hash = "sha256:81718493daf06d96d6bc76a91c23874dbf2fac0adbbf542831b805ee6e974696"},
+ {file = "google-api-core-1.16.0.tar.gz", hash = "sha256:92e962a087f1c4b8d1c5c88ade1c1dfd550047dcffb320c57ef6a534a20403e2"},
+ {file = "google_api_core-1.16.0-py2.py3-none-any.whl", hash = "sha256:859f7392676761f2b160c6ee030c3422135ada4458f0948c5690a6a7c8d86294"},
]
[package.dependencies]
-google-auth = ">=2.14.1,<3.0.0"
-googleapis-common-protos = ">=1.56.2,<2.0.0"
-proto-plus = [
- {version = ">=1.22.3,<2.0.0"},
- {version = ">=1.25.0,<2.0.0", markers = "python_version >= \"3.13\""},
-]
-protobuf = ">=3.19.5,<3.20.0 || >3.20.0,<3.20.1 || >3.20.1,<4.21.0 || >4.21.0,<4.21.1 || >4.21.1,<4.21.2 || >4.21.2,<4.21.3 || >4.21.3,<4.21.4 || >4.21.4,<4.21.5 || >4.21.5,<7.0.0"
-requests = ">=2.18.0,<3.0.0"
+google-auth = ">=0.4.0,<2.0dev"
+googleapis-common-protos = ">=1.6.0,<2.0dev"
+protobuf = ">=3.4.0"
+pytz = "*"
+requests = ">=2.18.0,<3.0.0dev"
+setuptools = ">=34.0.0"
+six = ">=1.10.0"
[package.extras]
-async-rest = ["google-auth[aiohttp] (>=2.35.0,<3.0.dev0)"]
-grpc = ["grpcio (>=1.33.2,<2.0dev)", "grpcio (>=1.49.1,<2.0dev) ; python_version >= \"3.11\"", "grpcio-status (>=1.33.2,<2.0.dev0)", "grpcio-status (>=1.49.1,<2.0.dev0) ; python_version >= \"3.11\""]
-grpcgcp = ["grpcio-gcp (>=0.2.2,<1.0.dev0)"]
-grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.dev0)"]
+grpc = ["grpcio (>=1.8.2,<2.0dev)"]
+grpcgcp = ["grpcio-gcp (>=0.2.2)"]
+grpcio-gcp = ["grpcio-gcp (>=0.2.2)"]
[[package]]
name = "google-auth"
-version = "2.38.0"
+version = "1.6.3"
description = "Google Authentication Library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_auth-2.38.0-py2.py3-none-any.whl", hash = "sha256:e7dae6694313f434a2727bf2906f27ad259bae090d7aa896590d86feec3d9d4a"},
- {file = "google_auth-2.38.0.tar.gz", hash = "sha256:8285113607d3b80a3f1543b75962447ba8a09fe85783432a784fdeef6ac094c4"},
+ {file = "google-auth-1.6.3.tar.gz", hash = "sha256:0f7c6a64927d34c1a474da92cfc59e552a5d3b940d3266606c6a28b72888b9e4"},
+ {file = "google_auth-1.6.3-py2.py3-none-any.whl", hash = "sha256:20705f6803fd2c4d1cc2dcb0df09d4dfcb9a7d51fd59e94a3a28231fd93119ed"},
]
[package.dependencies]
-cachetools = ">=2.0.0,<6.0"
+cachetools = ">=2.0.0"
pyasn1-modules = ">=0.2.1"
-rsa = ">=3.1.4,<5"
-
-[package.extras]
-aiohttp = ["aiohttp (>=3.6.2,<4.0.0.dev0)", "requests (>=2.20.0,<3.0.0.dev0)"]
-enterprise-cert = ["cryptography", "pyopenssl"]
-pyjwt = ["cryptography (>=38.0.3)", "pyjwt (>=2.0)"]
-pyopenssl = ["cryptography (>=38.0.3)", "pyopenssl (>=20.0.0)"]
-reauth = ["pyu2f (>=0.1.5)"]
-requests = ["requests (>=2.20.0,<3.0.0.dev0)"]
+rsa = ">=3.1.4"
+six = ">=1.9.0"
[[package]]
name = "google-auth-oauthlib"
-version = "1.2.1"
+version = "0.5.3"
description = "Google Authentication Library"
optional = true
python-versions = ">=3.6"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_auth_oauthlib-1.2.1-py2.py3-none-any.whl", hash = "sha256:2d58a27262d55aa1b87678c3ba7142a080098cbc2024f903c62355deb235d91f"},
- {file = "google_auth_oauthlib-1.2.1.tar.gz", hash = "sha256:afd0cad092a2eaa53cd8e8298557d6de1034c6cb4a740500b5357b648af97263"},
+ {file = "google-auth-oauthlib-0.5.3.tar.gz", hash = "sha256:307d21918d61a0741882ad1fd001c67e68ad81206451d05fc4d26f79de56fc90"},
+ {file = "google_auth_oauthlib-0.5.3-py2.py3-none-any.whl", hash = "sha256:9e8ff4ed2b21c174a2d6cc2172c698dbf0b1f686509774c663a83c495091fe09"},
]
[package.dependencies]
-google-auth = ">=2.15.0"
+google-auth = ">=1.0.0"
requests-oauthlib = ">=0.7.0"
[package.extras]
@@ -1744,115 +1735,58 @@ tool = ["click (>=6.0.0)"]
[[package]]
name = "google-cloud-core"
-version = "2.4.3"
+version = "1.4.0"
description = "Google Cloud API client core library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_cloud_core-2.4.3-py2.py3-none-any.whl", hash = "sha256:5130f9f4c14b4fafdff75c79448f9495cfade0d8775facf1b09c3bf67e027f6e"},
- {file = "google_cloud_core-2.4.3.tar.gz", hash = "sha256:1fab62d7102844b278fe6dead3af32408b1df3eb06f5c7e8634cbd40edc4da53"},
+ {file = "google-cloud-core-1.4.0.tar.gz", hash = "sha256:07a024a26c4eb14ee3df7e6e5021c04f8f7e9f0e83d3d47863229f3635f871ce"},
+ {file = "google_cloud_core-1.4.0-py2.py3-none-any.whl", hash = "sha256:e84d4a05c58fc905b9340118e257cd043e9c02a6d31168182204afe4afd521eb"},
]
[package.dependencies]
-google-api-core = ">=1.31.6,<2.0.dev0 || >2.3.0,<3.0.0dev"
-google-auth = ">=1.25.0,<3.0dev"
+google-api-core = ">=1.16.0,<2.0.0dev"
[package.extras]
-grpc = ["grpcio (>=1.38.0,<2.0dev)", "grpcio-status (>=1.38.0,<2.0.dev0)"]
+grpc = ["grpcio (>=1.8.2,<2.0dev)"]
[[package]]
name = "google-cloud-storage"
-version = "3.1.0"
+version = "1.23.0"
description = "Google Cloud Storage API client library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_cloud_storage-3.1.0-py2.py3-none-any.whl", hash = "sha256:eaf36966b68660a9633f03b067e4a10ce09f1377cae3ff9f2c699f69a81c66c6"},
- {file = "google_cloud_storage-3.1.0.tar.gz", hash = "sha256:944273179897c7c8a07ee15f2e6466a02da0c7c4b9ecceac2a26017cb2972049"},
+ {file = "google-cloud-storage-1.23.0.tar.gz", hash = "sha256:c66e876ae9547884fa42566a2ebfec51d280f488d7a058af9611ba90c78bed78"},
+ {file = "google_cloud_storage-1.23.0-py2.py3-none-any.whl", hash = "sha256:9f59c100d3940e38567c48d54cf1a2e7591a2f38e9693dfc11a242d5e54a1626"},
]
[package.dependencies]
-google-api-core = ">=2.15.0,<3.0.0dev"
-google-auth = ">=2.26.1,<3.0dev"
-google-cloud-core = ">=2.4.2,<3.0dev"
-google-crc32c = ">=1.0,<2.0dev"
-google-resumable-media = ">=2.7.2"
-requests = ">=2.18.0,<3.0.0dev"
-
-[package.extras]
-protobuf = ["protobuf (<6.0.0dev)"]
-tracing = ["opentelemetry-api (>=1.1.0)"]
-
-[[package]]
-name = "google-crc32c"
-version = "1.7.1"
-description = "A python wrapper of the C library 'Google CRC32C'"
-optional = true
-python-versions = ">=3.9"
-groups = ["main"]
-markers = "extra == \"gcsfs\""
-files = [
- {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:b07d48faf8292b4db7c3d64ab86f950c2e94e93a11fd47271c28ba458e4a0d76"},
- {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:7cc81b3a2fbd932a4313eb53cc7d9dde424088ca3a0337160f35d91826880c1d"},
- {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:1c67ca0a1f5b56162951a9dae987988679a7db682d6f97ce0f6381ebf0fbea4c"},
- {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc5319db92daa516b653600794d5b9f9439a9a121f3e162f94b0e1891c7933cb"},
- {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dcdf5a64adb747610140572ed18d011896e3b9ae5195f2514b7ff678c80f1603"},
- {file = "google_crc32c-1.7.1-cp310-cp310-win_amd64.whl", hash = "sha256:754561c6c66e89d55754106739e22fdaa93fafa8da7221b29c8b8e8270c6ec8a"},
- {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:6fbab4b935989e2c3610371963ba1b86afb09537fd0c633049be82afe153ac06"},
- {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:ed66cbe1ed9cbaaad9392b5259b3eba4a9e565420d734e6238813c428c3336c9"},
- {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ee6547b657621b6cbed3562ea7826c3e11cab01cd33b74e1f677690652883e77"},
- {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d68e17bad8f7dd9a49181a1f5a8f4b251c6dbc8cc96fb79f1d321dfd57d66f53"},
- {file = "google_crc32c-1.7.1-cp311-cp311-win_amd64.whl", hash = "sha256:6335de12921f06e1f774d0dd1fbea6bf610abe0887a1638f64d694013138be5d"},
- {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:2d73a68a653c57281401871dd4aeebbb6af3191dcac751a76ce430df4d403194"},
- {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:22beacf83baaf59f9d3ab2bbb4db0fb018da8e5aebdce07ef9f09fce8220285e"},
- {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:19eafa0e4af11b0a4eb3974483d55d2d77ad1911e6cf6f832e1574f6781fd337"},
- {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b6d86616faaea68101195c6bdc40c494e4d76f41e07a37ffdef270879c15fb65"},
- {file = "google_crc32c-1.7.1-cp312-cp312-win_amd64.whl", hash = "sha256:b7491bdc0c7564fcf48c0179d2048ab2f7c7ba36b84ccd3a3e1c3f7a72d3bba6"},
- {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:df8b38bdaf1629d62d51be8bdd04888f37c451564c2042d36e5812da9eff3c35"},
- {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:e42e20a83a29aa2709a0cf271c7f8aefaa23b7ab52e53b322585297bb94d4638"},
- {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:905a385140bf492ac300026717af339790921f411c0dfd9aa5a9e69a08ed32eb"},
- {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b211ddaf20f7ebeec5c333448582c224a7c90a9d98826fbab82c0ddc11348e6"},
- {file = "google_crc32c-1.7.1-cp313-cp313-win_amd64.whl", hash = "sha256:0f99eaa09a9a7e642a61e06742856eec8b19fc0037832e03f941fe7cf0c8e4db"},
- {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:32d1da0d74ec5634a05f53ef7df18fc646666a25efaaca9fc7dcfd4caf1d98c3"},
- {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e10554d4abc5238823112c2ad7e4560f96c7bf3820b202660373d769d9e6e4c9"},
- {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:9fc196f0b8d8bd2789352c6a522db03f89e83a0ed6b64315923c396d7a932315"},
- {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:bb5e35dcd8552f76eed9461a23de1030920a3c953c1982f324be8f97946e7127"},
- {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f2226b6a8da04f1d9e61d3e357f2460b9551c5e6950071437e122c958a18ae14"},
- {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1f2b3522222746fff0e04a9bd0a23ea003ba3cccc8cf21385c564deb1f223242"},
- {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3bda0fcb632d390e3ea8b6b07bf6b4f4a66c9d02dcd6fbf7ba00a197c143f582"},
- {file = "google_crc32c-1.7.1-cp39-cp39-win_amd64.whl", hash = "sha256:713121af19f1a617054c41f952294764e0c5443d5a5d9034b2cd60f5dd7e0349"},
- {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a8e9afc74168b0b2232fb32dd202c93e46b7d5e4bf03e66ba5dc273bb3559589"},
- {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa8136cc14dd27f34a3221c0f16fd42d8a40e4778273e61a3c19aedaa44daf6b"},
- {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:85fef7fae11494e747c9fd1359a527e5970fc9603c90764843caabd3a16a0a48"},
- {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6efb97eb4369d52593ad6f75e7e10d053cf00c48983f7a973105bc70b0ac4d82"},
- {file = "google_crc32c-1.7.1.tar.gz", hash = "sha256:2bff2305f98846f3e825dbeec9ee406f89da7962accdb29356e4eadc251bd472"},
-]
-
-[package.extras]
-testing = ["pytest"]
+google-auth = ">=1.2.0"
+google-cloud-core = ">=1.0.3,<2.0dev"
+google-resumable-media = ">=0.5.0,<0.6dev"
[[package]]
name = "google-resumable-media"
-version = "2.7.2"
+version = "0.5.1"
description = "Utilities for Google Media Downloads and Resumable Uploads"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_resumable_media-2.7.2-py2.py3-none-any.whl", hash = "sha256:3ce7551e9fe6d99e9a126101d2536612bb73486721951e9562fee0f90c6ababa"},
- {file = "google_resumable_media-2.7.2.tar.gz", hash = "sha256:5280aed4629f2b60b847b0d42f9857fd4935c11af266744df33d8074cae92fe0"},
+ {file = "google-resumable-media-0.5.1.tar.gz", hash = "sha256:97155236971970382b738921f978a6f86a7b5a0b0311703d991e065d3cb55773"},
+ {file = "google_resumable_media-0.5.1-py2.py3-none-any.whl", hash = "sha256:cdc64378dc9a7a7bf963a8d0c944c99b549dc0c195a9acbf1fcd465f380b9002"},
]
[package.dependencies]
-google-crc32c = ">=1.0,<2.0dev"
+six = "*"
[package.extras]
-aiohttp = ["aiohttp (>=3.6.2,<4.0.0dev)", "google-auth (>=1.22.0,<2.0dev)"]
requests = ["requests (>=2.18.0,<3.0.0dev)"]
[[package]]
@@ -3742,25 +3676,6 @@ files = [
{file = "propcache-0.3.1.tar.gz", hash = "sha256:40d980c33765359098837527e18eddefc9a24cea5b45e078a7f3bb5b032c6ecf"},
]
-[[package]]
-name = "proto-plus"
-version = "1.26.1"
-description = "Beautiful, Pythonic protocol buffers"
-optional = true
-python-versions = ">=3.7"
-groups = ["main"]
-markers = "extra == \"gcsfs\""
-files = [
- {file = "proto_plus-1.26.1-py3-none-any.whl", hash = "sha256:13285478c2dcf2abb829db158e1047e2f1e8d63a077d94263c2b88b043c75a66"},
- {file = "proto_plus-1.26.1.tar.gz", hash = "sha256:21a515a4c4c0088a773899e23c7bbade3d18f9c66c73edd4c7ee3816bc96a012"},
-]
-
-[package.dependencies]
-protobuf = ">=3.19.0,<7.0.0"
-
-[package.extras]
-testing = ["google-api-core (>=1.31.5)"]
-
[[package]]
name = "protobuf"
version = "6.30.1"
@@ -4477,7 +4392,7 @@ description = "World timezone definitions, modern and historical"
optional = true
python-versions = "*"
groups = ["main"]
-markers = "extra == \"pandas\" or extra == \"ray\""
+markers = "extra == \"gcsfs\" or extra == \"pandas\" or extra == \"ray\""
files = [
{file = "pytz-2025.2-py2.py3-none-any.whl", hash = "sha256:5ddf76296dd8c44c26eb8f4b6f35488f3ccbf6fbbd7adee0b7262d43f0ec2f00"},
{file = "pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3"},
@@ -5091,11 +5006,12 @@ version = "78.1.1"
description = "Easily download, build, install, upgrade, and uninstall Python packages"
optional = false
python-versions = ">=3.9"
-groups = ["dev"]
+groups = ["main", "dev"]
files = [
{file = "setuptools-78.1.1-py3-none-any.whl", hash = "sha256:c3a9c4211ff4c309edb8b8c4f1cbfa7ae324c4ba9f91ff254e3d305b9fd54561"},
{file = "setuptools-78.1.1.tar.gz", hash = "sha256:fcc17fd9cd898242f6b4adfaca46137a9edef687f43e6f78469692a5e70d851d"},
]
+markers = {main = "extra == \"gcsfs\""}
[package.extras]
check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\"", "ruff (>=0.8.0) ; sys_platform != \"cygwin\""]
@@ -6061,4 +5977,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "aa1196f4d24001f4d49d6a927568fea6361161c24a65b6ec9bb0b6380aabf6b8"
+content-hash = "9d4fe56235c1988928b6dc5eebc9374ed92e7d6cb52e5356b21c543196f31bda"
diff --git a/pyproject.toml b/pyproject.toml
index 6589a3f9aa..3296b83325 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -80,7 +80,7 @@ huggingface-hub = { version = ">=0.24.0", optional = true }
psycopg2-binary = { version = ">=2.9.6", optional = true }
sqlalchemy = { version = "^2.0.18", optional = true }
getdaft = { version = ">=0.2.12", optional = true }
-cachetools = "^5.5.0"
+cachetools = ">=5.5,<7.0"
pyiceberg-core = { version = "^0.4.0", optional = true }
polars = { version = "^1.21.0", optional = true }
thrift-sasl = { version = ">=0.4.3", optional = true }
From 767b6e85b84051272bf2660262cfe1033ea65f8b Mon Sep 17 00:00:00 2001
From: Alex Stephen <1325798+rambleraptor@users.noreply.github.com>
Date: Sat, 31 May 2025 05:28:19 -0700
Subject: [PATCH 038/112] Add `.vscode` folder to `.gitignore` (#2053)
# Rationale for this change
VSCode places all of its files in a folder now. We should add that
folder to gitignore in the same way we added the original files.
# Are these changes tested?
No tests necessary.
# Are there any user-facing changes?
None.
---
.gitignore | 1 +
1 file changed, 1 insertion(+)
diff --git a/.gitignore b/.gitignore
index 7043f0e7d4..064ce38fcc 100644
--- a/.gitignore
+++ b/.gitignore
@@ -35,6 +35,7 @@ coverage.xml
.project
.settings
bin/
+.vscode/
# Hive/metastore files
metastore_db/
From 6798d277f854d6c2c516020e78a713a7f6a02f71 Mon Sep 17 00:00:00 2001
From: Sung Yun <107272191+sungwy@users.noreply.github.com>
Date: Sun, 1 Jun 2025 16:50:49 -0400
Subject: [PATCH 039/112] Make `validation_history` private before release
(#2054)
# Rationale for this change
`validation_history` is a helper function that will be used for checking
for newly added manifests. Hence it would make sense to make this
function private before it is released.
# Are these changes tested?
Yes, integration tests and unit tests.
# Are there any user-facing changes?
This function has yet to be released
---
pyiceberg/table/update/validate.py | 4 ++--
tests/table/test_validate.py | 8 ++++----
2 files changed, 6 insertions(+), 6 deletions(-)
diff --git a/pyiceberg/table/update/validate.py b/pyiceberg/table/update/validate.py
index 55c34676e3..32aabac28c 100644
--- a/pyiceberg/table/update/validate.py
+++ b/pyiceberg/table/update/validate.py
@@ -27,7 +27,7 @@
VALIDATE_DATA_FILES_EXIST_OPERATIONS = {Operation.OVERWRITE, Operation.REPLACE, Operation.DELETE}
-def validation_history(
+def _validation_history(
table: Table,
from_snapshot: Snapshot,
to_snapshot: Snapshot,
@@ -100,7 +100,7 @@ def _deleted_data_files(
if parent_snapshot is None:
return
- manifests, snapshot_ids = validation_history(
+ manifests, snapshot_ids = _validation_history(
table,
parent_snapshot,
starting_snapshot,
diff --git a/tests/table/test_validate.py b/tests/table/test_validate.py
index 74a0b59566..1da8718805 100644
--- a/tests/table/test_validate.py
+++ b/tests/table/test_validate.py
@@ -25,7 +25,7 @@
from pyiceberg.manifest import ManifestContent, ManifestEntry, ManifestEntryStatus, ManifestFile
from pyiceberg.table import Table
from pyiceberg.table.snapshots import Operation, Snapshot, Summary
-from pyiceberg.table.update.validate import _deleted_data_files, _validate_deleted_data_files, validation_history
+from pyiceberg.table.update.validate import _deleted_data_files, _validate_deleted_data_files, _validation_history
@pytest.fixture
@@ -69,7 +69,7 @@ def mock_read_manifest_side_effect(self: Snapshot, io: FileIO) -> list[ManifestF
return []
with patch("pyiceberg.table.snapshots.Snapshot.manifests", new=mock_read_manifest_side_effect):
- manifests, snapshots = validation_history(
+ manifests, snapshots = _validation_history(
table,
oldest_snapshot,
newest_snapshot,
@@ -99,7 +99,7 @@ def test_validation_history_fails_on_snapshot_with_no_summary(
)
with patch("pyiceberg.table.update.validate.ancestors_between", return_value=[snapshot_with_no_summary]):
with pytest.raises(ValidationException):
- validation_history(
+ _validation_history(
table,
oldest_snapshot,
newest_snapshot,
@@ -129,7 +129,7 @@ def mock_read_manifest_side_effect(self: Snapshot, io: FileIO) -> list[ManifestF
with patch("pyiceberg.table.snapshots.Snapshot.manifests", new=mock_read_manifest_side_effect):
with patch("pyiceberg.table.update.validate.ancestors_between", return_value=missing_oldest_snapshot):
with pytest.raises(ValidationException):
- validation_history(
+ _validation_history(
table,
oldest_snapshot,
newest_snapshot,
From 7df5b8105ccb02dd8b32084ddfd26f5e2f553fa6 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 3 Jun 2025 00:19:43 -0400
Subject: [PATCH 040/112] Build: Bump datafusion from 46.0.0 to 47.0.0 (#2063)
Bumps [datafusion](https://github.com/apache/datafusion-python) from
46.0.0 to 47.0.0.
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 16 ++++++++--------
pyproject.toml | 2 +-
2 files changed, 9 insertions(+), 9 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 8aa1c21011..4fc07869f0 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1122,18 +1122,18 @@ files = [
[[package]]
name = "datafusion"
-version = "46.0.0"
+version = "47.0.0"
description = "Build and run queries against data"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "datafusion-46.0.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:4517b6d2d35fb7d9044519a9ef3388ed504d82cbbaaba4bf3fef1da673c8357a"},
- {file = "datafusion-46.0.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:2770a1606a9d0f1f9d64648e3777413bbcff0bee7b9a1ccb067f989e3a1c7b96"},
- {file = "datafusion-46.0.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc115441c17036ddd730bb6281ecc88269973691ec51279be173bd3cc3e5d219"},
- {file = "datafusion-46.0.0-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:f40d9eeaa0ac4776bf0cea943228c945ec5bae8032263ad7871c8d82e8a2725a"},
- {file = "datafusion-46.0.0-cp39-abi3-win_amd64.whl", hash = "sha256:2b783b3fd8ee91bedfcdb353a3e700d65b61552a84e072940f78e0cde461bb98"},
- {file = "datafusion-46.0.0.tar.gz", hash = "sha256:e8adb6b987068585bc12f63ea51776919b09376833b2412da6cc8ff77d85c553"},
+ {file = "datafusion-47.0.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:ccd83a8e49fb39be06ddfa87023200a9ddc93d181247654ac951fa5720219d08"},
+ {file = "datafusion-47.0.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:43677e6284b165727031aec14d4beaa97296e991960293c61dcb66a3a9ce59b8"},
+ {file = "datafusion-47.0.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d244ed32a2fae7c4dd292a6bfe092cc94b3b86c600eddb7d633609043d406bae"},
+ {file = "datafusion-47.0.0-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:b3304ec63fb89f27e4280226807fd033ed7f0ea36d2d69fecf68f257d975c24d"},
+ {file = "datafusion-47.0.0-cp39-abi3-win_amd64.whl", hash = "sha256:73c5d056908185c77eedcaea43a5a8ab5e1c2e747a3e34d36d3625e09a3dc2af"},
+ {file = "datafusion-47.0.0.tar.gz", hash = "sha256:19a6976731aa96a6f6e264c390c64b9e32051e866366bd69450bc77e67bc91b1"},
]
[package.dependencies]
@@ -5977,4 +5977,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "9d4fe56235c1988928b6dc5eebc9374ed92e7d6cb52e5356b21c543196f31bda"
+content-hash = "fb4fa1f20093b96c06b07ab1704e10534e365a46c590d7038a971e1bfe7b0438"
diff --git a/pyproject.toml b/pyproject.toml
index 3296b83325..d4a32eff11 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -100,7 +100,7 @@ pytest-mock = "3.14.1"
pyspark = "3.5.5"
cython = "3.1.1"
deptry = ">=0.14,<0.24"
-datafusion = ">=44,<47"
+datafusion = ">=44,<48"
docutils = "!=0.21.post1" # https://github.com/python-poetry/poetry/issues/9248#issuecomment-2026240520
[tool.poetry.group.docs.dependencies]
From 71915e32ac0831434eaf243cadc19691c77030eb Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 3 Jun 2025 00:20:16 -0400
Subject: [PATCH 041/112] Build: Bump huggingface-hub from 0.32.2 to 0.32.3
(#2061)
Bumps [huggingface-hub](https://github.com/huggingface/huggingface_hub)
from 0.32.2 to 0.32.3.
Release notes
Sourced from huggingface-hub's
releases.
[v0.32.3]: Handle env variables in tiny-agents, better
CLI exit and handling of MCP tool calls arguments
Full Changelog: https://github.com/huggingface/huggingface_hub/compare/v0.32.2...v0.32.3
This release introduces some improvements and bug fixes to
tiny-agents:
- [tiny-agents] Handle env variables in tiny-agents (Python client) #3129
- [Fix]
tiny-agents cli exit issues #3125
- Improve Handling of MCP Tool Call Arguments #3127
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 4fc07869f0..294befde94 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1950,15 +1950,15 @@ tests = ["pytest"]
[[package]]
name = "huggingface-hub"
-version = "0.32.2"
+version = "0.32.3"
description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
optional = true
python-versions = ">=3.8.0"
groups = ["main"]
markers = "extra == \"hf\""
files = [
- {file = "huggingface_hub-0.32.2-py3-none-any.whl", hash = "sha256:f8fcf14603237eadf96dbe577d30b330f8c27b4a0a31e8f6c94fdc25e021fdb8"},
- {file = "huggingface_hub-0.32.2.tar.gz", hash = "sha256:64a288b1eadad6b60bbfd50f0e52fd6cfa2ef77ab13c3e8a834a038ae929de54"},
+ {file = "huggingface_hub-0.32.3-py3-none-any.whl", hash = "sha256:e46f7ea7fe2b5e5f67cc4e37eb201140091946a314d7c2b134a9673dadd80b6a"},
+ {file = "huggingface_hub-0.32.3.tar.gz", hash = "sha256:752c889ebf3a63cbd39803f6d87ccc135a463bbcb36abfa2faff0ccbf1cec087"},
]
[package.dependencies]
From ac2bb7e69735b320bb6b2caa9b52f7ee9adb909d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 3 Jun 2025 00:20:33 -0400
Subject: [PATCH 042/112] Build: Bump getdaft from 0.4.16 to 0.4.18 (#2060)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [getdaft](https://github.com/Eventual-Inc/Daft) from 0.4.16 to
0.4.18.
Release notes
Sourced from getdaft's
releases.
v0.4.18
What's Changed 🚀
🐛 Bug Fixes
🔧 Maintenance
- chore: Upgrade Ruff ruleset to 3.9 and add
from __future__
import annotations @srilman (#4393)
Full Changelog: https://github.com/Eventual-Inc/Daft/compare/v0.4.17...v0.4.18
v0.4.17
What's Changed 🚀
✨ Features
🐛 Bug Fixes
♻️ Refactor
📖 Documentation
... (truncated)
Commits
b7af7d6
fix: Implement dedicated map growable (#4435)
00a302b
fix: update broken-link-checker.yml (#4440)
7573cdb
fix: casting from list(list(T)) to list(tensor(T, shape)) (#4437)
5db2e18
chore: Upgrade Ruff ruleset to 3.9 and add `from future
import annotation...
1714b53
fix: Ensure additional columns are passed through granular projection
splitti...
88baf41
fix: Improve UDF errors (#4424)
52e74fd
ci: Add retries to requirements installation (#4430)
2e60aff
feat(catalogs): enable Rust usage of Python catalogs and tables (#4394)
75cfc69
ci: Up broken link checker retries (#4429)
0eb5896
fix(dashboard): broadcast url (#4404)
- Additional commits viewable in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 14 +++++++-------
1 file changed, 7 insertions(+), 7 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 294befde94..72ac6e8f88 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1613,19 +1613,19 @@ gcsfuse = ["fusepy"]
[[package]]
name = "getdaft"
-version = "0.4.16"
+version = "0.4.18"
description = "Distributed Dataframes for Multimodal Data"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"daft\""
files = [
- {file = "getdaft-0.4.16-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:2ae85aa3f7ff95afa366e30b2645db05dc77fb1cf1b31aca4617062ed7bfd808"},
- {file = "getdaft-0.4.16-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:410a3fa24404ce302fd7278184fcda901393095a03b3ff35e7fb2ac06207c383"},
- {file = "getdaft-0.4.16-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:45f816ce0dc897c4a69a41f5b8bc241192dadfe63abcc6dd0d6a61c85bf0375f"},
- {file = "getdaft-0.4.16-cp39-abi3-manylinux_2_24_x86_64.whl", hash = "sha256:8ab96334e5447c60d07efe33f59b11f12377aaf5d44d686b76b6733ede28421c"},
- {file = "getdaft-0.4.16-cp39-abi3-win_amd64.whl", hash = "sha256:9834eefabb5930729e8d0122e2c0af7c908d0f222e0ef2f5b091a40598da0e78"},
- {file = "getdaft-0.4.16.tar.gz", hash = "sha256:3f407b710000f640e9c790b886615fe677a32b9dcbde3085fe026152f2233785"},
+ {file = "getdaft-0.4.18-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:ef7fd708e230ef7a80afe0b91d2ecc03d5ff3ec78826c5f39ff5a8fb15517ed2"},
+ {file = "getdaft-0.4.18-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:d2ae3a7271eaa6f93e830bea80ea7ded0d2c3814b97b4b615af41e6aac91268b"},
+ {file = "getdaft-0.4.18-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:626f644af9f16ea77accf80177151fa6817313d596633e46d893ca2b232b1856"},
+ {file = "getdaft-0.4.18-cp39-abi3-manylinux_2_24_x86_64.whl", hash = "sha256:4e5ad35750731499a39c6056620e09ef7f310e1364a7c9d6310341e5f7478ba2"},
+ {file = "getdaft-0.4.18-cp39-abi3-win_amd64.whl", hash = "sha256:def486c781c414b241a7843eb8c54c1ba105795cb14283d5986e395bd0510abc"},
+ {file = "getdaft-0.4.18.tar.gz", hash = "sha256:ff10119147a28cfaad949f1599d9e8317069d07f83098c8d40744c4f10b51398"},
]
[package.dependencies]
From 1d7abed67cc30d486ba14433fd2ca02e75699d27 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 3 Jun 2025 00:34:46 -0400
Subject: [PATCH 043/112] Build: Bump pyspark from 3.5.5 to 3.5.6 (#2062)
Bumps [pyspark](https://github.com/apache/spark) from 3.5.5 to 3.5.6.
Commits
303c18c
Preparing Spark release v3.5.6-rc1
5e17ddf
Revert "Preparing Spark release v3.5.6-rc1"
66bd391
Revert "Preparing development version 3.5.7-SNAPSHOT"
e2b0b37
Revert "Revert "[SPARK-52276][BUILD] Make ANSWER not to be
overwritten in rel...
35134dd
Revert "Revert "[SPARK-52225][BUILD][FOLLOW-UP] Change -it to
-ti in Docker e...
7c8c5b7
Revert "[SPARK-52225][BUILD][FOLLOW-UP] Change -it to -ti in Docker
execution...
4c7472b
Revert "[SPARK-52276][BUILD] Make ANSWER not to be overwritten in
release scr...
5f3d97e
[SPARK-52276][BUILD] Make ANSWER not to be overwritten in release
scripts
ef44dbd
[SPARK-52225][BUILD][FOLLOW-UP] Change -it to -ti in Docker execution in
rele...
0148bb2
Preparing development version 3.5.7-SNAPSHOT
- Additional commits viewable in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 6 +++---
pyproject.toml | 2 +-
2 files changed, 4 insertions(+), 4 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 72ac6e8f88..09ea410477 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -4259,13 +4259,13 @@ files = [
[[package]]
name = "pyspark"
-version = "3.5.5"
+version = "3.5.6"
description = "Apache Spark Python API"
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "pyspark-3.5.5.tar.gz", hash = "sha256:6effc9ce98edf231f4d683fd14f7270629bf8458c628d6a2620ded4bb34f3cb9"},
+ {file = "pyspark-3.5.6.tar.gz", hash = "sha256:f8b1c4360e41ab398c64904fae08740503bcb6bd389457d659fa6d9f2952cc48"},
]
[package.dependencies]
@@ -5977,4 +5977,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "fb4fa1f20093b96c06b07ab1704e10534e365a46c590d7038a971e1bfe7b0438"
+content-hash = "fb2fa0b2764e8528206e2e401ad35a12b8edf31350d130796d5bb279d8785b58"
diff --git a/pyproject.toml b/pyproject.toml
index d4a32eff11..d906dff646 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -97,7 +97,7 @@ requests-mock = "1.12.1"
moto = { version = "^5.0.2", extras = ["server"] }
typing-extensions = "4.13.2"
pytest-mock = "3.14.1"
-pyspark = "3.5.5"
+pyspark = "3.5.6"
cython = "3.1.1"
deptry = ">=0.14,<0.24"
datafusion = ">=44,<48"
From dd0fbe0c5488b198adf80e598724aefb59c04748 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sat, 7 Jun 2025 20:40:26 +0200
Subject: [PATCH 044/112] Leverage Iceberg-Rust for all the transforms (#1833)
# Rationale for this change
Testing out to use Iceberg Rust for all of the transforms. I think we
have some rounding error in
https://github.com/apache/iceberg-rust/pull/1128/
Closes https://github.com/apache/iceberg-python/issues/1591
# Are these changes tested?
# Are there any user-facing changes?
---------
Co-authored-by: Kevin Liu
---
poetry.lock | 54 ++++++-------
pyiceberg/transforms.py | 128 ++++++++++---------------------
pyproject.toml | 4 +-
tests/table/test_partitioning.py | 12 +--
tests/test_transforms.py | 35 +--------
5 files changed, 73 insertions(+), 160 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 09ea410477..f41ef7d861 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -58,7 +58,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -71,7 +71,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e2bc827c01f75803de77b134afdbf74fa74b62970eafdf190f3244931d7a5c0d"},
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e365034c5cf6cf74f57420b57682ea79e19eb29033399dd3f40de4d0171998fa"},
@@ -196,7 +196,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -248,7 +248,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -265,7 +265,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -467,7 +467,7 @@ files = [
{file = "boto3-1.37.3-py3-none-any.whl", hash = "sha256:2063b40af99fd02f6228ff52397b552ff3353831edaf8d25cc04801827ab9794"},
{file = "boto3-1.37.3.tar.gz", hash = "sha256:21f3ce0ef111297e63a6eb998a25197b8c10982970c320d4c6e8db08be2157be"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.3,<1.38.0"
@@ -488,7 +488,7 @@ files = [
{file = "botocore-1.37.3-py3-none-any.whl", hash = "sha256:d01bd3bf4c80e61fa88d636ad9f5c9f60a551d71549b481386c6b4efe0bb2b2e"},
{file = "botocore-1.37.3.tar.gz", hash = "sha256:fe8403eb55a88faf9b0f9da6615e5bee7be056d75e17af66c3c8f0a3b0648da4"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -1449,7 +1449,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5b6a66c18b5b9dd261ca98dffcb826a525334b2f29e7caa54e182255c5f6a65a"},
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d1b3eb7b05ea246510b43a7e53ed1653e55c2121019a97e60cad7efb881a97bb"},
@@ -2162,7 +2162,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -2980,7 +2980,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:b9f6392d98c0bd70676ae41474e2eecf4c7150cb419237a41f8f96043fcb81d1"},
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3501621d5e86f1a88521ea65d5cad0a0834c77b26f193747615b7c911e5422d2"},
@@ -3574,7 +3574,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:f27785888d2fdd918bc36de8b8739f2d6c791399552333721b58193f68ea3e98"},
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4e89cde74154c7b5957f87a355bb9c8ec929c167b59c83d90654ea36aeb6180"},
@@ -3882,7 +3882,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
+markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -4080,19 +4080,19 @@ windows-terminal = ["colorama (>=0.4.6)"]
[[package]]
name = "pyiceberg-core"
-version = "0.4.0"
+version = "0.5.1"
description = ""
optional = true
-python-versions = "*"
+python-versions = "~=3.9"
groups = ["main"]
-markers = "extra == \"pyiceberg-core\""
+markers = "extra == \"pyarrow\" or extra == \"pyiceberg-core\""
files = [
- {file = "pyiceberg_core-0.4.0-cp39-abi3-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:5aec569271c96e18428d542f9b7007117a7232c06017f95cb239d42e952ad3b4"},
- {file = "pyiceberg_core-0.4.0-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5e74773e58efa4df83aba6f6265cdd41e446fa66fa4e343ca86395fed9f209ae"},
- {file = "pyiceberg_core-0.4.0-cp39-abi3-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7675d21a54bf3753c740d8df78ad7efe33f438096844e479d4f3493f84830925"},
- {file = "pyiceberg_core-0.4.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7058ad935a40b1838e4cdc5febd768878c1a51f83dca005d5a52a7fa280a2489"},
- {file = "pyiceberg_core-0.4.0-cp39-abi3-win_amd64.whl", hash = "sha256:a83eb4c2307ae3dd321a9360828fb043a4add2cc9797bef0bafa20894488fb07"},
- {file = "pyiceberg_core-0.4.0.tar.gz", hash = "sha256:d2e6138707868477b806ed354aee9c476e437913a331cb9ad9ad46b4054cd11f"},
+ {file = "pyiceberg_core-0.5.1-cp39-abi3-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:f5bffe70f036d4409ab5f8a2a52353635dfa9afa665e84eb5fb0415d12171aaa"},
+ {file = "pyiceberg_core-0.5.1-cp39-abi3-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:aa5d0d00efeb7c03ad48b17577452f42398f398b41034d2695a294c872365fcd"},
+ {file = "pyiceberg_core-0.5.1-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6ae88f65f68898a680dde4d83179903d14b9ef2ca8f7b8715fca5d91a2aa24cd"},
+ {file = "pyiceberg_core-0.5.1-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:9d02f82292d2d01cc6b43f79eadf1209e84457cf0a2b945a2ed834245d6a45f6"},
+ {file = "pyiceberg_core-0.5.1-cp39-abi3-win_amd64.whl", hash = "sha256:aad22d3ea743fcf941336542fcf565e16b28b9131087d70e1b03924005334af3"},
+ {file = "pyiceberg_core-0.5.1.tar.gz", hash = "sha256:5359750e690169e06e4c20b74854f0048abbb5fe74b75919247d87cf2af2fcf4"},
]
[[package]]
@@ -4392,7 +4392,7 @@ description = "World timezone definitions, modern and historical"
optional = true
python-versions = "*"
groups = ["main"]
-markers = "extra == \"gcsfs\" or extra == \"pandas\" or extra == \"ray\""
+markers = "extra == \"pandas\" or extra == \"ray\" or extra == \"gcsfs\""
files = [
{file = "pytz-2025.2-py2.py3-none-any.whl", hash = "sha256:5ddf76296dd8c44c26eb8f4b6f35488f3ccbf6fbbd7adee0b7262d43f0ec2f00"},
{file = "pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3"},
@@ -4992,7 +4992,7 @@ files = [
{file = "s3transfer-0.11.3-py3-none-any.whl", hash = "sha256:ca855bdeb885174b5ffa95b9913622459d4ad8e331fc98eb01e6d5eb6a30655d"},
{file = "s3transfer-0.11.3.tar.gz", hash = "sha256:edae4977e3a122445660c7c114bba949f9d191bae3b34a096f18a1c8c354527a"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.36.0,<2.0a.0"
@@ -5426,7 +5426,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"daft\" or extra == \"hf\""
+markers = "extra == \"hf\" or extra == \"daft\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -5726,7 +5726,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7df647e8edd71f000a5208fe6ff8c382a1de8edfbccdbbfe649d263de07d8c34"},
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c69697d3adff5aa4f874b19c0e4ed65180ceed6318ec856ebc423aa5850d84f7"},
@@ -5964,7 +5964,7 @@ hive = ["thrift"]
hive-kerberos = ["kerberos", "thrift", "thrift-sasl"]
pandas = ["pandas", "pyarrow"]
polars = ["polars"]
-pyarrow = ["pyarrow"]
+pyarrow = ["pyarrow", "pyiceberg-core"]
pyiceberg-core = ["pyiceberg-core"]
ray = ["pandas", "pyarrow", "ray", "ray"]
rest-sigv4 = ["boto3"]
@@ -5977,4 +5977,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "fb2fa0b2764e8528206e2e401ad35a12b8edf31350d130796d5bb279d8785b58"
+content-hash = "93eda79de3991bea0b9520f69372d146f7fd3d128cd982ea5afb78e8ec5d94f2"
diff --git a/pyiceberg/transforms.py b/pyiceberg/transforms.py
index 19889a98e8..8dda85a48b 100644
--- a/pyiceberg/transforms.py
+++ b/pyiceberg/transforms.py
@@ -111,6 +111,36 @@ def _transform_literal(func: Callable[[L], L], lit: Literal[L]) -> Literal[L]:
return literal(func(lit.value))
+def _pyiceberg_transform_wrapper(
+ transform_func: Callable[["ArrayLike", Any], "ArrayLike"],
+ *args: Any,
+ expected_type: Optional["pa.DataType"] = None,
+) -> Callable[["ArrayLike"], "ArrayLike"]:
+ try:
+ import pyarrow as pa
+ except ModuleNotFoundError as e:
+ raise ModuleNotFoundError("For partition transforms, PyArrow needs to be installed") from e
+
+ def _transform(array: "ArrayLike") -> "ArrayLike":
+ def _cast_if_needed(arr: "ArrayLike") -> "ArrayLike":
+ if expected_type is not None:
+ return arr.cast(expected_type)
+ else:
+ return arr
+
+ if isinstance(array, pa.Array):
+ return _cast_if_needed(transform_func(array, *args))
+ elif isinstance(array, pa.ChunkedArray):
+ result_chunks = []
+ for arr in array.iterchunks():
+ result_chunks.append(_cast_if_needed(transform_func(arr, *args)))
+ return pa.chunked_array(result_chunks)
+ else:
+ raise ValueError(f"PyArrow array can only be of type pa.Array or pa.ChunkedArray, but found {type(array)}")
+
+ return _transform
+
+
class Transform(IcebergRootModel[str], ABC, Generic[S, T]):
"""Transform base class for concrete transforms.
@@ -175,27 +205,6 @@ def supports_pyarrow_transform(self) -> bool:
@abstractmethod
def pyarrow_transform(self, source: IcebergType) -> "Callable[[pa.Array], pa.Array]": ...
- def _pyiceberg_transform_wrapper(
- self, transform_func: Callable[["ArrayLike", Any], "ArrayLike"], *args: Any
- ) -> Callable[["ArrayLike"], "ArrayLike"]:
- try:
- import pyarrow as pa
- except ModuleNotFoundError as e:
- raise ModuleNotFoundError("For bucket/truncate transforms, PyArrow needs to be installed") from e
-
- def _transform(array: "ArrayLike") -> "ArrayLike":
- if isinstance(array, pa.Array):
- return transform_func(array, *args)
- elif isinstance(array, pa.ChunkedArray):
- result_chunks = []
- for arr in array.iterchunks():
- result_chunks.append(transform_func(arr, *args))
- return pa.chunked_array(result_chunks)
- else:
- raise ValueError(f"PyArrow array can only be of type pa.Array or pa.ChunkedArray, but found {type(array)}")
-
- return _transform
-
def parse_transform(v: Any) -> Transform[Any, Any]:
if isinstance(v, str):
@@ -375,7 +384,7 @@ def __repr__(self) -> str:
def pyarrow_transform(self, source: IcebergType) -> "Callable[[pa.Array], pa.Array]":
from pyiceberg_core import transform as pyiceberg_core_transform
- return self._pyiceberg_transform_wrapper(pyiceberg_core_transform.bucket, self._num_buckets)
+ return _pyiceberg_transform_wrapper(pyiceberg_core_transform.bucket, self._num_buckets)
@property
def supports_pyarrow_transform(self) -> bool:
@@ -501,22 +510,9 @@ def __repr__(self) -> str:
def pyarrow_transform(self, source: IcebergType) -> "Callable[[pa.Array], pa.Array]":
import pyarrow as pa
- import pyarrow.compute as pc
-
- if isinstance(source, DateType):
- epoch = pa.scalar(datetime.EPOCH_DATE)
- elif isinstance(source, TimestampType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP)
- elif isinstance(source, TimestamptzType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ)
- elif isinstance(source, TimestampNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP).cast(pa.timestamp("ns"))
- elif isinstance(source, TimestamptzNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ).cast(pa.timestamp("ns"))
- else:
- raise ValueError(f"Cannot apply year transform for type: {source}")
+ from pyiceberg_core import transform as pyiceberg_core_transform
- return lambda v: pc.years_between(epoch, v) if v is not None else None
+ return _pyiceberg_transform_wrapper(pyiceberg_core_transform.year, expected_type=pa.int32())
class MonthTransform(TimeTransform[S]):
@@ -575,28 +571,9 @@ def __repr__(self) -> str:
def pyarrow_transform(self, source: IcebergType) -> "Callable[[pa.Array], pa.Array]":
import pyarrow as pa
- import pyarrow.compute as pc
-
- if isinstance(source, DateType):
- epoch = pa.scalar(datetime.EPOCH_DATE)
- elif isinstance(source, TimestampType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP)
- elif isinstance(source, TimestamptzType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ)
- elif isinstance(source, TimestampNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP).cast(pa.timestamp("ns"))
- elif isinstance(source, TimestamptzNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ).cast(pa.timestamp("ns"))
- else:
- raise ValueError(f"Cannot apply month transform for type: {source}")
-
- def month_func(v: pa.Array) -> pa.Array:
- return pc.add(
- pc.multiply(pc.years_between(epoch, v), pa.scalar(12)),
- pc.add(pc.month(v), pa.scalar(-1)),
- )
+ from pyiceberg_core import transform as pyiceberg_core_transform
- return lambda v: month_func(v) if v is not None else None
+ return _pyiceberg_transform_wrapper(pyiceberg_core_transform.month, expected_type=pa.int32())
class DayTransform(TimeTransform[S]):
@@ -663,22 +640,9 @@ def __repr__(self) -> str:
def pyarrow_transform(self, source: IcebergType) -> "Callable[[pa.Array], pa.Array]":
import pyarrow as pa
- import pyarrow.compute as pc
-
- if isinstance(source, DateType):
- epoch = pa.scalar(datetime.EPOCH_DATE)
- elif isinstance(source, TimestampType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP)
- elif isinstance(source, TimestamptzType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ)
- elif isinstance(source, TimestampNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP).cast(pa.timestamp("ns"))
- elif isinstance(source, TimestamptzNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ).cast(pa.timestamp("ns"))
- else:
- raise ValueError(f"Cannot apply day transform for type: {source}")
+ from pyiceberg_core import transform as pyiceberg_core_transform
- return lambda v: pc.days_between(epoch, v) if v is not None else None
+ return _pyiceberg_transform_wrapper(pyiceberg_core_transform.day, expected_type=pa.int32())
class HourTransform(TimeTransform[S]):
@@ -728,21 +692,9 @@ def __repr__(self) -> str:
return "HourTransform()"
def pyarrow_transform(self, source: IcebergType) -> "Callable[[pa.Array], pa.Array]":
- import pyarrow as pa
- import pyarrow.compute as pc
-
- if isinstance(source, TimestampType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP)
- elif isinstance(source, TimestamptzType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ)
- elif isinstance(source, TimestampNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMP).cast(pa.timestamp("ns"))
- elif isinstance(source, TimestamptzNanoType):
- epoch = pa.scalar(datetime.EPOCH_TIMESTAMPTZ).cast(pa.timestamp("ns"))
- else:
- raise ValueError(f"Cannot apply hour transform for type: {source}")
+ from pyiceberg_core import transform as pyiceberg_core_transform
- return lambda v: pc.hours_between(epoch, v) if v is not None else None
+ return _pyiceberg_transform_wrapper(pyiceberg_core_transform.hour)
def _base64encode(buffer: bytes) -> str:
@@ -965,7 +917,7 @@ def __repr__(self) -> str:
def pyarrow_transform(self, source: IcebergType) -> "Callable[[pa.Array], pa.Array]":
from pyiceberg_core import transform as pyiceberg_core_transform
- return self._pyiceberg_transform_wrapper(pyiceberg_core_transform.truncate, self._width)
+ return _pyiceberg_transform_wrapper(pyiceberg_core_transform.truncate, self._width)
@property
def supports_pyarrow_transform(self) -> bool:
diff --git a/pyproject.toml b/pyproject.toml
index d906dff646..9699f28567 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -81,7 +81,7 @@ psycopg2-binary = { version = ">=2.9.6", optional = true }
sqlalchemy = { version = "^2.0.18", optional = true }
getdaft = { version = ">=0.2.12", optional = true }
cachetools = ">=5.5,<7.0"
-pyiceberg-core = { version = "^0.4.0", optional = true }
+pyiceberg-core = { version = "^0.5.1", optional = true }
polars = { version = "^1.21.0", optional = true }
thrift-sasl = { version = ">=0.4.3", optional = true }
kerberos = {version = "^1.3.1", optional = true}
@@ -289,7 +289,7 @@ generate-setup-file = false
script = "build-module.py"
[tool.poetry.extras]
-pyarrow = ["pyarrow"]
+pyarrow = ["pyarrow", "pyiceberg-core"]
pandas = ["pandas", "pyarrow"]
duckdb = ["duckdb", "pyarrow"]
ray = ["ray", "pyarrow", "pandas"]
diff --git a/tests/table/test_partitioning.py b/tests/table/test_partitioning.py
index 57ab3e328a..0fe22391c0 100644
--- a/tests/table/test_partitioning.py
+++ b/tests/table/test_partitioning.py
@@ -186,8 +186,8 @@ def test_partition_type(table_schema_simple: Schema) -> None:
(DecimalType(5, 9), Decimal(19.25)),
(DateType(), datetime.date(1925, 5, 22)),
(TimeType(), datetime.time(19, 25, 00)),
- (TimestampType(), datetime.datetime(19, 5, 1, 22, 1, 1)),
- (TimestamptzType(), datetime.datetime(19, 5, 1, 22, 1, 1, tzinfo=datetime.timezone.utc)),
+ (TimestampType(), datetime.datetime(2022, 5, 1, 22, 1, 1)),
+ (TimestamptzType(), datetime.datetime(2022, 5, 1, 22, 1, 1, tzinfo=datetime.timezone.utc)),
(StringType(), "abc"),
(UUIDType(), UUID("12345678-1234-5678-1234-567812345678").bytes),
(FixedType(5), 'b"\x8e\xd1\x87\x01"'),
@@ -208,13 +208,7 @@ def test_transform_consistency_with_pyarrow_transform(source_type: PrimitiveType
]
for t in all_transforms:
if t.can_transform(source_type):
- try:
- assert t.transform(source_type)(value) == t.pyarrow_transform(source_type)(pa.array([value])).to_pylist()[0]
- except ValueError as e:
- # Skipping unsupported feature
- if "FeatureUnsupported => Unsupported data type for truncate transform" in str(e):
- continue
- raise
+ assert t.transform(source_type)(value) == t.pyarrow_transform(source_type)(pa.array([value])).to_pylist()[0]
def test_deserialize_partition_field_v2() -> None:
diff --git a/tests/test_transforms.py b/tests/test_transforms.py
index d22c94cc93..f8d3ac9d10 100644
--- a/tests/test_transforms.py
+++ b/tests/test_transforms.py
@@ -16,7 +16,7 @@
# specific language governing permissions and limitations
# under the License.
# pylint: disable=eval-used,protected-access,redefined-outer-name
-from datetime import date, datetime
+from datetime import date
from decimal import Decimal
from typing import Annotated, Any, Callable, Optional, Union
from uuid import UUID
@@ -24,7 +24,6 @@
import mmh3 as mmh3
import pyarrow as pa
import pytest
-import pytz
from pydantic import (
BeforeValidator,
PlainSerializer,
@@ -1654,38 +1653,6 @@ def test_bucket_pyarrow_transforms(
assert expected == transform.pyarrow_transform(source_type)(input_arr)
-# pyiceberg_core currently does not support bucket transform on timestamp_ns and timestamptz_ns
-# https://github.com/apache/iceberg-rust/issues/1110
-@pytest.mark.parametrize(
- "source_type, input_arr, num_buckets",
- [
- (
- TimestampNanoType(),
- pa.array([datetime(1970, 1, 1, 0, 0, 0), datetime(2025, 2, 26, 1, 2, 3)], type=pa.timestamp(unit="ns")),
- 10,
- ),
- (
- TimestamptzNanoType(),
- pa.array(
- [datetime(1970, 1, 1, 0, 0, 0), datetime(2025, 2, 26, 1, 2, 3)],
- type=pa.timestamp(unit="ns", tz=pytz.timezone("Etc/GMT+10")),
- ),
- 10,
- ),
- ],
-)
-def test_unsupported_bucket_pyarrow_transform(
- source_type: PrimitiveType,
- input_arr: Union[pa.Array, pa.ChunkedArray],
- num_buckets: int,
-) -> None:
- transform: Transform[Any, Any] = BucketTransform(num_buckets=num_buckets)
- with pytest.raises(ValueError) as exc_info:
- transform.pyarrow_transform(source_type)(input_arr)
-
- assert "FeatureUnsupported => Unsupported data type for bucket transform" in str(exc_info.value)
-
-
@pytest.mark.parametrize(
"source_type, input_arr, expected, width",
[
From 401cfc382e452263e513966b0804227f75fa6f08 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 10 Jun 2025 06:34:15 +0200
Subject: [PATCH 045/112] Build: Bump cython from 3.1.1 to 3.1.2 (#2081)
---
poetry.lock | 160 ++++++++++++++++++++++++-------------------------
pyproject.toml | 2 +-
2 files changed, 81 insertions(+), 81 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index f41ef7d861..629211325f 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -58,7 +58,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -71,7 +71,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e2bc827c01f75803de77b134afdbf74fa74b62970eafdf190f3244931d7a5c0d"},
{file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e365034c5cf6cf74f57420b57682ea79e19eb29033399dd3f40de4d0171998fa"},
@@ -196,7 +196,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -248,7 +248,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -265,7 +265,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -467,7 +467,7 @@ files = [
{file = "boto3-1.37.3-py3-none-any.whl", hash = "sha256:2063b40af99fd02f6228ff52397b552ff3353831edaf8d25cc04801827ab9794"},
{file = "boto3-1.37.3.tar.gz", hash = "sha256:21f3ce0ef111297e63a6eb998a25197b8c10982970c320d4c6e8db08be2157be"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.3,<1.38.0"
@@ -488,7 +488,7 @@ files = [
{file = "botocore-1.37.3-py3-none-any.whl", hash = "sha256:d01bd3bf4c80e61fa88d636ad9f5c9f60a551d71549b481386c6b4efe0bb2b2e"},
{file = "botocore-1.37.3.tar.gz", hash = "sha256:fe8403eb55a88faf9b0f9da6615e5bee7be056d75e17af66c3c8f0a3b0648da4"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -1050,74 +1050,74 @@ test-randomorder = ["pytest-randomly"]
[[package]]
name = "cython"
-version = "3.1.1"
+version = "3.1.2"
description = "The Cython compiler for writing C extensions in the Python language."
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "cython-3.1.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0de7adff5b42d2556d073e9f321c2faa639a17fb195ec1de130327f60ec209d8"},
- {file = "cython-3.1.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9b61b99205308c96b1162de59bd67ecadcad3d166a4a1f03a3d9e826c39cd375"},
- {file = "cython-3.1.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d14186bd96783d13b8fd0e5b289f2e137a8a25479638b73a1c7e4a99a8d70753"},
- {file = "cython-3.1.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e3ccec55e2a534a712db14c6617b66f65ad149c014fad518fc3920f6edde770"},
- {file = "cython-3.1.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a585796939b09b3205b1980e4a55e745c0251e45a5c637afbcac3c6cc9ad6f90"},
- {file = "cython-3.1.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3fa4bd840de63509c74867b4b092541720a01db1e07351206011c34e0777dc96"},
- {file = "cython-3.1.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b68f1bc80387554eb43f2b62795c173bed9e37201f39dc5084ac437c90a79c9f"},
- {file = "cython-3.1.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:e851ab66a31794e40df1bc6f649cdc56c998c637f5a1b9410c97a90f6b6cb855"},
- {file = "cython-3.1.1-cp310-cp310-win32.whl", hash = "sha256:64915259276482fa23417b284d1fdc7e3a618ee2f819bb6ea7f974c075633df6"},
- {file = "cython-3.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:dee554f0a589377bdaea0eb70e212bf3f35dc6a51a2aa86c9351345e21fd2f07"},
- {file = "cython-3.1.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c360823e1063784efc2335617e0f28573d7a594c5a8a05d85e850a9621cccb1f"},
- {file = "cython-3.1.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:12e00b88147b03c148a95365f89dc1c45a0fc52f9c35aa75ff770ef65b615839"},
- {file = "cython-3.1.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ab644415458d782c16ba7252de9cec1e3125371641cafea2e53a8c1cf85dd58d"},
- {file = "cython-3.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c5cb6c054daadaf01a88c8f49f3edd9e829c9b76a82cbb4269e3f9878254540b"},
- {file = "cython-3.1.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:af8f62cc9339b75fe8434325083e6a7cae88c9c21efd74bbb6ba4e3623219469"},
- {file = "cython-3.1.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:689c1aad373556bd2ab1aa1c2dad8939a2891465a1fbd2cbbdd42b488fb40ec8"},
- {file = "cython-3.1.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:953046c190fa9ab9a09a546a909b847cdbb4c1fe34e9bfa4a15b6ee1585a86aa"},
- {file = "cython-3.1.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:755a991601b27dd3555310d0f95b19a05e622a80d7b4e7a91fa6f5f3ef3f3b80"},
- {file = "cython-3.1.1-cp311-cp311-win32.whl", hash = "sha256:83b2af5c327f7da4f08afc34fddfaf6d24fa0c000b6b70a527c8125e493b6080"},
- {file = "cython-3.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:141ffd6279411c562f6b707adc56b63e965a4fd7f21db83f5d4fcbd8c50ac546"},
- {file = "cython-3.1.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:9d7dc0e4d0cd491fac679a61e9ede348c64ca449f99a284f9a01851aa1dbc7f6"},
- {file = "cython-3.1.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:fd689910002adfac8734f237cdea1573e38345f27ed7fd445482813b65a29457"},
- {file = "cython-3.1.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:10f0434916994fe213ea7749268b88d77e3ebcbd1b99542cf64bb7d180f45470"},
- {file = "cython-3.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:873aac4ac0b0fb197557c0ac15458b780b9221daa4a716881cbd1a9016c8459f"},
- {file = "cython-3.1.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:23b886a6c8a50b1101ccef2f2f3dc9c699b77633ef5bb5007090226c2ad3f9c2"},
- {file = "cython-3.1.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:dff0e7dd53a0ca35b64cda843253d5cac944db26663dc097b3a1adf2c49514ad"},
- {file = "cython-3.1.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:0f7954b0b4b3302655d3caa6924261de5907a4e129bc22ace52fe9ae0cd5a758"},
- {file = "cython-3.1.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:dfa500fd7ae95ca152a5f8062b870532fa3e27efcef6d00612e1f28b9f72615f"},
- {file = "cython-3.1.1-cp312-cp312-win32.whl", hash = "sha256:cd748fab8e4426dbcb2e0fa2979558333934d24365e0de5672fbabfe337d880c"},
- {file = "cython-3.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:307f216ed319ea07644f2ef9974406c830f01bc8e677e2147e9bfcdf9e3ca8ad"},
- {file = "cython-3.1.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:cb5661941707bd41ec7a9c273d698113ac50392444f785088e9d9706c6a5937b"},
- {file = "cython-3.1.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:28b174f41718a7041cfbe0f48913020875ff1aaa4793942b2451ac6d2baf3f07"},
- {file = "cython-3.1.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c740a10cd0f50321d048c8ca318eefb4c42b8bffef982dcd89c946d374192702"},
- {file = "cython-3.1.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7da069ca769903c5dee56c5f7ab47b2b7b91030eee48912630db5f4f3ec5954a"},
- {file = "cython-3.1.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:24c640c0746d984789fe2787a098f06cda456ef2dd78b90164d17884b350839a"},
- {file = "cython-3.1.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:426d78565eb91d3366569b20e92b8f14bffef5f57b2acd05b60bbb9ce5c056a1"},
- {file = "cython-3.1.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:b181158b5761bdaf40f6854f016ab7ddff64d3db4fca55cb3ca0f73813dd76d6"},
- {file = "cython-3.1.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:7489559e6c5ecbba49d535c2e03cf77c2594a3190b6aca7da5b508ba1664a89a"},
- {file = "cython-3.1.1-cp313-cp313-win32.whl", hash = "sha256:263cb0e497910fb5e0a361ad1393b6d728b092178afecc56e8a786f3739960c3"},
- {file = "cython-3.1.1-cp313-cp313-win_amd64.whl", hash = "sha256:e000f0533eedf3d6dfbe30bb3c58a054c58f0a7778390342fa577a0dc47adab3"},
- {file = "cython-3.1.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:cdf53dc4b2a13bd072d6c2c18ac073dbf0f798555bc27ba4f7546a275eb16a0f"},
- {file = "cython-3.1.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:ce82070ccf92c3599d331b9eaaefd9d4562976fb86a8d6bccf05c4a0b8389f2a"},
- {file = "cython-3.1.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:020089f9c9f10269181f17660a2cada7d4577bd8eea24b7d2b14e6b64b6996be"},
- {file = "cython-3.1.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:402f86c00b08f875cd0990f0c4dc52eb3e0bc5d630066cdf3c798631976f1937"},
- {file = "cython-3.1.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:54a8934cb3bf13b1f8f6cbdae8e382e25a26e67de08ea6ebfd0a467131b67227"},
- {file = "cython-3.1.1-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:6ea77ad1e649cec38f8622ba28dcdfbe7bf519bc132abbcf5df759b3975b5a73"},
- {file = "cython-3.1.1-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:7e5cad896af896482240979b996bf4136b0d18dc40c56c72c5641bf0ea085dfb"},
- {file = "cython-3.1.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:16d9870654946375b28280371d370d541641d1071da123d0d64d2c7ebba0cc56"},
- {file = "cython-3.1.1-cp38-cp38-win32.whl", hash = "sha256:8aaa29e763adf3496ab9d371e3caed8da5d3ce5ff8fb57433e2a2f2b5036e5c8"},
- {file = "cython-3.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:011cdcbf7725f0cfc1abc55ec83d326e788050711272131daf3cc24a19c34bb2"},
- {file = "cython-3.1.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:40f50b07c479eaf33981d81cad274c68cf9fb81dbe79cbf991f59491c88a4705"},
- {file = "cython-3.1.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:a92f6bd395eadea6eed722a8188d3bdd49db1c9fa3c38710456d6148ab71bad7"},
- {file = "cython-3.1.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:268420b92307ae6c5a16e3cf0e2ba1ae3c861650e992893922a0ce08db07cfdb"},
- {file = "cython-3.1.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a19188ecd385cdc649e3fec370f38d5fd7f1651aeed0b3fb403180f38fc88e8a"},
- {file = "cython-3.1.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7fff6526bb6f4eea615663117b86de6ede0d17c477b600d3d8302be3502bd3c3"},
- {file = "cython-3.1.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:3192a61c2a532d3faccdff508bc8427de9530b587888218bfc0226eb33a84e11"},
- {file = "cython-3.1.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:56c6768a6f601f93daab7c2487f9f110548a896a91e00a6e119445ada2575323"},
- {file = "cython-3.1.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:50ad80e2f438e9127a87c10927e6ac16a987df39c248b19ab2cd31330129be3c"},
- {file = "cython-3.1.1-cp39-cp39-win32.whl", hash = "sha256:b194a65a0fd91f305d2d1e7010f44111774a28533e1e44dd2a76e7de81a219b9"},
- {file = "cython-3.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:c8b8be01fd40b3e38a76c60a524f956548a3a7566e5530a833a48a695f3d6c12"},
- {file = "cython-3.1.1-py3-none-any.whl", hash = "sha256:07621e044f332d18139df2ccfcc930151fd323c2f61a58c82f304cffc9eb5280"},
- {file = "cython-3.1.1.tar.gz", hash = "sha256:505ccd413669d5132a53834d792c707974248088c4f60c497deb1b416e366397"},
+ {file = "cython-3.1.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0f2add8b23cb19da3f546a688cd8f9e0bfc2776715ebf5e283bc3113b03ff008"},
+ {file = "cython-3.1.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0d6248a2ae155ca4c42d7fa6a9a05154d62e695d7736bc17e1b85da6dcc361df"},
+ {file = "cython-3.1.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:262bf49d9da64e2a34c86cbf8de4aa37daffb0f602396f116cca1ed47dc4b9f2"},
+ {file = "cython-3.1.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ae53ae93c699d5f113953a9869df2fc269d8e173f9aa0616c6d8d6e12b4e9827"},
+ {file = "cython-3.1.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b417c5d046ce676ee595ec7955ed47a68ad6f419cbf8c2a8708e55a3b38dfa35"},
+ {file = "cython-3.1.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:af127da4b956e0e906e552fad838dc3fb6b6384164070ceebb0d90982a8ae25a"},
+ {file = "cython-3.1.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:9be3d4954b46fd0f2dceac011d470f658eaf819132db52fbd1cf226ee60348db"},
+ {file = "cython-3.1.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:63da49672c4bb022b4de9d37bab6c29953dbf5a31a2f40dffd0cf0915dcd7a17"},
+ {file = "cython-3.1.2-cp310-cp310-win32.whl", hash = "sha256:2d8291dbbc1cb86b8d60c86fe9cbf99ec72de28cb157cbe869c95df4d32efa96"},
+ {file = "cython-3.1.2-cp310-cp310-win_amd64.whl", hash = "sha256:e1f30a1339e03c80968a371ef76bf27a6648c5646cccd14a97e731b6957db97a"},
+ {file = "cython-3.1.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:5548573e0912d7dc80579827493315384c462e2f15797b91a8ed177686d31eb9"},
+ {file = "cython-3.1.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:4bf3ea5bc50d80762c490f42846820a868a6406fdb5878ae9e4cc2f11b50228a"},
+ {file = "cython-3.1.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:20ce53951d06ab2bca39f153d9c5add1d631c2a44d58bf67288c9d631be9724e"},
+ {file = "cython-3.1.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e05a36224e3002d48c7c1c695b3771343bd16bc57eab60d6c5d5e08f3cbbafd8"},
+ {file = "cython-3.1.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:dbc0fc0777c7ab82297c01c61a1161093a22a41714f62e8c35188a309bd5db8e"},
+ {file = "cython-3.1.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:18161ef3dd0e90a944daa2be468dd27696712a5f792d6289e97d2a31298ad688"},
+ {file = "cython-3.1.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:ca45020950cd52d82189d6dfb6225737586be6fe7b0b9d3fadd7daca62eff531"},
+ {file = "cython-3.1.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:aaae97d6d07610224be2b73a93e9e3dd85c09aedfd8e47054e3ef5a863387dae"},
+ {file = "cython-3.1.2-cp311-cp311-win32.whl", hash = "sha256:3d439d9b19e7e70f6ff745602906d282a853dd5219d8e7abbf355de680c9d120"},
+ {file = "cython-3.1.2-cp311-cp311-win_amd64.whl", hash = "sha256:8efa44ee2f1876e40eb5e45f6513a19758077c56bf140623ccab43d31f873b61"},
+ {file = "cython-3.1.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:9c2c4b6f9a941c857b40168b3f3c81d514e509d985c2dcd12e1a4fea9734192e"},
+ {file = "cython-3.1.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:bdbc115bbe1b8c1dcbcd1b03748ea87fa967eb8dfc3a1a9bb243d4a382efcff4"},
+ {file = "cython-3.1.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c05111f89db1ca98edc0675cfaa62be47b3ff519a29876eb095532a9f9e052b8"},
+ {file = "cython-3.1.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f6e7188df8709be32cfdfadc7c3782e361c929df9132f95e1bbc90a340dca3c7"},
+ {file = "cython-3.1.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1c0ecc71e60a051732c2607b8eb8f2a03a5dac09b28e52b8af323c329db9987b"},
+ {file = "cython-3.1.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:f27143cf88835c8bcc9bf3304953f23f377d1d991e8942982fe7be344c7cfce3"},
+ {file = "cython-3.1.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:d8c43566701133f53bf13485839d8f3f309095fe0d3b9d0cd5873073394d2edc"},
+ {file = "cython-3.1.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:a3bb893e85f027a929c1764bb14db4c31cbdf8a96f59a78f608f2ba7cfbbce95"},
+ {file = "cython-3.1.2-cp312-cp312-win32.whl", hash = "sha256:12c5902f105e43ca9af7874cdf87a23627f98c15d5a4f6d38bc9d334845145c0"},
+ {file = "cython-3.1.2-cp312-cp312-win_amd64.whl", hash = "sha256:06789eb7bd2e55b38b9dd349e9309f794aee0fed99c26ea5c9562d463877763f"},
+ {file = "cython-3.1.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:cc22e5f18af436c894b90c257130346930fdc860d7f42b924548c591672beeef"},
+ {file = "cython-3.1.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:42c7bffb0fe9898996c7eef9eb74ce3654553c7a3a3f3da66e5a49f801904ce0"},
+ {file = "cython-3.1.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:88dc7fd54bfae78c366c6106a759f389000ea4dfe8ed9568af9d2f612825a164"},
+ {file = "cython-3.1.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:80d0ce057672ca50728153757d022842d5dcec536b50c79615a22dda2a874ea0"},
+ {file = "cython-3.1.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:eda6a43f1b78eae0d841698916eef661d15f8bc8439c266a964ea4c504f05612"},
+ {file = "cython-3.1.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:b4c516d103e87c2e9c1ab85227e4d91c7484c1ba29e25f8afbf67bae93fee164"},
+ {file = "cython-3.1.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:7542f1d18ab2cd22debc72974ec9e53437a20623d47d6001466e430538d7df54"},
+ {file = "cython-3.1.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:63335513c06dcec4ecdaa8598f36c969032149ffd92a461f641ee363dc83c7ad"},
+ {file = "cython-3.1.2-cp313-cp313-win32.whl", hash = "sha256:b377d542299332bfeb61ec09c57821b10f1597304394ba76544f4d07780a16df"},
+ {file = "cython-3.1.2-cp313-cp313-win_amd64.whl", hash = "sha256:8ab1319c77f15b0ae04b3fb03588df3afdec4cf79e90eeea5c961e0ebd8fdf72"},
+ {file = "cython-3.1.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:dbc1f225cb9f9be7a025589463507e10bb2d76a3258f8d308e0e2d0b966c556e"},
+ {file = "cython-3.1.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:c1661c1701c96e1866f839e238570c96a97535a81da76a26f45f99ede18b3897"},
+ {file = "cython-3.1.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:955bc6032d89ce380458266e65dcf5ae0ed1e7c03a7a4457e3e4773e90ba7373"},
+ {file = "cython-3.1.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0b58e859889dd0fc6c3a990445b930f692948b28328bb4f3ed84b51028b7e183"},
+ {file = "cython-3.1.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:992a6504aa3eed50dd1fc3d1fa998928b08c1188130bd526e177b6d7f3383ec4"},
+ {file = "cython-3.1.2-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:f3d03077938b02ec47a56aa156da7bfc2379193738397d4e88086db5b0a374e0"},
+ {file = "cython-3.1.2-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:b7e1d3c383a5f4ca5319248b9cb1b16a04fb36e153d651e558897171b7dbabb9"},
+ {file = "cython-3.1.2-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:58d4d45e40cadf4f602d96b7016cf24ccfe4d954c61fa30b79813db8ccb7818f"},
+ {file = "cython-3.1.2-cp38-cp38-win32.whl", hash = "sha256:919ff38a93f7c21829a519693b336979feb41a0f7ca35969402d7e211706100e"},
+ {file = "cython-3.1.2-cp38-cp38-win_amd64.whl", hash = "sha256:aca994519645ba8fb5e99c0f9d4be28d61435775552aaf893a158c583cd218a5"},
+ {file = "cython-3.1.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:fe7f1ee4c13f8a773bd6c66b3d25879f40596faeab49f97d28c39b16ace5fff9"},
+ {file = "cython-3.1.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:c9ec7d2baea122d94790624f743ff5b78f4e777bf969384be65b69d92fa4bc3f"},
+ {file = "cython-3.1.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:df57827185874f29240b02402e615547ab995d90182a852c6ec4f91bbae355a4"},
+ {file = "cython-3.1.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b1a69b9b4fe0a48a8271027c0703c71ab1993c4caca01791c0fd2e2bd9031aa"},
+ {file = "cython-3.1.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:970cc1558519f0f108c3e2f4b3480de4945228d9292612d5b2bb687e36c646b8"},
+ {file = "cython-3.1.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:604c39cd6d152498a940aeae28b6fd44481a255a3fdf1b0051c30f3873c88b7f"},
+ {file = "cython-3.1.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:855f2ae06438c7405997cf0df42d5b508ec3248272bb39df4a7a4a82a5f7c8cb"},
+ {file = "cython-3.1.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:9e3016ca7a86728bfcbdd52449521e859a977451f296a7ae4967cefa2ec498f7"},
+ {file = "cython-3.1.2-cp39-cp39-win32.whl", hash = "sha256:4896fc2b0f90820ea6fcf79a07e30822f84630a404d4e075784124262f6d0adf"},
+ {file = "cython-3.1.2-cp39-cp39-win_amd64.whl", hash = "sha256:a965b81eb4f5a5f3f6760b162cb4de3907c71a9ba25d74de1ad7a0e4856f0412"},
+ {file = "cython-3.1.2-py3-none-any.whl", hash = "sha256:d23fd7ffd7457205f08571a42b108a3cf993e83a59fe4d72b42e6fc592cf2639"},
+ {file = "cython-3.1.2.tar.gz", hash = "sha256:6bbf7a953fa6762dfecdec015e3b054ba51c0121a45ad851fa130f63f5331381"},
]
[[package]]
@@ -1449,7 +1449,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5b6a66c18b5b9dd261ca98dffcb826a525334b2f29e7caa54e182255c5f6a65a"},
{file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d1b3eb7b05ea246510b43a7e53ed1653e55c2121019a97e60cad7efb881a97bb"},
@@ -2162,7 +2162,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -2980,7 +2980,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:b9f6392d98c0bd70676ae41474e2eecf4c7150cb419237a41f8f96043fcb81d1"},
{file = "multidict-6.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3501621d5e86f1a88521ea65d5cad0a0834c77b26f193747615b7c911e5422d2"},
@@ -3574,7 +3574,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:f27785888d2fdd918bc36de8b8739f2d6c791399552333721b58193f68ea3e98"},
{file = "propcache-0.3.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4e89cde74154c7b5957f87a355bb9c8ec929c167b59c83d90654ea36aeb6180"},
@@ -3882,7 +3882,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
+markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -4392,7 +4392,7 @@ description = "World timezone definitions, modern and historical"
optional = true
python-versions = "*"
groups = ["main"]
-markers = "extra == \"pandas\" or extra == \"ray\" or extra == \"gcsfs\""
+markers = "extra == \"gcsfs\" or extra == \"pandas\" or extra == \"ray\""
files = [
{file = "pytz-2025.2-py2.py3-none-any.whl", hash = "sha256:5ddf76296dd8c44c26eb8f4b6f35488f3ccbf6fbbd7adee0b7262d43f0ec2f00"},
{file = "pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3"},
@@ -4992,7 +4992,7 @@ files = [
{file = "s3transfer-0.11.3-py3-none-any.whl", hash = "sha256:ca855bdeb885174b5ffa95b9913622459d4ad8e331fc98eb01e6d5eb6a30655d"},
{file = "s3transfer-0.11.3.tar.gz", hash = "sha256:edae4977e3a122445660c7c114bba949f9d191bae3b34a096f18a1c8c354527a"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.36.0,<2.0a.0"
@@ -5426,7 +5426,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"hf\" or extra == \"daft\""
+markers = "extra == \"daft\" or extra == \"hf\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -5726,7 +5726,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7df647e8edd71f000a5208fe6ff8c382a1de8edfbccdbbfe649d263de07d8c34"},
{file = "yarl-1.18.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c69697d3adff5aa4f874b19c0e4ed65180ceed6318ec856ebc423aa5850d84f7"},
@@ -5977,4 +5977,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "93eda79de3991bea0b9520f69372d146f7fd3d128cd982ea5afb78e8ec5d94f2"
+content-hash = "c14566bc5c7839ec2ae57bbb1f5652697ef38a9bcd0afd212b3c4d0b23fc56fa"
diff --git a/pyproject.toml b/pyproject.toml
index 9699f28567..f28414fa05 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -98,7 +98,7 @@ moto = { version = "^5.0.2", extras = ["server"] }
typing-extensions = "4.13.2"
pytest-mock = "3.14.1"
pyspark = "3.5.6"
-cython = "3.1.1"
+cython = "3.1.2"
deptry = ">=0.14,<0.24"
datafusion = ">=44,<48"
docutils = "!=0.21.post1" # https://github.com/python-poetry/poetry/issues/9248#issuecomment-2026240520
From c4104cc7d7ab9f16404109d7d1504b111c62cd79 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 10 Jun 2025 06:34:55 +0200
Subject: [PATCH 046/112] Build: Bump huggingface-hub from 0.32.3 to 0.32.4
(#2077)
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 629211325f..c92de0e522 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1950,15 +1950,15 @@ tests = ["pytest"]
[[package]]
name = "huggingface-hub"
-version = "0.32.3"
+version = "0.32.4"
description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
optional = true
python-versions = ">=3.8.0"
groups = ["main"]
markers = "extra == \"hf\""
files = [
- {file = "huggingface_hub-0.32.3-py3-none-any.whl", hash = "sha256:e46f7ea7fe2b5e5f67cc4e37eb201140091946a314d7c2b134a9673dadd80b6a"},
- {file = "huggingface_hub-0.32.3.tar.gz", hash = "sha256:752c889ebf3a63cbd39803f6d87ccc135a463bbcb36abfa2faff0ccbf1cec087"},
+ {file = "huggingface_hub-0.32.4-py3-none-any.whl", hash = "sha256:37abf8826b38d971f60d3625229221c36e53fe58060286db9baf619cfbf39767"},
+ {file = "huggingface_hub-0.32.4.tar.gz", hash = "sha256:f61d45cd338736f59fb0e97550b74c24ee771bcc92c05ae0766b9116abe720be"},
]
[package.dependencies]
From cdc7f1f86a59907d07c80f7ff6bec3981496c3aa Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 10 Jun 2025 06:35:17 +0200
Subject: [PATCH 047/112] Build: Bump requests from 2.32.3 to 2.32.4 (#2078)
---
poetry.lock | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index c92de0e522..cb4c47c5de 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -4689,19 +4689,19 @@ files = [
[[package]]
name = "requests"
-version = "2.32.3"
+version = "2.32.4"
description = "Python HTTP for Humans."
optional = false
python-versions = ">=3.8"
groups = ["main", "dev", "docs"]
files = [
- {file = "requests-2.32.3-py3-none-any.whl", hash = "sha256:70761cfe03c773ceb22aa2f671b4757976145175cdfca038c02654d061d6dcc6"},
- {file = "requests-2.32.3.tar.gz", hash = "sha256:55365417734eb18255590a9ff9eb97e9e1da868d4ccd6402399eaf68af20a760"},
+ {file = "requests-2.32.4-py3-none-any.whl", hash = "sha256:27babd3cda2a6d50b30443204ee89830707d396671944c998b5975b031ac2b2c"},
+ {file = "requests-2.32.4.tar.gz", hash = "sha256:27d0316682c8a29834d3264820024b62a36942083d52caf2f14c0591336d3422"},
]
[package.dependencies]
certifi = ">=2017.4.17"
-charset-normalizer = ">=2,<4"
+charset_normalizer = ">=2,<4"
idna = ">=2.5,<4"
urllib3 = ">=1.21.1,<3"
From 1251e4bbc0dc01d841dcc93e3dfc24302348aa34 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 10 Jun 2025 11:12:31 +0200
Subject: [PATCH 048/112] Build: Bump duckdb from 1.2.2 to 1.3.0 (#2080)
---
poetry.lock | 89 ++++++++++++++++++++++-------------------------------
1 file changed, 37 insertions(+), 52 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index cb4c47c5de..a9dafa1d55 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1255,64 +1255,49 @@ dates = ["pytz (>=2019.1)"]
[[package]]
name = "duckdb"
-version = "1.2.2"
+version = "1.3.0"
description = "DuckDB in-process database"
optional = true
python-versions = ">=3.7.0"
groups = ["main"]
markers = "extra == \"duckdb\""
files = [
- {file = "duckdb-1.2.2-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:6e5e6c333b550903ff11919ed1154c60c9b9d935db51afdb263babe523a8a69e"},
- {file = "duckdb-1.2.2-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:c1fcbc579de8e4fa7e34242fd6f419c1a39520073b1fe0c29ed6e60ed5553f38"},
- {file = "duckdb-1.2.2-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:690885060c4140922ffa2f6935291c6e74ddad0ca2cf33bff66474ce89312ab3"},
- {file = "duckdb-1.2.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8a382782980643f5ee827990b76f079b22f47786509061c0afac28afaa5b8bf5"},
- {file = "duckdb-1.2.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d7c33345570ed8c50c9fe340c2767470115cc02d330f25384104cfad1f6e54f5"},
- {file = "duckdb-1.2.2-cp310-cp310-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b744f8293ce649d802a9eabbf88e4930d672cf9de7d4fc9af5d14ceaeeec5805"},
- {file = "duckdb-1.2.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:c8680e81b0c77be9fc968c1dd4cd38395c34b18bb693cbfc7b7742c18221cc9b"},
- {file = "duckdb-1.2.2-cp310-cp310-win_amd64.whl", hash = "sha256:fb41f2035a70378b3021f724bb08b047ca4aa475850a3744c442570054af3c52"},
- {file = "duckdb-1.2.2-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:081110ffbc9d53c9740ef55482c93b97db2f8030d681d1658827d2e94f77da03"},
- {file = "duckdb-1.2.2-cp311-cp311-macosx_12_0_universal2.whl", hash = "sha256:53a154dbc074604036a537784ce5d1468edf263745a4363ca06fdb922f0d0a99"},
- {file = "duckdb-1.2.2-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:0353f80882c066f7b14451852395b7a360f3d4846a10555c4268eb49144ea11c"},
- {file = "duckdb-1.2.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b134a5002757af1ae44a9ae26c2fe963ffa09eb47a62779ce0c5eeb44bfc2f28"},
- {file = "duckdb-1.2.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd9c434127fd1575694e1cf19a393bed301f5d6e80b4bcdae80caa368a61a678"},
- {file = "duckdb-1.2.2-cp311-cp311-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:890f58855d127c25bc3a53f4c24b27e79391c4468c4fcc99bc10d87b5d4bd1c4"},
- {file = "duckdb-1.2.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:9a5002305cdd4e76c94b61b50abc5e3f4e32c9cb81116960bb4b74acbbc9c6c8"},
- {file = "duckdb-1.2.2-cp311-cp311-win_amd64.whl", hash = "sha256:cdb9999c6a109aa31196cdd22fc58a810a3d35d08181a25d1bf963988e89f0a5"},
- {file = "duckdb-1.2.2-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:f745379f44ad302560688855baaed9739c03b37a331338eda6a4ac655e4eb42f"},
- {file = "duckdb-1.2.2-cp312-cp312-macosx_12_0_universal2.whl", hash = "sha256:087713fc5958cae5eb59097856b3deaae0def021660c8f2052ec83fa8345174a"},
- {file = "duckdb-1.2.2-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:a1f96395319c447a31b9477881bd84b4cb8323d6f86f21ceaef355d22dd90623"},
- {file = "duckdb-1.2.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6aba3bc0acf4f8d52b94f7746c3b0007b78b517676d482dc516d63f48f967baf"},
- {file = "duckdb-1.2.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e5c1556775a9ebaa49b5c8d64718f155ac3e05b34a49e9c99443cf105e8b0371"},
- {file = "duckdb-1.2.2-cp312-cp312-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:d625cc7d2faacfb2fc83ebbe001ae75dda175b3d8dce6a51a71c199ffac3627a"},
- {file = "duckdb-1.2.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:73263f81545c5cb4360fbaf7b22a493e55ddf88fadbe639c43efb7bc8d7554c4"},
- {file = "duckdb-1.2.2-cp312-cp312-win_amd64.whl", hash = "sha256:b1c0c4d737fd2ab9681e4e78b9f361e0a827916a730e84fa91e76dca451b14d5"},
- {file = "duckdb-1.2.2-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:fb9a2c77236fae079185a990434cb9d8432902488ba990235c702fc2692d2dcd"},
- {file = "duckdb-1.2.2-cp313-cp313-macosx_12_0_universal2.whl", hash = "sha256:d8bb89e580cb9a3aaf42e4555bf265d3db9446abfb118e32150e1a5dfa4b5b15"},
- {file = "duckdb-1.2.2-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:88916d7f0532dc926bed84b50408c00dcbe6d2097d0de93c3ff647d8d57b4f83"},
- {file = "duckdb-1.2.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:30bece4f58a6c7bb0944a02dd1dc6de435a9daf8668fa31a9fe3a9923b20bd65"},
- {file = "duckdb-1.2.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2bd2c6373b8b54474724c2119f6939c4568c428e1d0be5bcb1f4e3d7f1b7c8bb"},
- {file = "duckdb-1.2.2-cp313-cp313-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:72f688a8b0df7030c5a28ca6072817c1f090979e08d28ee5912dee37c26a7d0c"},
- {file = "duckdb-1.2.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:26e9c349f56f7c99341b5c79bbaff5ba12a5414af0261e79bf1a6a2693f152f6"},
- {file = "duckdb-1.2.2-cp313-cp313-win_amd64.whl", hash = "sha256:e1aec7102670e59d83512cf47d32a6c77a79df9df0294c5e4d16b6259851e2e9"},
- {file = "duckdb-1.2.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d1b374e7e2c474d6cd65fd80a94ff7263baec4be14ea193db4076d54eab408f9"},
- {file = "duckdb-1.2.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0fc6512d26eac83521938d7de65645ec08b04c2dc7807d4e332590c667e9d78"},
- {file = "duckdb-1.2.2-cp37-cp37m-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:3b451d16c3931fdbc235a12a39217a2faa03fa7c84c8560e65bc9b706e876089"},
- {file = "duckdb-1.2.2-cp38-cp38-macosx_12_0_arm64.whl", hash = "sha256:f3f8e09029ae47d3b904d32a03149ffc938bb3fb8a3048dc7b2d0f2ab50e0f56"},
- {file = "duckdb-1.2.2-cp38-cp38-macosx_12_0_universal2.whl", hash = "sha256:cee19d0c5bcb143b851ebd3ffc91e3445c5c3ee3cc0106edd882dd5b4091d5c0"},
- {file = "duckdb-1.2.2-cp38-cp38-macosx_12_0_x86_64.whl", hash = "sha256:c0f86c5e4ab7d4007ca0baa1707486daa38869c43f552a56e9cd2a28d431c2ae"},
- {file = "duckdb-1.2.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:378ef6a3d1a8b50da5a89376cc0cc6f131102d4a27b4b3adef10b20f7a6ea49f"},
- {file = "duckdb-1.2.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b985d13e161c27e8b947af28658d460925bade61cb5d7431b8258a807cc83752"},
- {file = "duckdb-1.2.2-cp38-cp38-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:446a5db77caeb155bcc0874c162a51f6d023af4aa2563fffbdec555db7402a35"},
- {file = "duckdb-1.2.2-cp38-cp38-win_amd64.whl", hash = "sha256:0c1a3496695c7220ac83dde02fc1cf174359c8072a6880050c8ae6b5c62a2635"},
- {file = "duckdb-1.2.2-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:25ac669180f88fecca20f300b898e191f81aa674d51dde8a328bdeb28a572ab0"},
- {file = "duckdb-1.2.2-cp39-cp39-macosx_12_0_universal2.whl", hash = "sha256:d42e7e545d1059e6b73d0f0baa9ae34c90684bfd8c862e70b0d8ab92e01e0e3f"},
- {file = "duckdb-1.2.2-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:f3ce127bcecc723f1c7bddbc57f0526d11128cb05bfd81ffcd5e69e2dd5a1624"},
- {file = "duckdb-1.2.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2418937adb9d6d0ca823bd385b914495294db27bc2963749d54af6708757f679"},
- {file = "duckdb-1.2.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:14d41f899ce7979e7b3f9097ebce70da5c659db2d81d08c07a72b2b50f869859"},
- {file = "duckdb-1.2.2-cp39-cp39-manylinux_2_24_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:85e90a9c5307cf4d9151844e60c80f492618ea6e9b71081020e7d462e071ac8f"},
- {file = "duckdb-1.2.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:df8c8a4ec998139b8507213c44c50e24f62a36af1cfded87e8972173dc9f8baf"},
- {file = "duckdb-1.2.2-cp39-cp39-win_amd64.whl", hash = "sha256:6507ad2445cd3479853fb6473164b5eb5b22446d283c9892cfbbd0a85c5f361d"},
- {file = "duckdb-1.2.2.tar.gz", hash = "sha256:1e53555dece49201df08645dbfa4510c86440339889667702f936b7d28d39e43"},
+ {file = "duckdb-1.3.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:fc65c1e97aa010359c43c0342ea423e6efa3cb8c8e3f133b0765451ce674e3db"},
+ {file = "duckdb-1.3.0-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:8fc91b629646679e33806342510335ccbbeaf2b823186f0ae829fd48e7a63c66"},
+ {file = "duckdb-1.3.0-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:1a69b970553fd015c557238d427ef00be3c8ed58c3bc3641aef987e33f8bf614"},
+ {file = "duckdb-1.3.0-cp310-cp310-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:1003e84c07b84680cee6d06e4795b6e861892474704f7972058594a52c7473cf"},
+ {file = "duckdb-1.3.0-cp310-cp310-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:992239b54ca6f015ad0ed0d80f3492c065313c4641df0a226183b8860cb7f5b0"},
+ {file = "duckdb-1.3.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:0ba1c5af59e8147216149b814b1970b8f7e3c240494a9688171390db3c504b29"},
+ {file = "duckdb-1.3.0-cp310-cp310-win_amd64.whl", hash = "sha256:57b794ca28e22b23bd170506cb1d4704a3608e67f0fe33273db9777b69bdf26a"},
+ {file = "duckdb-1.3.0-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:60a58b85929754abb21db1e739d2f53eaef63e6015e62ba58eae3425030e7935"},
+ {file = "duckdb-1.3.0-cp311-cp311-macosx_12_0_universal2.whl", hash = "sha256:1d46b5a20f078b1b2284243e02a1fde7e12cbb8d205fce62e4700bcfe6a09881"},
+ {file = "duckdb-1.3.0-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:0044e5ffb2d46308099640a92f99980a44e12bb68642aa9e6b08acbf300d64a1"},
+ {file = "duckdb-1.3.0-cp311-cp311-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:5cb813de2ca2f5e7c77392a67bdcaa174bfd69ebbfdfc983024af270c77a0447"},
+ {file = "duckdb-1.3.0-cp311-cp311-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:7a0c993eb6df2b30b189ad747f3aea1b0b87b78ab7f80c6e7c57117b6e8dbfb0"},
+ {file = "duckdb-1.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:6728e209570d36ece66dd7249e5d6055326321137cd807f26300733283930cd4"},
+ {file = "duckdb-1.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:7e652b7c8dbdb91a94fd7d543d3e115d24a25aa0791a373a852e20cb7bb21154"},
+ {file = "duckdb-1.3.0-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:f24038fe9b83dcbaeafb1ed76ec3b3f38943c1c8d27ab464ad384db8a6658b61"},
+ {file = "duckdb-1.3.0-cp312-cp312-macosx_12_0_universal2.whl", hash = "sha256:956c85842841bef68f4a5388c6b225b933151a7c06d568390fc895fc44607913"},
+ {file = "duckdb-1.3.0-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:efe883d822ed56fcfbb6a7b397c13f6a0d2eaeb3bc4ef4510f84fadb3dfe416d"},
+ {file = "duckdb-1.3.0-cp312-cp312-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:3872a3a1b80ffba5264ea236a3754d0c41d3c7b01bdf8cdcb1c180fc1b8dc8e2"},
+ {file = "duckdb-1.3.0-cp312-cp312-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:30bf45ad78a5a997f378863e036e917b481d18d685e5c977cd0a3faf2e31fbaf"},
+ {file = "duckdb-1.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:85cbd8e1d65df8a0780023baf5045d3033fabd154799bc9ea6d9ab5728f41eb3"},
+ {file = "duckdb-1.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:8754c40dac0f26d9fb0363bbb5df02f7a61ce6a6728d5efc02c3bc925d7c89c3"},
+ {file = "duckdb-1.3.0-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:176b9818d940c52ac7f31c64a98cf172d7c19d2a006017c9c4e9c06c246e36bf"},
+ {file = "duckdb-1.3.0-cp313-cp313-macosx_12_0_universal2.whl", hash = "sha256:03981f7e8793f07a4a9a2ba387640e71d0a99ebcaf8693ab09f96d59e628b713"},
+ {file = "duckdb-1.3.0-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:a177d55a38a62fdf79b59a0eaa32531a1dbb443265f6d67f64992cc1e82b755c"},
+ {file = "duckdb-1.3.0-cp313-cp313-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b1c30e3749823147d5578bc3f01f35d1a0433a1c768908d946056ec8d6e1757e"},
+ {file = "duckdb-1.3.0-cp313-cp313-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:5855f3a564baf22eeeab70c120b51f5a11914f1f1634f03382daeb6b1dea4c62"},
+ {file = "duckdb-1.3.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:9b1fac15a48056f7c2739cf8800873063ba2f691e91a9b2fc167658a401ca76a"},
+ {file = "duckdb-1.3.0-cp313-cp313-win_amd64.whl", hash = "sha256:fbdfc1c0b83b90f780ae74038187ee696bb56ab727a289752372d7ec42dda65b"},
+ {file = "duckdb-1.3.0-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:5f6b5d725546ad30abc125a6813734b493fea694bc3123e991c480744573c2f1"},
+ {file = "duckdb-1.3.0-cp39-cp39-macosx_12_0_universal2.whl", hash = "sha256:fcbcc9b956b06cf5ee94629438ecab88de89b08b5620fcda93665c222ab18cd4"},
+ {file = "duckdb-1.3.0-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:2d32f2d44105e1705d8a0fb6d6d246fd69aff82c80ad23293266244b66b69012"},
+ {file = "duckdb-1.3.0-cp39-cp39-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:0aa7a5c0dcb780850e6da1227fb1d552af8e1a5091e02667ab6ace61ab49ce6c"},
+ {file = "duckdb-1.3.0-cp39-cp39-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:7cb254fd5405f3edbd7d962ba39c72e4ab90b37cb4d0e34846089796c8078419"},
+ {file = "duckdb-1.3.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:a7d337b58c59fd2cd9faae531b05d940f8d92bdc2e14cb6e9a5a37675ad2742d"},
+ {file = "duckdb-1.3.0-cp39-cp39-win_amd64.whl", hash = "sha256:b3cea3a345755c7dbcb58403dbab8befd499c82f0d27f893a4c1d4b8cf56ec54"},
+ {file = "duckdb-1.3.0.tar.gz", hash = "sha256:09aaa4b1dca24f4d1f231e7ae66b6413e317b7e04e2753541d42df6c8113fac7"},
]
[[package]]
From bdcb4b9d69fe5f30abcce8799c118722d7b60155 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 10 Jun 2025 11:12:47 +0200
Subject: [PATCH 049/112] Build: Bump pandas from 2.2.3 to 2.3.0 (#2079)
---
poetry.lock | 86 ++++++++++++++++++++++++++---------------------------
1 file changed, 43 insertions(+), 43 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index a9dafa1d55..25f7a2d48b 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -3333,55 +3333,55 @@ lint = ["black"]
[[package]]
name = "pandas"
-version = "2.2.3"
+version = "2.3.0"
description = "Powerful data structures for data analysis, time series, and statistics"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"pandas\" or extra == \"ray\""
files = [
- {file = "pandas-2.2.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:1948ddde24197a0f7add2bdc4ca83bf2b1ef84a1bc8ccffd95eda17fd836ecb5"},
- {file = "pandas-2.2.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:381175499d3802cde0eabbaf6324cce0c4f5d52ca6f8c377c29ad442f50f6348"},
- {file = "pandas-2.2.3-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:d9c45366def9a3dd85a6454c0e7908f2b3b8e9c138f5dc38fed7ce720d8453ed"},
- {file = "pandas-2.2.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:86976a1c5b25ae3f8ccae3a5306e443569ee3c3faf444dfd0f41cda24667ad57"},
- {file = "pandas-2.2.3-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:b8661b0238a69d7aafe156b7fa86c44b881387509653fdf857bebc5e4008ad42"},
- {file = "pandas-2.2.3-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:37e0aced3e8f539eccf2e099f65cdb9c8aa85109b0be6e93e2baff94264bdc6f"},
- {file = "pandas-2.2.3-cp310-cp310-win_amd64.whl", hash = "sha256:56534ce0746a58afaf7942ba4863e0ef81c9c50d3f0ae93e9497d6a41a057645"},
- {file = "pandas-2.2.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:66108071e1b935240e74525006034333f98bcdb87ea116de573a6a0dccb6c039"},
- {file = "pandas-2.2.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:7c2875855b0ff77b2a64a0365e24455d9990730d6431b9e0ee18ad8acee13dbd"},
- {file = "pandas-2.2.3-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:cd8d0c3be0515c12fed0bdbae072551c8b54b7192c7b1fda0ba56059a0179698"},
- {file = "pandas-2.2.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c124333816c3a9b03fbeef3a9f230ba9a737e9e5bb4060aa2107a86cc0a497fc"},
- {file = "pandas-2.2.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:63cc132e40a2e084cf01adf0775b15ac515ba905d7dcca47e9a251819c575ef3"},
- {file = "pandas-2.2.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:29401dbfa9ad77319367d36940cd8a0b3a11aba16063e39632d98b0e931ddf32"},
- {file = "pandas-2.2.3-cp311-cp311-win_amd64.whl", hash = "sha256:3fc6873a41186404dad67245896a6e440baacc92f5b716ccd1bc9ed2995ab2c5"},
- {file = "pandas-2.2.3-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:b1d432e8d08679a40e2a6d8b2f9770a5c21793a6f9f47fdd52c5ce1948a5a8a9"},
- {file = "pandas-2.2.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:a5a1595fe639f5988ba6a8e5bc9649af3baf26df3998a0abe56c02609392e0a4"},
- {file = "pandas-2.2.3-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:5de54125a92bb4d1c051c0659e6fcb75256bf799a732a87184e5ea503965bce3"},
- {file = "pandas-2.2.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fffb8ae78d8af97f849404f21411c95062db1496aeb3e56f146f0355c9989319"},
- {file = "pandas-2.2.3-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6dfcb5ee8d4d50c06a51c2fffa6cff6272098ad6540aed1a76d15fb9318194d8"},
- {file = "pandas-2.2.3-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:062309c1b9ea12a50e8ce661145c6aab431b1e99530d3cd60640e255778bd43a"},
- {file = "pandas-2.2.3-cp312-cp312-win_amd64.whl", hash = "sha256:59ef3764d0fe818125a5097d2ae867ca3fa64df032331b7e0917cf5d7bf66b13"},
- {file = "pandas-2.2.3-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f00d1345d84d8c86a63e476bb4955e46458b304b9575dcf71102b5c705320015"},
- {file = "pandas-2.2.3-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:3508d914817e153ad359d7e069d752cdd736a247c322d932eb89e6bc84217f28"},
- {file = "pandas-2.2.3-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:22a9d949bfc9a502d320aa04e5d02feab689d61da4e7764b62c30b991c42c5f0"},
- {file = "pandas-2.2.3-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f3a255b2c19987fbbe62a9dfd6cff7ff2aa9ccab3fc75218fd4b7530f01efa24"},
- {file = "pandas-2.2.3-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:800250ecdadb6d9c78eae4990da62743b857b470883fa27f652db8bdde7f6659"},
- {file = "pandas-2.2.3-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:6374c452ff3ec675a8f46fd9ab25c4ad0ba590b71cf0656f8b6daa5202bca3fb"},
- {file = "pandas-2.2.3-cp313-cp313-win_amd64.whl", hash = "sha256:61c5ad4043f791b61dd4752191d9f07f0ae412515d59ba8f005832a532f8736d"},
- {file = "pandas-2.2.3-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:3b71f27954685ee685317063bf13c7709a7ba74fc996b84fc6821c59b0f06468"},
- {file = "pandas-2.2.3-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:38cf8125c40dae9d5acc10fa66af8ea6fdf760b2714ee482ca691fc66e6fcb18"},
- {file = "pandas-2.2.3-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:ba96630bc17c875161df3818780af30e43be9b166ce51c9a18c1feae342906c2"},
- {file = "pandas-2.2.3-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1db71525a1538b30142094edb9adc10be3f3e176748cd7acc2240c2f2e5aa3a4"},
- {file = "pandas-2.2.3-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:15c0e1e02e93116177d29ff83e8b1619c93ddc9c49083f237d4312337a61165d"},
- {file = "pandas-2.2.3-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:ad5b65698ab28ed8d7f18790a0dc58005c7629f227be9ecc1072aa74c0c1d43a"},
- {file = "pandas-2.2.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:bc6b93f9b966093cb0fd62ff1a7e4c09e6d546ad7c1de191767baffc57628f39"},
- {file = "pandas-2.2.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:5dbca4c1acd72e8eeef4753eeca07de9b1db4f398669d5994086f788a5d7cc30"},
- {file = "pandas-2.2.3-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:8cd6d7cc958a3910f934ea8dbdf17b2364827bb4dafc38ce6eef6bb3d65ff09c"},
- {file = "pandas-2.2.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:99df71520d25fade9db7c1076ac94eb994f4d2673ef2aa2e86ee039b6746d20c"},
- {file = "pandas-2.2.3-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:31d0ced62d4ea3e231a9f228366919a5ea0b07440d9d4dac345376fd8e1477ea"},
- {file = "pandas-2.2.3-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:7eee9e7cea6adf3e3d24e304ac6b8300646e2a5d1cd3a3c2abed9101b0846761"},
- {file = "pandas-2.2.3-cp39-cp39-win_amd64.whl", hash = "sha256:4850ba03528b6dd51d6c5d273c46f183f39a9baf3f0143e566b89450965b105e"},
- {file = "pandas-2.2.3.tar.gz", hash = "sha256:4f18ba62b61d7e192368b84517265a99b4d7ee8912f8708660fb4a366cc82667"},
+ {file = "pandas-2.3.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:625466edd01d43b75b1883a64d859168e4556261a5035b32f9d743b67ef44634"},
+ {file = "pandas-2.3.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a6872d695c896f00df46b71648eea332279ef4077a409e2fe94220208b6bb675"},
+ {file = "pandas-2.3.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f4dd97c19bd06bc557ad787a15b6489d2614ddaab5d104a0310eb314c724b2d2"},
+ {file = "pandas-2.3.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:034abd6f3db8b9880aaee98f4f5d4dbec7c4829938463ec046517220b2f8574e"},
+ {file = "pandas-2.3.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:23c2b2dc5213810208ca0b80b8666670eb4660bbfd9d45f58592cc4ddcfd62e1"},
+ {file = "pandas-2.3.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:39ff73ec07be5e90330cc6ff5705c651ace83374189dcdcb46e6ff54b4a72cd6"},
+ {file = "pandas-2.3.0-cp310-cp310-win_amd64.whl", hash = "sha256:40cecc4ea5abd2921682b57532baea5588cc5f80f0231c624056b146887274d2"},
+ {file = "pandas-2.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:8adff9f138fc614347ff33812046787f7d43b3cef7c0f0171b3340cae333f6ca"},
+ {file = "pandas-2.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e5f08eb9a445d07720776df6e641975665c9ea12c9d8a331e0f6890f2dcd76ef"},
+ {file = "pandas-2.3.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fa35c266c8cd1a67d75971a1912b185b492d257092bdd2709bbdebe574ed228d"},
+ {file = "pandas-2.3.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:14a0cc77b0f089d2d2ffe3007db58f170dae9b9f54e569b299db871a3ab5bf46"},
+ {file = "pandas-2.3.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:c06f6f144ad0a1bf84699aeea7eff6068ca5c63ceb404798198af7eb86082e33"},
+ {file = "pandas-2.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ed16339bc354a73e0a609df36d256672c7d296f3f767ac07257801aa064ff73c"},
+ {file = "pandas-2.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:fa07e138b3f6c04addfeaf56cc7fdb96c3b68a3fe5e5401251f231fce40a0d7a"},
+ {file = "pandas-2.3.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:2eb4728a18dcd2908c7fccf74a982e241b467d178724545a48d0caf534b38ebf"},
+ {file = "pandas-2.3.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b9d8c3187be7479ea5c3d30c32a5d73d62a621166675063b2edd21bc47614027"},
+ {file = "pandas-2.3.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9ff730713d4c4f2f1c860e36c005c7cefc1c7c80c21c0688fd605aa43c9fcf09"},
+ {file = "pandas-2.3.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba24af48643b12ffe49b27065d3babd52702d95ab70f50e1b34f71ca703e2c0d"},
+ {file = "pandas-2.3.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:404d681c698e3c8a40a61d0cd9412cc7364ab9a9cc6e144ae2992e11a2e77a20"},
+ {file = "pandas-2.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:6021910b086b3ca756755e86ddc64e0ddafd5e58e076c72cb1585162e5ad259b"},
+ {file = "pandas-2.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:094e271a15b579650ebf4c5155c05dcd2a14fd4fdd72cf4854b2f7ad31ea30be"},
+ {file = "pandas-2.3.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:2c7e2fc25f89a49a11599ec1e76821322439d90820108309bf42130d2f36c983"},
+ {file = "pandas-2.3.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:c6da97aeb6a6d233fb6b17986234cc723b396b50a3c6804776351994f2a658fd"},
+ {file = "pandas-2.3.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb32dc743b52467d488e7a7c8039b821da2826a9ba4f85b89ea95274f863280f"},
+ {file = "pandas-2.3.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:213cd63c43263dbb522c1f8a7c9d072e25900f6975596f883f4bebd77295d4f3"},
+ {file = "pandas-2.3.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:1d2b33e68d0ce64e26a4acc2e72d747292084f4e8db4c847c6f5f6cbe56ed6d8"},
+ {file = "pandas-2.3.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:430a63bae10b5086995db1b02694996336e5a8ac9a96b4200572b413dfdfccb9"},
+ {file = "pandas-2.3.0-cp313-cp313-win_amd64.whl", hash = "sha256:4930255e28ff5545e2ca404637bcc56f031893142773b3468dc021c6c32a1390"},
+ {file = "pandas-2.3.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:f925f1ef673b4bd0271b1809b72b3270384f2b7d9d14a189b12b7fc02574d575"},
+ {file = "pandas-2.3.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e78ad363ddb873a631e92a3c063ade1ecfb34cae71e9a2be6ad100f875ac1042"},
+ {file = "pandas-2.3.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:951805d146922aed8357e4cc5671b8b0b9be1027f0619cea132a9f3f65f2f09c"},
+ {file = "pandas-2.3.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a881bc1309f3fce34696d07b00f13335c41f5f5a8770a33b09ebe23261cfc67"},
+ {file = "pandas-2.3.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:e1991bbb96f4050b09b5f811253c4f3cf05ee89a589379aa36cd623f21a31d6f"},
+ {file = "pandas-2.3.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:bb3be958022198531eb7ec2008cfc78c5b1eed51af8600c6c5d9160d89d8d249"},
+ {file = "pandas-2.3.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9efc0acbbffb5236fbdf0409c04edce96bec4bdaa649d49985427bd1ec73e085"},
+ {file = "pandas-2.3.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:75651c14fde635e680496148a8526b328e09fe0572d9ae9b638648c46a544ba3"},
+ {file = "pandas-2.3.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bf5be867a0541a9fb47a4be0c5790a4bccd5b77b92f0a59eeec9375fafc2aa14"},
+ {file = "pandas-2.3.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84141f722d45d0c2a89544dd29d35b3abfc13d2250ed7e68394eda7564bd6324"},
+ {file = "pandas-2.3.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:f95a2aef32614ed86216d3c450ab12a4e82084e8102e355707a1d96e33d51c34"},
+ {file = "pandas-2.3.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:e0f51973ba93a9f97185049326d75b942b9aeb472bec616a129806facb129ebb"},
+ {file = "pandas-2.3.0-cp39-cp39-win_amd64.whl", hash = "sha256:b198687ca9c8529662213538a9bb1e60fa0bf0f6af89292eb68fea28743fcd5a"},
+ {file = "pandas-2.3.0.tar.gz", hash = "sha256:34600ab34ebf1131a7613a260a61dbe8b62c188ec0ea4c296da7c9a06b004133"},
]
[package.dependencies]
From 25797930e84cd3fd001859012f0324d3fcae1c77 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 10 Jun 2025 11:13:08 +0200
Subject: [PATCH 050/112] Build: Bump mkdocstrings-python from 1.16.11 to
1.16.12 (#2076)
---
poetry.lock | 8 ++++----
pyproject.toml | 2 +-
2 files changed, 5 insertions(+), 5 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 25f7a2d48b..1bd4aed5ac 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -2650,14 +2650,14 @@ python-legacy = ["mkdocstrings-python-legacy (>=0.2.1)"]
[[package]]
name = "mkdocstrings-python"
-version = "1.16.11"
+version = "1.16.12"
description = "A Python handler for mkdocstrings."
optional = false
python-versions = ">=3.9"
groups = ["docs"]
files = [
- {file = "mkdocstrings_python-1.16.11-py3-none-any.whl", hash = "sha256:25d96cc9c1f9c272ea1bd8222c900b5f852bf46c984003e9c7c56eaa4696190f"},
- {file = "mkdocstrings_python-1.16.11.tar.gz", hash = "sha256:935f95efa887f99178e4a7becaaa1286fb35adafffd669b04fd611d97c00e5ce"},
+ {file = "mkdocstrings_python-1.16.12-py3-none-any.whl", hash = "sha256:22ded3a63b3d823d57457a70ff9860d5a4de9e8b1e482876fc9baabaf6f5f374"},
+ {file = "mkdocstrings_python-1.16.12.tar.gz", hash = "sha256:9b9eaa066e0024342d433e332a41095c4e429937024945fea511afe58f63175d"},
]
[package.dependencies]
@@ -5962,4 +5962,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "c14566bc5c7839ec2ae57bbb1f5652697ef38a9bcd0afd212b3c4d0b23fc56fa"
+content-hash = "7825a9ae7c96f9b76a4ef4727b45e9ebdd01c9de27a0a98579dd881af6396e3b"
diff --git a/pyproject.toml b/pyproject.toml
index f28414fa05..2dfcbcefcd 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -109,7 +109,7 @@ mkdocs = "1.6.1"
griffe = "1.7.3"
jinja2 = "3.1.6"
mkdocstrings = "0.29.1"
-mkdocstrings-python = "1.16.11"
+mkdocstrings-python = "1.16.12"
mkdocs-literate-nav = "0.6.2"
mkdocs-autorefs = "1.4.2"
mkdocs-gen-files = "0.5.0"
From e1b1ebe941a7b624d18618c19864e40d025dc134 Mon Sep 17 00:00:00 2001
From: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
Date: Wed, 11 Jun 2025 15:15:36 -0400
Subject: [PATCH 051/112] Robustify boto3 session handling (DynamoDB,
RestCatalog) (#2071)
3 changes here...
1. Basically a mirror of
https://github.com/apache/iceberg-python/pull/1920 but for dynamo
2. Allow users to pass in an existing client/pass initialize client with
properties in `RestCatalog` with sigv4 enabled. (closes #2070, closes
#2008)
3. Re-use of the client in `add_headers` in `RestCatalog` with sigv4.
(closes #2069 )
---------
Co-authored-by: Marko Grujic
---
poetry.lock | 2768 ++++++++++++++++------------
pyiceberg/catalog/dynamodb.py | 32 +-
pyiceberg/catalog/glue.py | 66 +-
pyiceberg/catalog/rest/__init__.py | 16 +-
pyproject.toml | 11 +-
tests/catalog/test_dynamodb.py | 8 +
6 files changed, 1639 insertions(+), 1262 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 1bd4aed5ac..bbd207d7bb 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -58,7 +58,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -66,98 +66,103 @@ files = [
[[package]]
name = "aiohttp"
-version = "3.11.14"
+version = "3.12.12"
description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
-files = [
- {file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e2bc827c01f75803de77b134afdbf74fa74b62970eafdf190f3244931d7a5c0d"},
- {file = "aiohttp-3.11.14-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e365034c5cf6cf74f57420b57682ea79e19eb29033399dd3f40de4d0171998fa"},
- {file = "aiohttp-3.11.14-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c32593ead1a8c6aabd58f9d7ee706e48beac796bb0cb71d6b60f2c1056f0a65f"},
- {file = "aiohttp-3.11.14-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b4e7c7ec4146a94a307ca4f112802a8e26d969018fabed526efc340d21d3e7d0"},
- {file = "aiohttp-3.11.14-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c8b2df9feac55043759aa89f722a967d977d80f8b5865a4153fc41c93b957efc"},
- {file = "aiohttp-3.11.14-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c7571f99525c76a6280f5fe8e194eeb8cb4da55586c3c61c59c33a33f10cfce7"},
- {file = "aiohttp-3.11.14-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b59d096b5537ec7c85954cb97d821aae35cfccce3357a2cafe85660cc6295628"},
- {file = "aiohttp-3.11.14-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b42dbd097abb44b3f1156b4bf978ec5853840802d6eee2784857be11ee82c6a0"},
- {file = "aiohttp-3.11.14-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:b05774864c87210c531b48dfeb2f7659407c2dda8643104fb4ae5e2c311d12d9"},
- {file = "aiohttp-3.11.14-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:4e2e8ef37d4bc110917d038807ee3af82700a93ab2ba5687afae5271b8bc50ff"},
- {file = "aiohttp-3.11.14-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e9faafa74dbb906b2b6f3eb9942352e9e9db8d583ffed4be618a89bd71a4e914"},
- {file = "aiohttp-3.11.14-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:7e7abe865504f41b10777ac162c727af14e9f4db9262e3ed8254179053f63e6d"},
- {file = "aiohttp-3.11.14-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:4848ae31ad44330b30f16c71e4f586cd5402a846b11264c412de99fa768f00f3"},
- {file = "aiohttp-3.11.14-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:2d0b46abee5b5737cb479cc9139b29f010a37b1875ee56d142aefc10686a390b"},
- {file = "aiohttp-3.11.14-cp310-cp310-win32.whl", hash = "sha256:a0d2c04a623ab83963576548ce098baf711a18e2c32c542b62322a0b4584b990"},
- {file = "aiohttp-3.11.14-cp310-cp310-win_amd64.whl", hash = "sha256:5409a59d5057f2386bb8b8f8bbcfb6e15505cedd8b2445db510563b5d7ea1186"},
- {file = "aiohttp-3.11.14-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:f296d637a50bb15fb6a229fbb0eb053080e703b53dbfe55b1e4bb1c5ed25d325"},
- {file = "aiohttp-3.11.14-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ec6cd1954ca2bbf0970f531a628da1b1338f594bf5da7e361e19ba163ecc4f3b"},
- {file = "aiohttp-3.11.14-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:572def4aad0a4775af66d5a2b5923c7de0820ecaeeb7987dcbccda2a735a993f"},
- {file = "aiohttp-3.11.14-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1c68e41c4d576cd6aa6c6d2eddfb32b2acfb07ebfbb4f9da991da26633a3db1a"},
- {file = "aiohttp-3.11.14-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:99b8bbfc8111826aa8363442c0fc1f5751456b008737ff053570f06a151650b3"},
- {file = "aiohttp-3.11.14-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4b0a200e85da5c966277a402736a96457b882360aa15416bf104ca81e6f5807b"},
- {file = "aiohttp-3.11.14-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d173c0ac508a2175f7c9a115a50db5fd3e35190d96fdd1a17f9cb10a6ab09aa1"},
- {file = "aiohttp-3.11.14-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:413fe39fd929329f697f41ad67936f379cba06fcd4c462b62e5b0f8061ee4a77"},
- {file = "aiohttp-3.11.14-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:65c75b14ee74e8eeff2886321e76188cbe938d18c85cff349d948430179ad02c"},
- {file = "aiohttp-3.11.14-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:321238a42ed463848f06e291c4bbfb3d15ba5a79221a82c502da3e23d7525d06"},
- {file = "aiohttp-3.11.14-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:59a05cdc636431f7ce843c7c2f04772437dd816a5289f16440b19441be6511f1"},
- {file = "aiohttp-3.11.14-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:daf20d9c3b12ae0fdf15ed92235e190f8284945563c4b8ad95b2d7a31f331cd3"},
- {file = "aiohttp-3.11.14-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:05582cb2d156ac7506e68b5eac83179faedad74522ed88f88e5861b78740dc0e"},
- {file = "aiohttp-3.11.14-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:12c5869e7ddf6b4b1f2109702b3cd7515667b437da90a5a4a50ba1354fe41881"},
- {file = "aiohttp-3.11.14-cp311-cp311-win32.whl", hash = "sha256:92868f6512714efd4a6d6cb2bfc4903b997b36b97baea85f744229f18d12755e"},
- {file = "aiohttp-3.11.14-cp311-cp311-win_amd64.whl", hash = "sha256:bccd2cb7aa5a3bfada72681bdb91637094d81639e116eac368f8b3874620a654"},
- {file = "aiohttp-3.11.14-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:70ab0f61c1a73d3e0342cedd9a7321425c27a7067bebeeacd509f96695b875fc"},
- {file = "aiohttp-3.11.14-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:602d4db80daf4497de93cb1ce00b8fc79969c0a7cf5b67bec96fa939268d806a"},
- {file = "aiohttp-3.11.14-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:3a8a0d127c10b8d89e69bbd3430da0f73946d839e65fec00ae48ca7916a31948"},
- {file = "aiohttp-3.11.14-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ca9f835cdfedcb3f5947304e85b8ca3ace31eef6346d8027a97f4de5fb687534"},
- {file = "aiohttp-3.11.14-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8aa5c68e1e68fff7cd3142288101deb4316b51f03d50c92de6ea5ce646e6c71f"},
- {file = "aiohttp-3.11.14-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3b512f1de1c688f88dbe1b8bb1283f7fbeb7a2b2b26e743bb2193cbadfa6f307"},
- {file = "aiohttp-3.11.14-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cc9253069158d57e27d47a8453d8a2c5a370dc461374111b5184cf2f147a3cc3"},
- {file = "aiohttp-3.11.14-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0b2501f1b981e70932b4a552fc9b3c942991c7ae429ea117e8fba57718cdeed0"},
- {file = "aiohttp-3.11.14-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:28a3d083819741592685762d51d789e6155411277050d08066537c5edc4066e6"},
- {file = "aiohttp-3.11.14-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:0df3788187559c262922846087e36228b75987f3ae31dd0a1e5ee1034090d42f"},
- {file = "aiohttp-3.11.14-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:9e73fa341d8b308bb799cf0ab6f55fc0461d27a9fa3e4582755a3d81a6af8c09"},
- {file = "aiohttp-3.11.14-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:51ba80d473eb780a329d73ac8afa44aa71dfb521693ccea1dea8b9b5c4df45ce"},
- {file = "aiohttp-3.11.14-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:8d1dd75aa4d855c7debaf1ef830ff2dfcc33f893c7db0af2423ee761ebffd22b"},
- {file = "aiohttp-3.11.14-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:41cf0cefd9e7b5c646c2ef529c8335e7eafd326f444cc1cdb0c47b6bc836f9be"},
- {file = "aiohttp-3.11.14-cp312-cp312-win32.whl", hash = "sha256:948abc8952aff63de7b2c83bfe3f211c727da3a33c3a5866a0e2cf1ee1aa950f"},
- {file = "aiohttp-3.11.14-cp312-cp312-win_amd64.whl", hash = "sha256:3b420d076a46f41ea48e5fcccb996f517af0d406267e31e6716f480a3d50d65c"},
- {file = "aiohttp-3.11.14-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:8d14e274828561db91e4178f0057a915f3af1757b94c2ca283cb34cbb6e00b50"},
- {file = "aiohttp-3.11.14-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f30fc72daf85486cdcdfc3f5e0aea9255493ef499e31582b34abadbfaafb0965"},
- {file = "aiohttp-3.11.14-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:4edcbe34e6dba0136e4cabf7568f5a434d89cc9de5d5155371acda275353d228"},
- {file = "aiohttp-3.11.14-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1a7169ded15505f55a87f8f0812c94c9412623c744227b9e51083a72a48b68a5"},
- {file = "aiohttp-3.11.14-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ad1f2fb9fe9b585ea4b436d6e998e71b50d2b087b694ab277b30e060c434e5db"},
- {file = "aiohttp-3.11.14-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:20412c7cc3720e47a47e63c0005f78c0c2370020f9f4770d7fc0075f397a9fb0"},
- {file = "aiohttp-3.11.14-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6dd9766da617855f7e85f27d2bf9a565ace04ba7c387323cd3e651ac4329db91"},
- {file = "aiohttp-3.11.14-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:599b66582f7276ebefbaa38adf37585e636b6a7a73382eb412f7bc0fc55fb73d"},
- {file = "aiohttp-3.11.14-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:b41693b7388324b80f9acfabd479bd1c84f0bc7e8f17bab4ecd9675e9ff9c734"},
- {file = "aiohttp-3.11.14-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:86135c32d06927339c8c5e64f96e4eee8825d928374b9b71a3c42379d7437058"},
- {file = "aiohttp-3.11.14-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:04eb541ce1e03edc1e3be1917a0f45ac703e913c21a940111df73a2c2db11d73"},
- {file = "aiohttp-3.11.14-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:dc311634f6f28661a76cbc1c28ecf3b3a70a8edd67b69288ab7ca91058eb5a33"},
- {file = "aiohttp-3.11.14-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:69bb252bfdca385ccabfd55f4cd740d421dd8c8ad438ded9637d81c228d0da49"},
- {file = "aiohttp-3.11.14-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:2b86efe23684b58a88e530c4ab5b20145f102916bbb2d82942cafec7bd36a647"},
- {file = "aiohttp-3.11.14-cp313-cp313-win32.whl", hash = "sha256:b9c60d1de973ca94af02053d9b5111c4fbf97158e139b14f1be68337be267be6"},
- {file = "aiohttp-3.11.14-cp313-cp313-win_amd64.whl", hash = "sha256:0a29be28e60e5610d2437b5b2fed61d6f3dcde898b57fb048aa5079271e7f6f3"},
- {file = "aiohttp-3.11.14-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:14fc03508359334edc76d35b2821832f092c8f092e4b356e74e38419dfe7b6de"},
- {file = "aiohttp-3.11.14-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:92007c89a8cb7be35befa2732b0b32bf3a394c1b22ef2dff0ef12537d98a7bda"},
- {file = "aiohttp-3.11.14-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:6d3986112e34eaa36e280dc8286b9dd4cc1a5bcf328a7f147453e188f6fe148f"},
- {file = "aiohttp-3.11.14-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:749f1eb10e51dbbcdba9df2ef457ec060554842eea4d23874a3e26495f9e87b1"},
- {file = "aiohttp-3.11.14-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:781c8bd423dcc4641298c8c5a2a125c8b1c31e11f828e8d35c1d3a722af4c15a"},
- {file = "aiohttp-3.11.14-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:997b57e38aa7dc6caab843c5e042ab557bc83a2f91b7bd302e3c3aebbb9042a1"},
- {file = "aiohttp-3.11.14-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a8b0321e40a833e381d127be993b7349d1564b756910b28b5f6588a159afef3"},
- {file = "aiohttp-3.11.14-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8778620396e554b758b59773ab29c03b55047841d8894c5e335f12bfc45ebd28"},
- {file = "aiohttp-3.11.14-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:e906da0f2bcbf9b26cc2b144929e88cb3bf943dd1942b4e5af066056875c7618"},
- {file = "aiohttp-3.11.14-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:87f0e003fb4dd5810c7fbf47a1239eaa34cd929ef160e0a54c570883125c4831"},
- {file = "aiohttp-3.11.14-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:7f2dadece8b85596ac3ab1ec04b00694bdd62abc31e5618f524648d18d9dd7fa"},
- {file = "aiohttp-3.11.14-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:fe846f0a98aa9913c2852b630cd39b4098f296e0907dd05f6c7b30d911afa4c3"},
- {file = "aiohttp-3.11.14-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:ced66c5c6ad5bcaf9be54560398654779ec1c3695f1a9cf0ae5e3606694a000a"},
- {file = "aiohttp-3.11.14-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:a40087b82f83bd671cbeb5f582c233d196e9653220404a798798bfc0ee189fff"},
- {file = "aiohttp-3.11.14-cp39-cp39-win32.whl", hash = "sha256:95d7787f2bcbf7cb46823036a8d64ccfbc2ffc7d52016b4044d901abceeba3db"},
- {file = "aiohttp-3.11.14-cp39-cp39-win_amd64.whl", hash = "sha256:22a8107896877212130c58f74e64b77f7007cb03cea8698be317272643602d45"},
- {file = "aiohttp-3.11.14.tar.gz", hash = "sha256:d6edc538c7480fa0a3b2bdd705f8010062d74700198da55d16498e1b49549b9c"},
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+files = [
+ {file = "aiohttp-3.12.12-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6f25e9d274d6abbb15254f76f100c3984d6b9ad6e66263cc60a465dd5c7e48f5"},
+ {file = "aiohttp-3.12.12-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b8ec3c1a1c13d24941b5b913607e57b9364e4c0ea69d5363181467492c4b2ba6"},
+ {file = "aiohttp-3.12.12-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:81ef2f9253c327c211cb7b06ea2edd90e637cf21c347b894d540466b8d304e08"},
+ {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:28ded835c3663fd41c9ad44685811b11e34e6ac9a7516a30bfce13f6abba4496"},
+ {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:a4b78ccf254fc10605b263996949a94ca3f50e4f9100e05137d6583e266b711e"},
+ {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4f4a5af90d5232c41bb857568fe7d11ed84408653ec9da1ff999cc30258b9bd1"},
+ {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ffa5205c2f53f1120e93fdf2eca41b0f6344db131bc421246ee82c1e1038a14a"},
+ {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f68301660f0d7a3eddfb84f959f78a8f9db98c76a49b5235508fa16edaad0f7c"},
+ {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:db874d3b0c92fdbb553751af9d2733b378c25cc83cd9dfba87f12fafd2dc9cd5"},
+ {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:5e53cf9c201b45838a2d07b1f2d5f7fec9666db7979240002ce64f9b8a1e0cf2"},
+ {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:8687cc5f32b4e328c233acd387d09a1b477007896b2f03c1c823a0fd05f63883"},
+ {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:5ee537ad29de716a3d8dc46c609908de0c25ffeebf93cd94a03d64cdc07d66d0"},
+ {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:411f821be5af6af11dc5bed6c6c1dc6b6b25b91737d968ec2756f9baa75e5f9b"},
+ {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:f90319d94cf5f9786773237f24bd235a7b5959089f1af8ec1154580a3434b503"},
+ {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:73b148e606f34e9d513c451fd65efe1091772659ca5703338a396a99f60108ff"},
+ {file = "aiohttp-3.12.12-cp310-cp310-win32.whl", hash = "sha256:d40e7bfd577fdc8a92b72f35dfbdd3ec90f1bc8a72a42037fefe34d4eca2d4a1"},
+ {file = "aiohttp-3.12.12-cp310-cp310-win_amd64.whl", hash = "sha256:65c7804a2343893d6dea9fce69811aea0a9ac47f68312cf2e3ee1668cd9a387f"},
+ {file = "aiohttp-3.12.12-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:38823fe0d8bc059b3eaedb263fe427d887c7032e72b4ef92c472953285f0e658"},
+ {file = "aiohttp-3.12.12-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:10237f2c34711215d04ed21da63852ce023608299554080a45c576215d9df81c"},
+ {file = "aiohttp-3.12.12-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:563ec477c0dc6d56fc7f943a3475b5acdb399c7686c30f5a98ada24bb7562c7a"},
+ {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f3d05c46a61aca7c47df74afff818bc06a251ab95d95ff80b53665edfe1e0bdf"},
+ {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:277c882916759b4a6b6dc7e2ceb124aad071b3c6456487808d9ab13e1b448d57"},
+ {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:216abf74b324b0f4e67041dd4fb2819613909a825904f8a51701fbcd40c09cd7"},
+ {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:65d6cefad286459b68e7f867b9586a821fb7f121057b88f02f536ef570992329"},
+ {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:feaaaff61966b5f4b4eae0b79fc79427f49484e4cfa5ab7d138ecd933ab540a8"},
+ {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a05917780b7cad1755784b16cfaad806bc16029a93d15f063ca60185b7d9ba05"},
+ {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:082c5ec6d262c1b2ee01c63f4fb9152c17f11692bf16f0f100ad94a7a287d456"},
+ {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:b265a3a8b379b38696ac78bdef943bdc4f4a5d6bed1a3fb5c75c6bab1ecea422"},
+ {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:2e0f2e208914ecbc4b2a3b7b4daa759d0c587d9a0b451bb0835ac47fae7fa735"},
+ {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:9923b025845b72f64d167bca221113377c8ffabd0a351dc18fb839d401ee8e22"},
+ {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:1ebb213445900527831fecc70e185bf142fdfe5f2a691075f22d63c65ee3c35a"},
+ {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:6fc369fb273a8328077d37798b77c1e65676709af5c182cb74bd169ca9defe81"},
+ {file = "aiohttp-3.12.12-cp311-cp311-win32.whl", hash = "sha256:58ecd10fda6a44c311cd3742cfd2aea8c4c600338e9f27cb37434d9f5ca9ddaa"},
+ {file = "aiohttp-3.12.12-cp311-cp311-win_amd64.whl", hash = "sha256:b0066e88f30be00badffb5ef8f2281532b9a9020863d873ae15f7c147770b6ec"},
+ {file = "aiohttp-3.12.12-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:98451ce9ce229d092f278a74a7c2a06b3aa72984673c87796126d7ccade893e9"},
+ {file = "aiohttp-3.12.12-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:adbac7286d89245e1aff42e948503fdc6edf6d5d65c8e305a67c40f6a8fb95f4"},
+ {file = "aiohttp-3.12.12-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0728882115bfa85cbd8d0f664c8ccc0cfd5bd3789dd837596785450ae52fac31"},
+ {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6bf3b9d9e767f9d0e09fb1a31516410fc741a62cc08754578c40abc497d09540"},
+ {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:c944860e86b9f77a462321a440ccf6fa10f5719bb9d026f6b0b11307b1c96c7b"},
+ {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3b1979e1f0c98c06fd0cd940988833b102fa3aa56751f6c40ffe85cabc51f6fd"},
+ {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:120b7dd084e96cfdad85acea2ce1e7708c70a26db913eabb8d7b417c728f5d84"},
+ {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e58f5ae79649ffa247081c2e8c85e31d29623cf2a3137dda985ae05c9478aae"},
+ {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9aa5f049e3e2745b0141f13e5a64e7c48b1a1427ed18bbb7957b348f282fee56"},
+ {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7163cc9cf3722d90f1822f8a38b211e3ae2fc651c63bb55449f03dc1b3ff1d44"},
+ {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:ef97c4d035b721de6607f3980fa3e4ef0ec3aca76474b5789b7fac286a8c4e23"},
+ {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:1c14448d6a86acadc3f7b2f4cc385d1fb390acb6f37dce27f86fe629410d92e3"},
+ {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:a1b6df6255cfc493454c79221183d64007dd5080bcda100db29b7ff181b8832c"},
+ {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:60fc7338dfb0626c2927bfbac4785de3ea2e2bbe3d328ba5f3ece123edda4977"},
+ {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:d2afc72207ef4c9d4ca9fcd00689a6a37ef2d625600c3d757b5c2b80c9d0cf9a"},
+ {file = "aiohttp-3.12.12-cp312-cp312-win32.whl", hash = "sha256:8098a48f93b2cbcdb5778e7c9a0e0375363e40ad692348e6e65c3b70d593b27c"},
+ {file = "aiohttp-3.12.12-cp312-cp312-win_amd64.whl", hash = "sha256:d1c1879b2e0fc337d7a1b63fe950553c2b9e93c071cf95928aeea1902d441403"},
+ {file = "aiohttp-3.12.12-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ea5d604318234427929d486954e3199aded65f41593ac57aa0241ab93dda3d15"},
+ {file = "aiohttp-3.12.12-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:e03ff38250b8b572dce6fcd7b6fb6ee398bb8a59e6aa199009c5322d721df4fc"},
+ {file = "aiohttp-3.12.12-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:71125b1fc2b6a94bccc63bbece620906a4dead336d2051f8af9cbf04480bc5af"},
+ {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:784a66f9f853a22c6b8c2bd0ff157f9b879700f468d6d72cfa99167df08c5c9c"},
+ {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:a5be0b58670b54301404bd1840e4902570a1c3be00358e2700919cb1ea73c438"},
+ {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ce8f13566fc7bf5a728275b434bc3bdea87a7ed3ad5f734102b02ca59d9b510f"},
+ {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d736e57d1901683bc9be648aa308cb73e646252c74b4c639c35dcd401ed385ea"},
+ {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2007eaa7aae9102f211c519d1ec196bd3cecb1944a095db19eeaf132b798738"},
+ {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2a813e61583cab6d5cdbaa34bc28863acdb92f9f46e11de1b3b9251a1e8238f6"},
+ {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:e408293aa910b0aea48b86a28eace41d497a85ba16c20f619f0c604597ef996c"},
+ {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:f3d31faf290f5a30acba46b388465b67c6dbe8655d183e9efe2f6a1d594e6d9d"},
+ {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:0b84731697325b023902aa643bd1726d999f5bc7854bc28b17ff410a81151d4b"},
+ {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:a324c6852b6e327811748446e56cc9bb6eaa58710557922183175816e82a4234"},
+ {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:22fd867fbd72612dcf670c90486dbcbaf702cb807fb0b42bc0b7a142a573574a"},
+ {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:3e092f1a970223794a4bf620a26c0e4e4e8e36bccae9b0b5da35e6d8ee598a03"},
+ {file = "aiohttp-3.12.12-cp313-cp313-win32.whl", hash = "sha256:7f5f5eb8717ef8ba15ab35fcde5a70ad28bbdc34157595d1cddd888a985f5aae"},
+ {file = "aiohttp-3.12.12-cp313-cp313-win_amd64.whl", hash = "sha256:ace2499bdd03c329c054dc4b47361f2b19d5aa470f7db5c7e0e989336761b33c"},
+ {file = "aiohttp-3.12.12-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:0d0b1c27c05a7d39a50e946ec5f94c3af4ffadd33fa5f20705df42fb0a72ca14"},
+ {file = "aiohttp-3.12.12-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:e5928847e6f7b7434921fbabf73fa5609d1f2bf4c25d9d4522b1fcc3b51995cb"},
+ {file = "aiohttp-3.12.12-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7678147c3c85a7ae61559b06411346272ed40a08f54bc05357079a63127c9718"},
+ {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f50057f36f2a1d8e750b273bb966bec9f69ee1e0a20725ae081610501f25d555"},
+ {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:e5e834f0f11ff5805d11f0f22b627c75eadfaf91377b457875e4e3affd0b924f"},
+ {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f94b2e2dea19d09745ef02ed483192260750f18731876a5c76f1c254b841443a"},
+ {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b434bfb49564dc1c318989a0ab1d3000d23e5cfd00d8295dc9d5a44324cdd42d"},
+ {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8ed76bc80177ddb7c5c93e1a6440b115ed2c92a3063420ac55206fd0832a6459"},
+ {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e1282a9acd378f2aed8dc79c01e702b1d5fd260ad083926a88ec7e987c4e0ade"},
+ {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:09a213c13fba321586edab1528b530799645b82bd64d79b779eb8d47ceea155a"},
+ {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:72eae16a9233561d315e72ae78ed9fc65ab3db0196e56cb2d329c755d694f137"},
+ {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:f25990c507dbbeefd5a6a17df32a4ace634f7b20a38211d1b9609410c7f67a24"},
+ {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:3a2aa255417c8ccf1b39359cd0a3d63ae3b5ced83958dbebc4d9113327c0536a"},
+ {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:a4c53b89b3f838e9c25f943d1257efff10b348cb56895f408ddbcb0ec953a2ad"},
+ {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:b5a49c2dcb32114455ad503e8354624d85ab311cbe032da03965882492a9cb98"},
+ {file = "aiohttp-3.12.12-cp39-cp39-win32.whl", hash = "sha256:74fddc0ba8cea6b9c5bd732eb9d97853543586596b86391f8de5d4f6c2a0e068"},
+ {file = "aiohttp-3.12.12-cp39-cp39-win_amd64.whl", hash = "sha256:ddf40ba4a1d0b4d232dc47d2b98ae7e937dcbc40bb5f2746bce0af490a64526f"},
+ {file = "aiohttp-3.12.12.tar.gz", hash = "sha256:05875595d2483d96cb61fa9f64e75262d7ac6251a7e3c811d8e26f7d721760bd"},
]
[package.dependencies]
-aiohappyeyeballs = ">=2.3.0"
+aiohappyeyeballs = ">=2.5.0"
aiosignal = ">=1.1.2"
async-timeout = {version = ">=4.0,<6.0", markers = "python_version < \"3.11\""}
attrs = ">=17.3.0"
@@ -167,7 +172,7 @@ propcache = ">=0.2.0"
yarl = ">=1.17.0,<2.0"
[package.extras]
-speedups = ["Brotli ; platform_python_implementation == \"CPython\"", "aiodns (>=3.2.0) ; sys_platform == \"linux\" or sys_platform == \"darwin\"", "brotlicffi ; platform_python_implementation != \"CPython\""]
+speedups = ["Brotli ; platform_python_implementation == \"CPython\"", "aiodns (>=3.3.0)", "brotlicffi ; platform_python_implementation != \"CPython\""]
[[package]]
name = "aioitertools"
@@ -196,7 +201,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -212,11 +217,25 @@ description = "A light, configurable Sphinx theme"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
+markers = "python_version < \"3.10\""
files = [
{file = "alabaster-0.7.16-py3-none-any.whl", hash = "sha256:b46733c07dce03ae4e150330b975c75737fa60f0a7c591b6c8bf4928a28e2c92"},
{file = "alabaster-0.7.16.tar.gz", hash = "sha256:75a8b99c28a5dad50dd7f8ccdd447a121ddb3892da9e53d1ca5cca3106d58d65"},
]
+[[package]]
+name = "alabaster"
+version = "1.0.0"
+description = "A light, configurable Sphinx theme"
+optional = false
+python-versions = ">=3.10"
+groups = ["dev"]
+markers = "python_version >= \"3.10\""
+files = [
+ {file = "alabaster-1.0.0-py3-none-any.whl", hash = "sha256:fc6786402dc3fcb2de3cabd5fe455a2db534b371124f1f21de8731783dec828b"},
+ {file = "alabaster-1.0.0.tar.gz", hash = "sha256:c00dca57bca26fa62a6d7d0a9fcce65f3e026e9bfe33e9c538fd3fbb2144fd9e"},
+]
+
[[package]]
name = "annotated-types"
version = "0.7.0"
@@ -248,7 +267,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -265,7 +284,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -277,21 +296,21 @@ tests-mypy = ["mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" a
[[package]]
name = "aws-sam-translator"
-version = "1.95.0"
+version = "1.98.0"
description = "AWS SAM Translator is a library that transform SAM templates into AWS CloudFormation templates"
optional = false
python-versions = "!=4.0,<=4.0,>=3.8"
groups = ["dev"]
files = [
- {file = "aws_sam_translator-1.95.0-py3-none-any.whl", hash = "sha256:c9e0f22cbe83c768f7d20a3afb7e654bd6bfc087b387528bd48e98366b82ae40"},
- {file = "aws_sam_translator-1.95.0.tar.gz", hash = "sha256:fd2b891fc4cbdde1e06130eaf2710de5cc74442a656b7859b3840691144494cf"},
+ {file = "aws_sam_translator-1.98.0-py3-none-any.whl", hash = "sha256:65e7afffdda2e6f715debc251ddae5deba079af41db5dd9ecd370d658b9d728e"},
+ {file = "aws_sam_translator-1.98.0.tar.gz", hash = "sha256:fe9fdf51b593aca4cde29f555e272b00d90662315c8078e9f5f3448dd962c66b"},
]
[package.dependencies]
boto3 = ">=1.19.5,<2.dev0"
jsonschema = ">=3.2,<5"
pydantic = ">=1.8,<1.10.15 || >1.10.15,<1.10.17 || >1.10.17,<3"
-typing-extensions = ">=4.4"
+typing_extensions = ">=4.4"
[package.extras]
dev = ["black (==24.3.0)", "boto3 (>=1.23,<2)", "boto3-stubs[appconfig,serverlessrepo] (>=1.19.5,<2.dev0)", "coverage (>=5.3,<8)", "dateparser (>=1.1,<2.0)", "mypy (>=1.3.0,<1.4.0)", "parameterized (>=0.7,<1.0)", "pytest (>=6.2,<8)", "pytest-cov (>=2.10,<5)", "pytest-env (>=0.6,<1)", "pytest-rerunfailures (>=9.1,<12)", "pytest-xdist (>=2.5,<4)", "pyyaml (>=6.0,<7.0)", "requests (>=2.28,<3.0)", "ruamel.yaml (==0.17.21)", "ruff (>=0.4.5,<0.5.0)", "tenacity (>=8.0,<9.0)", "types-PyYAML (>=6.0,<7.0)", "types-jsonschema (>=3.2,<4.0)"]
@@ -314,15 +333,15 @@ wrapt = "*"
[[package]]
name = "azure-core"
-version = "1.32.0"
+version = "1.34.0"
description = "Microsoft Azure Core Library for Python"
optional = true
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"adlfs\""
files = [
- {file = "azure_core-1.32.0-py3-none-any.whl", hash = "sha256:eac191a0efb23bfa83fddf321b27b122b4ec847befa3091fa736a5c32c50d7b4"},
- {file = "azure_core-1.32.0.tar.gz", hash = "sha256:22b3c35d6b2dae14990f6c1be2912bf23ffe50b220e708a28ab1bb92b1c730e5"},
+ {file = "azure_core-1.34.0-py3-none-any.whl", hash = "sha256:0615d3b756beccdb6624d1c0ae97284f38b78fb59a2a9839bf927c66fbbdddd6"},
+ {file = "azure_core-1.34.0.tar.gz", hash = "sha256:bdb544989f246a0ad1c85d72eeb45f2f835afdcbc5b45e43f0dbde7461c81ece"},
]
[package.dependencies]
@@ -332,6 +351,7 @@ typing-extensions = ">=4.6.0"
[package.extras]
aio = ["aiohttp (>=3.0)"]
+tracing = ["opentelemetry-api (>=1.26,<2.0)"]
[[package]]
name = "azure-datalake-store"
@@ -353,15 +373,15 @@ requests = ">=2.20.0"
[[package]]
name = "azure-identity"
-version = "1.21.0"
+version = "1.23.0"
description = "Microsoft Azure Identity Library for Python"
optional = true
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"adlfs\""
files = [
- {file = "azure_identity-1.21.0-py3-none-any.whl", hash = "sha256:258ea6325537352440f71b35c3dffe9d240eae4a5126c1b7ce5efd5766bd9fd9"},
- {file = "azure_identity-1.21.0.tar.gz", hash = "sha256:ea22ce6e6b0f429bc1b8d9212d5b9f9877bd4c82f1724bfa910760612c07a9a6"},
+ {file = "azure_identity-1.23.0-py3-none-any.whl", hash = "sha256:dbbeb64b8e5eaa81c44c565f264b519ff2de7ff0e02271c49f3cb492762a50b0"},
+ {file = "azure_identity-1.23.0.tar.gz", hash = "sha256:d9cdcad39adb49d4bb2953a217f62aec1f65bbb3c63c9076da2be2a47e53dde4"},
]
[package.dependencies]
@@ -373,15 +393,15 @@ typing-extensions = ">=4.0.0"
[[package]]
name = "azure-storage-blob"
-version = "12.25.0"
+version = "12.25.1"
description = "Microsoft Azure Blob Storage Client Library for Python"
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"adlfs\""
files = [
- {file = "azure_storage_blob-12.25.0-py3-none-any.whl", hash = "sha256:a38e18bf10258fb19028f343db0d3d373280c6427a619c98c06d76485805b755"},
- {file = "azure_storage_blob-12.25.0.tar.gz", hash = "sha256:42364ca8f9f49dbccd0acc10144ed47bb6770bf78719970b51915f048891abba"},
+ {file = "azure_storage_blob-12.25.1-py3-none-any.whl", hash = "sha256:1f337aab12e918ec3f1b638baada97550673911c4ceed892acc8e4e891b74167"},
+ {file = "azure_storage_blob-12.25.1.tar.gz", hash = "sha256:4f294ddc9bc47909ac66b8934bd26b50d2000278b10ad82cc109764fdc6e0e3b"},
]
[package.dependencies]
@@ -467,7 +487,7 @@ files = [
{file = "boto3-1.37.3-py3-none-any.whl", hash = "sha256:2063b40af99fd02f6228ff52397b552ff3353831edaf8d25cc04801827ab9794"},
{file = "boto3-1.37.3.tar.gz", hash = "sha256:21f3ce0ef111297e63a6eb998a25197b8c10982970c320d4c6e8db08be2157be"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.3,<1.38.0"
@@ -488,7 +508,7 @@ files = [
{file = "botocore-1.37.3-py3-none-any.whl", hash = "sha256:d01bd3bf4c80e61fa88d636ad9f5c9f60a551d71549b481386c6b4efe0bb2b2e"},
{file = "botocore-1.37.3.tar.gz", hash = "sha256:fe8403eb55a88faf9b0f9da6615e5bee7be056d75e17af66c3c8f0a3b0648da4"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -530,26 +550,26 @@ virtualenv = ["virtualenv (>=20.0.35)"]
[[package]]
name = "cachetools"
-version = "6.0.0"
+version = "5.5.2"
description = "Extensible memoizing collections and decorators"
optional = false
-python-versions = ">=3.9"
+python-versions = ">=3.7"
groups = ["main"]
files = [
- {file = "cachetools-6.0.0-py3-none-any.whl", hash = "sha256:82e73ba88f7b30228b5507dce1a1f878498fc669d972aef2dde4f3a3c24f103e"},
- {file = "cachetools-6.0.0.tar.gz", hash = "sha256:f225782b84438f828328fc2ad74346522f27e5b1440f4e9fd18b20ebfd1aa2cf"},
+ {file = "cachetools-5.5.2-py3-none-any.whl", hash = "sha256:d26a22bcc62eb95c3beabd9f1ee5e820d3d2704fe2967cbe350e20c8ffcd3f0a"},
+ {file = "cachetools-5.5.2.tar.gz", hash = "sha256:1a661caa9175d26759571b2e19580f9d6393969e5dfca11fdb1f947a23e640d4"},
]
[[package]]
name = "certifi"
-version = "2025.1.31"
+version = "2025.4.26"
description = "Python package for providing Mozilla's CA Bundle."
optional = false
python-versions = ">=3.6"
groups = ["main", "dev", "docs"]
files = [
- {file = "certifi-2025.1.31-py3-none-any.whl", hash = "sha256:ca78db4565a652026a4db2bcdf68f2fb589ea80d0be70e03929ed730746b84fe"},
- {file = "certifi-2025.1.31.tar.gz", hash = "sha256:3d5da6925056f6f18f119200434a4780a94263f10d1c21d032a6f6b2baa20651"},
+ {file = "certifi-2025.4.26-py3-none-any.whl", hash = "sha256:30350364dfe371162649852c63336a15c70c6510c2ad5015b21c2345311805f3"},
+ {file = "certifi-2025.4.26.tar.gz", hash = "sha256:0a816057ea3cdefcef70270d2c515e4506bbc954f417fa5ade2021213bb8f0c6"},
]
[[package]]
@@ -647,18 +667,18 @@ files = [
[[package]]
name = "cfn-lint"
-version = "1.32.1"
+version = "1.35.4"
description = "Checks CloudFormation templates for practices and behaviour that could potentially be improved"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "cfn_lint-1.32.1-py3-none-any.whl", hash = "sha256:a8ea63ac8daa69a66a54a796998362fd063d9ba1e9c1fc3c932213b0c027669c"},
- {file = "cfn_lint-1.32.1.tar.gz", hash = "sha256:10282c0ec7fc6391da4877d9381a6b954f3c54ddcc0d3c97ee86f4783b5ae680"},
+ {file = "cfn_lint-1.35.4-py3-none-any.whl", hash = "sha256:4649797b4a6975a8ca5ebbf51e568a52383fa5b7d591f92266b8803735e5a52f"},
+ {file = "cfn_lint-1.35.4.tar.gz", hash = "sha256:da38218367217b909884ec2efe361b3992868f140b1d5f37dc64a9e328d9ddb9"},
]
[package.dependencies]
-aws-sam-translator = ">=1.95.0"
+aws-sam-translator = ">=1.97.0"
jsonpatch = "*"
networkx = ">=2.4,<4"
pyyaml = ">5.4"
@@ -674,104 +694,104 @@ sarif = ["jschema_to_python (>=1.2.3,<1.3.0)", "sarif-om (>=1.0.4,<1.1.0)"]
[[package]]
name = "charset-normalizer"
-version = "3.4.1"
+version = "3.4.2"
description = "The Real First Universal Charset Detector. Open, modern and actively maintained alternative to Chardet."
optional = false
python-versions = ">=3.7"
groups = ["main", "dev", "docs"]
files = [
- {file = "charset_normalizer-3.4.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:91b36a978b5ae0ee86c394f5a54d6ef44db1de0815eb43de826d41d21e4af3de"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7461baadb4dc00fd9e0acbe254e3d7d2112e7f92ced2adc96e54ef6501c5f176"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e218488cd232553829be0664c2292d3af2eeeb94b32bea483cf79ac6a694e037"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:80ed5e856eb7f30115aaf94e4a08114ccc8813e6ed1b5efa74f9f82e8509858f"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b010a7a4fd316c3c484d482922d13044979e78d1861f0e0650423144c616a46a"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4532bff1b8421fd0a320463030c7520f56a79c9024a4e88f01c537316019005a"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:d973f03c0cb71c5ed99037b870f2be986c3c05e63622c017ea9816881d2dd247"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:3a3bd0dcd373514dcec91c411ddb9632c0d7d92aed7093b8c3bbb6d69ca74408"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:d9c3cdf5390dcd29aa8056d13e8e99526cda0305acc038b96b30352aff5ff2bb"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:2bdfe3ac2e1bbe5b59a1a63721eb3b95fc9b6817ae4a46debbb4e11f6232428d"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:eab677309cdb30d047996b36d34caeda1dc91149e4fdca0b1a039b3f79d9a807"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-win32.whl", hash = "sha256:c0429126cf75e16c4f0ad00ee0eae4242dc652290f940152ca8c75c3a4b6ee8f"},
- {file = "charset_normalizer-3.4.1-cp310-cp310-win_amd64.whl", hash = "sha256:9f0b8b1c6d84c8034a44893aba5e767bf9c7a211e313a9605d9c617d7083829f"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:8bfa33f4f2672964266e940dd22a195989ba31669bd84629f05fab3ef4e2d125"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:28bf57629c75e810b6ae989f03c0828d64d6b26a5e205535585f96093e405ed1"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f08ff5e948271dc7e18a35641d2f11a4cd8dfd5634f55228b691e62b37125eb3"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:234ac59ea147c59ee4da87a0c0f098e9c8d169f4dc2a159ef720f1a61bbe27cd"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd4ec41f914fa74ad1b8304bbc634b3de73d2a0889bd32076342a573e0779e00"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:eea6ee1db730b3483adf394ea72f808b6e18cf3cb6454b4d86e04fa8c4327a12"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:c96836c97b1238e9c9e3fe90844c947d5afbf4f4c92762679acfe19927d81d77"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:4d86f7aff21ee58f26dcf5ae81a9addbd914115cdebcbb2217e4f0ed8982e146"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:09b5e6733cbd160dcc09589227187e242a30a49ca5cefa5a7edd3f9d19ed53fd"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:5777ee0881f9499ed0f71cc82cf873d9a0ca8af166dfa0af8ec4e675b7df48e6"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:237bdbe6159cff53b4f24f397d43c6336c6b0b42affbe857970cefbb620911c8"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-win32.whl", hash = "sha256:8417cb1f36cc0bc7eaba8ccb0e04d55f0ee52df06df3ad55259b9a323555fc8b"},
- {file = "charset_normalizer-3.4.1-cp311-cp311-win_amd64.whl", hash = "sha256:d7f50a1f8c450f3925cb367d011448c39239bb3eb4117c36a6d354794de4ce76"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:73d94b58ec7fecbc7366247d3b0b10a21681004153238750bb67bd9012414545"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dad3e487649f498dd991eeb901125411559b22e8d7ab25d3aeb1af367df5efd7"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c30197aa96e8eed02200a83fba2657b4c3acd0f0aa4bdc9f6c1af8e8962e0757"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2369eea1ee4a7610a860d88f268eb39b95cb588acd7235e02fd5a5601773d4fa"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc2722592d8998c870fa4e290c2eec2c1569b87fe58618e67d38b4665dfa680d"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ffc9202a29ab3920fa812879e95a9e78b2465fd10be7fcbd042899695d75e616"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:804a4d582ba6e5b747c625bf1255e6b1507465494a40a2130978bda7b932c90b"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:0f55e69f030f7163dffe9fd0752b32f070566451afe180f99dbeeb81f511ad8d"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:c4c3e6da02df6fa1410a7680bd3f63d4f710232d3139089536310d027950696a"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:5df196eb874dae23dcfb968c83d4f8fdccb333330fe1fc278ac5ceeb101003a9"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e358e64305fe12299a08e08978f51fc21fac060dcfcddd95453eabe5b93ed0e1"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-win32.whl", hash = "sha256:9b23ca7ef998bc739bf6ffc077c2116917eabcc901f88da1b9856b210ef63f35"},
- {file = "charset_normalizer-3.4.1-cp312-cp312-win_amd64.whl", hash = "sha256:6ff8a4a60c227ad87030d76e99cd1698345d4491638dfa6673027c48b3cd395f"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:aabfa34badd18f1da5ec1bc2715cadc8dca465868a4e73a0173466b688f29dda"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:22e14b5d70560b8dd51ec22863f370d1e595ac3d024cb8ad7d308b4cd95f8313"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8436c508b408b82d87dc5f62496973a1805cd46727c34440b0d29d8a2f50a6c9"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2d074908e1aecee37a7635990b2c6d504cd4766c7bc9fc86d63f9c09af3fa11b"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:955f8851919303c92343d2f66165294848d57e9bba6cf6e3625485a70a038d11"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:44ecbf16649486d4aebafeaa7ec4c9fed8b88101f4dd612dcaf65d5e815f837f"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:0924e81d3d5e70f8126529951dac65c1010cdf117bb75eb02dd12339b57749dd"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:2967f74ad52c3b98de4c3b32e1a44e32975e008a9cd2a8cc8966d6a5218c5cb2"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:c75cb2a3e389853835e84a2d8fb2b81a10645b503eca9bcb98df6b5a43eb8886"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:09b26ae6b1abf0d27570633b2b078a2a20419c99d66fb2823173d73f188ce601"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:fa88b843d6e211393a37219e6a1c1df99d35e8fd90446f1118f4216e307e48cd"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-win32.whl", hash = "sha256:eb8178fe3dba6450a3e024e95ac49ed3400e506fd4e9e5c32d30adda88cbd407"},
- {file = "charset_normalizer-3.4.1-cp313-cp313-win_amd64.whl", hash = "sha256:b1ac5992a838106edb89654e0aebfc24f5848ae2547d22c2c3f66454daa11971"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f30bf9fd9be89ecb2360c7d94a711f00c09b976258846efe40db3d05828e8089"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:97f68b8d6831127e4787ad15e6757232e14e12060bec17091b85eb1486b91d8d"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7974a0b5ecd505609e3b19742b60cee7aa2aa2fb3151bc917e6e2646d7667dcf"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fc54db6c8593ef7d4b2a331b58653356cf04f67c960f584edb7c3d8c97e8f39e"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:311f30128d7d333eebd7896965bfcfbd0065f1716ec92bd5638d7748eb6f936a"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-musllinux_1_2_aarch64.whl", hash = "sha256:7d053096f67cd1241601111b698f5cad775f97ab25d81567d3f59219b5f1adbd"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-musllinux_1_2_i686.whl", hash = "sha256:807f52c1f798eef6cf26beb819eeb8819b1622ddfeef9d0977a8502d4db6d534"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-musllinux_1_2_ppc64le.whl", hash = "sha256:dccbe65bd2f7f7ec22c4ff99ed56faa1e9f785482b9bbd7c717e26fd723a1d1e"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-musllinux_1_2_s390x.whl", hash = "sha256:2fb9bd477fdea8684f78791a6de97a953c51831ee2981f8e4f583ff3b9d9687e"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-musllinux_1_2_x86_64.whl", hash = "sha256:01732659ba9b5b873fc117534143e4feefecf3b2078b0a6a2e925271bb6f4cfa"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-win32.whl", hash = "sha256:7a4f97a081603d2050bfaffdefa5b02a9ec823f8348a572e39032caa8404a487"},
- {file = "charset_normalizer-3.4.1-cp37-cp37m-win_amd64.whl", hash = "sha256:7b1bef6280950ee6c177b326508f86cad7ad4dff12454483b51d8b7d673a2c5d"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:ecddf25bee22fe4fe3737a399d0d177d72bc22be6913acfab364b40bce1ba83c"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8c60ca7339acd497a55b0ea5d506b2a2612afb2826560416f6894e8b5770d4a9"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b7b2d86dd06bfc2ade3312a83a5c364c7ec2e3498f8734282c6c3d4b07b346b8"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dd78cfcda14a1ef52584dbb008f7ac81c1328c0f58184bf9a84c49c605002da6"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6e27f48bcd0957c6d4cb9d6fa6b61d192d0b13d5ef563e5f2ae35feafc0d179c"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:01ad647cdd609225c5350561d084b42ddf732f4eeefe6e678765636791e78b9a"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:619a609aa74ae43d90ed2e89bdd784765de0a25ca761b93e196d938b8fd1dbbd"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:89149166622f4db9b4b6a449256291dc87a99ee53151c74cbd82a53c8c2f6ccd"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:7709f51f5f7c853f0fb938bcd3bc59cdfdc5203635ffd18bf354f6967ea0f824"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:345b0426edd4e18138d6528aed636de7a9ed169b4aaf9d61a8c19e39d26838ca"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:0907f11d019260cdc3f94fbdb23ff9125f6b5d1039b76003b5b0ac9d6a6c9d5b"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-win32.whl", hash = "sha256:ea0d8d539afa5eb2728aa1932a988a9a7af94f18582ffae4bc10b3fbdad0626e"},
- {file = "charset_normalizer-3.4.1-cp38-cp38-win_amd64.whl", hash = "sha256:329ce159e82018d646c7ac45b01a430369d526569ec08516081727a20e9e4af4"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:b97e690a2118911e39b4042088092771b4ae3fc3aa86518f84b8cf6888dbdb41"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:78baa6d91634dfb69ec52a463534bc0df05dbd546209b79a3880a34487f4b84f"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1a2bc9f351a75ef49d664206d51f8e5ede9da246602dc2d2726837620ea034b2"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:75832c08354f595c760a804588b9357d34ec00ba1c940c15e31e96d902093770"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0af291f4fe114be0280cdd29d533696a77b5b49cfde5467176ecab32353395c4"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0167ddc8ab6508fe81860a57dd472b2ef4060e8d378f0cc555707126830f2537"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:2a75d49014d118e4198bcee5ee0a6f25856b29b12dbf7cd012791f8a6cc5c496"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:363e2f92b0f0174b2f8238240a1a30142e3db7b957a5dd5689b0e75fb717cc78"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:ab36c8eb7e454e34e60eb55ca5d241a5d18b2c6244f6827a30e451c42410b5f7"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:4c0907b1928a36d5a998d72d64d8eaa7244989f7aaaf947500d3a800c83a3fd6"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:04432ad9479fa40ec0f387795ddad4437a2b50417c69fa275e212933519ff294"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-win32.whl", hash = "sha256:3bed14e9c89dcb10e8f3a29f9ccac4955aebe93c71ae803af79265c9ca5644c5"},
- {file = "charset_normalizer-3.4.1-cp39-cp39-win_amd64.whl", hash = "sha256:49402233c892a461407c512a19435d1ce275543138294f7ef013f0b63d5d3765"},
- {file = "charset_normalizer-3.4.1-py3-none-any.whl", hash = "sha256:d98b1668f06378c6dbefec3b92299716b931cd4e6061f3c875a71ced1780ab85"},
- {file = "charset_normalizer-3.4.1.tar.gz", hash = "sha256:44251f18cd68a75b56585dd00dae26183e102cd5e0f9f1466e6df5da2ed64ea3"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7c48ed483eb946e6c04ccbe02c6b4d1d48e51944b6db70f697e089c193404941"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b2d318c11350e10662026ad0eb71bb51c7812fc8590825304ae0bdd4ac283acd"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9cbfacf36cb0ec2897ce0ebc5d08ca44213af24265bd56eca54bee7923c48fd6"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:18dd2e350387c87dabe711b86f83c9c78af772c748904d372ade190b5c7c9d4d"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8075c35cd58273fee266c58c0c9b670947c19df5fb98e7b66710e04ad4e9ff86"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5bf4545e3b962767e5c06fe1738f951f77d27967cb2caa64c28be7c4563e162c"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:7a6ab32f7210554a96cd9e33abe3ddd86732beeafc7a28e9955cdf22ffadbab0"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b33de11b92e9f75a2b545d6e9b6f37e398d86c3e9e9653c4864eb7e89c5773ef"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:8755483f3c00d6c9a77f490c17e6ab0c8729e39e6390328e42521ef175380ae6"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:68a328e5f55ec37c57f19ebb1fdc56a248db2e3e9ad769919a58672958e8f366"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:21b2899062867b0e1fde9b724f8aecb1af14f2778d69aacd1a5a1853a597a5db"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-win32.whl", hash = "sha256:e8082b26888e2f8b36a042a58307d5b917ef2b1cacab921ad3323ef91901c71a"},
+ {file = "charset_normalizer-3.4.2-cp310-cp310-win_amd64.whl", hash = "sha256:f69a27e45c43520f5487f27627059b64aaf160415589230992cec34c5e18a509"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:be1e352acbe3c78727a16a455126d9ff83ea2dfdcbc83148d2982305a04714c2"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aa88ca0b1932e93f2d961bf3addbb2db902198dca337d88c89e1559e066e7645"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d524ba3f1581b35c03cb42beebab4a13e6cdad7b36246bd22541fa585a56cccd"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:28a1005facc94196e1fb3e82a3d442a9d9110b8434fc1ded7a24a2983c9888d8"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fdb20a30fe1175ecabed17cbf7812f7b804b8a315a25f24678bcdf120a90077f"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0f5d9ed7f254402c9e7d35d2f5972c9bbea9040e99cd2861bd77dc68263277c7"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:efd387a49825780ff861998cd959767800d54f8308936b21025326de4b5a42b9"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:f0aa37f3c979cf2546b73e8222bbfa3dc07a641585340179d768068e3455e544"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:e70e990b2137b29dc5564715de1e12701815dacc1d056308e2b17e9095372a82"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:0c8c57f84ccfc871a48a47321cfa49ae1df56cd1d965a09abe84066f6853b9c0"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:6b66f92b17849b85cad91259efc341dce9c1af48e2173bf38a85c6329f1033e5"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-win32.whl", hash = "sha256:daac4765328a919a805fa5e2720f3e94767abd632ae410a9062dff5412bae65a"},
+ {file = "charset_normalizer-3.4.2-cp311-cp311-win_amd64.whl", hash = "sha256:e53efc7c7cee4c1e70661e2e112ca46a575f90ed9ae3fef200f2a25e954f4b28"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:0c29de6a1a95f24b9a1aa7aefd27d2487263f00dfd55a77719b530788f75cff7"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cddf7bd982eaa998934a91f69d182aec997c6c468898efe6679af88283b498d3"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:fcbe676a55d7445b22c10967bceaaf0ee69407fbe0ece4d032b6eb8d4565982a"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d41c4d287cfc69060fa91cae9683eacffad989f1a10811995fa309df656ec214"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4e594135de17ab3866138f496755f302b72157d115086d100c3f19370839dd3a"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cf713fe9a71ef6fd5adf7a79670135081cd4431c2943864757f0fa3a65b1fafd"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:a370b3e078e418187da8c3674eddb9d983ec09445c99a3a263c2011993522981"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:a955b438e62efdf7e0b7b52a64dc5c3396e2634baa62471768a64bc2adb73d5c"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:7222ffd5e4de8e57e03ce2cef95a4c43c98fcb72ad86909abdfc2c17d227fc1b"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:bee093bf902e1d8fc0ac143c88902c3dfc8941f7ea1d6a8dd2bcb786d33db03d"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:dedb8adb91d11846ee08bec4c8236c8549ac721c245678282dcb06b221aab59f"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-win32.whl", hash = "sha256:db4c7bf0e07fc3b7d89ac2a5880a6a8062056801b83ff56d8464b70f65482b6c"},
+ {file = "charset_normalizer-3.4.2-cp312-cp312-win_amd64.whl", hash = "sha256:5a9979887252a82fefd3d3ed2a8e3b937a7a809f65dcb1e068b090e165bbe99e"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:926ca93accd5d36ccdabd803392ddc3e03e6d4cd1cf17deff3b989ab8e9dbcf0"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:eba9904b0f38a143592d9fc0e19e2df0fa2e41c3c3745554761c5f6447eedabf"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3fddb7e2c84ac87ac3a947cb4e66d143ca5863ef48e4a5ecb83bd48619e4634e"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:98f862da73774290f251b9df8d11161b6cf25b599a66baf087c1ffe340e9bfd1"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6c9379d65defcab82d07b2a9dfbfc2e95bc8fe0ebb1b176a3190230a3ef0e07c"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e635b87f01ebc977342e2697d05b56632f5f879a4f15955dfe8cef2448b51691"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:1c95a1e2902a8b722868587c0e1184ad5c55631de5afc0eb96bc4b0d738092c0"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:ef8de666d6179b009dce7bcb2ad4c4a779f113f12caf8dc77f0162c29d20490b"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:32fc0341d72e0f73f80acb0a2c94216bd704f4f0bce10aedea38f30502b271ff"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:289200a18fa698949d2b39c671c2cc7a24d44096784e76614899a7ccf2574b7b"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:4a476b06fbcf359ad25d34a057b7219281286ae2477cc5ff5e3f70a246971148"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-win32.whl", hash = "sha256:aaeeb6a479c7667fbe1099af9617c83aaca22182d6cf8c53966491a0f1b7ffb7"},
+ {file = "charset_normalizer-3.4.2-cp313-cp313-win_amd64.whl", hash = "sha256:aa6af9e7d59f9c12b33ae4e9450619cf2488e2bbe9b44030905877f0b2324980"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1cad5f45b3146325bb38d6855642f6fd609c3f7cad4dbaf75549bf3b904d3184"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b2680962a4848b3c4f155dc2ee64505a9c57186d0d56b43123b17ca3de18f0fa"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:36b31da18b8890a76ec181c3cf44326bf2c48e36d393ca1b72b3f484113ea344"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f4074c5a429281bf056ddd4c5d3b740ebca4d43ffffe2ef4bf4d2d05114299da"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c9e36a97bee9b86ef9a1cf7bb96747eb7a15c2f22bdb5b516434b00f2a599f02"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-musllinux_1_2_aarch64.whl", hash = "sha256:1b1bde144d98e446b056ef98e59c256e9294f6b74d7af6846bf5ffdafd687a7d"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-musllinux_1_2_i686.whl", hash = "sha256:915f3849a011c1f593ab99092f3cecfcb4d65d8feb4a64cf1bf2d22074dc0ec4"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-musllinux_1_2_ppc64le.whl", hash = "sha256:fb707f3e15060adf5b7ada797624a6c6e0138e2a26baa089df64c68ee98e040f"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-musllinux_1_2_s390x.whl", hash = "sha256:25a23ea5c7edc53e0f29bae2c44fcb5a1aa10591aae107f2a2b2583a9c5cbc64"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-musllinux_1_2_x86_64.whl", hash = "sha256:770cab594ecf99ae64c236bc9ee3439c3f46be49796e265ce0cc8bc17b10294f"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-win32.whl", hash = "sha256:6a0289e4589e8bdfef02a80478f1dfcb14f0ab696b5a00e1f4b8a14a307a3c58"},
+ {file = "charset_normalizer-3.4.2-cp37-cp37m-win_amd64.whl", hash = "sha256:6fc1f5b51fa4cecaa18f2bd7a003f3dd039dd615cd69a2afd6d3b19aed6775f2"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:76af085e67e56c8816c3ccf256ebd136def2ed9654525348cfa744b6802b69eb"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e45ba65510e2647721e35323d6ef54c7974959f6081b58d4ef5d87c60c84919a"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:046595208aae0120559a67693ecc65dd75d46f7bf687f159127046628178dc45"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:75d10d37a47afee94919c4fab4c22b9bc2a8bf7d4f46f87363bcf0573f3ff4f5"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6333b3aa5a12c26b2a4d4e7335a28f1475e0e5e17d69d55141ee3cab736f66d1"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e8323a9b031aa0393768b87f04b4164a40037fb2a3c11ac06a03ffecd3618027"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:24498ba8ed6c2e0b56d4acbf83f2d989720a93b41d712ebd4f4979660db4417b"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:844da2b5728b5ce0e32d863af26f32b5ce61bc4273a9c720a9f3aa9df73b1455"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:65c981bdbd3f57670af8b59777cbfae75364b483fa8a9f420f08094531d54a01"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:3c21d4fca343c805a52c0c78edc01e3477f6dd1ad7c47653241cf2a206d4fc58"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:dc7039885fa1baf9be153a0626e337aa7ec8bf96b0128605fb0d77788ddc1681"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-win32.whl", hash = "sha256:8272b73e1c5603666618805fe821edba66892e2870058c94c53147602eab29c7"},
+ {file = "charset_normalizer-3.4.2-cp38-cp38-win_amd64.whl", hash = "sha256:70f7172939fdf8790425ba31915bfbe8335030f05b9913d7ae00a87d4395620a"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:005fa3432484527f9732ebd315da8da8001593e2cf46a3d817669f062c3d9ed4"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e92fca20c46e9f5e1bb485887d074918b13543b1c2a1185e69bb8d17ab6236a7"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:50bf98d5e563b83cc29471fa114366e6806bc06bc7a25fd59641e41445327836"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:721c76e84fe669be19c5791da68232ca2e05ba5185575086e384352e2c309597"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:82d8fd25b7f4675d0c47cf95b594d4e7b158aca33b76aa63d07186e13c0e0ab7"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b3daeac64d5b371dea99714f08ffc2c208522ec6b06fbc7866a450dd446f5c0f"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:dccab8d5fa1ef9bfba0590ecf4d46df048d18ffe3eec01eeb73a42e0d9e7a8ba"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:aaf27faa992bfee0264dc1f03f4c75e9fcdda66a519db6b957a3f826e285cf12"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:eb30abc20df9ab0814b5a2524f23d75dcf83cde762c161917a2b4b7b55b1e518"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:c72fbbe68c6f32f251bdc08b8611c7b3060612236e960ef848e0a517ddbe76c5"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:982bb1e8b4ffda883b3d0a521e23abcd6fd17418f6d2c4118d257a10199c0ce3"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-win32.whl", hash = "sha256:43e0933a0eff183ee85833f341ec567c0980dae57c464d8a508e1b2ceb336471"},
+ {file = "charset_normalizer-3.4.2-cp39-cp39-win_amd64.whl", hash = "sha256:d11b54acf878eef558599658b0ffca78138c8c3655cf4f3a4a673c437e67732e"},
+ {file = "charset_normalizer-3.4.2-py3-none-any.whl", hash = "sha256:7f56930ab0abd1c45cd15be65cc741c28b1c9a34876ce8c17a2fa107810c0af0"},
+ {file = "charset_normalizer-3.4.2.tar.gz", hash = "sha256:5baececa9ecba31eff645232d59845c07aa030f0c81ee70184a90d35099a0e63"},
]
[[package]]
@@ -781,6 +801,7 @@ description = "Composable command line interface toolkit"
optional = false
python-versions = ">=3.7"
groups = ["main", "dev", "docs"]
+markers = "python_version < \"3.10\""
files = [
{file = "click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2"},
{file = "click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a"},
@@ -789,6 +810,22 @@ files = [
[package.dependencies]
colorama = {version = "*", markers = "platform_system == \"Windows\""}
+[[package]]
+name = "click"
+version = "8.2.1"
+description = "Composable command line interface toolkit"
+optional = false
+python-versions = ">=3.10"
+groups = ["main", "dev", "docs"]
+markers = "python_version >= \"3.10\""
+files = [
+ {file = "click-8.2.1-py3-none-any.whl", hash = "sha256:61a3265b914e850b85317d0b3109c7f8cd35a670f963866005d6ef1d5175a12b"},
+ {file = "click-8.2.1.tar.gz", hash = "sha256:27c491cc05d968d271d5a1db13e3b5a184636d9d930f148c50b038f0d0646202"},
+]
+
+[package.dependencies]
+colorama = {version = "*", markers = "platform_system == \"Windows\""}
+
[[package]]
name = "colorama"
version = "0.4.6"
@@ -887,165 +924,180 @@ toml = ["tomli ; python_full_version <= \"3.11.0a6\""]
[[package]]
name = "cramjam"
-version = "2.9.1"
+version = "2.10.0"
description = "Thin Python bindings to de/compression algorithms in Rust"
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"snappy\""
files = [
- {file = "cramjam-2.9.1-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:8e82464d1e00fbbb12958999b8471ba5e9f3d9711954505a0a7b378762332e6f"},
- {file = "cramjam-2.9.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6d2df8a6511cc08ef1fccd2e0c65e2ebc9f57574ec8376052a76851af5398810"},
- {file = "cramjam-2.9.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:21ea784e6c3f1843d3523ae0f03651dd06058b39eeb64beb82ee3b100fa83662"},
- {file = "cramjam-2.9.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8e0c5d98a4e791f0bbd0ffcb7dae879baeb2dcc357348a8dc2be0a8c10403a2a"},
- {file = "cramjam-2.9.1-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:e076fd87089197cb61117c63dbe7712ad5eccb93968860eb3bae09b767bac813"},
- {file = "cramjam-2.9.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:6d86b44933aea0151e4a2e1e6935448499849045c38167d288ca4c59d5b8cd4e"},
- {file = "cramjam-2.9.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7eb032549dec897b942ddcf80c1cdccbcb40629f15fc902731dbe6362da49326"},
- {file = "cramjam-2.9.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cf29b4def86ec503e329fe138842a9b79a997e3beb6c7809b05665a0d291edff"},
- {file = "cramjam-2.9.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:a36adf7d13b7accfa206e1c917f08924eb905b45aa8e62176509afa7b14db71e"},
- {file = "cramjam-2.9.1-cp310-cp310-musllinux_1_1_armv7l.whl", hash = "sha256:cf4ea758d98b6fad1b4b2d808d0de690d3162ac56c26968aea0af6524e3eb736"},
- {file = "cramjam-2.9.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:4826d6d81ea490fa7a3ae7a4b9729866a945ffac1f77fe57b71e49d6e1b21efd"},
- {file = "cramjam-2.9.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:335103317475bf992953c58838152a4761fc3c87354000edbfc4d7e57cf05909"},
- {file = "cramjam-2.9.1-cp310-cp310-win32.whl", hash = "sha256:258120cb1e3afc3443f756f9de161ed63eed56a2c31f6093e81c571c0f2dc9f6"},
- {file = "cramjam-2.9.1-cp310-cp310-win_amd64.whl", hash = "sha256:c60e5996aa02547d12bc2740d44e90e006b0f93100f53206f7abe6732ad56e69"},
- {file = "cramjam-2.9.1-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:b9db1debe48060e41a5b91af9193c524e473c57f6105462c5524a41f5aabdb88"},
- {file = "cramjam-2.9.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f6f18f0242212d3409d26ce3874937b5b979cebd61f08b633a6ea893c32fc7b6"},
- {file = "cramjam-2.9.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:b5b1cd7d39242b2b903cf09cd4696b3a6e04dc537ffa9f3ac8668edae76eecb6"},
- {file = "cramjam-2.9.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a47de0a68f5f4d9951250ef5af31f2a7228132caa9ed60994234f7eb98090d33"},
- {file = "cramjam-2.9.1-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:e13c9a697881e5e38148958612dc6856967f5ff8cd7bba5ff751f2d6ac020aa4"},
- {file = "cramjam-2.9.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ba560244bc1335b420b74e91e35f9d4e7f307a3be3a4603ce0f0d7e15a0acdf0"},
- {file = "cramjam-2.9.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8d47fd41ce260cf4f0ff0e788de961fab9e9c6844a05ce55d06ce31e06107bdc"},
- {file = "cramjam-2.9.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84d154fbadece82935396eb6bcb502085d944d2fd13b07a94348364344370c2c"},
- {file = "cramjam-2.9.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:038df668ffb94d64d67b6ecc59cbd206745a425ffc0402897dde12d89fa6a870"},
- {file = "cramjam-2.9.1-cp311-cp311-musllinux_1_1_armv7l.whl", hash = "sha256:4125d8cd86fa08495d310e80926c2f0563f157b76862e7479f9b2cf94823ea0c"},
- {file = "cramjam-2.9.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:4206ebdd1d1ef0f3f86c8c2f7c426aa4af6094f4f41e274601fd4c4569f37454"},
- {file = "cramjam-2.9.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:ab687bef5c493732b9a4ab870542ee43f5eae0025f9c684c7cb399c3a85cb380"},
- {file = "cramjam-2.9.1-cp311-cp311-win32.whl", hash = "sha256:dda7698b6d7caeae1047adafebc4b43b2a82478234f6c2b45bc3edad854e0600"},
- {file = "cramjam-2.9.1-cp311-cp311-win_amd64.whl", hash = "sha256:872b00ff83e84bcbdc7e951af291ebe65eed20b09c47e7c4af21c312f90b796f"},
- {file = "cramjam-2.9.1-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:79417957972553502b217a0093532e48893c8b4ca30ccc941cefe9c72379df7c"},
- {file = "cramjam-2.9.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:ce2b94117f373defc876f88e74e44049a9969223dbca3240415b71752d0422fb"},
- {file = "cramjam-2.9.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:67040e0fd84404885ec716a806bee6110f9960c3647e0ef1670aab3b7375a70a"},
- {file = "cramjam-2.9.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0bedb84e068b53c944bd08dcb501fd00d67daa8a917922356dd559b484ce7eab"},
- {file = "cramjam-2.9.1-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:06e3f97a379386d97debf08638a78b3d3850fdf6124755eb270b54905a169930"},
- {file = "cramjam-2.9.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:11118675e9c7952ececabc62f023290ee4f8ecf0bee0d2c7eb8d1c402ee9769d"},
- {file = "cramjam-2.9.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6b7de6b61b11545570e4d6033713f3599525efc615ee353a822be8f6b0c65b77"},
- {file = "cramjam-2.9.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:57ca8f3775324a9de3ee6f05ca172687ba258c0dea79f7e3a6b4112834982f2a"},
- {file = "cramjam-2.9.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:9847dd6f288f1c56359f52acb48ff2df848ff3e3bff34d23855bbcf7016427cc"},
- {file = "cramjam-2.9.1-cp312-cp312-musllinux_1_1_armv7l.whl", hash = "sha256:8d1248dfa7f151e893ce819670f00879e4b7650b8d4c01279ce4f12140d68dd2"},
- {file = "cramjam-2.9.1-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:9da6d970281083bae91b914362de325414aa03c01fc806f6bb2cc006322ec834"},
- {file = "cramjam-2.9.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:1c33bc095db5733c841a102b8693062be5db8cdac17b9782ebc00577c6a94480"},
- {file = "cramjam-2.9.1-cp312-cp312-win32.whl", hash = "sha256:9e9193cd4bb57e7acd3af24891526299244bfed88168945efdaa09af4e50720f"},
- {file = "cramjam-2.9.1-cp312-cp312-win_amd64.whl", hash = "sha256:15955dd75e80f66c1ea271167a5347661d9bdc365f894a57698c383c9b7d465c"},
- {file = "cramjam-2.9.1-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:5a7797a2fff994fc5e323f7a967a35a3e37e3006ed21d64dcded086502f482af"},
- {file = "cramjam-2.9.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:d51b9b140b1df39a44bff7896d98a10da345b7d5f5ce92368d328c1c2c829167"},
- {file = "cramjam-2.9.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:07ac76b7f992556e7aa910244be11ece578cdf84f4d5d5297461f9a895e18312"},
- {file = "cramjam-2.9.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d90a72608c7550cd7eba914668f6277bfb0b24f074d1f1bd9d061fcb6f2adbd6"},
- {file = "cramjam-2.9.1-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:56495975401b1821dbe1f29cf222e23556232209a2fdb809fe8156d120ca9c7f"},
- {file = "cramjam-2.9.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3b695259e71fde6d5be66b77a4474523ced9ffe9fe8a34cb9b520ec1241a14d3"},
- {file = "cramjam-2.9.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ab1e69dc4831bbb79b6d547077aae89074c83e8ad94eba1a3d80e94d2424fd02"},
- {file = "cramjam-2.9.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:440b489902bfb7a26d3fec1ca888007615336ff763d2a32a2fc40586548a0dbf"},
- {file = "cramjam-2.9.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:217fe22b41f8c3dce03852f828b059abfad11d1344a1df2f43d3eb8634b18d75"},
- {file = "cramjam-2.9.1-cp313-cp313-musllinux_1_1_armv7l.whl", hash = "sha256:95f3646ddc98af25af25d5692ae65966488a283813336ea9cf41b22e542e7c0d"},
- {file = "cramjam-2.9.1-cp313-cp313-musllinux_1_1_i686.whl", hash = "sha256:6b19fc60ead1cae9795a5b359599da3a1c95d38f869bdfb51c441fd76b04e926"},
- {file = "cramjam-2.9.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:8dc5207567459d049696f62a1fdfb220f3fe6aa0d722285d44753e12504dac6c"},
- {file = "cramjam-2.9.1-cp313-cp313-win32.whl", hash = "sha256:fbfe35929a61b914de9e5dbacde0cfbba86cbf5122f9285a24c14ed0b645490b"},
- {file = "cramjam-2.9.1-cp313-cp313-win_amd64.whl", hash = "sha256:06068bd191a82ad4fc1ac23d6f8627fb5e37ec4be0431711b9a2dbacaccfeddb"},
- {file = "cramjam-2.9.1-cp38-cp38-macosx_10_12_x86_64.whl", hash = "sha256:6a2ca4d3c683d28d3217821029eb08d3487d5043d7eb455df11ff3cacfd4c916"},
- {file = "cramjam-2.9.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:008b49b455b396acc5459dfb06fb9d56049c4097ee8e590892a4d3da9a711da3"},
- {file = "cramjam-2.9.1-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:45c18cc13156e8697a8d3f9e57e49a69b00e14a103196efab0893fae1a5257f8"},
- {file = "cramjam-2.9.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d14a0efb21e0fec0631bcd66040b06e6a0fe10825f3aacffded38c1c978bdff9"},
- {file = "cramjam-2.9.1-cp38-cp38-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:3f815fb0eba625af45139af4f90f5fc2ddda61b171c2cc3ab63d44b40c5c7768"},
- {file = "cramjam-2.9.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:04828cbfad7384f06a4a7d0d927c3e85ef11dc5a40b9cf5f3e29ac4e23ecd678"},
- {file = "cramjam-2.9.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b0944a7c3a78f940c06d1b29bdce91a17798d80593dd01ebfeb842761e48a8b5"},
- {file = "cramjam-2.9.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ec769e5b16251704502277a1163dcf2611551452d7590ff4cc422b7b0367fc96"},
- {file = "cramjam-2.9.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:3ba79c7d2cc5adb897b690c05dd9b67c4d401736d207314b99315f7be3cd94fd"},
- {file = "cramjam-2.9.1-cp38-cp38-musllinux_1_1_armv7l.whl", hash = "sha256:d35923fb5411bde30b53c0696dff8e24c8a38b010b89544834c53f4462fd71df"},
- {file = "cramjam-2.9.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:da0cc0efdbfb8ee2361f89f38ded03d11678f37e392afff7a97b09c55dadfc83"},
- {file = "cramjam-2.9.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:f89924858712b8b936f04f3d690e72825a3e5127a140b434c79030c1c5a887ce"},
- {file = "cramjam-2.9.1-cp38-cp38-win32.whl", hash = "sha256:5925a738b8478f223ab9756fc794e3cabd5917fd7846f66adcf1d5fc2bf9864c"},
- {file = "cramjam-2.9.1-cp38-cp38-win_amd64.whl", hash = "sha256:b7ac273498a2c6772d67707e101b74014c0d9413bb4711c51d8ec311de59b4b1"},
- {file = "cramjam-2.9.1-cp39-cp39-macosx_10_12_x86_64.whl", hash = "sha256:af39006faddfc6253beb93ca821d544931cfee7f0177b99ff106dfd8fd6a2cd8"},
- {file = "cramjam-2.9.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:b3291be0d3f73d5774d69013be4ab33978c777363b5312d14f62f77817c2f75a"},
- {file = "cramjam-2.9.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:1539fd758f0e57fad7913cebff8baaee871bb561ddf6fa710a427b74da6b6778"},
- {file = "cramjam-2.9.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ff362f68bd68ac0eccb445209238d589bba728fb6d7f2e9dc199e0ec3a61d6e0"},
- {file = "cramjam-2.9.1-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:23b9786d1d17686fb8d600ade2a19374c7188d4b8867efa9af0d8274a220aec7"},
- {file = "cramjam-2.9.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8bc9c2c748aaf91863d89c4583f529c1c709485c94f8dfeb3ee48662d88e3258"},
- {file = "cramjam-2.9.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fd0fa9a0e7f18224b6d2d1d69dbdc3aecec80ef1393c59244159b131604a4395"},
- {file = "cramjam-2.9.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2ceef6e09ee22457997370882aa3c69de01e6dd0aaa2f953e1e87ad11641d042"},
- {file = "cramjam-2.9.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:1376f6fdbf0b30712413a0b4e51663a4938ae2f6b449f8e4635dbb3694db83cf"},
- {file = "cramjam-2.9.1-cp39-cp39-musllinux_1_1_armv7l.whl", hash = "sha256:342fb946f8d3e9e35b837288b03ab23cfbe0bb5a30e582ed805ef79706823a96"},
- {file = "cramjam-2.9.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:a237064a6e2c2256c9a1cf2beb7c971382190c0f1eb2e810e02e971881756132"},
- {file = "cramjam-2.9.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:53145fc9f2319c1245d4329e1da8cfacd6e35e27090c07c0b9d453ae2bbdac3e"},
- {file = "cramjam-2.9.1-cp39-cp39-win32.whl", hash = "sha256:8a9f52c27292c21457f43c4ce124939302a9acfb62295e7cda8667310563a5a3"},
- {file = "cramjam-2.9.1-cp39-cp39-win_amd64.whl", hash = "sha256:8097ee39b61c86848a443c0b25b2df1de6b331fd512b20836a4f5cfde51ab255"},
- {file = "cramjam-2.9.1-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:86824c695688fcd06c5ac9bbd3fea9bdfb4cca194b1e706fbf11a629df48d2b4"},
- {file = "cramjam-2.9.1-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:27571bfa5a5d618604696747d0dc1d2a99b5906c967c8dee53c13a7107edfde6"},
- {file = "cramjam-2.9.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fb01f6e38719818778144d3165a89ea1ad9dc58c6342b7f20aa194c70f34cbd1"},
- {file = "cramjam-2.9.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b5cef5cf40725fe64592af9ec163e7389855077700678a1d94bec549403a74d"},
- {file = "cramjam-2.9.1-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:ac48b978aa0675f62b642750e798c394a64d25ce852e4e541f69bef9a564c2f0"},
- {file = "cramjam-2.9.1.tar.gz", hash = "sha256:336cc591d86cbd225d256813779f46624f857bc9c779db126271eff9ddc524ae"},
+ {file = "cramjam-2.10.0-cp310-cp310-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:26c44f17938cf00a339899ce6ea7ba12af7b1210d707a80a7f14724fba39869b"},
+ {file = "cramjam-2.10.0-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:ce208a3e4043b8ce89e5d90047da16882456ea395577b1ee07e8215dce7d7c91"},
+ {file = "cramjam-2.10.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2c24907c972aca7b56c8326307e15d78f56199852dda1e67e4e54c2672afede4"},
+ {file = "cramjam-2.10.0-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:f25db473667774725e4f34e738d644ffb205bf0bdc0e8146870a1104c5f42e4a"},
+ {file = "cramjam-2.10.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:51eb00c72d4a93e4a2ddcc751ba2a7a1318026247e80742866912ec82b39e5ce"},
+ {file = "cramjam-2.10.0-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:def47645b1b970fd97f063da852b0ddc4f5bdee9af8d5b718d9682c7b828d89d"},
+ {file = "cramjam-2.10.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:42dcd7c83104edae70004a8dc494e4e57de4940e3019e5d2cbec2830d5908a85"},
+ {file = "cramjam-2.10.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e0744e391ea8baf0ddea5a180b0aa71a6a302490c14d7a37add730bf0172c7c6"},
+ {file = "cramjam-2.10.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5018c7414047f640b126df02e9286a8da7cc620798cea2b39bac79731c2ee336"},
+ {file = "cramjam-2.10.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:4b201aacc7a06079b063cfbcf5efe78b1e65c7279b2828d06ffaa90a8316579d"},
+ {file = "cramjam-2.10.0-cp310-cp310-musllinux_1_1_armv7l.whl", hash = "sha256:5264ac242697fbb1cfffa79d0153cbc4c088538bd99d60cfa374e8a8b83e2bb5"},
+ {file = "cramjam-2.10.0-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:e193918c81139361f3f45db19696d31847601f2c0e79a38618f34d7bff6ee704"},
+ {file = "cramjam-2.10.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:22a7ab05c62b0a71fcd6db4274af1508c5ea039a43fb143ac50a62f86e6f32f7"},
+ {file = "cramjam-2.10.0-cp310-cp310-win32.whl", hash = "sha256:2464bdf0e2432e0f07a834f48c16022cd7f4648ed18badf52c32c13d6722518c"},
+ {file = "cramjam-2.10.0-cp310-cp310-win_amd64.whl", hash = "sha256:73b6ffc8ffe6546462ccc7e34ca3acd9eb3984e1232645f498544a7eab6b8aca"},
+ {file = "cramjam-2.10.0-cp311-cp311-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:fb73ee9616e3efd2cf3857b019c66f9bf287bb47139ea48425850da2ae508670"},
+ {file = "cramjam-2.10.0-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:acef0e2c4d9f38428721a0ec878dee3fb73a35e640593d99c9803457dbb65214"},
+ {file = "cramjam-2.10.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5b21b1672814ecce88f1da76635f0483d2d877d4cb8998db3692792f46279bf1"},
+ {file = "cramjam-2.10.0-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:7699d61c712bc77907c48fe63a21fffa03c4dd70401e1d14e368af031fde7c21"},
+ {file = "cramjam-2.10.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3484f1595eef64cefed05804d7ec8a88695f89086c49b086634e44c16f3d4769"},
+ {file = "cramjam-2.10.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:38fba4594dd0e2b7423ef403039e63774086ebb0696d9060db20093f18a2f43e"},
+ {file = "cramjam-2.10.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b07fe3e48c881a75a11f722e1d5b052173b5e7c78b22518f659b8c9b4ac4c937"},
+ {file = "cramjam-2.10.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3596b6ceaf85f872c1e56295c6ec80bb15fdd71e7ed9e0e5c3e654563dcc40a2"},
+ {file = "cramjam-2.10.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e1c03360c1760f8608dc5ce1ddd7e5491180765360cae8104b428d5f86fbe1b9"},
+ {file = "cramjam-2.10.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:3e0b70fe7796b63b87cb7ebfaad0ebaca7574fdf177311952f74b8bda6522fb8"},
+ {file = "cramjam-2.10.0-cp311-cp311-musllinux_1_1_armv7l.whl", hash = "sha256:d61a21e4153589bd53ffe71b553f93f2afbc8fb7baf63c91a83c933347473083"},
+ {file = "cramjam-2.10.0-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:91ab85752a08dc875a05742cfda0234d7a70fadda07dd0b0582cfe991911f332"},
+ {file = "cramjam-2.10.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:c6afff7e9da53afb8d11eae27a20ee5709e2943b39af6c949b38424d0f271569"},
+ {file = "cramjam-2.10.0-cp311-cp311-win32.whl", hash = "sha256:adf484b06063134ae604d4fc826d942af7e751c9d0b2fcab5bf1058a8ebe242b"},
+ {file = "cramjam-2.10.0-cp311-cp311-win_amd64.whl", hash = "sha256:9e20ebea6ec77232cd12e4084c8be6d03534dc5f3d027d365b32766beafce6c3"},
+ {file = "cramjam-2.10.0-cp312-cp312-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:0acb17e3681138b48300b27d3409742c81d5734ec39c650a60a764c135197840"},
+ {file = "cramjam-2.10.0-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:647553c44cf6b5ce2d9b56e743cc1eab886940d776b36438183e807bb5a7a42b"},
+ {file = "cramjam-2.10.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:5c52805c7ccb533fe42d3d36c91d237c97c3b6551cd6b32f98b79eeb30d0f139"},
+ {file = "cramjam-2.10.0-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:337ceb50bde7708b2a4068f3000625c23ceb1b2497edce2e21fd08ef58549170"},
+ {file = "cramjam-2.10.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1c071765bdd5eefa3b2157a61e84d72e161b63f95eb702a0133fee293800a619"},
+ {file = "cramjam-2.10.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8b40d46d2aa566f8e3def953279cce0191e47364b453cda492db12a84dd97f78"},
+ {file = "cramjam-2.10.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4c7bab3703babb93c9dd4444ac9797d01ec46cf521e247d3319bfb292414d053"},
+ {file = "cramjam-2.10.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ba19308b8e19cdaadfbf47142f52b705d2cbfb8edd84a8271573e50fa7fa022d"},
+ {file = "cramjam-2.10.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:de3e4be5aa71b73c2640c9b86e435ec033592f7f79787937f8342259106a63ae"},
+ {file = "cramjam-2.10.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:11c5ef0c70d6bdd8e1d8afed8b0430709b22decc3865eb6c0656aa00117a7b3d"},
+ {file = "cramjam-2.10.0-cp312-cp312-musllinux_1_1_armv7l.whl", hash = "sha256:86b29e349064821ceeb14d60d01a11a0788f94e73ed4b3a5c3f9fac7aa4e2cd7"},
+ {file = "cramjam-2.10.0-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:2c7008bb54bdc5d130c0e8581925dfcbdc6f0a4d2051de7a153bfced9a31910f"},
+ {file = "cramjam-2.10.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:3a94fe7024137ed8bf200308000d106874afe52ff203f852f43b3547eddfa10e"},
+ {file = "cramjam-2.10.0-cp312-cp312-win32.whl", hash = "sha256:ce11be5722c9d433c5e1eb3980f16eb7d80828b9614f089e28f4f1724fc8973f"},
+ {file = "cramjam-2.10.0-cp312-cp312-win_amd64.whl", hash = "sha256:a01e89e99ba066dfa2df40fe99a2371565f4a3adc6811a73c8019d9929a312e8"},
+ {file = "cramjam-2.10.0-cp313-cp313-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:8bb0b6aaaa5f37091e05d756a3337faf0ddcffe8a68dbe8a710731b0d555ec8f"},
+ {file = "cramjam-2.10.0-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:27b2625c0840b9a5522eba30b165940084391762492e03b9d640fca5074016ae"},
+ {file = "cramjam-2.10.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:4ba90f7b8f986934f33aad8cc029cf7c74842d3ecd5eda71f7531330d38a8dc4"},
+ {file = "cramjam-2.10.0-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:6655d04942f7c02087a6bba4bdc8d88961aa8ddf3fb9a05b3bad06d2d1ca321b"},
+ {file = "cramjam-2.10.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7dda9be2caf067ac21c4aa63497833e0984908b66849c07aaa42b1cfa93f5e1c"},
+ {file = "cramjam-2.10.0-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:afa36aa006d7692718fce427ecb276211918447f806f80c19096a627f5122e3d"},
+ {file = "cramjam-2.10.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d46fd5a9e8eb5d56eccc6191a55e3e1e2b3ab24b19ab87563a2299a39c855fd7"},
+ {file = "cramjam-2.10.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e3012564760394dff89e7a10c5a244f8885cd155aec07bdbe2d6dc46be398614"},
+ {file = "cramjam-2.10.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2d216ed4aca2090eabdd354204ae55ed3e13333d1a5b271981543696e634672"},
+ {file = "cramjam-2.10.0-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:44c2660ee7c4c269646955e4e40c2693f803fbad12398bb31b2ad00cfc6027b8"},
+ {file = "cramjam-2.10.0-cp313-cp313-musllinux_1_1_armv7l.whl", hash = "sha256:636a48e2d01fe8d7955e9523efd2f8efce55a0221f3b5d5b4bdf37c7ff056bf1"},
+ {file = "cramjam-2.10.0-cp313-cp313-musllinux_1_1_i686.whl", hash = "sha256:44c15f6117031a84497433b5f55d30ee72d438fdcba9778fec0c5ca5d416aa96"},
+ {file = "cramjam-2.10.0-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:76e4e42f2ecf1aca0a710adaa23000a192efb81a2aee3bcc16761f1777f08a74"},
+ {file = "cramjam-2.10.0-cp313-cp313-win32.whl", hash = "sha256:5b34f4678d386c64d3be402fdf67f75e8f1869627ea2ec4decd43e828d3b6fba"},
+ {file = "cramjam-2.10.0-cp313-cp313-win_amd64.whl", hash = "sha256:88754dd516f0e2f4dd242880b8e760dc854e917315a17fe3fc626475bea9b252"},
+ {file = "cramjam-2.10.0-cp38-cp38-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:645827af834a64145ba4b06f703342b2dbe1d40d1a48fb04e82373bd95cf68e2"},
+ {file = "cramjam-2.10.0-cp38-cp38-macosx_10_12_x86_64.whl", hash = "sha256:570c81f991033e624874475ade96b601f1db2c51b3e69c324072adcfb23ef5aa"},
+ {file = "cramjam-2.10.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:06ad4a8b368d30ded1d932d9eed647962fbe44923269185a6bbd5e0d11cc39ab"},
+ {file = "cramjam-2.10.0-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:bcedda2ef2560e6e62cac03734ab1ad28616206b4d4f2d138440b4f43e18c395"},
+ {file = "cramjam-2.10.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:68362d87372a90b9717536238c81d74d7feb4a14392ac239ceb61c1c199a9bac"},
+ {file = "cramjam-2.10.0-cp38-cp38-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ff7b95bd299c9360e7cb8d226002d58e2917f594ea5af0373efc713f896622b9"},
+ {file = "cramjam-2.10.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a2742eea6e336961167c5b6a2393fa04d54bdb10980f0d60ea36ed0a824e9a20"},
+ {file = "cramjam-2.10.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8695857e0b0b5289fabb6c200b95e2b18d8575551ddd9d50746b3d78b6fb5aa8"},
+ {file = "cramjam-2.10.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ac5a8a3ef660e6869a7761cd0664223eb546b2d17e9121c8ab0ad46353635611"},
+ {file = "cramjam-2.10.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:6d86c1e2006fe82a8679ed851c2462a6019b57255b3902d16ac35df4a37f6cdd"},
+ {file = "cramjam-2.10.0-cp38-cp38-musllinux_1_1_armv7l.whl", hash = "sha256:a094ca72440364bc1d0a793555875e515b0d7cc0eef171f4cd49c7e4855ba06e"},
+ {file = "cramjam-2.10.0-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:05793857773ec62101edf2c0d22d8edc955707727124f637d2f6cc138e5f97aa"},
+ {file = "cramjam-2.10.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:b8dee2e4a402dac2df110e7b02fae49507a63b44b6fd91350cf069f31545a925"},
+ {file = "cramjam-2.10.0-cp38-cp38-win32.whl", hash = "sha256:001fc2572adc655406fb899087f57a740e58a800b05acdccac8bf5759b617d90"},
+ {file = "cramjam-2.10.0-cp38-cp38-win_amd64.whl", hash = "sha256:9cadef44f5ad4c5b4d06ba3c28464d70241a40539c0343b1821ba43102b6a9fc"},
+ {file = "cramjam-2.10.0-cp39-cp39-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:967f5f0f22bf5dba4e4d7abe9594b28f5da95606225a50555926ff6e975d84dd"},
+ {file = "cramjam-2.10.0-cp39-cp39-macosx_10_12_x86_64.whl", hash = "sha256:260732e3b5c56d6182586f3a7fc5e3f3641b27bfbad5883e8d8e292af85a6870"},
+ {file = "cramjam-2.10.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:eafdc9d1721afcb4be9d20b980b61d404a592c19067197976a4077f52727bd1a"},
+ {file = "cramjam-2.10.0-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.whl", hash = "sha256:28a13c0317e71121b2059ffa8beefa2b185be241c52f740f6eb261f0067186db"},
+ {file = "cramjam-2.10.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4b3e0067ae3513e4cbd0efbabbe5a2bcfa2c2d4bddc67188eeb0751b9a02fdb7"},
+ {file = "cramjam-2.10.0-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:112638a4cdf806509d2d2661cb519d239d731bd5fd2e95f211c48ac0f0deeab5"},
+ {file = "cramjam-2.10.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7ddbf6a3d3def7ae46638ebf87d7746ccebf22f885a87884ac24d97943af3f30"},
+ {file = "cramjam-2.10.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a2923b8cd2fcbd22e0842decb66bf925a9e95bda165490d037c355e5df8fef68"},
+ {file = "cramjam-2.10.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7ab6f36c772109c974890eafff2a841ddbf38ea1293b01a778b28f26089a890d"},
+ {file = "cramjam-2.10.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:17dda15edf256362edb30dcb1d5ecdcd727d946c6be0d1b130e736f3f49487dc"},
+ {file = "cramjam-2.10.0-cp39-cp39-musllinux_1_1_armv7l.whl", hash = "sha256:92fd6e784ade210c3522bc627b3938821d12fac52acefe4d6630460e243e28de"},
+ {file = "cramjam-2.10.0-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:a120fc0514c9ed9a4051d040ddd36176241d4f54c4a37d8e4f3d29ac9bdb4c3a"},
+ {file = "cramjam-2.10.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:a71ab695a16c6d5aeae1f02fcc37fbd1ae876e8fb339337aca187012a3d6c0a2"},
+ {file = "cramjam-2.10.0-cp39-cp39-win32.whl", hash = "sha256:61b7f3c81e5e9015e73e5f423706b2f5e85a07ce79dea35645fad93505ff06cf"},
+ {file = "cramjam-2.10.0-cp39-cp39-win_amd64.whl", hash = "sha256:0d27fe3e316f9ae7fe1367b6daf0ffc993c1c66edae588165ac0f41f91a5a6b1"},
+ {file = "cramjam-2.10.0-pp310-pypy310_pp73-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:77192bc1a9897ecd91cf977a5d5f990373e35a8d028c9141c8c3d3680a4a4cd7"},
+ {file = "cramjam-2.10.0-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:50b59e981f219d6840ac43cda8e885aff1457944ddbabaa16ac047690bfd6ad1"},
+ {file = "cramjam-2.10.0-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:d84581c869d279fab437182d5db2b590d44975084e8d50b164947f7aaa2c5f25"},
+ {file = "cramjam-2.10.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:04f54bea9ce39c440d1ac6901fe4d647f9218dd5cd8fe903c6fe9c42bf5e1f3b"},
+ {file = "cramjam-2.10.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cddd12ee5a2ef4100478db7f5563a9cdb8bc0a067fbd8ccd1ecdc446d2e6a41a"},
+ {file = "cramjam-2.10.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:35bcecff38648908a4833928a892a1e7a32611171785bef27015107426bc1d9d"},
+ {file = "cramjam-2.10.0-pp311-pypy311_pp73-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:1e826469cfbb6dcd5b967591e52855073267835229674cfa3d327088805855da"},
+ {file = "cramjam-2.10.0-pp311-pypy311_pp73-macosx_10_12_x86_64.whl", hash = "sha256:1a200b74220dcd80c2bb99e3bfe1cdb1e4ed0f5c071959f4316abd65f9ef1e39"},
+ {file = "cramjam-2.10.0-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:2e419b65538786fc1f0cf776612262d4bf6c9449983d3fc0d0acfd86594fe551"},
+ {file = "cramjam-2.10.0-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bf1321a40da930edeff418d561dfb03e6d59d5b8ab5cbab1c4b03ff0aa4c6d21"},
+ {file = "cramjam-2.10.0-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a04376601c8f9714fb3a6a0a1699b85aab665d9d952a2a31fb37cf70e1be1fba"},
+ {file = "cramjam-2.10.0-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:2c1eb6e6c3d5c1cc3f7c7f8a52e034340a3c454641f019687fa94077c05da5c2"},
+ {file = "cramjam-2.10.0.tar.gz", hash = "sha256:e821dd487384ae8004e977c3b13135ad6665ccf8c9874e68441cad1146e66d8a"},
]
[package.extras]
-dev = ["black (==22.3.0)", "hypothesis", "numpy", "pytest (>=5.30)", "pytest-benchmark", "pytest-xdist"]
+dev = ["black (==22.3.0)", "hypothesis (<6.123.0)", "numpy", "pytest (>=5.30)", "pytest-benchmark", "pytest-xdist"]
[[package]]
name = "cryptography"
-version = "44.0.2"
+version = "45.0.4"
description = "cryptography is a package which provides cryptographic recipes and primitives to Python developers."
optional = false
python-versions = "!=3.9.0,!=3.9.1,>=3.7"
groups = ["main", "dev"]
files = [
- {file = "cryptography-44.0.2-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:efcfe97d1b3c79e486554efddeb8f6f53a4cdd4cf6086642784fa31fc384e1d7"},
- {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:29ecec49f3ba3f3849362854b7253a9f59799e3763b0c9d0826259a88efa02f1"},
- {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bc821e161ae88bfe8088d11bb39caf2916562e0a2dc7b6d56714a48b784ef0bb"},
- {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:3c00b6b757b32ce0f62c574b78b939afab9eecaf597c4d624caca4f9e71e7843"},
- {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:7bdcd82189759aba3816d1f729ce42ffded1ac304c151d0a8e89b9996ab863d5"},
- {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:4973da6ca3db4405c54cd0b26d328be54c7747e89e284fcff166132eb7bccc9c"},
- {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:4e389622b6927d8133f314949a9812972711a111d577a5d1f4bee5e58736b80a"},
- {file = "cryptography-44.0.2-cp37-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:f514ef4cd14bb6fb484b4a60203e912cfcb64f2ab139e88c2274511514bf7308"},
- {file = "cryptography-44.0.2-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:1bc312dfb7a6e5d66082c87c34c8a62176e684b6fe3d90fcfe1568de675e6688"},
- {file = "cryptography-44.0.2-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:3b721b8b4d948b218c88cb8c45a01793483821e709afe5f622861fc6182b20a7"},
- {file = "cryptography-44.0.2-cp37-abi3-win32.whl", hash = "sha256:51e4de3af4ec3899d6d178a8c005226491c27c4ba84101bfb59c901e10ca9f79"},
- {file = "cryptography-44.0.2-cp37-abi3-win_amd64.whl", hash = "sha256:c505d61b6176aaf982c5717ce04e87da5abc9a36a5b39ac03905c4aafe8de7aa"},
- {file = "cryptography-44.0.2-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:8e0ddd63e6bf1161800592c71ac794d3fb8001f2caebe0966e77c5234fa9efc3"},
- {file = "cryptography-44.0.2-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:81276f0ea79a208d961c433a947029e1a15948966658cf6710bbabb60fcc2639"},
- {file = "cryptography-44.0.2-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9a1e657c0f4ea2a23304ee3f964db058c9e9e635cc7019c4aa21c330755ef6fd"},
- {file = "cryptography-44.0.2-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:6210c05941994290f3f7f175a4a57dbbb2afd9273657614c506d5976db061181"},
- {file = "cryptography-44.0.2-cp39-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:d1c3572526997b36f245a96a2b1713bf79ce99b271bbcf084beb6b9b075f29ea"},
- {file = "cryptography-44.0.2-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:b042d2a275c8cee83a4b7ae30c45a15e6a4baa65a179a0ec2d78ebb90e4f6699"},
- {file = "cryptography-44.0.2-cp39-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:d03806036b4f89e3b13b6218fefea8d5312e450935b1a2d55f0524e2ed7c59d9"},
- {file = "cryptography-44.0.2-cp39-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:c7362add18b416b69d58c910caa217f980c5ef39b23a38a0880dfd87bdf8cd23"},
- {file = "cryptography-44.0.2-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:8cadc6e3b5a1f144a039ea08a0bdb03a2a92e19c46be3285123d32029f40a922"},
- {file = "cryptography-44.0.2-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:6f101b1f780f7fc613d040ca4bdf835c6ef3b00e9bd7125a4255ec574c7916e4"},
- {file = "cryptography-44.0.2-cp39-abi3-win32.whl", hash = "sha256:3dc62975e31617badc19a906481deacdeb80b4bb454394b4098e3f2525a488c5"},
- {file = "cryptography-44.0.2-cp39-abi3-win_amd64.whl", hash = "sha256:5f6f90b72d8ccadb9c6e311c775c8305381db88374c65fa1a68250aa8a9cb3a6"},
- {file = "cryptography-44.0.2-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:af4ff3e388f2fa7bff9f7f2b31b87d5651c45731d3e8cfa0944be43dff5cfbdb"},
- {file = "cryptography-44.0.2-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:0529b1d5a0105dd3731fa65680b45ce49da4d8115ea76e9da77a875396727b41"},
- {file = "cryptography-44.0.2-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:7ca25849404be2f8e4b3c59483d9d3c51298a22c1c61a0e84415104dacaf5562"},
- {file = "cryptography-44.0.2-pp310-pypy310_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:268e4e9b177c76d569e8a145a6939eca9a5fec658c932348598818acf31ae9a5"},
- {file = "cryptography-44.0.2-pp310-pypy310_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:9eb9d22b0a5d8fd9925a7764a054dca914000607dff201a24c791ff5c799e1fa"},
- {file = "cryptography-44.0.2-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:2bf7bf75f7df9715f810d1b038870309342bff3069c5bd8c6b96128cb158668d"},
- {file = "cryptography-44.0.2-pp311-pypy311_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:909c97ab43a9c0c0b0ada7a1281430e4e5ec0458e6d9244c0e821bbf152f061d"},
- {file = "cryptography-44.0.2-pp311-pypy311_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:96e7a5e9d6e71f9f4fca8eebfd603f8e86c5225bb18eb621b2c1e50b290a9471"},
- {file = "cryptography-44.0.2-pp311-pypy311_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:d1b3031093a366ac767b3feb8bcddb596671b3aaff82d4050f984da0c248b615"},
- {file = "cryptography-44.0.2-pp311-pypy311_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:04abd71114848aa25edb28e225ab5f268096f44cf0127f3d36975bdf1bdf3390"},
- {file = "cryptography-44.0.2.tar.gz", hash = "sha256:c63454aa261a0cf0c5b4718349629793e9e634993538db841165b3df74f37ec0"},
+ {file = "cryptography-45.0.4-cp311-abi3-macosx_10_9_universal2.whl", hash = "sha256:425a9a6ac2823ee6e46a76a21a4e8342d8fa5c01e08b823c1f19a8b74f096069"},
+ {file = "cryptography-45.0.4-cp311-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:680806cf63baa0039b920f4976f5f31b10e772de42f16310a6839d9f21a26b0d"},
+ {file = "cryptography-45.0.4-cp311-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:4ca0f52170e821bc8da6fc0cc565b7bb8ff8d90d36b5e9fdd68e8a86bdf72036"},
+ {file = "cryptography-45.0.4-cp311-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:f3fe7a5ae34d5a414957cc7f457e2b92076e72938423ac64d215722f6cf49a9e"},
+ {file = "cryptography-45.0.4-cp311-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:25eb4d4d3e54595dc8adebc6bbd5623588991d86591a78c2548ffb64797341e2"},
+ {file = "cryptography-45.0.4-cp311-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:ce1678a2ccbe696cf3af15a75bb72ee008d7ff183c9228592ede9db467e64f1b"},
+ {file = "cryptography-45.0.4-cp311-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:49fe9155ab32721b9122975e168a6760d8ce4cffe423bcd7ca269ba41b5dfac1"},
+ {file = "cryptography-45.0.4-cp311-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:2882338b2a6e0bd337052e8b9007ced85c637da19ef9ecaf437744495c8c2999"},
+ {file = "cryptography-45.0.4-cp311-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:23b9c3ea30c3ed4db59e7b9619272e94891f8a3a5591d0b656a7582631ccf750"},
+ {file = "cryptography-45.0.4-cp311-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:b0a97c927497e3bc36b33987abb99bf17a9a175a19af38a892dc4bbb844d7ee2"},
+ {file = "cryptography-45.0.4-cp311-abi3-win32.whl", hash = "sha256:e00a6c10a5c53979d6242f123c0a97cff9f3abed7f064fc412c36dc521b5f257"},
+ {file = "cryptography-45.0.4-cp311-abi3-win_amd64.whl", hash = "sha256:817ee05c6c9f7a69a16200f0c90ab26d23a87701e2a284bd15156783e46dbcc8"},
+ {file = "cryptography-45.0.4-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:964bcc28d867e0f5491a564b7debb3ffdd8717928d315d12e0d7defa9e43b723"},
+ {file = "cryptography-45.0.4-cp37-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:6a5bf57554e80f75a7db3d4b1dacaa2764611ae166ab42ea9a72bcdb5d577637"},
+ {file = "cryptography-45.0.4-cp37-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:46cf7088bf91bdc9b26f9c55636492c1cce3e7aaf8041bbf0243f5e5325cfb2d"},
+ {file = "cryptography-45.0.4-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:7bedbe4cc930fa4b100fc845ea1ea5788fcd7ae9562e669989c11618ae8d76ee"},
+ {file = "cryptography-45.0.4-cp37-abi3-manylinux_2_28_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:eaa3e28ea2235b33220b949c5a0d6cf79baa80eab2eb5607ca8ab7525331b9ff"},
+ {file = "cryptography-45.0.4-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:7ef2dde4fa9408475038fc9aadfc1fb2676b174e68356359632e980c661ec8f6"},
+ {file = "cryptography-45.0.4-cp37-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:6a3511ae33f09094185d111160fd192c67aa0a2a8d19b54d36e4c78f651dc5ad"},
+ {file = "cryptography-45.0.4-cp37-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:06509dc70dd71fa56eaa138336244e2fbaf2ac164fc9b5e66828fccfd2b680d6"},
+ {file = "cryptography-45.0.4-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:5f31e6b0a5a253f6aa49be67279be4a7e5a4ef259a9f33c69f7d1b1191939872"},
+ {file = "cryptography-45.0.4-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:944e9ccf67a9594137f942d5b52c8d238b1b4e46c7a0c2891b7ae6e01e7c80a4"},
+ {file = "cryptography-45.0.4-cp37-abi3-win32.whl", hash = "sha256:c22fe01e53dc65edd1945a2e6f0015e887f84ced233acecb64b4daadb32f5c97"},
+ {file = "cryptography-45.0.4-cp37-abi3-win_amd64.whl", hash = "sha256:627ba1bc94f6adf0b0a2e35d87020285ead22d9f648c7e75bb64f367375f3b22"},
+ {file = "cryptography-45.0.4-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:a77c6fb8d76e9c9f99f2f3437c1a4ac287b34eaf40997cfab1e9bd2be175ac39"},
+ {file = "cryptography-45.0.4-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:7aad98a25ed8ac917fdd8a9c1e706e5a0956e06c498be1f713b61734333a4507"},
+ {file = "cryptography-45.0.4-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:3530382a43a0e524bc931f187fc69ef4c42828cf7d7f592f7f249f602b5a4ab0"},
+ {file = "cryptography-45.0.4-pp310-pypy310_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:6b613164cb8425e2f8db5849ffb84892e523bf6d26deb8f9bb76ae86181fa12b"},
+ {file = "cryptography-45.0.4-pp310-pypy310_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:96d4819e25bf3b685199b304a0029ce4a3caf98947ce8a066c9137cc78ad2c58"},
+ {file = "cryptography-45.0.4-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:b97737a3ffbea79eebb062eb0d67d72307195035332501722a9ca86bab9e3ab2"},
+ {file = "cryptography-45.0.4-pp311-pypy311_pp73-macosx_10_9_x86_64.whl", hash = "sha256:4828190fb6c4bcb6ebc6331f01fe66ae838bb3bd58e753b59d4b22eb444b996c"},
+ {file = "cryptography-45.0.4-pp311-pypy311_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:03dbff8411206713185b8cebe31bc5c0eb544799a50c09035733716b386e61a4"},
+ {file = "cryptography-45.0.4-pp311-pypy311_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:51dfbd4d26172d31150d84c19bbe06c68ea4b7f11bbc7b3a5e146b367c311349"},
+ {file = "cryptography-45.0.4-pp311-pypy311_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:0339a692de47084969500ee455e42c58e449461e0ec845a34a6a9b9bf7df7fb8"},
+ {file = "cryptography-45.0.4-pp311-pypy311_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:0cf13c77d710131d33e63626bd55ae7c0efb701ebdc2b3a7952b9b23a0412862"},
+ {file = "cryptography-45.0.4-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:bbc505d1dc469ac12a0a064214879eac6294038d6b24ae9f71faae1448a9608d"},
+ {file = "cryptography-45.0.4.tar.gz", hash = "sha256:7405ade85c83c37682c8fe65554759800a4a8c54b2d96e0f8ad114d31b808d57"},
]
markers = {main = "extra == \"adlfs\""}
[package.dependencies]
-cffi = {version = ">=1.12", markers = "platform_python_implementation != \"PyPy\""}
+cffi = {version = ">=1.14", markers = "platform_python_implementation != \"PyPy\""}
[package.extras]
-docs = ["sphinx (>=5.3.0)", "sphinx-rtd-theme (>=3.0.0) ; python_version >= \"3.8\""]
+docs = ["sphinx (>=5.3.0)", "sphinx-inline-tabs ; python_full_version >= \"3.8.0\"", "sphinx-rtd-theme (>=3.0.0) ; python_full_version >= \"3.8.0\""]
docstest = ["pyenchant (>=3)", "readme-renderer (>=30.0)", "sphinxcontrib-spelling (>=7.3.1)"]
-nox = ["nox (>=2024.4.15)", "nox[uv] (>=2024.3.2) ; python_version >= \"3.8\""]
-pep8test = ["check-sdist ; python_version >= \"3.8\"", "click (>=8.0.1)", "mypy (>=1.4)", "ruff (>=0.3.6)"]
+nox = ["nox (>=2024.4.15)", "nox[uv] (>=2024.3.2) ; python_full_version >= \"3.8.0\""]
+pep8test = ["check-sdist ; python_full_version >= \"3.8.0\"", "click (>=8.0.1)", "mypy (>=1.4)", "ruff (>=0.3.6)"]
sdist = ["build (>=1.0.0)"]
ssh = ["bcrypt (>=3.1.5)"]
-test = ["certifi (>=2024)", "cryptography-vectors (==44.0.2)", "pretend (>=0.7)", "pytest (>=7.4.0)", "pytest-benchmark (>=4.0)", "pytest-cov (>=2.10.1)", "pytest-xdist (>=3.5.0)"]
+test = ["certifi (>=2024)", "cryptography-vectors (==45.0.4)", "pretend (>=0.7)", "pytest (>=7.4.0)", "pytest-benchmark (>=4.0)", "pytest-cov (>=2.10.1)", "pytest-xdist (>=3.5.0)"]
test-randomorder = ["pytest-randomly"]
[[package]]
@@ -1302,17 +1354,20 @@ files = [
[[package]]
name = "exceptiongroup"
-version = "1.2.2"
+version = "1.3.0"
description = "Backport of PEP 654 (exception groups)"
optional = false
python-versions = ">=3.7"
groups = ["dev"]
markers = "python_version <= \"3.10\""
files = [
- {file = "exceptiongroup-1.2.2-py3-none-any.whl", hash = "sha256:3111b9d131c238bec2f8f516e123e14ba243563fb135d3fe885990585aa7795b"},
- {file = "exceptiongroup-1.2.2.tar.gz", hash = "sha256:47c2edf7c6738fafb49fd34290706d1a1a2f4d1c6df275526b62cbb4aa5393cc"},
+ {file = "exceptiongroup-1.3.0-py3-none-any.whl", hash = "sha256:4d111e6e0c13d0644cad6ddaa7ed0261a0b36971f6d23e7ec9b4b9097da78a10"},
+ {file = "exceptiongroup-1.3.0.tar.gz", hash = "sha256:b241f5885f560bc56a59ee63ca4c6a8bfa46ae4ad651af316d4e81817bb9fd88"},
]
+[package.dependencies]
+typing-extensions = {version = ">=4.6.0", markers = "python_version < \"3.13\""}
+
[package.extras]
test = ["pytest (>=6)"]
@@ -1429,117 +1484,129 @@ Werkzeug = ">=0.7"
[[package]]
name = "frozenlist"
-version = "1.5.0"
+version = "1.7.0"
description = "A list-like structure which implements collections.abc.MutableSequence"
optional = true
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
-files = [
- {file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5b6a66c18b5b9dd261ca98dffcb826a525334b2f29e7caa54e182255c5f6a65a"},
- {file = "frozenlist-1.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d1b3eb7b05ea246510b43a7e53ed1653e55c2121019a97e60cad7efb881a97bb"},
- {file = "frozenlist-1.5.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:15538c0cbf0e4fa11d1e3a71f823524b0c46299aed6e10ebb4c2089abd8c3bec"},
- {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e79225373c317ff1e35f210dd5f1344ff31066ba8067c307ab60254cd3a78ad5"},
- {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9272fa73ca71266702c4c3e2d4a28553ea03418e591e377a03b8e3659d94fa76"},
- {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:498524025a5b8ba81695761d78c8dd7382ac0b052f34e66939c42df860b8ff17"},
- {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:92b5278ed9d50fe610185ecd23c55d8b307d75ca18e94c0e7de328089ac5dcba"},
- {file = "frozenlist-1.5.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7f3c8c1dacd037df16e85227bac13cca58c30da836c6f936ba1df0c05d046d8d"},
- {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:f2ac49a9bedb996086057b75bf93538240538c6d9b38e57c82d51f75a73409d2"},
- {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e66cc454f97053b79c2ab09c17fbe3c825ea6b4de20baf1be28919460dd7877f"},
- {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:5a3ba5f9a0dfed20337d3e966dc359784c9f96503674c2faf015f7fe8e96798c"},
- {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:6321899477db90bdeb9299ac3627a6a53c7399c8cd58d25da094007402b039ab"},
- {file = "frozenlist-1.5.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:76e4753701248476e6286f2ef492af900ea67d9706a0155335a40ea21bf3b2f5"},
- {file = "frozenlist-1.5.0-cp310-cp310-win32.whl", hash = "sha256:977701c081c0241d0955c9586ffdd9ce44f7a7795df39b9151cd9a6fd0ce4cfb"},
- {file = "frozenlist-1.5.0-cp310-cp310-win_amd64.whl", hash = "sha256:189f03b53e64144f90990d29a27ec4f7997d91ed3d01b51fa39d2dbe77540fd4"},
- {file = "frozenlist-1.5.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:fd74520371c3c4175142d02a976aee0b4cb4a7cc912a60586ffd8d5929979b30"},
- {file = "frozenlist-1.5.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:2f3f7a0fbc219fb4455264cae4d9f01ad41ae6ee8524500f381de64ffaa077d5"},
- {file = "frozenlist-1.5.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:f47c9c9028f55a04ac254346e92977bf0f166c483c74b4232bee19a6697e4778"},
- {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0996c66760924da6e88922756d99b47512a71cfd45215f3570bf1e0b694c206a"},
- {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a2fe128eb4edeabe11896cb6af88fca5346059f6c8d807e3b910069f39157869"},
- {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1a8ea951bbb6cacd492e3948b8da8c502a3f814f5d20935aae74b5df2b19cf3d"},
- {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:de537c11e4aa01d37db0d403b57bd6f0546e71a82347a97c6a9f0dcc532b3a45"},
- {file = "frozenlist-1.5.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9c2623347b933fcb9095841f1cc5d4ff0b278addd743e0e966cb3d460278840d"},
- {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:cee6798eaf8b1416ef6909b06f7dc04b60755206bddc599f52232606e18179d3"},
- {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:f5f9da7f5dbc00a604fe74aa02ae7c98bcede8a3b8b9666f9f86fc13993bc71a"},
- {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:90646abbc7a5d5c7c19461d2e3eeb76eb0b204919e6ece342feb6032c9325ae9"},
- {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:bdac3c7d9b705d253b2ce370fde941836a5f8b3c5c2b8fd70940a3ea3af7f4f2"},
- {file = "frozenlist-1.5.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:03d33c2ddbc1816237a67f66336616416e2bbb6beb306e5f890f2eb22b959cdf"},
- {file = "frozenlist-1.5.0-cp311-cp311-win32.whl", hash = "sha256:237f6b23ee0f44066219dae14c70ae38a63f0440ce6750f868ee08775073f942"},
- {file = "frozenlist-1.5.0-cp311-cp311-win_amd64.whl", hash = "sha256:0cc974cc93d32c42e7b0f6cf242a6bd941c57c61b618e78b6c0a96cb72788c1d"},
- {file = "frozenlist-1.5.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:31115ba75889723431aa9a4e77d5f398f5cf976eea3bdf61749731f62d4a4a21"},
- {file = "frozenlist-1.5.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:7437601c4d89d070eac8323f121fcf25f88674627505334654fd027b091db09d"},
- {file = "frozenlist-1.5.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:7948140d9f8ece1745be806f2bfdf390127cf1a763b925c4a805c603df5e697e"},
- {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:feeb64bc9bcc6b45c6311c9e9b99406660a9c05ca8a5b30d14a78555088b0b3a"},
- {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:683173d371daad49cffb8309779e886e59c2f369430ad28fe715f66d08d4ab1a"},
- {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7d57d8f702221405a9d9b40f9da8ac2e4a1a8b5285aac6100f3393675f0a85ee"},
- {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:30c72000fbcc35b129cb09956836c7d7abf78ab5416595e4857d1cae8d6251a6"},
- {file = "frozenlist-1.5.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:000a77d6034fbad9b6bb880f7ec073027908f1b40254b5d6f26210d2dab1240e"},
- {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:5d7f5a50342475962eb18b740f3beecc685a15b52c91f7d975257e13e029eca9"},
- {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:87f724d055eb4785d9be84e9ebf0f24e392ddfad00b3fe036e43f489fafc9039"},
- {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:6e9080bb2fb195a046e5177f10d9d82b8a204c0736a97a153c2466127de87784"},
- {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9b93d7aaa36c966fa42efcaf716e6b3900438632a626fb09c049f6a2f09fc631"},
- {file = "frozenlist-1.5.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:52ef692a4bc60a6dd57f507429636c2af8b6046db8b31b18dac02cbc8f507f7f"},
- {file = "frozenlist-1.5.0-cp312-cp312-win32.whl", hash = "sha256:29d94c256679247b33a3dc96cce0f93cbc69c23bf75ff715919332fdbb6a32b8"},
- {file = "frozenlist-1.5.0-cp312-cp312-win_amd64.whl", hash = "sha256:8969190d709e7c48ea386db202d708eb94bdb29207a1f269bab1196ce0dcca1f"},
- {file = "frozenlist-1.5.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:7a1a048f9215c90973402e26c01d1cff8a209e1f1b53f72b95c13db61b00f953"},
- {file = "frozenlist-1.5.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:dd47a5181ce5fcb463b5d9e17ecfdb02b678cca31280639255ce9d0e5aa67af0"},
- {file = "frozenlist-1.5.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:1431d60b36d15cda188ea222033eec8e0eab488f39a272461f2e6d9e1a8e63c2"},
- {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6482a5851f5d72767fbd0e507e80737f9c8646ae7fd303def99bfe813f76cf7f"},
- {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:44c49271a937625619e862baacbd037a7ef86dd1ee215afc298a417ff3270608"},
- {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:12f78f98c2f1c2429d42e6a485f433722b0061d5c0b0139efa64f396efb5886b"},
- {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ce3aa154c452d2467487765e3adc730a8c153af77ad84096bc19ce19a2400840"},
- {file = "frozenlist-1.5.0-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9b7dc0c4338e6b8b091e8faf0db3168a37101943e687f373dce00959583f7439"},
- {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:45e0896250900b5aa25180f9aec243e84e92ac84bd4a74d9ad4138ef3f5c97de"},
- {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:561eb1c9579d495fddb6da8959fd2a1fca2c6d060d4113f5844b433fc02f2641"},
- {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:df6e2f325bfee1f49f81aaac97d2aa757c7646534a06f8f577ce184afe2f0a9e"},
- {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:140228863501b44b809fb39ec56b5d4071f4d0aa6d216c19cbb08b8c5a7eadb9"},
- {file = "frozenlist-1.5.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:7707a25d6a77f5d27ea7dc7d1fc608aa0a478193823f88511ef5e6b8a48f9d03"},
- {file = "frozenlist-1.5.0-cp313-cp313-win32.whl", hash = "sha256:31a9ac2b38ab9b5a8933b693db4939764ad3f299fcaa931a3e605bc3460e693c"},
- {file = "frozenlist-1.5.0-cp313-cp313-win_amd64.whl", hash = "sha256:11aabdd62b8b9c4b84081a3c246506d1cddd2dd93ff0ad53ede5defec7886b28"},
- {file = "frozenlist-1.5.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:dd94994fc91a6177bfaafd7d9fd951bc8689b0a98168aa26b5f543868548d3ca"},
- {file = "frozenlist-1.5.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:2d0da8bbec082bf6bf18345b180958775363588678f64998c2b7609e34719b10"},
- {file = "frozenlist-1.5.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:73f2e31ea8dd7df61a359b731716018c2be196e5bb3b74ddba107f694fbd7604"},
- {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:828afae9f17e6de596825cf4228ff28fbdf6065974e5ac1410cecc22f699d2b3"},
- {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f1577515d35ed5649d52ab4319db757bb881ce3b2b796d7283e6634d99ace307"},
- {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2150cc6305a2c2ab33299453e2968611dacb970d2283a14955923062c8d00b10"},
- {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a72b7a6e3cd2725eff67cd64c8f13335ee18fc3c7befc05aed043d24c7b9ccb9"},
- {file = "frozenlist-1.5.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c16d2fa63e0800723139137d667e1056bee1a1cf7965153d2d104b62855e9b99"},
- {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:17dcc32fc7bda7ce5875435003220a457bcfa34ab7924a49a1c19f55b6ee185c"},
- {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:97160e245ea33d8609cd2b8fd997c850b56db147a304a262abc2b3be021a9171"},
- {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:f1e6540b7fa044eee0bb5111ada694cf3dc15f2b0347ca125ee9ca984d5e9e6e"},
- {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:91d6c171862df0a6c61479d9724f22efb6109111017c87567cfeb7b5d1449fdf"},
- {file = "frozenlist-1.5.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:c1fac3e2ace2eb1052e9f7c7db480818371134410e1f5c55d65e8f3ac6d1407e"},
- {file = "frozenlist-1.5.0-cp38-cp38-win32.whl", hash = "sha256:b97f7b575ab4a8af9b7bc1d2ef7f29d3afee2226bd03ca3875c16451ad5a7723"},
- {file = "frozenlist-1.5.0-cp38-cp38-win_amd64.whl", hash = "sha256:374ca2dabdccad8e2a76d40b1d037f5bd16824933bf7bcea3e59c891fd4a0923"},
- {file = "frozenlist-1.5.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:9bbcdfaf4af7ce002694a4e10a0159d5a8d20056a12b05b45cea944a4953f972"},
- {file = "frozenlist-1.5.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:1893f948bf6681733aaccf36c5232c231e3b5166d607c5fa77773611df6dc336"},
- {file = "frozenlist-1.5.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:2b5e23253bb709ef57a8e95e6ae48daa9ac5f265637529e4ce6b003a37b2621f"},
- {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0f253985bb515ecd89629db13cb58d702035ecd8cfbca7d7a7e29a0e6d39af5f"},
- {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:04a5c6babd5e8fb7d3c871dc8b321166b80e41b637c31a995ed844a6139942b6"},
- {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a9fe0f1c29ba24ba6ff6abf688cb0b7cf1efab6b6aa6adc55441773c252f7411"},
- {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:226d72559fa19babe2ccd920273e767c96a49b9d3d38badd7c91a0fdeda8ea08"},
- {file = "frozenlist-1.5.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:15b731db116ab3aedec558573c1a5eec78822b32292fe4f2f0345b7f697745c2"},
- {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:366d8f93e3edfe5a918c874702f78faac300209a4d5bf38352b2c1bdc07a766d"},
- {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:1b96af8c582b94d381a1c1f51ffaedeb77c821c690ea5f01da3d70a487dd0a9b"},
- {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:c03eff4a41bd4e38415cbed054bbaff4a075b093e2394b6915dca34a40d1e38b"},
- {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:50cf5e7ee9b98f22bdecbabf3800ae78ddcc26e4a435515fc72d97903e8488e0"},
- {file = "frozenlist-1.5.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1e76bfbc72353269c44e0bc2cfe171900fbf7f722ad74c9a7b638052afe6a00c"},
- {file = "frozenlist-1.5.0-cp39-cp39-win32.whl", hash = "sha256:666534d15ba8f0fda3f53969117383d5dc021266b3c1a42c9ec4855e4b58b9d3"},
- {file = "frozenlist-1.5.0-cp39-cp39-win_amd64.whl", hash = "sha256:5c28f4b5dbef8a0d8aad0d4de24d1e9e981728628afaf4ea0792f5d0939372f0"},
- {file = "frozenlist-1.5.0-py3-none-any.whl", hash = "sha256:d994863bba198a4a518b467bb971c56e1db3f180a25c6cf7bb1949c267f748c3"},
- {file = "frozenlist-1.5.0.tar.gz", hash = "sha256:81d5af29e61b9c8348e876d442253723928dce6433e0e76cd925cd83f1b4b817"},
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+files = [
+ {file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cc4df77d638aa2ed703b878dd093725b72a824c3c546c076e8fdf276f78ee84a"},
+ {file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:716a9973a2cc963160394f701964fe25012600f3d311f60c790400b00e568b61"},
+ {file = "frozenlist-1.7.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a0fd1bad056a3600047fb9462cff4c5322cebc59ebf5d0a3725e0ee78955001d"},
+ {file = "frozenlist-1.7.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3789ebc19cb811163e70fe2bd354cea097254ce6e707ae42e56f45e31e96cb8e"},
+ {file = "frozenlist-1.7.0-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:af369aa35ee34f132fcfad5be45fbfcde0e3a5f6a1ec0712857f286b7d20cca9"},
+ {file = "frozenlist-1.7.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ac64b6478722eeb7a3313d494f8342ef3478dff539d17002f849101b212ef97c"},
+ {file = "frozenlist-1.7.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f89f65d85774f1797239693cef07ad4c97fdd0639544bad9ac4b869782eb1981"},
+ {file = "frozenlist-1.7.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1073557c941395fdfcfac13eb2456cb8aad89f9de27bae29fabca8e563b12615"},
+ {file = "frozenlist-1.7.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1ed8d2fa095aae4bdc7fdd80351009a48d286635edffee66bf865e37a9125c50"},
+ {file = "frozenlist-1.7.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:24c34bea555fe42d9f928ba0a740c553088500377448febecaa82cc3e88aa1fa"},
+ {file = "frozenlist-1.7.0-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:69cac419ac6a6baad202c85aaf467b65ac860ac2e7f2ac1686dc40dbb52f6577"},
+ {file = "frozenlist-1.7.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:960d67d0611f4c87da7e2ae2eacf7ea81a5be967861e0c63cf205215afbfac59"},
+ {file = "frozenlist-1.7.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:41be2964bd4b15bf575e5daee5a5ce7ed3115320fb3c2b71fca05582ffa4dc9e"},
+ {file = "frozenlist-1.7.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:46d84d49e00c9429238a7ce02dc0be8f6d7cd0cd405abd1bebdc991bf27c15bd"},
+ {file = "frozenlist-1.7.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:15900082e886edb37480335d9d518cec978afc69ccbc30bd18610b7c1b22a718"},
+ {file = "frozenlist-1.7.0-cp310-cp310-win32.whl", hash = "sha256:400ddd24ab4e55014bba442d917203c73b2846391dd42ca5e38ff52bb18c3c5e"},
+ {file = "frozenlist-1.7.0-cp310-cp310-win_amd64.whl", hash = "sha256:6eb93efb8101ef39d32d50bce242c84bcbddb4f7e9febfa7b524532a239b4464"},
+ {file = "frozenlist-1.7.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:aa51e147a66b2d74de1e6e2cf5921890de6b0f4820b257465101d7f37b49fb5a"},
+ {file = "frozenlist-1.7.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:9b35db7ce1cd71d36ba24f80f0c9e7cff73a28d7a74e91fe83e23d27c7828750"},
+ {file = "frozenlist-1.7.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:34a69a85e34ff37791e94542065c8416c1afbf820b68f720452f636d5fb990cd"},
+ {file = "frozenlist-1.7.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4a646531fa8d82c87fe4bb2e596f23173caec9185bfbca5d583b4ccfb95183e2"},
+ {file = "frozenlist-1.7.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:79b2ffbba483f4ed36a0f236ccb85fbb16e670c9238313709638167670ba235f"},
+ {file = "frozenlist-1.7.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a26f205c9ca5829cbf82bb2a84b5c36f7184c4316617d7ef1b271a56720d6b30"},
+ {file = "frozenlist-1.7.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bcacfad3185a623fa11ea0e0634aac7b691aa925d50a440f39b458e41c561d98"},
+ {file = "frozenlist-1.7.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:72c1b0fe8fe451b34f12dce46445ddf14bd2a5bcad7e324987194dc8e3a74c86"},
+ {file = "frozenlist-1.7.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:61d1a5baeaac6c0798ff6edfaeaa00e0e412d49946c53fae8d4b8e8b3566c4ae"},
+ {file = "frozenlist-1.7.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:7edf5c043c062462f09b6820de9854bf28cc6cc5b6714b383149745e287181a8"},
+ {file = "frozenlist-1.7.0-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:d50ac7627b3a1bd2dcef6f9da89a772694ec04d9a61b66cf87f7d9446b4a0c31"},
+ {file = "frozenlist-1.7.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:ce48b2fece5aeb45265bb7a58259f45027db0abff478e3077e12b05b17fb9da7"},
+ {file = "frozenlist-1.7.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:fe2365ae915a1fafd982c146754e1de6ab3478def8a59c86e1f7242d794f97d5"},
+ {file = "frozenlist-1.7.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:45a6f2fdbd10e074e8814eb98b05292f27bad7d1883afbe009d96abdcf3bc898"},
+ {file = "frozenlist-1.7.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:21884e23cffabb157a9dd7e353779077bf5b8f9a58e9b262c6caad2ef5f80a56"},
+ {file = "frozenlist-1.7.0-cp311-cp311-win32.whl", hash = "sha256:284d233a8953d7b24f9159b8a3496fc1ddc00f4db99c324bd5fb5f22d8698ea7"},
+ {file = "frozenlist-1.7.0-cp311-cp311-win_amd64.whl", hash = "sha256:387cbfdcde2f2353f19c2f66bbb52406d06ed77519ac7ee21be0232147c2592d"},
+ {file = "frozenlist-1.7.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:3dbf9952c4bb0e90e98aec1bd992b3318685005702656bc6f67c1a32b76787f2"},
+ {file = "frozenlist-1.7.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:1f5906d3359300b8a9bb194239491122e6cf1444c2efb88865426f170c262cdb"},
+ {file = "frozenlist-1.7.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:3dabd5a8f84573c8d10d8859a50ea2dec01eea372031929871368c09fa103478"},
+ {file = "frozenlist-1.7.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aa57daa5917f1738064f302bf2626281a1cb01920c32f711fbc7bc36111058a8"},
+ {file = "frozenlist-1.7.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:c193dda2b6d49f4c4398962810fa7d7c78f032bf45572b3e04dd5249dff27e08"},
+ {file = "frozenlist-1.7.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:bfe2b675cf0aaa6d61bf8fbffd3c274b3c9b7b1623beb3809df8a81399a4a9c4"},
+ {file = "frozenlist-1.7.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8fc5d5cda37f62b262405cf9652cf0856839c4be8ee41be0afe8858f17f4c94b"},
+ {file = "frozenlist-1.7.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b0d5ce521d1dd7d620198829b87ea002956e4319002ef0bc8d3e6d045cb4646e"},
+ {file = "frozenlist-1.7.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:488d0a7d6a0008ca0db273c542098a0fa9e7dfaa7e57f70acef43f32b3f69dca"},
+ {file = "frozenlist-1.7.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:15a7eaba63983d22c54d255b854e8108e7e5f3e89f647fc854bd77a237e767df"},
+ {file = "frozenlist-1.7.0-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:1eaa7e9c6d15df825bf255649e05bd8a74b04a4d2baa1ae46d9c2d00b2ca2cb5"},
+ {file = "frozenlist-1.7.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:e4389e06714cfa9d47ab87f784a7c5be91d3934cd6e9a7b85beef808297cc025"},
+ {file = "frozenlist-1.7.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:73bd45e1488c40b63fe5a7df892baf9e2a4d4bb6409a2b3b78ac1c6236178e01"},
+ {file = "frozenlist-1.7.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:99886d98e1643269760e5fe0df31e5ae7050788dd288947f7f007209b8c33f08"},
+ {file = "frozenlist-1.7.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:290a172aae5a4c278c6da8a96222e6337744cd9c77313efe33d5670b9f65fc43"},
+ {file = "frozenlist-1.7.0-cp312-cp312-win32.whl", hash = "sha256:426c7bc70e07cfebc178bc4c2bf2d861d720c4fff172181eeb4a4c41d4ca2ad3"},
+ {file = "frozenlist-1.7.0-cp312-cp312-win_amd64.whl", hash = "sha256:563b72efe5da92e02eb68c59cb37205457c977aa7a449ed1b37e6939e5c47c6a"},
+ {file = "frozenlist-1.7.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ee80eeda5e2a4e660651370ebffd1286542b67e268aa1ac8d6dbe973120ef7ee"},
+ {file = "frozenlist-1.7.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:d1a81c85417b914139e3a9b995d4a1c84559afc839a93cf2cb7f15e6e5f6ed2d"},
+ {file = "frozenlist-1.7.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:cbb65198a9132ebc334f237d7b0df163e4de83fb4f2bdfe46c1e654bdb0c5d43"},
+ {file = "frozenlist-1.7.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dab46c723eeb2c255a64f9dc05b8dd601fde66d6b19cdb82b2e09cc6ff8d8b5d"},
+ {file = "frozenlist-1.7.0-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:6aeac207a759d0dedd2e40745575ae32ab30926ff4fa49b1635def65806fddee"},
+ {file = "frozenlist-1.7.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:bd8c4e58ad14b4fa7802b8be49d47993182fdd4023393899632c88fd8cd994eb"},
+ {file = "frozenlist-1.7.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:04fb24d104f425da3540ed83cbfc31388a586a7696142004c577fa61c6298c3f"},
+ {file = "frozenlist-1.7.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6a5c505156368e4ea6b53b5ac23c92d7edc864537ff911d2fb24c140bb175e60"},
+ {file = "frozenlist-1.7.0-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8bd7eb96a675f18aa5c553eb7ddc24a43c8c18f22e1f9925528128c052cdbe00"},
+ {file = "frozenlist-1.7.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:05579bf020096fe05a764f1f84cd104a12f78eaab68842d036772dc6d4870b4b"},
+ {file = "frozenlist-1.7.0-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:376b6222d114e97eeec13d46c486facd41d4f43bab626b7c3f6a8b4e81a5192c"},
+ {file = "frozenlist-1.7.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:0aa7e176ebe115379b5b1c95b4096fb1c17cce0847402e227e712c27bdb5a949"},
+ {file = "frozenlist-1.7.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:3fbba20e662b9c2130dc771e332a99eff5da078b2b2648153a40669a6d0e36ca"},
+ {file = "frozenlist-1.7.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:f3f4410a0a601d349dd406b5713fec59b4cee7e71678d5b17edda7f4655a940b"},
+ {file = "frozenlist-1.7.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:e2cdfaaec6a2f9327bf43c933c0319a7c429058e8537c508964a133dffee412e"},
+ {file = "frozenlist-1.7.0-cp313-cp313-win32.whl", hash = "sha256:5fc4df05a6591c7768459caba1b342d9ec23fa16195e744939ba5914596ae3e1"},
+ {file = "frozenlist-1.7.0-cp313-cp313-win_amd64.whl", hash = "sha256:52109052b9791a3e6b5d1b65f4b909703984b770694d3eb64fad124c835d7cba"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:a6f86e4193bb0e235ef6ce3dde5cbabed887e0b11f516ce8a0f4d3b33078ec2d"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:82d664628865abeb32d90ae497fb93df398a69bb3434463d172b80fc25b0dd7d"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:912a7e8375a1c9a68325a902f3953191b7b292aa3c3fb0d71a216221deca460b"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9537c2777167488d539bc5de2ad262efc44388230e5118868e172dd4a552b146"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:f34560fb1b4c3e30ba35fa9a13894ba39e5acfc5f60f57d8accde65f46cc5e74"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:acd03d224b0175f5a850edc104ac19040d35419eddad04e7cf2d5986d98427f1"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f2038310bc582f3d6a09b3816ab01737d60bf7b1ec70f5356b09e84fb7408ab1"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b8c05e4c8e5f36e5e088caa1bf78a687528f83c043706640a92cb76cd6999384"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:765bb588c86e47d0b68f23c1bee323d4b703218037765dcf3f25c838c6fecceb"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:32dc2e08c67d86d0969714dd484fd60ff08ff81d1a1e40a77dd34a387e6ebc0c"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:c0303e597eb5a5321b4de9c68e9845ac8f290d2ab3f3e2c864437d3c5a30cd65"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:a47f2abb4e29b3a8d0b530f7c3598badc6b134562b1a5caee867f7c62fee51e3"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:3d688126c242a6fabbd92e02633414d40f50bb6002fa4cf995a1d18051525657"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:4e7e9652b3d367c7bd449a727dc79d5043f48b88d0cbfd4f9f1060cf2b414104"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:1a85e345b4c43db8b842cab1feb41be5cc0b10a1830e6295b69d7310f99becaf"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-win32.whl", hash = "sha256:3a14027124ddb70dfcee5148979998066897e79f89f64b13328595c4bdf77c81"},
+ {file = "frozenlist-1.7.0-cp313-cp313t-win_amd64.whl", hash = "sha256:3bf8010d71d4507775f658e9823210b7427be36625b387221642725b515dcf3e"},
+ {file = "frozenlist-1.7.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:cea3dbd15aea1341ea2de490574a4a37ca080b2ae24e4b4f4b51b9057b4c3630"},
+ {file = "frozenlist-1.7.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:7d536ee086b23fecc36c2073c371572374ff50ef4db515e4e503925361c24f71"},
+ {file = "frozenlist-1.7.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:dfcebf56f703cb2e346315431699f00db126d158455e513bd14089d992101e44"},
+ {file = "frozenlist-1.7.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:974c5336e61d6e7eb1ea5b929cb645e882aadab0095c5a6974a111e6479f8878"},
+ {file = "frozenlist-1.7.0-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:c70db4a0ab5ab20878432c40563573229a7ed9241506181bba12f6b7d0dc41cb"},
+ {file = "frozenlist-1.7.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1137b78384eebaf70560a36b7b229f752fb64d463d38d1304939984d5cb887b6"},
+ {file = "frozenlist-1.7.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e793a9f01b3e8b5c0bc646fb59140ce0efcc580d22a3468d70766091beb81b35"},
+ {file = "frozenlist-1.7.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:74739ba8e4e38221d2c5c03d90a7e542cb8ad681915f4ca8f68d04f810ee0a87"},
+ {file = "frozenlist-1.7.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1e63344c4e929b1a01e29bc184bbb5fd82954869033765bfe8d65d09e336a677"},
+ {file = "frozenlist-1.7.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:2ea2a7369eb76de2217a842f22087913cdf75f63cf1307b9024ab82dfb525938"},
+ {file = "frozenlist-1.7.0-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:836b42f472a0e006e02499cef9352ce8097f33df43baaba3e0a28a964c26c7d2"},
+ {file = "frozenlist-1.7.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:e22b9a99741294b2571667c07d9f8cceec07cb92aae5ccda39ea1b6052ed4319"},
+ {file = "frozenlist-1.7.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:9a19e85cc503d958abe5218953df722748d87172f71b73cf3c9257a91b999890"},
+ {file = "frozenlist-1.7.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:f22dac33bb3ee8fe3e013aa7b91dc12f60d61d05b7fe32191ffa84c3aafe77bd"},
+ {file = "frozenlist-1.7.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:9ccec739a99e4ccf664ea0775149f2749b8a6418eb5b8384b4dc0a7d15d304cb"},
+ {file = "frozenlist-1.7.0-cp39-cp39-win32.whl", hash = "sha256:b3950f11058310008a87757f3eee16a8e1ca97979833239439586857bc25482e"},
+ {file = "frozenlist-1.7.0-cp39-cp39-win_amd64.whl", hash = "sha256:43a82fce6769c70f2f5a06248b614a7d268080a9d20f7457ef10ecee5af82b63"},
+ {file = "frozenlist-1.7.0-py3-none-any.whl", hash = "sha256:9a5af342e34f7e97caf8c995864c7a396418ae2859cc6fdf1b1073020d516a7e"},
+ {file = "frozenlist-1.7.0.tar.gz", hash = "sha256:2e310d81923c2437ea8670467121cc3e9b0f76d3043cc1d2331d56c7fb7a3a8f"},
]
[[package]]
name = "fsspec"
-version = "2025.3.0"
+version = "2025.5.1"
description = "File-system specification"
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main"]
files = [
- {file = "fsspec-2025.3.0-py3-none-any.whl", hash = "sha256:efb87af3efa9103f94ca91a7f8cb7a4df91af9f74fc106c9c7ea0efd7277c1b3"},
- {file = "fsspec-2025.3.0.tar.gz", hash = "sha256:a935fd1ea872591f2b5148907d103488fc523295e6c64b835cfad8c3eca44972"},
+ {file = "fsspec-2025.5.1-py3-none-any.whl", hash = "sha256:24d3a2e663d5fc735ab256263c4075f374a174c3410c0b25e5bd1970bceaa462"},
+ {file = "fsspec-2025.5.1.tar.gz", hash = "sha256:2e55e47a540b91843b755e83ded97c6e897fa0942b11490113f09e9c443c2475"},
]
[package.extras]
@@ -1572,21 +1639,21 @@ tqdm = ["tqdm"]
[[package]]
name = "gcsfs"
-version = "2025.3.0"
+version = "2025.5.1"
description = "Convenient Filesystem interface over GCS"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "gcsfs-2025.3.0-py2.py3-none-any.whl", hash = "sha256:afbc2b26a481de66519e9cce7762340ef4781ce01c6663af0d63eda10f6d2c9c"},
- {file = "gcsfs-2025.3.0.tar.gz", hash = "sha256:f68d7bc24bd4b944cd55a6963b9fd722c7bd5791f46c6aebacc380e648292c04"},
+ {file = "gcsfs-2025.5.1-py2.py3-none-any.whl", hash = "sha256:48712471ff71ac83d3e2152ba4dc232874698466e344d5e700feba06b0a0de7b"},
+ {file = "gcsfs-2025.5.1.tar.gz", hash = "sha256:ba945530cf4857cd9d599ccb3ae729c65c39088880b11c4df1fecac30df5f3e3"},
]
[package.dependencies]
aiohttp = "<4.0.0a0 || >4.0.0a0,<4.0.0a1 || >4.0.0a1"
decorator = ">4.1.2"
-fsspec = "2025.3.0"
+fsspec = "2025.5.1"
google-auth = ">=1.2"
google-auth-oauthlib = "*"
google-cloud-storage = "*"
@@ -1654,65 +1721,76 @@ dev = ["flake8", "markdown", "twine", "wheel"]
[[package]]
name = "google-api-core"
-version = "1.16.0"
+version = "2.25.0"
description = "Google API client core library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-api-core-1.16.0.tar.gz", hash = "sha256:92e962a087f1c4b8d1c5c88ade1c1dfd550047dcffb320c57ef6a534a20403e2"},
- {file = "google_api_core-1.16.0-py2.py3-none-any.whl", hash = "sha256:859f7392676761f2b160c6ee030c3422135ada4458f0948c5690a6a7c8d86294"},
+ {file = "google_api_core-2.25.0-py3-none-any.whl", hash = "sha256:1db79d1281dcf9f3d10023283299ba38f3dc9f639ec41085968fd23e5bcf512e"},
+ {file = "google_api_core-2.25.0.tar.gz", hash = "sha256:9b548e688702f82a34ed8409fb8a6961166f0b7795032f0be8f48308dff4333a"},
]
[package.dependencies]
-google-auth = ">=0.4.0,<2.0dev"
-googleapis-common-protos = ">=1.6.0,<2.0dev"
-protobuf = ">=3.4.0"
-pytz = "*"
-requests = ">=2.18.0,<3.0.0dev"
-setuptools = ">=34.0.0"
-six = ">=1.10.0"
+google-auth = ">=2.14.1,<3.0.0"
+googleapis-common-protos = ">=1.56.2,<2.0.0"
+proto-plus = [
+ {version = ">=1.22.3,<2.0.0"},
+ {version = ">=1.25.0,<2.0.0", markers = "python_version >= \"3.13\""},
+]
+protobuf = ">=3.19.5,<3.20.0 || >3.20.0,<3.20.1 || >3.20.1,<4.21.0 || >4.21.0,<4.21.1 || >4.21.1,<4.21.2 || >4.21.2,<4.21.3 || >4.21.3,<4.21.4 || >4.21.4,<4.21.5 || >4.21.5,<7.0.0"
+requests = ">=2.18.0,<3.0.0"
[package.extras]
-grpc = ["grpcio (>=1.8.2,<2.0dev)"]
-grpcgcp = ["grpcio-gcp (>=0.2.2)"]
-grpcio-gcp = ["grpcio-gcp (>=0.2.2)"]
+async-rest = ["google-auth[aiohttp] (>=2.35.0,<3.0.0)"]
+grpc = ["grpcio (>=1.33.2,<2.0.0)", "grpcio (>=1.49.1,<2.0.0) ; python_version >= \"3.11\"", "grpcio-status (>=1.33.2,<2.0.0)", "grpcio-status (>=1.49.1,<2.0.0) ; python_version >= \"3.11\""]
+grpcgcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
+grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
[[package]]
name = "google-auth"
-version = "1.6.3"
+version = "2.40.3"
description = "Google Authentication Library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-auth-1.6.3.tar.gz", hash = "sha256:0f7c6a64927d34c1a474da92cfc59e552a5d3b940d3266606c6a28b72888b9e4"},
- {file = "google_auth-1.6.3-py2.py3-none-any.whl", hash = "sha256:20705f6803fd2c4d1cc2dcb0df09d4dfcb9a7d51fd59e94a3a28231fd93119ed"},
+ {file = "google_auth-2.40.3-py2.py3-none-any.whl", hash = "sha256:1370d4593e86213563547f97a92752fc658456fe4514c809544f330fed45a7ca"},
+ {file = "google_auth-2.40.3.tar.gz", hash = "sha256:500c3a29adedeb36ea9cf24b8d10858e152f2412e3ca37829b3fa18e33d63b77"},
]
[package.dependencies]
-cachetools = ">=2.0.0"
+cachetools = ">=2.0.0,<6.0"
pyasn1-modules = ">=0.2.1"
-rsa = ">=3.1.4"
-six = ">=1.9.0"
+rsa = ">=3.1.4,<5"
+
+[package.extras]
+aiohttp = ["aiohttp (>=3.6.2,<4.0.0)", "requests (>=2.20.0,<3.0.0)"]
+enterprise-cert = ["cryptography", "pyopenssl"]
+pyjwt = ["cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "pyjwt (>=2.0)"]
+pyopenssl = ["cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "pyopenssl (>=20.0.0)"]
+reauth = ["pyu2f (>=0.1.5)"]
+requests = ["requests (>=2.20.0,<3.0.0)"]
+testing = ["aiohttp (<3.10.0)", "aiohttp (>=3.6.2,<4.0.0)", "aioresponses", "cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "flask", "freezegun", "grpcio", "mock", "oauth2client", "packaging", "pyjwt (>=2.0)", "pyopenssl (<24.3.0)", "pyopenssl (>=20.0.0)", "pytest", "pytest-asyncio", "pytest-cov", "pytest-localserver", "pyu2f (>=0.1.5)", "requests (>=2.20.0,<3.0.0)", "responses", "urllib3"]
+urllib3 = ["packaging", "urllib3"]
[[package]]
name = "google-auth-oauthlib"
-version = "0.5.3"
+version = "1.2.2"
description = "Google Authentication Library"
optional = true
python-versions = ">=3.6"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-auth-oauthlib-0.5.3.tar.gz", hash = "sha256:307d21918d61a0741882ad1fd001c67e68ad81206451d05fc4d26f79de56fc90"},
- {file = "google_auth_oauthlib-0.5.3-py2.py3-none-any.whl", hash = "sha256:9e8ff4ed2b21c174a2d6cc2172c698dbf0b1f686509774c663a83c495091fe09"},
+ {file = "google_auth_oauthlib-1.2.2-py3-none-any.whl", hash = "sha256:fd619506f4b3908b5df17b65f39ca8d66ea56986e5472eb5978fd8f3786f00a2"},
+ {file = "google_auth_oauthlib-1.2.2.tar.gz", hash = "sha256:11046fb8d3348b296302dd939ace8af0a724042e8029c1b872d87fabc9f41684"},
]
[package.dependencies]
-google-auth = ">=1.0.0"
+google-auth = ">=2.15.0"
requests-oauthlib = ">=0.7.0"
[package.extras]
@@ -1720,71 +1798,128 @@ tool = ["click (>=6.0.0)"]
[[package]]
name = "google-cloud-core"
-version = "1.4.0"
+version = "2.4.3"
description = "Google Cloud API client core library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-cloud-core-1.4.0.tar.gz", hash = "sha256:07a024a26c4eb14ee3df7e6e5021c04f8f7e9f0e83d3d47863229f3635f871ce"},
- {file = "google_cloud_core-1.4.0-py2.py3-none-any.whl", hash = "sha256:e84d4a05c58fc905b9340118e257cd043e9c02a6d31168182204afe4afd521eb"},
+ {file = "google_cloud_core-2.4.3-py2.py3-none-any.whl", hash = "sha256:5130f9f4c14b4fafdff75c79448f9495cfade0d8775facf1b09c3bf67e027f6e"},
+ {file = "google_cloud_core-2.4.3.tar.gz", hash = "sha256:1fab62d7102844b278fe6dead3af32408b1df3eb06f5c7e8634cbd40edc4da53"},
]
[package.dependencies]
-google-api-core = ">=1.16.0,<2.0.0dev"
+google-api-core = ">=1.31.6,<2.0.dev0 || >2.3.0,<3.0.0dev"
+google-auth = ">=1.25.0,<3.0dev"
[package.extras]
-grpc = ["grpcio (>=1.8.2,<2.0dev)"]
+grpc = ["grpcio (>=1.38.0,<2.0dev)", "grpcio-status (>=1.38.0,<2.0.dev0)"]
[[package]]
name = "google-cloud-storage"
-version = "1.23.0"
+version = "3.1.0"
description = "Google Cloud Storage API client library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-cloud-storage-1.23.0.tar.gz", hash = "sha256:c66e876ae9547884fa42566a2ebfec51d280f488d7a058af9611ba90c78bed78"},
- {file = "google_cloud_storage-1.23.0-py2.py3-none-any.whl", hash = "sha256:9f59c100d3940e38567c48d54cf1a2e7591a2f38e9693dfc11a242d5e54a1626"},
+ {file = "google_cloud_storage-3.1.0-py2.py3-none-any.whl", hash = "sha256:eaf36966b68660a9633f03b067e4a10ce09f1377cae3ff9f2c699f69a81c66c6"},
+ {file = "google_cloud_storage-3.1.0.tar.gz", hash = "sha256:944273179897c7c8a07ee15f2e6466a02da0c7c4b9ecceac2a26017cb2972049"},
]
[package.dependencies]
-google-auth = ">=1.2.0"
-google-cloud-core = ">=1.0.3,<2.0dev"
-google-resumable-media = ">=0.5.0,<0.6dev"
+google-api-core = ">=2.15.0,<3.0.0dev"
+google-auth = ">=2.26.1,<3.0dev"
+google-cloud-core = ">=2.4.2,<3.0dev"
+google-crc32c = ">=1.0,<2.0dev"
+google-resumable-media = ">=2.7.2"
+requests = ">=2.18.0,<3.0.0dev"
+
+[package.extras]
+protobuf = ["protobuf (<6.0.0dev)"]
+tracing = ["opentelemetry-api (>=1.1.0)"]
+
+[[package]]
+name = "google-crc32c"
+version = "1.7.1"
+description = "A python wrapper of the C library 'Google CRC32C'"
+optional = true
+python-versions = ">=3.9"
+groups = ["main"]
+markers = "extra == \"gcsfs\""
+files = [
+ {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:b07d48faf8292b4db7c3d64ab86f950c2e94e93a11fd47271c28ba458e4a0d76"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:7cc81b3a2fbd932a4313eb53cc7d9dde424088ca3a0337160f35d91826880c1d"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:1c67ca0a1f5b56162951a9dae987988679a7db682d6f97ce0f6381ebf0fbea4c"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc5319db92daa516b653600794d5b9f9439a9a121f3e162f94b0e1891c7933cb"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dcdf5a64adb747610140572ed18d011896e3b9ae5195f2514b7ff678c80f1603"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-win_amd64.whl", hash = "sha256:754561c6c66e89d55754106739e22fdaa93fafa8da7221b29c8b8e8270c6ec8a"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:6fbab4b935989e2c3610371963ba1b86afb09537fd0c633049be82afe153ac06"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:ed66cbe1ed9cbaaad9392b5259b3eba4a9e565420d734e6238813c428c3336c9"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ee6547b657621b6cbed3562ea7826c3e11cab01cd33b74e1f677690652883e77"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d68e17bad8f7dd9a49181a1f5a8f4b251c6dbc8cc96fb79f1d321dfd57d66f53"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-win_amd64.whl", hash = "sha256:6335de12921f06e1f774d0dd1fbea6bf610abe0887a1638f64d694013138be5d"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:2d73a68a653c57281401871dd4aeebbb6af3191dcac751a76ce430df4d403194"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:22beacf83baaf59f9d3ab2bbb4db0fb018da8e5aebdce07ef9f09fce8220285e"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:19eafa0e4af11b0a4eb3974483d55d2d77ad1911e6cf6f832e1574f6781fd337"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b6d86616faaea68101195c6bdc40c494e4d76f41e07a37ffdef270879c15fb65"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-win_amd64.whl", hash = "sha256:b7491bdc0c7564fcf48c0179d2048ab2f7c7ba36b84ccd3a3e1c3f7a72d3bba6"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:df8b38bdaf1629d62d51be8bdd04888f37c451564c2042d36e5812da9eff3c35"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:e42e20a83a29aa2709a0cf271c7f8aefaa23b7ab52e53b322585297bb94d4638"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:905a385140bf492ac300026717af339790921f411c0dfd9aa5a9e69a08ed32eb"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b211ddaf20f7ebeec5c333448582c224a7c90a9d98826fbab82c0ddc11348e6"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-win_amd64.whl", hash = "sha256:0f99eaa09a9a7e642a61e06742856eec8b19fc0037832e03f941fe7cf0c8e4db"},
+ {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:32d1da0d74ec5634a05f53ef7df18fc646666a25efaaca9fc7dcfd4caf1d98c3"},
+ {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e10554d4abc5238823112c2ad7e4560f96c7bf3820b202660373d769d9e6e4c9"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:9fc196f0b8d8bd2789352c6a522db03f89e83a0ed6b64315923c396d7a932315"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:bb5e35dcd8552f76eed9461a23de1030920a3c953c1982f324be8f97946e7127"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f2226b6a8da04f1d9e61d3e357f2460b9551c5e6950071437e122c958a18ae14"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1f2b3522222746fff0e04a9bd0a23ea003ba3cccc8cf21385c564deb1f223242"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3bda0fcb632d390e3ea8b6b07bf6b4f4a66c9d02dcd6fbf7ba00a197c143f582"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-win_amd64.whl", hash = "sha256:713121af19f1a617054c41f952294764e0c5443d5a5d9034b2cd60f5dd7e0349"},
+ {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a8e9afc74168b0b2232fb32dd202c93e46b7d5e4bf03e66ba5dc273bb3559589"},
+ {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa8136cc14dd27f34a3221c0f16fd42d8a40e4778273e61a3c19aedaa44daf6b"},
+ {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:85fef7fae11494e747c9fd1359a527e5970fc9603c90764843caabd3a16a0a48"},
+ {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6efb97eb4369d52593ad6f75e7e10d053cf00c48983f7a973105bc70b0ac4d82"},
+ {file = "google_crc32c-1.7.1.tar.gz", hash = "sha256:2bff2305f98846f3e825dbeec9ee406f89da7962accdb29356e4eadc251bd472"},
+]
+
+[package.extras]
+testing = ["pytest"]
[[package]]
name = "google-resumable-media"
-version = "0.5.1"
+version = "2.7.2"
description = "Utilities for Google Media Downloads and Resumable Uploads"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-resumable-media-0.5.1.tar.gz", hash = "sha256:97155236971970382b738921f978a6f86a7b5a0b0311703d991e065d3cb55773"},
- {file = "google_resumable_media-0.5.1-py2.py3-none-any.whl", hash = "sha256:cdc64378dc9a7a7bf963a8d0c944c99b549dc0c195a9acbf1fcd465f380b9002"},
+ {file = "google_resumable_media-2.7.2-py2.py3-none-any.whl", hash = "sha256:3ce7551e9fe6d99e9a126101d2536612bb73486721951e9562fee0f90c6ababa"},
+ {file = "google_resumable_media-2.7.2.tar.gz", hash = "sha256:5280aed4629f2b60b847b0d42f9857fd4935c11af266744df33d8074cae92fe0"},
]
[package.dependencies]
-six = "*"
+google-crc32c = ">=1.0,<2.0dev"
[package.extras]
+aiohttp = ["aiohttp (>=3.6.2,<4.0.0dev)", "google-auth (>=1.22.0,<2.0dev)"]
requests = ["requests (>=2.18.0,<3.0.0dev)"]
[[package]]
name = "googleapis-common-protos"
-version = "1.69.2"
+version = "1.70.0"
description = "Common protobufs used in Google APIs"
optional = true
python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "googleapis_common_protos-1.69.2-py3-none-any.whl", hash = "sha256:0b30452ff9c7a27d80bfc5718954063e8ab53dd3697093d3bc99581f5fd24212"},
- {file = "googleapis_common_protos-1.69.2.tar.gz", hash = "sha256:3e1b904a27a33c821b4b749fd31d334c0c9c30e6113023d495e48979a3dc9c5f"},
+ {file = "googleapis_common_protos-1.70.0-py3-none-any.whl", hash = "sha256:b8bfcca8c25a2bb253e0e0b0adaf8c00773e5e6af6fd92397576680b807e0fd8"},
+ {file = "googleapis_common_protos-1.70.0.tar.gz", hash = "sha256:0e1b44e0ea153e6594f9f394fef15193a68aaaea2d843f83e2742717ca753257"},
]
[package.dependencies]
@@ -1810,86 +1945,67 @@ typing-extensions = {version = ">=4,<5", markers = "python_version < \"3.10\""}
[[package]]
name = "greenlet"
-version = "3.1.1"
+version = "3.2.3"
description = "Lightweight in-process concurrent programming"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=3.9"
groups = ["main"]
markers = "python_version < \"3.14\" and (platform_machine == \"aarch64\" or platform_machine == \"ppc64le\" or platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"AMD64\" or platform_machine == \"win32\" or platform_machine == \"WIN32\") and (extra == \"sql-postgres\" or extra == \"sql-sqlite\")"
files = [
- {file = "greenlet-3.1.1-cp310-cp310-macosx_11_0_universal2.whl", hash = "sha256:0bbae94a29c9e5c7e4a2b7f0aae5c17e8e90acbfd3bf6270eeba60c39fce3563"},
- {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0fde093fb93f35ca72a556cf72c92ea3ebfda3d79fc35bb19fbe685853869a83"},
- {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:36b89d13c49216cadb828db8dfa6ce86bbbc476a82d3a6c397f0efae0525bdd0"},
- {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:94b6150a85e1b33b40b1464a3f9988dcc5251d6ed06842abff82e42632fac120"},
- {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:93147c513fac16385d1036b7e5b102c7fbbdb163d556b791f0f11eada7ba65dc"},
- {file = "greenlet-3.1.1-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:da7a9bff22ce038e19bf62c4dd1ec8391062878710ded0a845bcf47cc0200617"},
- {file = "greenlet-3.1.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:b2795058c23988728eec1f36a4e5e4ebad22f8320c85f3587b539b9ac84128d7"},
- {file = "greenlet-3.1.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:ed10eac5830befbdd0c32f83e8aa6288361597550ba669b04c48f0f9a2c843c6"},
- {file = "greenlet-3.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:77c386de38a60d1dfb8e55b8c1101d68c79dfdd25c7095d51fec2dd800892b80"},
- {file = "greenlet-3.1.1-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:e4d333e558953648ca09d64f13e6d8f0523fa705f51cae3f03b5983489958c70"},
- {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09fc016b73c94e98e29af67ab7b9a879c307c6731a2c9da0db5a7d9b7edd1159"},
- {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d5e975ca70269d66d17dd995dafc06f1b06e8cb1ec1e9ed54c1d1e4a7c4cf26e"},
- {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3b2813dc3de8c1ee3f924e4d4227999285fd335d1bcc0d2be6dc3f1f6a318ec1"},
- {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e347b3bfcf985a05e8c0b7d462ba6f15b1ee1c909e2dcad795e49e91b152c383"},
- {file = "greenlet-3.1.1-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9e8f8c9cb53cdac7ba9793c276acd90168f416b9ce36799b9b885790f8ad6c0a"},
- {file = "greenlet-3.1.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:62ee94988d6b4722ce0028644418d93a52429e977d742ca2ccbe1c4f4a792511"},
- {file = "greenlet-3.1.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1776fd7f989fc6b8d8c8cb8da1f6b82c5814957264d1f6cf818d475ec2bf6395"},
- {file = "greenlet-3.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:48ca08c771c268a768087b408658e216133aecd835c0ded47ce955381105ba39"},
- {file = "greenlet-3.1.1-cp312-cp312-macosx_11_0_universal2.whl", hash = "sha256:4afe7ea89de619adc868e087b4d2359282058479d7cfb94970adf4b55284574d"},
- {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f406b22b7c9a9b4f8aa9d2ab13d6ae0ac3e85c9a809bd590ad53fed2bf70dc79"},
- {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c3a701fe5a9695b238503ce5bbe8218e03c3bcccf7e204e455e7462d770268aa"},
- {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2846930c65b47d70b9d178e89c7e1a69c95c1f68ea5aa0a58646b7a96df12441"},
- {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:99cfaa2110534e2cf3ba31a7abcac9d328d1d9f1b95beede58294a60348fba36"},
- {file = "greenlet-3.1.1-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1443279c19fca463fc33e65ef2a935a5b09bb90f978beab37729e1c3c6c25fe9"},
- {file = "greenlet-3.1.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:b7cede291382a78f7bb5f04a529cb18e068dd29e0fb27376074b6d0317bf4dd0"},
- {file = "greenlet-3.1.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:23f20bb60ae298d7d8656c6ec6db134bca379ecefadb0b19ce6f19d1f232a942"},
- {file = "greenlet-3.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:7124e16b4c55d417577c2077be379514321916d5790fa287c9ed6f23bd2ffd01"},
- {file = "greenlet-3.1.1-cp313-cp313-macosx_11_0_universal2.whl", hash = "sha256:05175c27cb459dcfc05d026c4232f9de8913ed006d42713cb8a5137bd49375f1"},
- {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:935e943ec47c4afab8965954bf49bfa639c05d4ccf9ef6e924188f762145c0ff"},
- {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:667a9706c970cb552ede35aee17339a18e8f2a87a51fba2ed39ceeeb1004798a"},
- {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b8a678974d1f3aa55f6cc34dc480169d58f2e6d8958895d68845fa4ab566509e"},
- {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:efc0f674aa41b92da8c49e0346318c6075d734994c3c4e4430b1c3f853e498e4"},
- {file = "greenlet-3.1.1-cp313-cp313-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:0153404a4bb921f0ff1abeb5ce8a5131da56b953eda6e14b88dc6bbc04d2049e"},
- {file = "greenlet-3.1.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:275f72decf9932639c1c6dd1013a1bc266438eb32710016a1c742df5da6e60a1"},
- {file = "greenlet-3.1.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:c4aab7f6381f38a4b42f269057aee279ab0fc7bf2e929e3d4abfae97b682a12c"},
- {file = "greenlet-3.1.1-cp313-cp313-win_amd64.whl", hash = "sha256:b42703b1cf69f2aa1df7d1030b9d77d3e584a70755674d60e710f0af570f3761"},
- {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f1695e76146579f8c06c1509c7ce4dfe0706f49c6831a817ac04eebb2fd02011"},
- {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7876452af029456b3f3549b696bb36a06db7c90747740c5302f74a9e9fa14b13"},
- {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4ead44c85f8ab905852d3de8d86f6f8baf77109f9da589cb4fa142bd3b57b475"},
- {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8320f64b777d00dd7ccdade271eaf0cad6636343293a25074cc5566160e4de7b"},
- {file = "greenlet-3.1.1-cp313-cp313t-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:6510bf84a6b643dabba74d3049ead221257603a253d0a9873f55f6a59a65f822"},
- {file = "greenlet-3.1.1-cp313-cp313t-musllinux_1_1_aarch64.whl", hash = "sha256:04b013dc07c96f83134b1e99888e7a79979f1a247e2a9f59697fa14b5862ed01"},
- {file = "greenlet-3.1.1-cp313-cp313t-musllinux_1_1_x86_64.whl", hash = "sha256:411f015496fec93c1c8cd4e5238da364e1da7a124bcb293f085bf2860c32c6f6"},
- {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:47da355d8687fd65240c364c90a31569a133b7b60de111c255ef5b606f2ae291"},
- {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:98884ecf2ffb7d7fe6bd517e8eb99d31ff7855a840fa6d0d63cd07c037f6a981"},
- {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f1d4aeb8891338e60d1ab6127af1fe45def5259def8094b9c7e34690c8858803"},
- {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:db32b5348615a04b82240cc67983cb315309e88d444a288934ee6ceaebcad6cc"},
- {file = "greenlet-3.1.1-cp37-cp37m-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:dcc62f31eae24de7f8dce72134c8651c58000d3b1868e01392baea7c32c247de"},
- {file = "greenlet-3.1.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:1d3755bcb2e02de341c55b4fca7a745a24a9e7212ac953f6b3a48d117d7257aa"},
- {file = "greenlet-3.1.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:b8da394b34370874b4572676f36acabac172602abf054cbc4ac910219f3340af"},
- {file = "greenlet-3.1.1-cp37-cp37m-win32.whl", hash = "sha256:a0dfc6c143b519113354e780a50381508139b07d2177cb6ad6a08278ec655798"},
- {file = "greenlet-3.1.1-cp37-cp37m-win_amd64.whl", hash = "sha256:54558ea205654b50c438029505def3834e80f0869a70fb15b871c29b4575ddef"},
- {file = "greenlet-3.1.1-cp38-cp38-macosx_11_0_universal2.whl", hash = "sha256:346bed03fe47414091be4ad44786d1bd8bef0c3fcad6ed3dee074a032ab408a9"},
- {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dfc59d69fc48664bc693842bd57acfdd490acafda1ab52c7836e3fc75c90a111"},
- {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d21e10da6ec19b457b82636209cbe2331ff4306b54d06fa04b7c138ba18c8a81"},
- {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:37b9de5a96111fc15418819ab4c4432e4f3c2ede61e660b1e33971eba26ef9ba"},
- {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6ef9ea3f137e5711f0dbe5f9263e8c009b7069d8a1acea822bd5e9dae0ae49c8"},
- {file = "greenlet-3.1.1-cp38-cp38-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:85f3ff71e2e60bd4b4932a043fbbe0f499e263c628390b285cb599154a3b03b1"},
- {file = "greenlet-3.1.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:95ffcf719966dd7c453f908e208e14cde192e09fde6c7186c8f1896ef778d8cd"},
- {file = "greenlet-3.1.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:03a088b9de532cbfe2ba2034b2b85e82df37874681e8c470d6fb2f8c04d7e4b7"},
- {file = "greenlet-3.1.1-cp38-cp38-win32.whl", hash = "sha256:8b8b36671f10ba80e159378df9c4f15c14098c4fd73a36b9ad715f057272fbef"},
- {file = "greenlet-3.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:7017b2be767b9d43cc31416aba48aab0d2309ee31b4dbf10a1d38fb7972bdf9d"},
- {file = "greenlet-3.1.1-cp39-cp39-macosx_11_0_universal2.whl", hash = "sha256:396979749bd95f018296af156201d6211240e7a23090f50a8d5d18c370084dc3"},
- {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ca9d0ff5ad43e785350894d97e13633a66e2b50000e8a183a50a88d834752d42"},
- {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f6ff3b14f2df4c41660a7dec01045a045653998784bf8cfcb5a525bdffffbc8f"},
- {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:94ebba31df2aa506d7b14866fed00ac141a867e63143fe5bca82a8e503b36437"},
- {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:73aaad12ac0ff500f62cebed98d8789198ea0e6f233421059fa68a5aa7220145"},
- {file = "greenlet-3.1.1-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:63e4844797b975b9af3a3fb8f7866ff08775f5426925e1e0bbcfe7932059a12c"},
- {file = "greenlet-3.1.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:7939aa3ca7d2a1593596e7ac6d59391ff30281ef280d8632fa03d81f7c5f955e"},
- {file = "greenlet-3.1.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:d0028e725ee18175c6e422797c407874da24381ce0690d6b9396c204c7f7276e"},
- {file = "greenlet-3.1.1-cp39-cp39-win32.whl", hash = "sha256:5e06afd14cbaf9e00899fae69b24a32f2196c19de08fcb9f4779dd4f004e5e7c"},
- {file = "greenlet-3.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:3319aa75e0e0639bc15ff54ca327e8dc7a6fe404003496e3c6925cd3142e0e22"},
- {file = "greenlet-3.1.1.tar.gz", hash = "sha256:4ce3ac6cdb6adf7946475d7ef31777c26d94bccc377e070a7986bd2d5c515467"},
+ {file = "greenlet-3.2.3-cp310-cp310-macosx_11_0_universal2.whl", hash = "sha256:1afd685acd5597349ee6d7a88a8bec83ce13c106ac78c196ee9dde7c04fe87be"},
+ {file = "greenlet-3.2.3-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:761917cac215c61e9dc7324b2606107b3b292a8349bdebb31503ab4de3f559ac"},
+ {file = "greenlet-3.2.3-cp310-cp310-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:a433dbc54e4a37e4fff90ef34f25a8c00aed99b06856f0119dcf09fbafa16392"},
+ {file = "greenlet-3.2.3-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:72e77ed69312bab0434d7292316d5afd6896192ac4327d44f3d613ecb85b037c"},
+ {file = "greenlet-3.2.3-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:68671180e3849b963649254a882cd544a3c75bfcd2c527346ad8bb53494444db"},
+ {file = "greenlet-3.2.3-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:49c8cfb18fb419b3d08e011228ef8a25882397f3a859b9fe1436946140b6756b"},
+ {file = "greenlet-3.2.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:efc6dc8a792243c31f2f5674b670b3a95d46fa1c6a912b8e310d6f542e7b0712"},
+ {file = "greenlet-3.2.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:731e154aba8e757aedd0781d4b240f1225b075b4409f1bb83b05ff410582cf00"},
+ {file = "greenlet-3.2.3-cp310-cp310-win_amd64.whl", hash = "sha256:96c20252c2f792defe9a115d3287e14811036d51e78b3aaddbee23b69b216302"},
+ {file = "greenlet-3.2.3-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:784ae58bba89fa1fa5733d170d42486580cab9decda3484779f4759345b29822"},
+ {file = "greenlet-3.2.3-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:0921ac4ea42a5315d3446120ad48f90c3a6b9bb93dd9b3cf4e4d84a66e42de83"},
+ {file = "greenlet-3.2.3-cp311-cp311-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:d2971d93bb99e05f8c2c0c2f4aa9484a18d98c4c3bd3c62b65b7e6ae33dfcfaf"},
+ {file = "greenlet-3.2.3-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:c667c0bf9d406b77a15c924ef3285e1e05250948001220368e039b6aa5b5034b"},
+ {file = "greenlet-3.2.3-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:592c12fb1165be74592f5de0d70f82bc5ba552ac44800d632214b76089945147"},
+ {file = "greenlet-3.2.3-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:29e184536ba333003540790ba29829ac14bb645514fbd7e32af331e8202a62a5"},
+ {file = "greenlet-3.2.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:93c0bb79844a367782ec4f429d07589417052e621aa39a5ac1fb99c5aa308edc"},
+ {file = "greenlet-3.2.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:751261fc5ad7b6705f5f76726567375bb2104a059454e0226e1eef6c756748ba"},
+ {file = "greenlet-3.2.3-cp311-cp311-win_amd64.whl", hash = "sha256:83a8761c75312361aa2b5b903b79da97f13f556164a7dd2d5448655425bd4c34"},
+ {file = "greenlet-3.2.3-cp312-cp312-macosx_11_0_universal2.whl", hash = "sha256:25ad29caed5783d4bd7a85c9251c651696164622494c00802a139c00d639242d"},
+ {file = "greenlet-3.2.3-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:88cd97bf37fe24a6710ec6a3a7799f3f81d9cd33317dcf565ff9950c83f55e0b"},
+ {file = "greenlet-3.2.3-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:baeedccca94880d2f5666b4fa16fc20ef50ba1ee353ee2d7092b383a243b0b0d"},
+ {file = "greenlet-3.2.3-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:be52af4b6292baecfa0f397f3edb3c6092ce071b499dd6fe292c9ac9f2c8f264"},
+ {file = "greenlet-3.2.3-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:0cc73378150b8b78b0c9fe2ce56e166695e67478550769536a6742dca3651688"},
+ {file = "greenlet-3.2.3-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:706d016a03e78df129f68c4c9b4c4f963f7d73534e48a24f5f5a7101ed13dbbb"},
+ {file = "greenlet-3.2.3-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:419e60f80709510c343c57b4bb5a339d8767bf9aef9b8ce43f4f143240f88b7c"},
+ {file = "greenlet-3.2.3-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:93d48533fade144203816783373f27a97e4193177ebaaf0fc396db19e5d61163"},
+ {file = "greenlet-3.2.3-cp312-cp312-win_amd64.whl", hash = "sha256:7454d37c740bb27bdeddfc3f358f26956a07d5220818ceb467a483197d84f849"},
+ {file = "greenlet-3.2.3-cp313-cp313-macosx_11_0_universal2.whl", hash = "sha256:500b8689aa9dd1ab26872a34084503aeddefcb438e2e7317b89b11eaea1901ad"},
+ {file = "greenlet-3.2.3-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:a07d3472c2a93117af3b0136f246b2833fdc0b542d4a9799ae5f41c28323faef"},
+ {file = "greenlet-3.2.3-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:8704b3768d2f51150626962f4b9a9e4a17d2e37c8a8d9867bbd9fa4eb938d3b3"},
+ {file = "greenlet-3.2.3-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:5035d77a27b7c62db6cf41cf786cfe2242644a7a337a0e155c80960598baab95"},
+ {file = "greenlet-3.2.3-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:2d8aa5423cd4a396792f6d4580f88bdc6efcb9205891c9d40d20f6e670992efb"},
+ {file = "greenlet-3.2.3-cp313-cp313-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:2c724620a101f8170065d7dded3f962a2aea7a7dae133a009cada42847e04a7b"},
+ {file = "greenlet-3.2.3-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:873abe55f134c48e1f2a6f53f7d1419192a3d1a4e873bace00499a4e45ea6af0"},
+ {file = "greenlet-3.2.3-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:024571bbce5f2c1cfff08bf3fbaa43bbc7444f580ae13b0099e95d0e6e67ed36"},
+ {file = "greenlet-3.2.3-cp313-cp313-win_amd64.whl", hash = "sha256:5195fb1e75e592dd04ce79881c8a22becdfa3e6f500e7feb059b1e6fdd54d3e3"},
+ {file = "greenlet-3.2.3-cp314-cp314-macosx_11_0_universal2.whl", hash = "sha256:3d04332dddb10b4a211b68111dabaee2e1a073663d117dc10247b5b1642bac86"},
+ {file = "greenlet-3.2.3-cp314-cp314-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:8186162dffde068a465deab08fc72c767196895c39db26ab1c17c0b77a6d8b97"},
+ {file = "greenlet-3.2.3-cp314-cp314-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:f4bfbaa6096b1b7a200024784217defedf46a07c2eee1a498e94a1b5f8ec5728"},
+ {file = "greenlet-3.2.3-cp314-cp314-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:ed6cfa9200484d234d8394c70f5492f144b20d4533f69262d530a1a082f6ee9a"},
+ {file = "greenlet-3.2.3-cp314-cp314-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:02b0df6f63cd15012bed5401b47829cfd2e97052dc89da3cfaf2c779124eb892"},
+ {file = "greenlet-3.2.3-cp314-cp314-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:86c2d68e87107c1792e2e8d5399acec2487a4e993ab76c792408e59394d52141"},
+ {file = "greenlet-3.2.3-cp314-cp314-win_amd64.whl", hash = "sha256:8c47aae8fbbfcf82cc13327ae802ba13c9c36753b67e760023fd116bc124a62a"},
+ {file = "greenlet-3.2.3-cp39-cp39-macosx_11_0_universal2.whl", hash = "sha256:42efc522c0bd75ffa11a71e09cd8a399d83fafe36db250a87cf1dacfaa15dc64"},
+ {file = "greenlet-3.2.3-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:d760f9bdfe79bff803bad32b4d8ffb2c1d2ce906313fc10a83976ffb73d64ca7"},
+ {file = "greenlet-3.2.3-cp39-cp39-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:8324319cbd7b35b97990090808fdc99c27fe5338f87db50514959f8059999805"},
+ {file = "greenlet-3.2.3-cp39-cp39-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:8c37ef5b3787567d322331d5250e44e42b58c8c713859b8a04c6065f27efbf72"},
+ {file = "greenlet-3.2.3-cp39-cp39-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:ce539fb52fb774d0802175d37fcff5c723e2c7d249c65916257f0a940cee8904"},
+ {file = "greenlet-3.2.3-cp39-cp39-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:003c930e0e074db83559edc8705f3a2d066d4aa8c2f198aff1e454946efd0f26"},
+ {file = "greenlet-3.2.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:7e70ea4384b81ef9e84192e8a77fb87573138aa5d4feee541d8014e452b434da"},
+ {file = "greenlet-3.2.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:22eb5ba839c4b2156f18f76768233fe44b23a31decd9cc0d4cc8141c211fd1b4"},
+ {file = "greenlet-3.2.3-cp39-cp39-win32.whl", hash = "sha256:4532f0d25df67f896d137431b13f4cdce89f7e3d4a96387a41290910df4d3a57"},
+ {file = "greenlet-3.2.3-cp39-cp39-win_amd64.whl", hash = "sha256:aaa7aae1e7f75eaa3ae400ad98f8644bb81e1dc6ba47ce8a93d3f17274e08322"},
+ {file = "greenlet-3.2.3.tar.gz", hash = "sha256:8b0dd8ae4c0d6f5e54ee55ba935eeb3d735a9b58a8a1e5b5cbab64e01a39f365"},
]
[package.extras]
@@ -1913,21 +2029,21 @@ colorama = ">=0.4"
[[package]]
name = "hf-xet"
-version = "1.1.2"
+version = "1.1.3"
description = "Fast transfer of large files with the Hugging Face Hub."
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"hf\" and (platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"arm64\" or platform_machine == \"aarch64\")"
files = [
- {file = "hf_xet-1.1.2-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:dfd1873fd648488c70735cb60f7728512bca0e459e61fcd107069143cd798469"},
- {file = "hf_xet-1.1.2-cp37-abi3-macosx_11_0_arm64.whl", hash = "sha256:29b584983b2d977c44157d9241dcf0fd50acde0b7bff8897fe4386912330090d"},
- {file = "hf_xet-1.1.2-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b29ac84298147fe9164cc55ad994ba47399f90b5d045b0b803b99cf5f06d8ec"},
- {file = "hf_xet-1.1.2-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:d921ba32615676e436a0d15e162331abc9ed43d440916b1d836dc27ce1546173"},
- {file = "hf_xet-1.1.2-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:d9b03c34e13c44893ab6e8fea18ee8d2a6878c15328dd3aabedbdd83ee9f2ed3"},
- {file = "hf_xet-1.1.2-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:01b18608955b3d826307d37da8bd38b28a46cd2d9908b3a3655d1363274f941a"},
- {file = "hf_xet-1.1.2-cp37-abi3-win_amd64.whl", hash = "sha256:3562902c81299b09f3582ddfb324400c6a901a2f3bc854f83556495755f4954c"},
- {file = "hf_xet-1.1.2.tar.gz", hash = "sha256:3712d6d4819d3976a1c18e36db9f503e296283f9363af818f50703506ed63da3"},
+ {file = "hf_xet-1.1.3-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:c3b508b5f583a75641aebf732853deb058953370ce8184f5dabc49f803b0819b"},
+ {file = "hf_xet-1.1.3-cp37-abi3-macosx_11_0_arm64.whl", hash = "sha256:b788a61977fbe6b5186e66239e2a329a3f0b7e7ff50dad38984c0c74f44aeca1"},
+ {file = "hf_xet-1.1.3-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd2da210856444a34aad8ada2fc12f70dabed7cc20f37e90754d1d9b43bc0534"},
+ {file = "hf_xet-1.1.3-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:8203f52827e3df65981984936654a5b390566336956f65765a8aa58c362bb841"},
+ {file = "hf_xet-1.1.3-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:30c575a5306f8e6fda37edb866762140a435037365eba7a17ce7bd0bc0216a8b"},
+ {file = "hf_xet-1.1.3-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:7c1a6aa6abed1f696f8099aa9796ca04c9ee778a58728a115607de9cc4638ff1"},
+ {file = "hf_xet-1.1.3-cp37-abi3-win_amd64.whl", hash = "sha256:b578ae5ac9c056296bb0df9d018e597c8dc6390c5266f35b5c44696003cde9f3"},
+ {file = "hf_xet-1.1.3.tar.gz", hash = "sha256:a5f09b1dd24e6ff6bcedb4b0ddab2d81824098bb002cf8b4ffa780545fa348c3"},
]
[package.extras]
@@ -1935,15 +2051,15 @@ tests = ["pytest"]
[[package]]
name = "huggingface-hub"
-version = "0.32.4"
+version = "0.32.5"
description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
optional = true
python-versions = ">=3.8.0"
groups = ["main"]
markers = "extra == \"hf\""
files = [
- {file = "huggingface_hub-0.32.4-py3-none-any.whl", hash = "sha256:37abf8826b38d971f60d3625229221c36e53fe58060286db9baf619cfbf39767"},
- {file = "huggingface_hub-0.32.4.tar.gz", hash = "sha256:f61d45cd338736f59fb0e97550b74c24ee771bcc92c05ae0766b9116abe720be"},
+ {file = "huggingface_hub-0.32.5-py3-none-any.whl", hash = "sha256:6df8d5f42034a1b61daac60eed04acf348d337a4bd83aa448d4235cfb003e379"},
+ {file = "huggingface_hub-0.32.5.tar.gz", hash = "sha256:8328f848218e3212647cec77eab9fdfc2590e8117d979b925439bc01042a20de"},
]
[package.dependencies]
@@ -1975,14 +2091,14 @@ typing = ["types-PyYAML", "types-requests", "types-simplejson", "types-toml", "t
[[package]]
name = "identify"
-version = "2.6.9"
+version = "2.6.12"
description = "File identification library for Python"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "identify-2.6.9-py2.py3-none-any.whl", hash = "sha256:c98b4322da415a8e5a70ff6e51fbc2d2932c015532d77e9f8537b4ba7813b150"},
- {file = "identify-2.6.9.tar.gz", hash = "sha256:d40dfe3142a1421d8518e3d3985ef5ac42890683e32306ad614a29490abeb6bf"},
+ {file = "identify-2.6.12-py2.py3-none-any.whl", hash = "sha256:ad9672d5a72e0d2ff7c5c8809b62dfa60458626352fb0eb7b55e69bdc45334a2"},
+ {file = "identify-2.6.12.tar.gz", hash = "sha256:d8de45749f1efb108badef65ee8386f0f7bb19a7f26185f74de6367bffbaf0e6"},
]
[package.extras]
@@ -2017,14 +2133,14 @@ files = [
[[package]]
name = "importlib-metadata"
-version = "8.6.1"
+version = "8.7.0"
description = "Read metadata from Python packages"
optional = false
python-versions = ">=3.9"
groups = ["main", "dev", "docs"]
files = [
- {file = "importlib_metadata-8.6.1-py3-none-any.whl", hash = "sha256:02a89390c1e15fdfdc0d7c6b25cb3e62650d0494005c97d6f148bf5b9787525e"},
- {file = "importlib_metadata-8.6.1.tar.gz", hash = "sha256:310b41d755445d74569f993ccfc22838295d9fe005425094fad953d7f15c8580"},
+ {file = "importlib_metadata-8.7.0-py3-none-any.whl", hash = "sha256:e5dd1551894c77868a30651cef00984d50e1002d06942a7101d34870c5f02afd"},
+ {file = "importlib_metadata-8.7.0.tar.gz", hash = "sha256:d13b81ad223b890aa16c5471f2ac3056cf76c5f10f82d6f9292f0b415f389000"},
]
markers = {main = "extra == \"daft\" and python_version < \"3.10\"", dev = "python_full_version < \"3.10.2\"", docs = "python_version < \"3.10\""}
@@ -2147,18 +2263,18 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
-version = "1.0.4"
+version = "1.1.0"
description = "The ultimate Python library for JOSE RFCs, including JWS, JWE, JWK, JWA, JWT"
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "joserfc-1.0.4-py3-none-any.whl", hash = "sha256:ecf3a5999f89d3a663485ab7c4f633541586d6f44e664ee760197299f39ed51b"},
- {file = "joserfc-1.0.4.tar.gz", hash = "sha256:dc3fc216cfcfc952d4c0d4b06c759a04711af0b667e5973adc47dbb1ba784127"},
+ {file = "joserfc-1.1.0-py3-none-any.whl", hash = "sha256:9493512cfffb9bc3001e8f609fe0eb7e95b71f3d3b374ede93de94b4b6b520f5"},
+ {file = "joserfc-1.1.0.tar.gz", hash = "sha256:a8f3442b04c233f742f7acde0d0dcd926414e9542a6337096b2b4e5f435f36c1"},
]
[package.dependencies]
@@ -2212,14 +2328,14 @@ files = [
[[package]]
name = "jsonschema"
-version = "4.23.0"
+version = "4.24.0"
description = "An implementation of JSON Schema validation for Python"
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "jsonschema-4.23.0-py3-none-any.whl", hash = "sha256:fbadb6f8b144a8f8cf9f0b89ba94501d143e50411a1278633f56a7acf7fd5566"},
- {file = "jsonschema-4.23.0.tar.gz", hash = "sha256:d71497fef26351a33265337fa77ffeb82423f3ea21283cd9467bb03999266bc4"},
+ {file = "jsonschema-4.24.0-py3-none-any.whl", hash = "sha256:a462455f19f5faf404a7902952b6f0e3ce868f3ee09a359b05eca6673bd8412d"},
+ {file = "jsonschema-4.24.0.tar.gz", hash = "sha256:0b4e8069eb12aedfa881333004bccaec24ecef5a8a6a4b6df142b2cc9599d196"},
]
markers = {main = "extra == \"ray\""}
@@ -2253,14 +2369,14 @@ requests = ">=2.31.0,<3.0.0"
[[package]]
name = "jsonschema-specifications"
-version = "2024.10.1"
+version = "2025.4.1"
description = "The JSON Schema meta-schemas and vocabularies, exposed as a Registry"
optional = false
python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "jsonschema_specifications-2024.10.1-py3-none-any.whl", hash = "sha256:a09a0680616357d9a0ecf05c12ad234479f549239d0f5b55f3deea67475da9bf"},
- {file = "jsonschema_specifications-2024.10.1.tar.gz", hash = "sha256:0f38b83639958ce1152d02a7f062902c41c8fd20d558b0c34344292d417ae272"},
+ {file = "jsonschema_specifications-2025.4.1-py3-none-any.whl", hash = "sha256:4653bffbd6584f7de83a67e0d620ef16900b390ddc7939d56684d6c81e33f1af"},
+ {file = "jsonschema_specifications-2025.4.1.tar.gz", hash = "sha256:630159c9f4dbea161a6a2205c3011cc4f18ff381b189fff48bb39b9bf26ae608"},
]
markers = {main = "extra == \"ray\""}
@@ -2284,68 +2400,45 @@ files = [
[[package]]
name = "lazy-object-proxy"
-version = "1.10.0"
+version = "1.11.0"
description = "A fast and thorough lazy object proxy."
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "lazy-object-proxy-1.10.0.tar.gz", hash = "sha256:78247b6d45f43a52ef35c25b5581459e85117225408a4128a3daf8bf9648ac69"},
- {file = "lazy_object_proxy-1.10.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:855e068b0358ab916454464a884779c7ffa312b8925c6f7401e952dcf3b89977"},
- {file = "lazy_object_proxy-1.10.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7ab7004cf2e59f7c2e4345604a3e6ea0d92ac44e1c2375527d56492014e690c3"},
- {file = "lazy_object_proxy-1.10.0-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dc0d2fc424e54c70c4bc06787e4072c4f3b1aa2f897dfdc34ce1013cf3ceef05"},
- {file = "lazy_object_proxy-1.10.0-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:e2adb09778797da09d2b5ebdbceebf7dd32e2c96f79da9052b2e87b6ea495895"},
- {file = "lazy_object_proxy-1.10.0-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:b1f711e2c6dcd4edd372cf5dec5c5a30d23bba06ee012093267b3376c079ec83"},
- {file = "lazy_object_proxy-1.10.0-cp310-cp310-win32.whl", hash = "sha256:76a095cfe6045c7d0ca77db9934e8f7b71b14645f0094ffcd842349ada5c5fb9"},
- {file = "lazy_object_proxy-1.10.0-cp310-cp310-win_amd64.whl", hash = "sha256:b4f87d4ed9064b2628da63830986c3d2dca7501e6018347798313fcf028e2fd4"},
- {file = "lazy_object_proxy-1.10.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:fec03caabbc6b59ea4a638bee5fce7117be8e99a4103d9d5ad77f15d6f81020c"},
- {file = "lazy_object_proxy-1.10.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:02c83f957782cbbe8136bee26416686a6ae998c7b6191711a04da776dc9e47d4"},
- {file = "lazy_object_proxy-1.10.0-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:009e6bb1f1935a62889ddc8541514b6a9e1fcf302667dcb049a0be5c8f613e56"},
- {file = "lazy_object_proxy-1.10.0-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:75fc59fc450050b1b3c203c35020bc41bd2695ed692a392924c6ce180c6f1dc9"},
- {file = "lazy_object_proxy-1.10.0-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:782e2c9b2aab1708ffb07d4bf377d12901d7a1d99e5e410d648d892f8967ab1f"},
- {file = "lazy_object_proxy-1.10.0-cp311-cp311-win32.whl", hash = "sha256:edb45bb8278574710e68a6b021599a10ce730d156e5b254941754a9cc0b17d03"},
- {file = "lazy_object_proxy-1.10.0-cp311-cp311-win_amd64.whl", hash = "sha256:e271058822765ad5e3bca7f05f2ace0de58a3f4e62045a8c90a0dfd2f8ad8cc6"},
- {file = "lazy_object_proxy-1.10.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:e98c8af98d5707dcdecc9ab0863c0ea6e88545d42ca7c3feffb6b4d1e370c7ba"},
- {file = "lazy_object_proxy-1.10.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:952c81d415b9b80ea261d2372d2a4a2332a3890c2b83e0535f263ddfe43f0d43"},
- {file = "lazy_object_proxy-1.10.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:80b39d3a151309efc8cc48675918891b865bdf742a8616a337cb0090791a0de9"},
- {file = "lazy_object_proxy-1.10.0-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:e221060b701e2aa2ea991542900dd13907a5c90fa80e199dbf5a03359019e7a3"},
- {file = "lazy_object_proxy-1.10.0-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:92f09ff65ecff3108e56526f9e2481b8116c0b9e1425325e13245abfd79bdb1b"},
- {file = "lazy_object_proxy-1.10.0-cp312-cp312-win32.whl", hash = "sha256:3ad54b9ddbe20ae9f7c1b29e52f123120772b06dbb18ec6be9101369d63a4074"},
- {file = "lazy_object_proxy-1.10.0-cp312-cp312-win_amd64.whl", hash = "sha256:127a789c75151db6af398b8972178afe6bda7d6f68730c057fbbc2e96b08d282"},
- {file = "lazy_object_proxy-1.10.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:9e4ed0518a14dd26092614412936920ad081a424bdcb54cc13349a8e2c6d106a"},
- {file = "lazy_object_proxy-1.10.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5ad9e6ed739285919aa9661a5bbed0aaf410aa60231373c5579c6b4801bd883c"},
- {file = "lazy_object_proxy-1.10.0-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2fc0a92c02fa1ca1e84fc60fa258458e5bf89d90a1ddaeb8ed9cc3147f417255"},
- {file = "lazy_object_proxy-1.10.0-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:0aefc7591920bbd360d57ea03c995cebc204b424524a5bd78406f6e1b8b2a5d8"},
- {file = "lazy_object_proxy-1.10.0-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:5faf03a7d8942bb4476e3b62fd0f4cf94eaf4618e304a19865abf89a35c0bbee"},
- {file = "lazy_object_proxy-1.10.0-cp38-cp38-win32.whl", hash = "sha256:e333e2324307a7b5d86adfa835bb500ee70bfcd1447384a822e96495796b0ca4"},
- {file = "lazy_object_proxy-1.10.0-cp38-cp38-win_amd64.whl", hash = "sha256:cb73507defd385b7705c599a94474b1d5222a508e502553ef94114a143ec6696"},
- {file = "lazy_object_proxy-1.10.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:366c32fe5355ef5fc8a232c5436f4cc66e9d3e8967c01fb2e6302fd6627e3d94"},
- {file = "lazy_object_proxy-1.10.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2297f08f08a2bb0d32a4265e98a006643cd7233fb7983032bd61ac7a02956b3b"},
- {file = "lazy_object_proxy-1.10.0-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:18dd842b49456aaa9a7cf535b04ca4571a302ff72ed8740d06b5adcd41fe0757"},
- {file = "lazy_object_proxy-1.10.0-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:217138197c170a2a74ca0e05bddcd5f1796c735c37d0eee33e43259b192aa424"},
- {file = "lazy_object_proxy-1.10.0-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:9a3a87cf1e133e5b1994144c12ca4aa3d9698517fe1e2ca82977781b16955658"},
- {file = "lazy_object_proxy-1.10.0-cp39-cp39-win32.whl", hash = "sha256:30b339b2a743c5288405aa79a69e706a06e02958eab31859f7f3c04980853b70"},
- {file = "lazy_object_proxy-1.10.0-cp39-cp39-win_amd64.whl", hash = "sha256:a899b10e17743683b293a729d3a11f2f399e8a90c73b089e29f5d0fe3509f0dd"},
- {file = "lazy_object_proxy-1.10.0-pp310.pp311.pp312.pp38.pp39-none-any.whl", hash = "sha256:80fa48bd89c8f2f456fc0765c11c23bf5af827febacd2f523ca5bc1893fcc09d"},
+ {file = "lazy_object_proxy-1.11.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:132bc8a34f2f2d662a851acfd1b93df769992ed1b81e2b1fda7db3e73b0d5a18"},
+ {file = "lazy_object_proxy-1.11.0-cp310-cp310-win_amd64.whl", hash = "sha256:01261a3afd8621a1accb5682df2593dc7ec7d21d38f411011a5712dcd418fbed"},
+ {file = "lazy_object_proxy-1.11.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:090935756cc041e191f22f4f9c7fd4fe9a454717067adf5b1bbd2ce3046b556e"},
+ {file = "lazy_object_proxy-1.11.0-cp311-cp311-win_amd64.whl", hash = "sha256:76ec715017f06410f57df442c1a8d66e6b5f7035077785b129817f5ae58810a4"},
+ {file = "lazy_object_proxy-1.11.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:9a9f39098e93a63618a79eef2889ae3cf0605f676cd4797fdfd49fcd7ddc318b"},
+ {file = "lazy_object_proxy-1.11.0-cp312-cp312-win_amd64.whl", hash = "sha256:ee13f67f4fcd044ef27bfccb1c93d39c100046fec1fad6e9a1fcdfd17492aeb3"},
+ {file = "lazy_object_proxy-1.11.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:fd4c84eafd8dd15ea16f7d580758bc5c2ce1f752faec877bb2b1f9f827c329cd"},
+ {file = "lazy_object_proxy-1.11.0-cp313-cp313-win_amd64.whl", hash = "sha256:d2503427bda552d3aefcac92f81d9e7ca631e680a2268cbe62cd6a58de6409b7"},
+ {file = "lazy_object_proxy-1.11.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:0613116156801ab3fccb9e2b05ed83b08ea08c2517fdc6c6bc0d4697a1a376e3"},
+ {file = "lazy_object_proxy-1.11.0-cp313-cp313t-win_amd64.whl", hash = "sha256:bb03c507d96b65f617a6337dedd604399d35face2cdf01526b913fb50c4cb6e8"},
+ {file = "lazy_object_proxy-1.11.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:28c174db37946f94b97a97b579932ff88f07b8d73a46b6b93322b9ac06794a3b"},
+ {file = "lazy_object_proxy-1.11.0-cp39-cp39-win_amd64.whl", hash = "sha256:d662f0669e27704495ff1f647070eb8816931231c44e583f4d0701b7adf6272f"},
+ {file = "lazy_object_proxy-1.11.0-py3-none-any.whl", hash = "sha256:a56a5093d433341ff7da0e89f9b486031ccd222ec8e52ec84d0ec1cdc819674b"},
+ {file = "lazy_object_proxy-1.11.0.tar.gz", hash = "sha256:18874411864c9fbbbaa47f9fc1dd7aea754c86cfde21278ef427639d1dd78e9c"},
]
[[package]]
name = "markdown"
-version = "3.7"
+version = "3.8"
description = "Python implementation of John Gruber's Markdown."
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["docs"]
files = [
- {file = "Markdown-3.7-py3-none-any.whl", hash = "sha256:7eb6df5690b81a1d7942992c97fad2938e956e79df20cbc6186e9c3a77b1c803"},
- {file = "markdown-3.7.tar.gz", hash = "sha256:2ae2471477cfd02dbbf038d5d9bc226d40def84b4fe2986e49b59b6b472bbed2"},
+ {file = "markdown-3.8-py3-none-any.whl", hash = "sha256:794a929b79c5af141ef5ab0f2f642d0f7b1872981250230e72682346f7cc90dc"},
+ {file = "markdown-3.8.tar.gz", hash = "sha256:7df81e63f0df5c4b24b7d156eb81e4690595239b7d70937d0409f1b0de319c6f"},
]
[package.dependencies]
importlib-metadata = {version = ">=4.4", markers = "python_version < \"3.10\""}
[package.extras]
-docs = ["mdx-gh-links (>=0.2)", "mkdocs (>=1.5)", "mkdocs-gen-files", "mkdocs-literate-nav", "mkdocs-nature (>=0.6)", "mkdocs-section-index", "mkdocstrings[python]"]
+docs = ["mdx_gh_links (>=0.2)", "mkdocs (>=1.6)", "mkdocs-gen-files", "mkdocs-literate-nav", "mkdocs-nature (>=0.6)", "mkdocs-section-index", "mkdocstrings[python]"]
testing = ["coverage", "pyyaml"]
[[package]]
@@ -2845,15 +2938,15 @@ tests = ["pytest (>=4.6)"]
[[package]]
name = "msal"
-version = "1.32.0"
+version = "1.32.3"
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 = ">=3.7"
groups = ["main"]
markers = "extra == \"adlfs\""
files = [
- {file = "msal-1.32.0-py3-none-any.whl", hash = "sha256:9dbac5384a10bbbf4dae5c7ea0d707d14e087b92c5aa4954b3feaa2d1aa0bcb7"},
- {file = "msal-1.32.0.tar.gz", hash = "sha256:5445fe3af1da6be484991a7ab32eaa82461dc2347de105b76af92c610c3335c2"},
+ {file = "msal-1.32.3-py3-none-any.whl", hash = "sha256:b2798db57760b1961b142f027ffb7c8169536bf77316e99a0df5c4aaebb11569"},
+ {file = "msal-1.32.3.tar.gz", hash = "sha256:5eea038689c78a5a70ca8ecbe1245458b55a857bd096efb6989c69ba15985d35"},
]
[package.dependencies]
@@ -2960,118 +3053,144 @@ files = [
[[package]]
name = "multidict"
-version = "6.2.0"
+version = "6.4.4"
description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
-files = [
- {file = "multidict-6.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:b9f6392d98c0bd70676ae41474e2eecf4c7150cb419237a41f8f96043fcb81d1"},
- {file = "multidict-6.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:3501621d5e86f1a88521ea65d5cad0a0834c77b26f193747615b7c911e5422d2"},
- {file = "multidict-6.2.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:32ed748ff9ac682eae7859790d3044b50e3076c7d80e17a44239683769ff485e"},
- {file = "multidict-6.2.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cc826b9a8176e686b67aa60fd6c6a7047b0461cae5591ea1dc73d28f72332a8a"},
- {file = "multidict-6.2.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:214207dcc7a6221d9942f23797fe89144128a71c03632bf713d918db99bd36de"},
- {file = "multidict-6.2.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:05fefbc3cddc4e36da209a5e49f1094bbece9a581faa7f3589201fd95df40e5d"},
- {file = "multidict-6.2.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e851e6363d0dbe515d8de81fd544a2c956fdec6f8a049739562286727d4a00c3"},
- {file = "multidict-6.2.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:32c9b4878f48be3e75808ea7e499d6223b1eea6d54c487a66bc10a1871e3dc6a"},
- {file = "multidict-6.2.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:7243c5a6523c5cfeca76e063efa5f6a656d1d74c8b1fc64b2cd1e84e507f7e2a"},
- {file = "multidict-6.2.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:0e5a644e50ef9fb87878d4d57907f03a12410d2aa3b93b3acdf90a741df52c49"},
- {file = "multidict-6.2.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:0dc25a3293c50744796e87048de5e68996104d86d940bb24bc3ec31df281b191"},
- {file = "multidict-6.2.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:a49994481b99cd7dedde07f2e7e93b1d86c01c0fca1c32aded18f10695ae17eb"},
- {file = "multidict-6.2.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:641cf2e3447c9ecff2f7aa6e9eee9eaa286ea65d57b014543a4911ff2799d08a"},
- {file = "multidict-6.2.0-cp310-cp310-win32.whl", hash = "sha256:0c383d28857f66f5aebe3e91d6cf498da73af75fbd51cedbe1adfb85e90c0460"},
- {file = "multidict-6.2.0-cp310-cp310-win_amd64.whl", hash = "sha256:a33273a541f1e1a8219b2a4ed2de355848ecc0254264915b9290c8d2de1c74e1"},
- {file = "multidict-6.2.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:84e87a7d75fa36839a3a432286d719975362d230c70ebfa0948549cc38bd5b46"},
- {file = "multidict-6.2.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:8de4d42dffd5ced9117af2ce66ba8722402541a3aa98ffdf78dde92badb68932"},
- {file = "multidict-6.2.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e7d91a230c7f8af86c904a5a992b8c064b66330544693fd6759c3d6162382ecf"},
- {file = "multidict-6.2.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9f6cad071960ba1914fa231677d21b1b4a3acdcce463cee41ea30bc82e6040cf"},
- {file = "multidict-6.2.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0f74f2fc51555f4b037ef278efc29a870d327053aba5cb7d86ae572426c7cccc"},
- {file = "multidict-6.2.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:14ed9ed1bfedd72a877807c71113deac292bf485159a29025dfdc524c326f3e1"},
- {file = "multidict-6.2.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4ac3fcf9a2d369bd075b2c2965544036a27ccd277fc3c04f708338cc57533081"},
- {file = "multidict-6.2.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2fc6af8e39f7496047c7876314f4317736eac82bf85b54c7c76cf1a6f8e35d98"},
- {file = "multidict-6.2.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:5f8cb1329f42fadfb40d6211e5ff568d71ab49be36e759345f91c69d1033d633"},
- {file = "multidict-6.2.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:5389445f0173c197f4a3613713b5fb3f3879df1ded2a1a2e4bc4b5b9c5441b7e"},
- {file = "multidict-6.2.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:94a7bb972178a8bfc4055db80c51efd24baefaced5e51c59b0d598a004e8305d"},
- {file = "multidict-6.2.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:da51d8928ad8b4244926fe862ba1795f0b6e68ed8c42cd2f822d435db9c2a8f4"},
- {file = "multidict-6.2.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:063be88bd684782a0715641de853e1e58a2f25b76388538bd62d974777ce9bc2"},
- {file = "multidict-6.2.0-cp311-cp311-win32.whl", hash = "sha256:52b05e21ff05729fbea9bc20b3a791c3c11da61649ff64cce8257c82a020466d"},
- {file = "multidict-6.2.0-cp311-cp311-win_amd64.whl", hash = "sha256:1e2a2193d3aa5cbf5758f6d5680a52aa848e0cf611da324f71e5e48a9695cc86"},
- {file = "multidict-6.2.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:437c33561edb6eb504b5a30203daf81d4a9b727e167e78b0854d9a4e18e8950b"},
- {file = "multidict-6.2.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:9f49585f4abadd2283034fc605961f40c638635bc60f5162276fec075f2e37a4"},
- {file = "multidict-6.2.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:5dd7106d064d05896ce28c97da3f46caa442fe5a43bc26dfb258e90853b39b44"},
- {file = "multidict-6.2.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e25b11a0417475f093d0f0809a149aff3943c2c56da50fdf2c3c88d57fe3dfbd"},
- {file = "multidict-6.2.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ac380cacdd3b183338ba63a144a34e9044520a6fb30c58aa14077157a033c13e"},
- {file = "multidict-6.2.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:61d5541f27533f803a941d3a3f8a3d10ed48c12cf918f557efcbf3cd04ef265c"},
- {file = "multidict-6.2.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:facaf11f21f3a4c51b62931feb13310e6fe3475f85e20d9c9fdce0d2ea561b87"},
- {file = "multidict-6.2.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:095a2eabe8c43041d3e6c2cb8287a257b5f1801c2d6ebd1dd877424f1e89cf29"},
- {file = "multidict-6.2.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:a0cc398350ef31167e03f3ca7c19313d4e40a662adcb98a88755e4e861170bdd"},
- {file = "multidict-6.2.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:7c611345bbe7cb44aabb877cb94b63e86f2d0db03e382667dbd037866d44b4f8"},
- {file = "multidict-6.2.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:8cd1a0644ccaf27e9d2f6d9c9474faabee21f0578fe85225cc5af9a61e1653df"},
- {file = "multidict-6.2.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:89b3857652183b8206a891168af47bac10b970d275bba1f6ee46565a758c078d"},
- {file = "multidict-6.2.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:125dd82b40f8c06d08d87b3510beaccb88afac94e9ed4a6f6c71362dc7dbb04b"},
- {file = "multidict-6.2.0-cp312-cp312-win32.whl", hash = "sha256:76b34c12b013d813e6cb325e6bd4f9c984db27758b16085926bbe7ceeaace626"},
- {file = "multidict-6.2.0-cp312-cp312-win_amd64.whl", hash = "sha256:0b183a959fb88ad1be201de2c4bdf52fa8e46e6c185d76201286a97b6f5ee65c"},
- {file = "multidict-6.2.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:5c5e7d2e300d5cb3b2693b6d60d3e8c8e7dd4ebe27cd17c9cb57020cac0acb80"},
- {file = "multidict-6.2.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:256d431fe4583c5f1e0f2e9c4d9c22f3a04ae96009b8cfa096da3a8723db0a16"},
- {file = "multidict-6.2.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:a3c0ff89fe40a152e77b191b83282c9664357dce3004032d42e68c514ceff27e"},
- {file = "multidict-6.2.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ef7d48207926edbf8b16b336f779c557dd8f5a33035a85db9c4b0febb0706817"},
- {file = "multidict-6.2.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1f3c099d3899b14e1ce52262eb82a5f5cb92157bb5106bf627b618c090a0eadc"},
- {file = "multidict-6.2.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e16e7297f29a544f49340012d6fc08cf14de0ab361c9eb7529f6a57a30cbfda1"},
- {file = "multidict-6.2.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:042028348dc5a1f2be6c666437042a98a5d24cee50380f4c0902215e5ec41844"},
- {file = "multidict-6.2.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:08549895e6a799bd551cf276f6e59820aa084f0f90665c0f03dd3a50db5d3c48"},
- {file = "multidict-6.2.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:4ccfd74957ef53fa7380aaa1c961f523d582cd5e85a620880ffabd407f8202c0"},
- {file = "multidict-6.2.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:83b78c680d4b15d33042d330c2fa31813ca3974197bddb3836a5c635a5fd013f"},
- {file = "multidict-6.2.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:b4c153863dd6569f6511845922c53e39c8d61f6e81f228ad5443e690fca403de"},
- {file = "multidict-6.2.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:98aa8325c7f47183b45588af9c434533196e241be0a4e4ae2190b06d17675c02"},
- {file = "multidict-6.2.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:9e658d1373c424457ddf6d55ec1db93c280b8579276bebd1f72f113072df8a5d"},
- {file = "multidict-6.2.0-cp313-cp313-win32.whl", hash = "sha256:3157126b028c074951839233647bd0e30df77ef1fedd801b48bdcad242a60f4e"},
- {file = "multidict-6.2.0-cp313-cp313-win_amd64.whl", hash = "sha256:2e87f1926e91855ae61769ba3e3f7315120788c099677e0842e697b0bfb659f2"},
- {file = "multidict-6.2.0-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:2529ddbdaa424b2c6c2eb668ea684dd6b75b839d0ad4b21aad60c168269478d7"},
- {file = "multidict-6.2.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:13551d0e2d7201f0959725a6a769b6f7b9019a168ed96006479c9ac33fe4096b"},
- {file = "multidict-6.2.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:d1996ee1330e245cd3aeda0887b4409e3930524c27642b046e4fae88ffa66c5e"},
- {file = "multidict-6.2.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c537da54ce4ff7c15e78ab1292e5799d0d43a2108e006578a57f531866f64025"},
- {file = "multidict-6.2.0-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0f249badb360b0b4d694307ad40f811f83df4da8cef7b68e429e4eea939e49dd"},
- {file = "multidict-6.2.0-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:48d39b1824b8d6ea7de878ef6226efbe0773f9c64333e1125e0efcfdd18a24c7"},
- {file = "multidict-6.2.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b99aac6bb2c37db336fa03a39b40ed4ef2818bf2dfb9441458165ebe88b793af"},
- {file = "multidict-6.2.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:07bfa8bc649783e703263f783f73e27fef8cd37baaad4389816cf6a133141331"},
- {file = "multidict-6.2.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:b2c00ad31fbc2cbac85d7d0fcf90853b2ca2e69d825a2d3f3edb842ef1544a2c"},
- {file = "multidict-6.2.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:0d57a01a2a9fa00234aace434d8c131f0ac6e0ac6ef131eda5962d7e79edfb5b"},
- {file = "multidict-6.2.0-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:abf5b17bc0cf626a8a497d89ac691308dbd825d2ac372aa990b1ca114e470151"},
- {file = "multidict-6.2.0-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:f7716f7e7138252d88607228ce40be22660d6608d20fd365d596e7ca0738e019"},
- {file = "multidict-6.2.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:d5a36953389f35f0a4e88dc796048829a2f467c9197265504593f0e420571547"},
- {file = "multidict-6.2.0-cp313-cp313t-win32.whl", hash = "sha256:e653d36b1bf48fa78c7fcebb5fa679342e025121ace8c87ab05c1cefd33b34fc"},
- {file = "multidict-6.2.0-cp313-cp313t-win_amd64.whl", hash = "sha256:ca23db5fb195b5ef4fd1f77ce26cadefdf13dba71dab14dadd29b34d457d7c44"},
- {file = "multidict-6.2.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:b4f3d66dd0354b79761481fc15bdafaba0b9d9076f1f42cc9ce10d7fcbda205a"},
- {file = "multidict-6.2.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6e2a2d6749e1ff2c9c76a72c6530d5baa601205b14e441e6d98011000f47a7ac"},
- {file = "multidict-6.2.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:cca83a629f77402cfadd58352e394d79a61c8015f1694b83ab72237ec3941f88"},
- {file = "multidict-6.2.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:781b5dd1db18c9e9eacc419027b0acb5073bdec9de1675c0be25ceb10e2ad133"},
- {file = "multidict-6.2.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:cf8d370b2fea27fb300825ec3984334f7dd54a581bde6456799ba3776915a656"},
- {file = "multidict-6.2.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:25bb96338512e2f46f615a2bb7c6012fe92a4a5ebd353e5020836a7e33120349"},
- {file = "multidict-6.2.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:19e2819b0b468174de25c0ceed766606a07cedeab132383f1e83b9a4e96ccb4f"},
- {file = "multidict-6.2.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6aed763b6a1b28c46c055692836879328f0b334a6d61572ee4113a5d0c859872"},
- {file = "multidict-6.2.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:a1133414b771619aa3c3000701c11b2e4624a7f492f12f256aedde97c28331a2"},
- {file = "multidict-6.2.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:639556758c36093b35e2e368ca485dada6afc2bd6a1b1207d85ea6dfc3deab27"},
- {file = "multidict-6.2.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:163f4604e76639f728d127293d24c3e208b445b463168af3d031b92b0998bb90"},
- {file = "multidict-6.2.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:2325105e16d434749e1be8022f942876a936f9bece4ec41ae244e3d7fae42aaf"},
- {file = "multidict-6.2.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:e4371591e621579cb6da8401e4ea405b33ff25a755874a3567c4075ca63d56e2"},
- {file = "multidict-6.2.0-cp39-cp39-win32.whl", hash = "sha256:d1175b0e0d6037fab207f05774a176d71210ebd40b1c51f480a04b65ec5c786d"},
- {file = "multidict-6.2.0-cp39-cp39-win_amd64.whl", hash = "sha256:ad81012b24b88aad4c70b2cbc2dad84018783221b7f923e926f4690ff8569da3"},
- {file = "multidict-6.2.0-py3-none-any.whl", hash = "sha256:5d26547423e5e71dcc562c4acdc134b900640a39abd9066d7326a7cc2324c530"},
- {file = "multidict-6.2.0.tar.gz", hash = "sha256:0085b0afb2446e57050140240a8595846ed64d1cbd26cef936bfab3192c673b8"},
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+files = [
+ {file = "multidict-6.4.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:8adee3ac041145ffe4488ea73fa0a622b464cc25340d98be76924d0cda8545ff"},
+ {file = "multidict-6.4.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b61e98c3e2a861035aaccd207da585bdcacef65fe01d7a0d07478efac005e028"},
+ {file = "multidict-6.4.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:75493f28dbadecdbb59130e74fe935288813301a8554dc32f0c631b6bdcdf8b0"},
+ {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4ffc3c6a37e048b5395ee235e4a2a0d639c2349dffa32d9367a42fc20d399772"},
+ {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:87cb72263946b301570b0f63855569a24ee8758aaae2cd182aae7d95fbc92ca7"},
+ {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9bbf7bd39822fd07e3609b6b4467af4c404dd2b88ee314837ad1830a7f4a8299"},
+ {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d1f7cbd4f1f44ddf5fd86a8675b7679176eae770f2fc88115d6dddb6cefb59bc"},
+ {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bb5ac9e5bfce0e6282e7f59ff7b7b9a74aa8e5c60d38186a4637f5aa764046ad"},
+ {file = "multidict-6.4.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4efc31dfef8c4eeb95b6b17d799eedad88c4902daba39ce637e23a17ea078915"},
+ {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:9fcad2945b1b91c29ef2b4050f590bfcb68d8ac8e0995a74e659aa57e8d78e01"},
+ {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:d877447e7368c7320832acb7159557e49b21ea10ffeb135c1077dbbc0816b598"},
+ {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:33a12ebac9f380714c298cbfd3e5b9c0c4e89c75fe612ae496512ee51028915f"},
+ {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:0f14ea68d29b43a9bf37953881b1e3eb75b2739e896ba4a6aa4ad4c5b9ffa145"},
+ {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:0327ad2c747a6600e4797d115d3c38a220fdb28e54983abe8964fd17e95ae83c"},
+ {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:d1a20707492db9719a05fc62ee215fd2c29b22b47c1b1ba347f9abc831e26683"},
+ {file = "multidict-6.4.4-cp310-cp310-win32.whl", hash = "sha256:d83f18315b9fca5db2452d1881ef20f79593c4aa824095b62cb280019ef7aa3d"},
+ {file = "multidict-6.4.4-cp310-cp310-win_amd64.whl", hash = "sha256:9c17341ee04545fd962ae07330cb5a39977294c883485c8d74634669b1f7fe04"},
+ {file = "multidict-6.4.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:4f5f29794ac0e73d2a06ac03fd18870adc0135a9d384f4a306a951188ed02f95"},
+ {file = "multidict-6.4.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c04157266344158ebd57b7120d9b0b35812285d26d0e78193e17ef57bfe2979a"},
+ {file = "multidict-6.4.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:bb61ffd3ab8310d93427e460f565322c44ef12769f51f77277b4abad7b6f7223"},
+ {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5e0ba18a9afd495f17c351d08ebbc4284e9c9f7971d715f196b79636a4d0de44"},
+ {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:9faf1b1dcaadf9f900d23a0e6d6c8eadd6a95795a0e57fcca73acce0eb912065"},
+ {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a4d1cb1327c6082c4fce4e2a438483390964c02213bc6b8d782cf782c9b1471f"},
+ {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:941f1bec2f5dbd51feeb40aea654c2747f811ab01bdd3422a48a4e4576b7d76a"},
+ {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e5f8a146184da7ea12910a4cec51ef85e44f6268467fb489c3caf0cd512f29c2"},
+ {file = "multidict-6.4.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:232b7237e57ec3c09be97206bfb83a0aa1c5d7d377faa019c68a210fa35831f1"},
+ {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:55ae0721c1513e5e3210bca4fc98456b980b0c2c016679d3d723119b6b202c42"},
+ {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:51d662c072579f63137919d7bb8fc250655ce79f00c82ecf11cab678f335062e"},
+ {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:0e05c39962baa0bb19a6b210e9b1422c35c093b651d64246b6c2e1a7e242d9fd"},
+ {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:d5b1cc3ab8c31d9ebf0faa6e3540fb91257590da330ffe6d2393d4208e638925"},
+ {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:93ec84488a384cd7b8a29c2c7f467137d8a73f6fe38bb810ecf29d1ade011a7c"},
+ {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:b308402608493638763abc95f9dc0030bbd6ac6aff784512e8ac3da73a88af08"},
+ {file = "multidict-6.4.4-cp311-cp311-win32.whl", hash = "sha256:343892a27d1a04d6ae455ecece12904d242d299ada01633d94c4f431d68a8c49"},
+ {file = "multidict-6.4.4-cp311-cp311-win_amd64.whl", hash = "sha256:73484a94f55359780c0f458bbd3c39cb9cf9c182552177d2136e828269dee529"},
+ {file = "multidict-6.4.4-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:dc388f75a1c00000824bf28b7633e40854f4127ede80512b44c3cfeeea1839a2"},
+ {file = "multidict-6.4.4-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:98af87593a666f739d9dba5d0ae86e01b0e1a9cfcd2e30d2d361fbbbd1a9162d"},
+ {file = "multidict-6.4.4-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:aff4cafea2d120327d55eadd6b7f1136a8e5a0ecf6fb3b6863e8aca32cd8e50a"},
+ {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:169c4ba7858176b797fe551d6e99040c531c775d2d57b31bcf4de6d7a669847f"},
+ {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:b9eb4c59c54421a32b3273d4239865cb14ead53a606db066d7130ac80cc8ec93"},
+ {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7cf3bd54c56aa16fdb40028d545eaa8d051402b61533c21e84046e05513d5780"},
+ {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f682c42003c7264134bfe886376299db4cc0c6cd06a3295b41b347044bcb5482"},
+ {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a920f9cf2abdf6e493c519492d892c362007f113c94da4c239ae88429835bad1"},
+ {file = "multidict-6.4.4-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:530d86827a2df6504526106b4c104ba19044594f8722d3e87714e847c74a0275"},
+ {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:ecde56ea2439b96ed8a8d826b50c57364612ddac0438c39e473fafad7ae1c23b"},
+ {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:dc8c9736d8574b560634775ac0def6bdc1661fc63fa27ffdfc7264c565bcb4f2"},
+ {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:7f3d3b3c34867579ea47cbd6c1f2ce23fbfd20a273b6f9e3177e256584f1eacc"},
+ {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:87a728af265e08f96b6318ebe3c0f68b9335131f461efab2fc64cc84a44aa6ed"},
+ {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9f193eeda1857f8e8d3079a4abd258f42ef4a4bc87388452ed1e1c4d2b0c8740"},
+ {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:be06e73c06415199200e9a2324a11252a3d62030319919cde5e6950ffeccf72e"},
+ {file = "multidict-6.4.4-cp312-cp312-win32.whl", hash = "sha256:622f26ea6a7e19b7c48dd9228071f571b2fbbd57a8cd71c061e848f281550e6b"},
+ {file = "multidict-6.4.4-cp312-cp312-win_amd64.whl", hash = "sha256:5e2bcda30d5009996ff439e02a9f2b5c3d64a20151d34898c000a6281faa3781"},
+ {file = "multidict-6.4.4-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:82ffabefc8d84c2742ad19c37f02cde5ec2a1ee172d19944d380f920a340e4b9"},
+ {file = "multidict-6.4.4-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:6a2f58a66fe2c22615ad26156354005391e26a2f3721c3621504cd87c1ea87bf"},
+ {file = "multidict-6.4.4-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:5883d6ee0fd9d8a48e9174df47540b7545909841ac82354c7ae4cbe9952603bd"},
+ {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9abcf56a9511653fa1d052bfc55fbe53dbee8f34e68bd6a5a038731b0ca42d15"},
+ {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:6ed5ae5605d4ad5a049fad2a28bb7193400700ce2f4ae484ab702d1e3749c3f9"},
+ {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:bbfcb60396f9bcfa63e017a180c3105b8c123a63e9d1428a36544e7d37ca9e20"},
+ {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b0f1987787f5f1e2076b59692352ab29a955b09ccc433c1f6b8e8e18666f608b"},
+ {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1d0121ccce8c812047d8d43d691a1ad7641f72c4f730474878a5aeae1b8ead8c"},
+ {file = "multidict-6.4.4-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:83ec4967114295b8afd120a8eec579920c882831a3e4c3331d591a8e5bfbbc0f"},
+ {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:995f985e2e268deaf17867801b859a282e0448633f1310e3704b30616d269d69"},
+ {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:d832c608f94b9f92a0ec8b7e949be7792a642b6e535fcf32f3e28fab69eeb046"},
+ {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:d21c1212171cf7da703c5b0b7a0e85be23b720818aef502ad187d627316d5645"},
+ {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:cbebaa076aaecad3d4bb4c008ecc73b09274c952cf6a1b78ccfd689e51f5a5b0"},
+ {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:c93a6fb06cc8e5d3628b2b5fda215a5db01e8f08fc15fadd65662d9b857acbe4"},
+ {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8cd8f81f1310182362fb0c7898145ea9c9b08a71081c5963b40ee3e3cac589b1"},
+ {file = "multidict-6.4.4-cp313-cp313-win32.whl", hash = "sha256:3e9f1cd61a0ab857154205fb0b1f3d3ace88d27ebd1409ab7af5096e409614cd"},
+ {file = "multidict-6.4.4-cp313-cp313-win_amd64.whl", hash = "sha256:8ffb40b74400e4455785c2fa37eba434269149ec525fc8329858c862e4b35373"},
+ {file = "multidict-6.4.4-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:6a602151dbf177be2450ef38966f4be3467d41a86c6a845070d12e17c858a156"},
+ {file = "multidict-6.4.4-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:0d2b9712211b860d123815a80b859075d86a4d54787e247d7fbee9db6832cf1c"},
+ {file = "multidict-6.4.4-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:d2fa86af59f8fc1972e121ade052145f6da22758f6996a197d69bb52f8204e7e"},
+ {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:50855d03e9e4d66eab6947ba688ffb714616f985838077bc4b490e769e48da51"},
+ {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:5bce06b83be23225be1905dcdb6b789064fae92499fbc458f59a8c0e68718601"},
+ {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:66ed0731f8e5dfd8369a883b6e564aca085fb9289aacabd9decd70568b9a30de"},
+ {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:329ae97fc2f56f44d91bc47fe0972b1f52d21c4b7a2ac97040da02577e2daca2"},
+ {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c27e5dcf520923d6474d98b96749e6805f7677e93aaaf62656005b8643f907ab"},
+ {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:058cc59b9e9b143cc56715e59e22941a5d868c322242278d28123a5d09cdf6b0"},
+ {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:69133376bc9a03f8c47343d33f91f74a99c339e8b58cea90433d8e24bb298031"},
+ {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:d6b15c55721b1b115c5ba178c77104123745b1417527ad9641a4c5e2047450f0"},
+ {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:a887b77f51d3d41e6e1a63cf3bc7ddf24de5939d9ff69441387dfefa58ac2e26"},
+ {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:632a3bf8f1787f7ef7d3c2f68a7bde5be2f702906f8b5842ad6da9d974d0aab3"},
+ {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:a145c550900deb7540973c5cdb183b0d24bed6b80bf7bddf33ed8f569082535e"},
+ {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:cc5d83c6619ca5c9672cb78b39ed8542f1975a803dee2cda114ff73cbb076edd"},
+ {file = "multidict-6.4.4-cp313-cp313t-win32.whl", hash = "sha256:3312f63261b9df49be9d57aaa6abf53a6ad96d93b24f9cc16cf979956355ce6e"},
+ {file = "multidict-6.4.4-cp313-cp313t-win_amd64.whl", hash = "sha256:ba852168d814b2c73333073e1c7116d9395bea69575a01b0b3c89d2d5a87c8fb"},
+ {file = "multidict-6.4.4-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:603f39bd1cf85705c6c1ba59644b480dfe495e6ee2b877908de93322705ad7cf"},
+ {file = "multidict-6.4.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:fc60f91c02e11dfbe3ff4e1219c085695c339af72d1641800fe6075b91850c8f"},
+ {file = "multidict-6.4.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:496bcf01c76a70a31c3d746fd39383aad8d685ce6331e4c709e9af4ced5fa221"},
+ {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4219390fb5bf8e548e77b428bb36a21d9382960db5321b74d9d9987148074d6b"},
+ {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:3ef4e9096ff86dfdcbd4a78253090ba13b1d183daa11b973e842465d94ae1772"},
+ {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:49a29d7133b1fc214e818bbe025a77cc6025ed9a4f407d2850373ddde07fd04a"},
+ {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e32053d6d3a8b0dfe49fde05b496731a0e6099a4df92154641c00aa76786aef5"},
+ {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8cc403092a49509e8ef2d2fd636a8ecefc4698cc57bbe894606b14579bc2a955"},
+ {file = "multidict-6.4.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5363f9b2a7f3910e5c87d8b1855c478c05a2dc559ac57308117424dfaad6805c"},
+ {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:2e543a40e4946cf70a88a3be87837a3ae0aebd9058ba49e91cacb0b2cd631e2b"},
+ {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:60d849912350da557fe7de20aa8cf394aada6980d0052cc829eeda4a0db1c1db"},
+ {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:19d08b4f22eae45bb018b9f06e2838c1e4b853c67628ef8ae126d99de0da6395"},
+ {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:d693307856d1ef08041e8b6ff01d5b4618715007d288490ce2c7e29013c12b9a"},
+ {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:fad6daaed41021934917f4fb03ca2db8d8a4d79bf89b17ebe77228eb6710c003"},
+ {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:c10d17371bff801af0daf8b073c30b6cf14215784dc08cd5c43ab5b7b8029bbc"},
+ {file = "multidict-6.4.4-cp39-cp39-win32.whl", hash = "sha256:7e23f2f841fcb3ebd4724a40032d32e0892fbba4143e43d2a9e7695c5e50e6bd"},
+ {file = "multidict-6.4.4-cp39-cp39-win_amd64.whl", hash = "sha256:4d7b50b673ffb4ff4366e7ab43cf1f0aef4bd3608735c5fbdf0bdb6f690da411"},
+ {file = "multidict-6.4.4-py3-none-any.whl", hash = "sha256:bd4557071b561a8b3b6075c3ce93cf9bfb6182cb241805c3d66ced3b75eff4ac"},
+ {file = "multidict-6.4.4.tar.gz", hash = "sha256:69ee9e6ba214b5245031b76233dd95408a0fd57fdb019ddcc1ead4790932a8e8"},
]
[package.dependencies]
typing-extensions = {version = ">=4.1.0", markers = "python_version < \"3.11\""}
+[[package]]
+name = "mypy-boto3-dynamodb"
+version = "1.38.4"
+description = "Type annotations for boto3 DynamoDB 1.38.4 service generated with mypy-boto3-builder 8.10.1"
+optional = false
+python-versions = ">=3.8"
+groups = ["dev"]
+files = [
+ {file = "mypy_boto3_dynamodb-1.38.4-py3-none-any.whl", hash = "sha256:6b29d89c649eeb1e894118bee002cb8b1304c78da735b1503aa08e46b0abfdec"},
+ {file = "mypy_boto3_dynamodb-1.38.4.tar.gz", hash = "sha256:5cf3787631e312b3d75f89a6cbbbd4ad786a76f5d565af023febf03fbf23c0b5"},
+]
+
+[package.dependencies]
+typing-extensions = {version = "*", markers = "python_version < \"3.12\""}
+
[[package]]
name = "mypy-boto3-glue"
version = "1.38.22"
description = "Type annotations for boto3 Glue 1.38.22 service generated with mypy-boto3-builder 8.11.0"
-optional = true
+optional = false
python-versions = ">=3.8"
-groups = ["main"]
-markers = "extra == \"glue\""
+groups = ["dev"]
files = [
{file = "mypy_boto3_glue-1.38.22-py3-none-any.whl", hash = "sha256:4fe34c858cbee41e8ad30382305c01b0dd9c1da4c84f894860b9249ddabb4a58"},
{file = "mypy_boto3_glue-1.38.22.tar.gz", hash = "sha256:a9c529fafaaa9845d39c3204b3fb6cbbb633fa747faf6a084a2b2a381ef12a2b"},
@@ -3103,6 +3222,7 @@ description = "Python package for creating and manipulating graphs and networks"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
+markers = "python_version < \"3.10\""
files = [
{file = "networkx-3.2.1-py3-none-any.whl", hash = "sha256:f18c69adc97877c42332c170849c96cefa91881c99a7cb3e95b7c659ebdc1ec2"},
{file = "networkx-3.2.1.tar.gz", hash = "sha256:9f1bb5cf3409bf324e0a722c20bdb4c20ee39bf1c30ce8ae499c8502b0b5e0c6"},
@@ -3115,6 +3235,49 @@ doc = ["nb2plots (>=0.7)", "nbconvert (<7.9)", "numpydoc (>=1.6)", "pillow (>=9.
extra = ["lxml (>=4.6)", "pydot (>=1.4.2)", "pygraphviz (>=1.11)", "sympy (>=1.10)"]
test = ["pytest (>=7.2)", "pytest-cov (>=4.0)"]
+[[package]]
+name = "networkx"
+version = "3.4.2"
+description = "Python package for creating and manipulating graphs and networks"
+optional = false
+python-versions = ">=3.10"
+groups = ["dev"]
+markers = "python_version == \"3.10\""
+files = [
+ {file = "networkx-3.4.2-py3-none-any.whl", hash = "sha256:df5d4365b724cf81b8c6a7312509d0c22386097011ad1abe274afd5e9d3bbc5f"},
+ {file = "networkx-3.4.2.tar.gz", hash = "sha256:307c3669428c5362aab27c8a1260aa8f47c4e91d3891f48be0141738d8d053e1"},
+]
+
+[package.extras]
+default = ["matplotlib (>=3.7)", "numpy (>=1.24)", "pandas (>=2.0)", "scipy (>=1.10,!=1.11.0,!=1.11.1)"]
+developer = ["changelist (==0.5)", "mypy (>=1.1)", "pre-commit (>=3.2)", "rtoml"]
+doc = ["intersphinx-registry", "myst-nb (>=1.1)", "numpydoc (>=1.8.0)", "pillow (>=9.4)", "pydata-sphinx-theme (>=0.15)", "sphinx (>=7.3)", "sphinx-gallery (>=0.16)", "texext (>=0.6.7)"]
+example = ["cairocffi (>=1.7)", "contextily (>=1.6)", "igraph (>=0.11)", "momepy (>=0.7.2)", "osmnx (>=1.9)", "scikit-learn (>=1.5)", "seaborn (>=0.13)"]
+extra = ["lxml (>=4.6)", "pydot (>=3.0.1)", "pygraphviz (>=1.14)", "sympy (>=1.10)"]
+test = ["pytest (>=7.2)", "pytest-cov (>=4.0)"]
+
+[[package]]
+name = "networkx"
+version = "3.5"
+description = "Python package for creating and manipulating graphs and networks"
+optional = false
+python-versions = ">=3.11"
+groups = ["dev"]
+markers = "python_version >= \"3.11\""
+files = [
+ {file = "networkx-3.5-py3-none-any.whl", hash = "sha256:0030d386a9a06dee3565298b4a734b68589749a544acbb6c412dc9e2489ec6ec"},
+ {file = "networkx-3.5.tar.gz", hash = "sha256:d4c6f9cf81f52d69230866796b82afbccdec3db7ae4fbd1b65ea750feed50037"},
+]
+
+[package.extras]
+default = ["matplotlib (>=3.8)", "numpy (>=1.25)", "pandas (>=2.0)", "scipy (>=1.11.2)"]
+developer = ["mypy (>=1.15)", "pre-commit (>=4.1)"]
+doc = ["intersphinx-registry", "myst-nb (>=1.1)", "numpydoc (>=1.8.0)", "pillow (>=10)", "pydata-sphinx-theme (>=0.16)", "sphinx (>=8.0)", "sphinx-gallery (>=0.18)", "texext (>=0.6.7)"]
+example = ["cairocffi (>=1.7)", "contextily (>=1.6)", "igraph (>=0.11)", "momepy (>=0.7.2)", "osmnx (>=2.0.0)", "scikit-learn (>=1.5)", "seaborn (>=0.13)"]
+extra = ["lxml (>=4.6)", "pydot (>=3.0.1)", "pygraphviz (>=1.14)", "sympy (>=1.10)"]
+test = ["pytest (>=7.2)", "pytest-cov (>=4.0)", "pytest-xdist (>=3.0)"]
+test-extras = ["pytest-mpl", "pytest-randomly"]
+
[[package]]
name = "nodeenv"
version = "1.9.1"
@@ -3185,68 +3348,130 @@ files = [
[[package]]
name = "numpy"
-version = "2.2.4"
+version = "2.2.6"
description = "Fundamental package for array computing in Python"
optional = true
python-versions = ">=3.10"
groups = ["main"]
-markers = "(extra == \"pandas\" or extra == \"ray\") and python_version >= \"3.10\""
-files = [
- {file = "numpy-2.2.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:8146f3550d627252269ac42ae660281d673eb6f8b32f113538e0cc2a9aed42b9"},
- {file = "numpy-2.2.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:e642d86b8f956098b564a45e6f6ce68a22c2c97a04f5acd3f221f57b8cb850ae"},
- {file = "numpy-2.2.4-cp310-cp310-macosx_14_0_arm64.whl", hash = "sha256:a84eda42bd12edc36eb5b53bbcc9b406820d3353f1994b6cfe453a33ff101775"},
- {file = "numpy-2.2.4-cp310-cp310-macosx_14_0_x86_64.whl", hash = "sha256:4ba5054787e89c59c593a4169830ab362ac2bee8a969249dc56e5d7d20ff8df9"},
- {file = "numpy-2.2.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7716e4a9b7af82c06a2543c53ca476fa0b57e4d760481273e09da04b74ee6ee2"},
- {file = "numpy-2.2.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:adf8c1d66f432ce577d0197dceaac2ac00c0759f573f28516246351c58a85020"},
- {file = "numpy-2.2.4-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:218f061d2faa73621fa23d6359442b0fc658d5b9a70801373625d958259eaca3"},
- {file = "numpy-2.2.4-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:df2f57871a96bbc1b69733cd4c51dc33bea66146b8c63cacbfed73eec0883017"},
- {file = "numpy-2.2.4-cp310-cp310-win32.whl", hash = "sha256:a0258ad1f44f138b791327961caedffbf9612bfa504ab9597157806faa95194a"},
- {file = "numpy-2.2.4-cp310-cp310-win_amd64.whl", hash = "sha256:0d54974f9cf14acf49c60f0f7f4084b6579d24d439453d5fc5805d46a165b542"},
- {file = "numpy-2.2.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:e9e0a277bb2eb5d8a7407e14688b85fd8ad628ee4e0c7930415687b6564207a4"},
- {file = "numpy-2.2.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9eeea959168ea555e556b8188da5fa7831e21d91ce031e95ce23747b7609f8a4"},
- {file = "numpy-2.2.4-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:bd3ad3b0a40e713fc68f99ecfd07124195333f1e689387c180813f0e94309d6f"},
- {file = "numpy-2.2.4-cp311-cp311-macosx_14_0_x86_64.whl", hash = "sha256:cf28633d64294969c019c6df4ff37f5698e8326db68cc2b66576a51fad634880"},
- {file = "numpy-2.2.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2fa8fa7697ad1646b5c93de1719965844e004fcad23c91228aca1cf0800044a1"},
- {file = "numpy-2.2.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f4162988a360a29af158aeb4a2f4f09ffed6a969c9776f8f3bdee9b06a8ab7e5"},
- {file = "numpy-2.2.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:892c10d6a73e0f14935c31229e03325a7b3093fafd6ce0af704be7f894d95687"},
- {file = "numpy-2.2.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:db1f1c22173ac1c58db249ae48aa7ead29f534b9a948bc56828337aa84a32ed6"},
- {file = "numpy-2.2.4-cp311-cp311-win32.whl", hash = "sha256:ea2bb7e2ae9e37d96835b3576a4fa4b3a97592fbea8ef7c3587078b0068b8f09"},
- {file = "numpy-2.2.4-cp311-cp311-win_amd64.whl", hash = "sha256:f7de08cbe5551911886d1ab60de58448c6df0f67d9feb7d1fb21e9875ef95e91"},
- {file = "numpy-2.2.4-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:a7b9084668aa0f64e64bd00d27ba5146ef1c3a8835f3bd912e7a9e01326804c4"},
- {file = "numpy-2.2.4-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:dbe512c511956b893d2dacd007d955a3f03d555ae05cfa3ff1c1ff6df8851854"},
- {file = "numpy-2.2.4-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:bb649f8b207ab07caebba230d851b579a3c8711a851d29efe15008e31bb4de24"},
- {file = "numpy-2.2.4-cp312-cp312-macosx_14_0_x86_64.whl", hash = "sha256:f34dc300df798742b3d06515aa2a0aee20941c13579d7a2f2e10af01ae4901ee"},
- {file = "numpy-2.2.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c3f7ac96b16955634e223b579a3e5798df59007ca43e8d451a0e6a50f6bfdfba"},
- {file = "numpy-2.2.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4f92084defa704deadd4e0a5ab1dc52d8ac9e8a8ef617f3fbb853e79b0ea3592"},
- {file = "numpy-2.2.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7a4e84a6283b36632e2a5b56e121961f6542ab886bc9e12f8f9818b3c266bfbb"},
- {file = "numpy-2.2.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:11c43995255eb4127115956495f43e9343736edb7fcdb0d973defd9de14cd84f"},
- {file = "numpy-2.2.4-cp312-cp312-win32.whl", hash = "sha256:65ef3468b53269eb5fdb3a5c09508c032b793da03251d5f8722b1194f1790c00"},
- {file = "numpy-2.2.4-cp312-cp312-win_amd64.whl", hash = "sha256:2aad3c17ed2ff455b8eaafe06bcdae0062a1db77cb99f4b9cbb5f4ecb13c5146"},
- {file = "numpy-2.2.4-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:1cf4e5c6a278d620dee9ddeb487dc6a860f9b199eadeecc567f777daace1e9e7"},
- {file = "numpy-2.2.4-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:1974afec0b479e50438fc3648974268f972e2d908ddb6d7fb634598cdb8260a0"},
- {file = "numpy-2.2.4-cp313-cp313-macosx_14_0_arm64.whl", hash = "sha256:79bd5f0a02aa16808fcbc79a9a376a147cc1045f7dfe44c6e7d53fa8b8a79392"},
- {file = "numpy-2.2.4-cp313-cp313-macosx_14_0_x86_64.whl", hash = "sha256:3387dd7232804b341165cedcb90694565a6015433ee076c6754775e85d86f1fc"},
- {file = "numpy-2.2.4-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6f527d8fdb0286fd2fd97a2a96c6be17ba4232da346931d967a0630050dfd298"},
- {file = "numpy-2.2.4-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bce43e386c16898b91e162e5baaad90c4b06f9dcbe36282490032cec98dc8ae7"},
- {file = "numpy-2.2.4-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:31504f970f563d99f71a3512d0c01a645b692b12a63630d6aafa0939e52361e6"},
- {file = "numpy-2.2.4-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:81413336ef121a6ba746892fad881a83351ee3e1e4011f52e97fba79233611fd"},
- {file = "numpy-2.2.4-cp313-cp313-win32.whl", hash = "sha256:f486038e44caa08dbd97275a9a35a283a8f1d2f0ee60ac260a1790e76660833c"},
- {file = "numpy-2.2.4-cp313-cp313-win_amd64.whl", hash = "sha256:207a2b8441cc8b6a2a78c9ddc64d00d20c303d79fba08c577752f080c4007ee3"},
- {file = "numpy-2.2.4-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:8120575cb4882318c791f839a4fd66161a6fa46f3f0a5e613071aae35b5dd8f8"},
- {file = "numpy-2.2.4-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:a761ba0fa886a7bb33c6c8f6f20213735cb19642c580a931c625ee377ee8bd39"},
- {file = "numpy-2.2.4-cp313-cp313t-macosx_14_0_arm64.whl", hash = "sha256:ac0280f1ba4a4bfff363a99a6aceed4f8e123f8a9b234c89140f5e894e452ecd"},
- {file = "numpy-2.2.4-cp313-cp313t-macosx_14_0_x86_64.whl", hash = "sha256:879cf3a9a2b53a4672a168c21375166171bc3932b7e21f622201811c43cdd3b0"},
- {file = "numpy-2.2.4-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f05d4198c1bacc9124018109c5fba2f3201dbe7ab6e92ff100494f236209c960"},
- {file = "numpy-2.2.4-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2f085ce2e813a50dfd0e01fbfc0c12bbe5d2063d99f8b29da30e544fb6483b8"},
- {file = "numpy-2.2.4-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:92bda934a791c01d6d9d8e038363c50918ef7c40601552a58ac84c9613a665bc"},
- {file = "numpy-2.2.4-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:ee4d528022f4c5ff67332469e10efe06a267e32f4067dc76bb7e2cddf3cd25ff"},
- {file = "numpy-2.2.4-cp313-cp313t-win32.whl", hash = "sha256:05c076d531e9998e7e694c36e8b349969c56eadd2cdcd07242958489d79a7286"},
- {file = "numpy-2.2.4-cp313-cp313t-win_amd64.whl", hash = "sha256:188dcbca89834cc2e14eb2f106c96d6d46f200fe0200310fc29089657379c58d"},
- {file = "numpy-2.2.4-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:7051ee569db5fbac144335e0f3b9c2337e0c8d5c9fee015f259a5bd70772b7e8"},
- {file = "numpy-2.2.4-pp310-pypy310_pp73-macosx_14_0_x86_64.whl", hash = "sha256:ab2939cd5bec30a7430cbdb2287b63151b77cf9624de0532d629c9a1c59b1d5c"},
- {file = "numpy-2.2.4-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d0f35b19894a9e08639fd60a1ec1978cb7f5f7f1eace62f38dd36be8aecdef4d"},
- {file = "numpy-2.2.4-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:b4adfbbc64014976d2f91084915ca4e626fbf2057fb81af209c1a6d776d23e3d"},
- {file = "numpy-2.2.4.tar.gz", hash = "sha256:9ba03692a45d3eef66559efe1d1096c4b9b75c0986b5dff5530c378fb8331d4f"},
+markers = "(extra == \"pandas\" or extra == \"ray\") and python_version == \"3.10\""
+files = [
+ {file = "numpy-2.2.6-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b412caa66f72040e6d268491a59f2c43bf03eb6c96dd8f0307829feb7fa2b6fb"},
+ {file = "numpy-2.2.6-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8e41fd67c52b86603a91c1a505ebaef50b3314de0213461c7a6e99c9a3beff90"},
+ {file = "numpy-2.2.6-cp310-cp310-macosx_14_0_arm64.whl", hash = "sha256:37e990a01ae6ec7fe7fa1c26c55ecb672dd98b19c3d0e1d1f326fa13cb38d163"},
+ {file = "numpy-2.2.6-cp310-cp310-macosx_14_0_x86_64.whl", hash = "sha256:5a6429d4be8ca66d889b7cf70f536a397dc45ba6faeb5f8c5427935d9592e9cf"},
+ {file = "numpy-2.2.6-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:efd28d4e9cd7d7a8d39074a4d44c63eda73401580c5c76acda2ce969e0a38e83"},
+ {file = "numpy-2.2.6-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fc7b73d02efb0e18c000e9ad8b83480dfcd5dfd11065997ed4c6747470ae8915"},
+ {file = "numpy-2.2.6-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:74d4531beb257d2c3f4b261bfb0fc09e0f9ebb8842d82a7b4209415896adc680"},
+ {file = "numpy-2.2.6-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:8fc377d995680230e83241d8a96def29f204b5782f371c532579b4f20607a289"},
+ {file = "numpy-2.2.6-cp310-cp310-win32.whl", hash = "sha256:b093dd74e50a8cba3e873868d9e93a85b78e0daf2e98c6797566ad8044e8363d"},
+ {file = "numpy-2.2.6-cp310-cp310-win_amd64.whl", hash = "sha256:f0fd6321b839904e15c46e0d257fdd101dd7f530fe03fd6359c1ea63738703f3"},
+ {file = "numpy-2.2.6-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f9f1adb22318e121c5c69a09142811a201ef17ab257a1e66ca3025065b7f53ae"},
+ {file = "numpy-2.2.6-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:c820a93b0255bc360f53eca31a0e676fd1101f673dda8da93454a12e23fc5f7a"},
+ {file = "numpy-2.2.6-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:3d70692235e759f260c3d837193090014aebdf026dfd167834bcba43e30c2a42"},
+ {file = "numpy-2.2.6-cp311-cp311-macosx_14_0_x86_64.whl", hash = "sha256:481b49095335f8eed42e39e8041327c05b0f6f4780488f61286ed3c01368d491"},
+ {file = "numpy-2.2.6-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b64d8d4d17135e00c8e346e0a738deb17e754230d7e0810ac5012750bbd85a5a"},
+ {file = "numpy-2.2.6-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba10f8411898fc418a521833e014a77d3ca01c15b0c6cdcce6a0d2897e6dbbdf"},
+ {file = "numpy-2.2.6-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:bd48227a919f1bafbdda0583705e547892342c26fb127219d60a5c36882609d1"},
+ {file = "numpy-2.2.6-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:9551a499bf125c1d4f9e250377c1ee2eddd02e01eac6644c080162c0c51778ab"},
+ {file = "numpy-2.2.6-cp311-cp311-win32.whl", hash = "sha256:0678000bb9ac1475cd454c6b8c799206af8107e310843532b04d49649c717a47"},
+ {file = "numpy-2.2.6-cp311-cp311-win_amd64.whl", hash = "sha256:e8213002e427c69c45a52bbd94163084025f533a55a59d6f9c5b820774ef3303"},
+ {file = "numpy-2.2.6-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:41c5a21f4a04fa86436124d388f6ed60a9343a6f767fced1a8a71c3fbca038ff"},
+ {file = "numpy-2.2.6-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:de749064336d37e340f640b05f24e9e3dd678c57318c7289d222a8a2f543e90c"},
+ {file = "numpy-2.2.6-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:894b3a42502226a1cac872f840030665f33326fc3dac8e57c607905773cdcde3"},
+ {file = "numpy-2.2.6-cp312-cp312-macosx_14_0_x86_64.whl", hash = "sha256:71594f7c51a18e728451bb50cc60a3ce4e6538822731b2933209a1f3614e9282"},
+ {file = "numpy-2.2.6-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f2618db89be1b4e05f7a1a847a9c1c0abd63e63a1607d892dd54668dd92faf87"},
+ {file = "numpy-2.2.6-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd83c01228a688733f1ded5201c678f0c53ecc1006ffbc404db9f7a899ac6249"},
+ {file = "numpy-2.2.6-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:37c0ca431f82cd5fa716eca9506aefcabc247fb27ba69c5062a6d3ade8cf8f49"},
+ {file = "numpy-2.2.6-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:fe27749d33bb772c80dcd84ae7e8df2adc920ae8297400dabec45f0dedb3f6de"},
+ {file = "numpy-2.2.6-cp312-cp312-win32.whl", hash = "sha256:4eeaae00d789f66c7a25ac5f34b71a7035bb474e679f410e5e1a94deb24cf2d4"},
+ {file = "numpy-2.2.6-cp312-cp312-win_amd64.whl", hash = "sha256:c1f9540be57940698ed329904db803cf7a402f3fc200bfe599334c9bd84a40b2"},
+ {file = "numpy-2.2.6-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:0811bb762109d9708cca4d0b13c4f67146e3c3b7cf8d34018c722adb2d957c84"},
+ {file = "numpy-2.2.6-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:287cc3162b6f01463ccd86be154f284d0893d2b3ed7292439ea97eafa8170e0b"},
+ {file = "numpy-2.2.6-cp313-cp313-macosx_14_0_arm64.whl", hash = "sha256:f1372f041402e37e5e633e586f62aa53de2eac8d98cbfb822806ce4bbefcb74d"},
+ {file = "numpy-2.2.6-cp313-cp313-macosx_14_0_x86_64.whl", hash = "sha256:55a4d33fa519660d69614a9fad433be87e5252f4b03850642f88993f7b2ca566"},
+ {file = "numpy-2.2.6-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f92729c95468a2f4f15e9bb94c432a9229d0d50de67304399627a943201baa2f"},
+ {file = "numpy-2.2.6-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1bc23a79bfabc5d056d106f9befb8d50c31ced2fbc70eedb8155aec74a45798f"},
+ {file = "numpy-2.2.6-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:e3143e4451880bed956e706a3220b4e5cf6172ef05fcc397f6f36a550b1dd868"},
+ {file = "numpy-2.2.6-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:b4f13750ce79751586ae2eb824ba7e1e8dba64784086c98cdbbcc6a42112ce0d"},
+ {file = "numpy-2.2.6-cp313-cp313-win32.whl", hash = "sha256:5beb72339d9d4fa36522fc63802f469b13cdbe4fdab4a288f0c441b74272ebfd"},
+ {file = "numpy-2.2.6-cp313-cp313-win_amd64.whl", hash = "sha256:b0544343a702fa80c95ad5d3d608ea3599dd54d4632df855e4c8d24eb6ecfa1c"},
+ {file = "numpy-2.2.6-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:0bca768cd85ae743b2affdc762d617eddf3bcf8724435498a1e80132d04879e6"},
+ {file = "numpy-2.2.6-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:fc0c5673685c508a142ca65209b4e79ed6740a4ed6b2267dbba90f34b0b3cfda"},
+ {file = "numpy-2.2.6-cp313-cp313t-macosx_14_0_arm64.whl", hash = "sha256:5bd4fc3ac8926b3819797a7c0e2631eb889b4118a9898c84f585a54d475b7e40"},
+ {file = "numpy-2.2.6-cp313-cp313t-macosx_14_0_x86_64.whl", hash = "sha256:fee4236c876c4e8369388054d02d0e9bb84821feb1a64dd59e137e6511a551f8"},
+ {file = "numpy-2.2.6-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e1dda9c7e08dc141e0247a5b8f49cf05984955246a327d4c48bda16821947b2f"},
+ {file = "numpy-2.2.6-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f447e6acb680fd307f40d3da4852208af94afdfab89cf850986c3ca00562f4fa"},
+ {file = "numpy-2.2.6-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:389d771b1623ec92636b0786bc4ae56abafad4a4c513d36a55dce14bd9ce8571"},
+ {file = "numpy-2.2.6-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:8e9ace4a37db23421249ed236fdcdd457d671e25146786dfc96835cd951aa7c1"},
+ {file = "numpy-2.2.6-cp313-cp313t-win32.whl", hash = "sha256:038613e9fb8c72b0a41f025a7e4c3f0b7a1b5d768ece4796b674c8f3fe13efff"},
+ {file = "numpy-2.2.6-cp313-cp313t-win_amd64.whl", hash = "sha256:6031dd6dfecc0cf9f668681a37648373bddd6421fff6c66ec1624eed0180ee06"},
+ {file = "numpy-2.2.6-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:0b605b275d7bd0c640cad4e5d30fa701a8d59302e127e5f79138ad62762c3e3d"},
+ {file = "numpy-2.2.6-pp310-pypy310_pp73-macosx_14_0_x86_64.whl", hash = "sha256:7befc596a7dc9da8a337f79802ee8adb30a552a94f792b9c9d18c840055907db"},
+ {file = "numpy-2.2.6-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ce47521a4754c8f4593837384bd3424880629f718d87c5d44f8ed763edd63543"},
+ {file = "numpy-2.2.6-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:d042d24c90c41b54fd506da306759e06e568864df8ec17ccc17e9e884634fd00"},
+ {file = "numpy-2.2.6.tar.gz", hash = "sha256:e29554e2bef54a90aa5cc07da6ce955accb83f21ab5de01a62c8478897b264fd"},
+]
+
+[[package]]
+name = "numpy"
+version = "2.3.0"
+description = "Fundamental package for array computing in Python"
+optional = true
+python-versions = ">=3.11"
+groups = ["main"]
+markers = "python_version >= \"3.11\" and (extra == \"pandas\" or extra == \"ray\")"
+files = [
+ {file = "numpy-2.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c3c9fdde0fa18afa1099d6257eb82890ea4f3102847e692193b54e00312a9ae9"},
+ {file = "numpy-2.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:46d16f72c2192da7b83984aa5455baee640e33a9f1e61e656f29adf55e406c2b"},
+ {file = "numpy-2.3.0-cp311-cp311-macosx_14_0_arm64.whl", hash = "sha256:a0be278be9307c4ab06b788f2a077f05e180aea817b3e41cebbd5aaf7bd85ed3"},
+ {file = "numpy-2.3.0-cp311-cp311-macosx_14_0_x86_64.whl", hash = "sha256:99224862d1412d2562248d4710126355d3a8db7672170a39d6909ac47687a8a4"},
+ {file = "numpy-2.3.0-cp311-cp311-manylinux_2_28_aarch64.whl", hash = "sha256:2393a914db64b0ead0ab80c962e42d09d5f385802006a6c87835acb1f58adb96"},
+ {file = "numpy-2.3.0-cp311-cp311-manylinux_2_28_x86_64.whl", hash = "sha256:7729c8008d55e80784bd113787ce876ca117185c579c0d626f59b87d433ea779"},
+ {file = "numpy-2.3.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:06d4fb37a8d383b769281714897420c5cc3545c79dc427df57fc9b852ee0bf58"},
+ {file = "numpy-2.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:c39ec392b5db5088259c68250e342612db82dc80ce044cf16496cf14cf6bc6f8"},
+ {file = "numpy-2.3.0-cp311-cp311-win32.whl", hash = "sha256:ee9d3ee70d62827bc91f3ea5eee33153212c41f639918550ac0475e3588da59f"},
+ {file = "numpy-2.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:43c55b6a860b0eb44d42341438b03513cf3879cb3617afb749ad49307e164edd"},
+ {file = "numpy-2.3.0-cp311-cp311-win_arm64.whl", hash = "sha256:2e6a1409eee0cb0316cb64640a49a49ca44deb1a537e6b1121dc7c458a1299a8"},
+ {file = "numpy-2.3.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:389b85335838155a9076e9ad7f8fdba0827496ec2d2dc32ce69ce7898bde03ba"},
+ {file = "numpy-2.3.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:9498f60cd6bb8238d8eaf468a3d5bb031d34cd12556af53510f05fcf581c1b7e"},
+ {file = "numpy-2.3.0-cp312-cp312-macosx_14_0_arm64.whl", hash = "sha256:622a65d40d8eb427d8e722fd410ac3ad4958002f109230bc714fa551044ebae2"},
+ {file = "numpy-2.3.0-cp312-cp312-macosx_14_0_x86_64.whl", hash = "sha256:b9446d9d8505aadadb686d51d838f2b6688c9e85636a0c3abaeb55ed54756459"},
+ {file = "numpy-2.3.0-cp312-cp312-manylinux_2_28_aarch64.whl", hash = "sha256:50080245365d75137a2bf46151e975de63146ae6d79f7e6bd5c0e85c9931d06a"},
+ {file = "numpy-2.3.0-cp312-cp312-manylinux_2_28_x86_64.whl", hash = "sha256:c24bb4113c66936eeaa0dc1e47c74770453d34f46ee07ae4efd853a2ed1ad10a"},
+ {file = "numpy-2.3.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:4d8d294287fdf685281e671886c6dcdf0291a7c19db3e5cb4178d07ccf6ecc67"},
+ {file = "numpy-2.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:6295f81f093b7f5769d1728a6bd8bf7466de2adfa771ede944ce6711382b89dc"},
+ {file = "numpy-2.3.0-cp312-cp312-win32.whl", hash = "sha256:e6648078bdd974ef5d15cecc31b0c410e2e24178a6e10bf511e0557eed0f2570"},
+ {file = "numpy-2.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:0898c67a58cdaaf29994bc0e2c65230fd4de0ac40afaf1584ed0b02cd74c6fdd"},
+ {file = "numpy-2.3.0-cp312-cp312-win_arm64.whl", hash = "sha256:bd8df082b6c4695753ad6193018c05aac465d634834dca47a3ae06d4bb22d9ea"},
+ {file = "numpy-2.3.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:5754ab5595bfa2c2387d241296e0381c21f44a4b90a776c3c1d39eede13a746a"},
+ {file = "numpy-2.3.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:d11fa02f77752d8099573d64e5fe33de3229b6632036ec08f7080f46b6649959"},
+ {file = "numpy-2.3.0-cp313-cp313-macosx_14_0_arm64.whl", hash = "sha256:aba48d17e87688a765ab1cd557882052f238e2f36545dfa8e29e6a91aef77afe"},
+ {file = "numpy-2.3.0-cp313-cp313-macosx_14_0_x86_64.whl", hash = "sha256:4dc58865623023b63b10d52f18abaac3729346a7a46a778381e0e3af4b7f3beb"},
+ {file = "numpy-2.3.0-cp313-cp313-manylinux_2_28_aarch64.whl", hash = "sha256:df470d376f54e052c76517393fa443758fefcdd634645bc9c1f84eafc67087f0"},
+ {file = "numpy-2.3.0-cp313-cp313-manylinux_2_28_x86_64.whl", hash = "sha256:87717eb24d4a8a64683b7a4e91ace04e2f5c7c77872f823f02a94feee186168f"},
+ {file = "numpy-2.3.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:d8fa264d56882b59dcb5ea4d6ab6f31d0c58a57b41aec605848b6eb2ef4a43e8"},
+ {file = "numpy-2.3.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:e651756066a0eaf900916497e20e02fe1ae544187cb0fe88de981671ee7f6270"},
+ {file = "numpy-2.3.0-cp313-cp313-win32.whl", hash = "sha256:e43c3cce3b6ae5f94696669ff2a6eafd9a6b9332008bafa4117af70f4b88be6f"},
+ {file = "numpy-2.3.0-cp313-cp313-win_amd64.whl", hash = "sha256:81ae0bf2564cf475f94be4a27ef7bcf8af0c3e28da46770fc904da9abd5279b5"},
+ {file = "numpy-2.3.0-cp313-cp313-win_arm64.whl", hash = "sha256:c8738baa52505fa6e82778580b23f945e3578412554d937093eac9205e845e6e"},
+ {file = "numpy-2.3.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:39b27d8b38942a647f048b675f134dd5a567f95bfff481f9109ec308515c51d8"},
+ {file = "numpy-2.3.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:0eba4a1ea88f9a6f30f56fdafdeb8da3774349eacddab9581a21234b8535d3d3"},
+ {file = "numpy-2.3.0-cp313-cp313t-macosx_14_0_arm64.whl", hash = "sha256:b0f1f11d0a1da54927436505a5a7670b154eac27f5672afc389661013dfe3d4f"},
+ {file = "numpy-2.3.0-cp313-cp313t-macosx_14_0_x86_64.whl", hash = "sha256:690d0a5b60a47e1f9dcec7b77750a4854c0d690e9058b7bef3106e3ae9117808"},
+ {file = "numpy-2.3.0-cp313-cp313t-manylinux_2_28_aarch64.whl", hash = "sha256:8b51ead2b258284458e570942137155978583e407babc22e3d0ed7af33ce06f8"},
+ {file = "numpy-2.3.0-cp313-cp313t-manylinux_2_28_x86_64.whl", hash = "sha256:aaf81c7b82c73bd9b45e79cfb9476cb9c29e937494bfe9092c26aece812818ad"},
+ {file = "numpy-2.3.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:f420033a20b4f6a2a11f585f93c843ac40686a7c3fa514060a97d9de93e5e72b"},
+ {file = "numpy-2.3.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:d344ca32ab482bcf8735d8f95091ad081f97120546f3d250240868430ce52555"},
+ {file = "numpy-2.3.0-cp313-cp313t-win32.whl", hash = "sha256:48a2e8eaf76364c32a1feaa60d6925eaf32ed7a040183b807e02674305beef61"},
+ {file = "numpy-2.3.0-cp313-cp313t-win_amd64.whl", hash = "sha256:ba17f93a94e503551f154de210e4d50c5e3ee20f7e7a1b5f6ce3f22d419b93bb"},
+ {file = "numpy-2.3.0-cp313-cp313t-win_arm64.whl", hash = "sha256:f14e016d9409680959691c109be98c436c6249eaf7f118b424679793607b5944"},
+ {file = "numpy-2.3.0-pp311-pypy311_pp73-macosx_10_15_x86_64.whl", hash = "sha256:80b46117c7359de8167cc00a2c7d823bdd505e8c7727ae0871025a86d668283b"},
+ {file = "numpy-2.3.0-pp311-pypy311_pp73-macosx_14_0_arm64.whl", hash = "sha256:5814a0f43e70c061f47abd5857d120179609ddc32a613138cbb6c4e9e2dbdda5"},
+ {file = "numpy-2.3.0-pp311-pypy311_pp73-macosx_14_0_x86_64.whl", hash = "sha256:ef6c1e88fd6b81ac6d215ed71dc8cd027e54d4bf1d2682d362449097156267a2"},
+ {file = "numpy-2.3.0-pp311-pypy311_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:33a5a12a45bb82d9997e2c0b12adae97507ad7c347546190a18ff14c28bbca12"},
+ {file = "numpy-2.3.0-pp311-pypy311_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:54dfc8681c1906d239e95ab1508d0a533c4a9505e52ee2d71a5472b04437ef97"},
+ {file = "numpy-2.3.0-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:e017a8a251ff4d18d71f139e28bdc7c31edba7a507f72b1414ed902cbe48c74d"},
+ {file = "numpy-2.3.0.tar.gz", hash = "sha256:581f87f9e9e9db2cba2141400e160e9dd644ee248788d6f90636eeb8fd9260a6"},
]
[[package]]
@@ -3286,14 +3511,14 @@ rfc3339-validator = "*"
[[package]]
name = "openapi-spec-validator"
-version = "0.7.1"
+version = "0.7.2"
description = "OpenAPI 2.0 (aka Swagger) and OpenAPI 3 spec validator"
optional = false
-python-versions = ">=3.8.0,<4.0.0"
+python-versions = "<4.0.0,>=3.8.0"
groups = ["dev"]
files = [
- {file = "openapi_spec_validator-0.7.1-py3-none-any.whl", hash = "sha256:3c81825043f24ccbcd2f4b149b11e8231abce5ba84f37065e14ec947d8f4e959"},
- {file = "openapi_spec_validator-0.7.1.tar.gz", hash = "sha256:8577b85a8268685da6f8aa30990b83b7960d4d1117e901d451b5d572605e5ec7"},
+ {file = "openapi_spec_validator-0.7.2-py3-none-any.whl", hash = "sha256:4bbdc0894ec85f1d1bea1d6d9c8b2c3c8d7ccaa13577ef40da9c006c9fd0eb60"},
+ {file = "openapi_spec_validator-0.7.2.tar.gz", hash = "sha256:cc029309b5c5dbc7859df0372d55e9d1ff43e96d678b9ba087f7c56fc586f734"},
]
[package.dependencies]
@@ -3304,14 +3529,14 @@ openapi-schema-validator = ">=0.6.0,<0.7.0"
[[package]]
name = "packaging"
-version = "24.2"
+version = "25.0"
description = "Core utilities for Python packages"
optional = false
python-versions = ">=3.8"
groups = ["main", "dev", "docs"]
files = [
- {file = "packaging-24.2-py3-none-any.whl", hash = "sha256:09abb1bccd265c01f4a3aa3f7a7db064b36514d2cba19a2f694fe6150451a759"},
- {file = "packaging-24.2.tar.gz", hash = "sha256:c228a6dc5e932d346bc5739379109d49e8853dd8223571c7c5b55260edc0b97f"},
+ {file = "packaging-25.0-py3-none-any.whl", hash = "sha256:29572ef2b1f17581046b3a2227d5c611fb25ec70ca1ba8554b24b0e69331a484"},
+ {file = "packaging-25.0.tar.gz", hash = "sha256:d443872c98d677bf60f6a1f2f8c1cb748e8fe762d2bf9d3148b5599295b0fc4f"},
]
markers = {main = "extra == \"ray\" or extra == \"hf\""}
@@ -3445,14 +3670,14 @@ files = [
[[package]]
name = "platformdirs"
-version = "4.3.7"
+version = "4.3.8"
description = "A small Python package for determining appropriate platform-specific dirs, e.g. a `user data dir`."
optional = false
python-versions = ">=3.9"
groups = ["dev", "docs"]
files = [
- {file = "platformdirs-4.3.7-py3-none-any.whl", hash = "sha256:a03875334331946f13c549dbd8f4bac7a13a50a895a0eb1e8c6a8ace80d40a94"},
- {file = "platformdirs-4.3.7.tar.gz", hash = "sha256:eb437d586b6a0986388f0d6f74aa0cde27b48d0e3d66843640bfb6bdcdb6e351"},
+ {file = "platformdirs-4.3.8-py3-none-any.whl", hash = "sha256:ff7059bb7eb1179e2685604f4aaf157cfd9535242bd23742eadc3c13542139b4"},
+ {file = "platformdirs-4.3.8.tar.gz", hash = "sha256:3d512d96e16bcb959a814c9f348431070822a6496326a4be0911c40b5a74c2bc"},
]
[package.extras]
@@ -3462,19 +3687,19 @@ type = ["mypy (>=1.14.1)"]
[[package]]
name = "pluggy"
-version = "1.5.0"
+version = "1.6.0"
description = "plugin and hook calling mechanisms for python"
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "pluggy-1.5.0-py3-none-any.whl", hash = "sha256:44e1ad92c8ca002de6377e165f3e0f1be63266ab4d554740532335b9d75ea669"},
- {file = "pluggy-1.5.0.tar.gz", hash = "sha256:2cffa88e94fdc978c4c574f15f9e59b7f4201d439195c3715ca9e2486f1d0cf1"},
+ {file = "pluggy-1.6.0-py3-none-any.whl", hash = "sha256:e920276dd6813095e9377c0bc5566d94c932c33b27a3e3945d8389c374dd4746"},
+ {file = "pluggy-1.6.0.tar.gz", hash = "sha256:7dcc130b76258d33b90f61b658791dede3486c3e6bfb003ee5c9bfb396dd22f3"},
]
[package.extras]
dev = ["pre-commit", "tox"]
-testing = ["pytest", "pytest-benchmark"]
+testing = ["coverage", "pytest", "pytest-benchmark"]
[[package]]
name = "ply"
@@ -3490,20 +3715,20 @@ files = [
[[package]]
name = "polars"
-version = "1.29.0"
+version = "1.30.0"
description = "Blazingly fast DataFrame library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"polars\""
files = [
- {file = "polars-1.29.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:d053ee3217df31468caf2f5ddb9fd0f3a94fd42afdf7d9abe23d9d424adca02b"},
- {file = "polars-1.29.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:14131078e365eae5ccda3e67383cd43c0c0598d7f760bdf1cb4082566c5494ce"},
- {file = "polars-1.29.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:54f6902da333f99208b8d27765d580ba0299b412787c0564275912122c228e40"},
- {file = "polars-1.29.0-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:7a0ac6a11088279af4d715f4b58068835f551fa5368504a53401743006115e78"},
- {file = "polars-1.29.0-cp39-abi3-win_amd64.whl", hash = "sha256:f5aac4656e58b1e12f9481950981ef68b5b0e53dd4903bd72472efd2d09a74c8"},
- {file = "polars-1.29.0-cp39-abi3-win_arm64.whl", hash = "sha256:0c105b07b980b77fe88c3200b015bf4695e53185385f0f244c13e2d1027c7bbf"},
- {file = "polars-1.29.0.tar.gz", hash = "sha256:d2acb71fce1ff0ea76db5f648abd91a7a6c460fafabce9a2e8175184efa00d02"},
+ {file = "polars-1.30.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:4c33bc97c29b7112f0e689a2f8a33143973a3ff466c70b25c7fd1880225de6dd"},
+ {file = "polars-1.30.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:e3d05914c364b8e39a5b10dcf97e84d76e516b3b1693880bf189a93aab3ca00d"},
+ {file = "polars-1.30.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a52af3862082b868c1febeae650af8ae8a2105d2cb28f0449179a7b44f54ccf"},
+ {file = "polars-1.30.0-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:ffb3ef133454275d4254442257c5f71dd6e393ce365c97997dadeb6fa9d6d4b5"},
+ {file = "polars-1.30.0-cp39-abi3-win_amd64.whl", hash = "sha256:c26b633a9bd530c5fc09d317fca3bb3e16c772bd7df7549a9d8ec1934773cc5d"},
+ {file = "polars-1.30.0-cp39-abi3-win_arm64.whl", hash = "sha256:476f1bde65bc7b4d9f80af370645c2981b5798d67c151055e58534e89e96f2a8"},
+ {file = "polars-1.30.0.tar.gz", hash = "sha256:dfe94ae84a5efd9ba74e616e3e125b24ca155494a931890a8f17480737c4db45"},
]
[package.extras]
@@ -3554,131 +3779,150 @@ virtualenv = ">=20.10.0"
[[package]]
name = "propcache"
-version = "0.3.1"
+version = "0.3.2"
description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
-files = [
- {file = "propcache-0.3.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:f27785888d2fdd918bc36de8b8739f2d6c791399552333721b58193f68ea3e98"},
- {file = "propcache-0.3.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:d4e89cde74154c7b5957f87a355bb9c8ec929c167b59c83d90654ea36aeb6180"},
- {file = "propcache-0.3.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:730178f476ef03d3d4d255f0c9fa186cb1d13fd33ffe89d39f2cda4da90ceb71"},
- {file = "propcache-0.3.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:967a8eec513dbe08330f10137eacb427b2ca52118769e82ebcfcab0fba92a649"},
- {file = "propcache-0.3.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5b9145c35cc87313b5fd480144f8078716007656093d23059e8993d3a8fa730f"},
- {file = "propcache-0.3.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9e64e948ab41411958670f1093c0a57acfdc3bee5cf5b935671bbd5313bcf229"},
- {file = "propcache-0.3.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:319fa8765bfd6a265e5fa661547556da381e53274bc05094fc9ea50da51bfd46"},
- {file = "propcache-0.3.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c66d8ccbc902ad548312b96ed8d5d266d0d2c6d006fd0f66323e9d8f2dd49be7"},
- {file = "propcache-0.3.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:2d219b0dbabe75e15e581fc1ae796109b07c8ba7d25b9ae8d650da582bed01b0"},
- {file = "propcache-0.3.1-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:cd6a55f65241c551eb53f8cf4d2f4af33512c39da5d9777694e9d9c60872f519"},
- {file = "propcache-0.3.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:9979643ffc69b799d50d3a7b72b5164a2e97e117009d7af6dfdd2ab906cb72cd"},
- {file = "propcache-0.3.1-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:4cf9e93a81979f1424f1a3d155213dc928f1069d697e4353edb8a5eba67c6259"},
- {file = "propcache-0.3.1-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:2fce1df66915909ff6c824bbb5eb403d2d15f98f1518e583074671a30fe0c21e"},
- {file = "propcache-0.3.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:4d0dfdd9a2ebc77b869a0b04423591ea8823f791293b527dc1bb896c1d6f1136"},
- {file = "propcache-0.3.1-cp310-cp310-win32.whl", hash = "sha256:1f6cc0ad7b4560e5637eb2c994e97b4fa41ba8226069c9277eb5ea7101845b42"},
- {file = "propcache-0.3.1-cp310-cp310-win_amd64.whl", hash = "sha256:47ef24aa6511e388e9894ec16f0fbf3313a53ee68402bc428744a367ec55b833"},
- {file = "propcache-0.3.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:7f30241577d2fef2602113b70ef7231bf4c69a97e04693bde08ddab913ba0ce5"},
- {file = "propcache-0.3.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:43593c6772aa12abc3af7784bff4a41ffa921608dd38b77cf1dfd7f5c4e71371"},
- {file = "propcache-0.3.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a75801768bbe65499495660b777e018cbe90c7980f07f8aa57d6be79ea6f71da"},
- {file = "propcache-0.3.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f6f1324db48f001c2ca26a25fa25af60711e09b9aaf4b28488602776f4f9a744"},
- {file = "propcache-0.3.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5cdb0f3e1eb6dfc9965d19734d8f9c481b294b5274337a8cb5cb01b462dcb7e0"},
- {file = "propcache-0.3.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1eb34d90aac9bfbced9a58b266f8946cb5935869ff01b164573a7634d39fbcb5"},
- {file = "propcache-0.3.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f35c7070eeec2cdaac6fd3fe245226ed2a6292d3ee8c938e5bb645b434c5f256"},
- {file = "propcache-0.3.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b23c11c2c9e6d4e7300c92e022046ad09b91fd00e36e83c44483df4afa990073"},
- {file = "propcache-0.3.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:3e19ea4ea0bf46179f8a3652ac1426e6dcbaf577ce4b4f65be581e237340420d"},
- {file = "propcache-0.3.1-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:bd39c92e4c8f6cbf5f08257d6360123af72af9f4da75a690bef50da77362d25f"},
- {file = "propcache-0.3.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:b0313e8b923b3814d1c4a524c93dfecea5f39fa95601f6a9b1ac96cd66f89ea0"},
- {file = "propcache-0.3.1-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:e861ad82892408487be144906a368ddbe2dc6297074ade2d892341b35c59844a"},
- {file = "propcache-0.3.1-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:61014615c1274df8da5991a1e5da85a3ccb00c2d4701ac6f3383afd3ca47ab0a"},
- {file = "propcache-0.3.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:71ebe3fe42656a2328ab08933d420df5f3ab121772eef78f2dc63624157f0ed9"},
- {file = "propcache-0.3.1-cp311-cp311-win32.whl", hash = "sha256:58aa11f4ca8b60113d4b8e32d37e7e78bd8af4d1a5b5cb4979ed856a45e62005"},
- {file = "propcache-0.3.1-cp311-cp311-win_amd64.whl", hash = "sha256:9532ea0b26a401264b1365146c440a6d78269ed41f83f23818d4b79497aeabe7"},
- {file = "propcache-0.3.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:f78eb8422acc93d7b69964012ad7048764bb45a54ba7a39bb9e146c72ea29723"},
- {file = "propcache-0.3.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:89498dd49c2f9a026ee057965cdf8192e5ae070ce7d7a7bd4b66a8e257d0c976"},
- {file = "propcache-0.3.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:09400e98545c998d57d10035ff623266927cb784d13dd2b31fd33b8a5316b85b"},
- {file = "propcache-0.3.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aa8efd8c5adc5a2c9d3b952815ff8f7710cefdcaf5f2c36d26aff51aeca2f12f"},
- {file = "propcache-0.3.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c2fe5c910f6007e716a06d269608d307b4f36e7babee5f36533722660e8c4a70"},
- {file = "propcache-0.3.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a0ab8cf8cdd2194f8ff979a43ab43049b1df0b37aa64ab7eca04ac14429baeb7"},
- {file = "propcache-0.3.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:563f9d8c03ad645597b8d010ef4e9eab359faeb11a0a2ac9f7b4bc8c28ebef25"},
- {file = "propcache-0.3.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fb6e0faf8cb6b4beea5d6ed7b5a578254c6d7df54c36ccd3d8b3eb00d6770277"},
- {file = "propcache-0.3.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:1c5c7ab7f2bb3f573d1cb921993006ba2d39e8621019dffb1c5bc94cdbae81e8"},
- {file = "propcache-0.3.1-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:050b571b2e96ec942898f8eb46ea4bfbb19bd5502424747e83badc2d4a99a44e"},
- {file = "propcache-0.3.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:e1c4d24b804b3a87e9350f79e2371a705a188d292fd310e663483af6ee6718ee"},
- {file = "propcache-0.3.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:e4fe2a6d5ce975c117a6bb1e8ccda772d1e7029c1cca1acd209f91d30fa72815"},
- {file = "propcache-0.3.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:feccd282de1f6322f56f6845bf1207a537227812f0a9bf5571df52bb418d79d5"},
- {file = "propcache-0.3.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ec314cde7314d2dd0510c6787326bbffcbdc317ecee6b7401ce218b3099075a7"},
- {file = "propcache-0.3.1-cp312-cp312-win32.whl", hash = "sha256:7d2d5a0028d920738372630870e7d9644ce437142197f8c827194fca404bf03b"},
- {file = "propcache-0.3.1-cp312-cp312-win_amd64.whl", hash = "sha256:88c423efef9d7a59dae0614eaed718449c09a5ac79a5f224a8b9664d603f04a3"},
- {file = "propcache-0.3.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:f1528ec4374617a7a753f90f20e2f551121bb558fcb35926f99e3c42367164b8"},
- {file = "propcache-0.3.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:dc1915ec523b3b494933b5424980831b636fe483d7d543f7afb7b3bf00f0c10f"},
- {file = "propcache-0.3.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:a110205022d077da24e60b3df8bcee73971be9575dec5573dd17ae5d81751111"},
- {file = "propcache-0.3.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d249609e547c04d190e820d0d4c8ca03ed4582bcf8e4e160a6969ddfb57b62e5"},
- {file = "propcache-0.3.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5ced33d827625d0a589e831126ccb4f5c29dfdf6766cac441d23995a65825dcb"},
- {file = "propcache-0.3.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4114c4ada8f3181af20808bedb250da6bae56660e4b8dfd9cd95d4549c0962f7"},
- {file = "propcache-0.3.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:975af16f406ce48f1333ec5e912fe11064605d5c5b3f6746969077cc3adeb120"},
- {file = "propcache-0.3.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a34aa3a1abc50740be6ac0ab9d594e274f59960d3ad253cd318af76b996dd654"},
- {file = "propcache-0.3.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:9cec3239c85ed15bfaded997773fdad9fb5662b0a7cbc854a43f291eb183179e"},
- {file = "propcache-0.3.1-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:05543250deac8e61084234d5fc54f8ebd254e8f2b39a16b1dce48904f45b744b"},
- {file = "propcache-0.3.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:5cb5918253912e088edbf023788de539219718d3b10aef334476b62d2b53de53"},
- {file = "propcache-0.3.1-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:f3bbecd2f34d0e6d3c543fdb3b15d6b60dd69970c2b4c822379e5ec8f6f621d5"},
- {file = "propcache-0.3.1-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:aca63103895c7d960a5b9b044a83f544b233c95e0dcff114389d64d762017af7"},
- {file = "propcache-0.3.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:5a0a9898fdb99bf11786265468571e628ba60af80dc3f6eb89a3545540c6b0ef"},
- {file = "propcache-0.3.1-cp313-cp313-win32.whl", hash = "sha256:3a02a28095b5e63128bcae98eb59025924f121f048a62393db682f049bf4ac24"},
- {file = "propcache-0.3.1-cp313-cp313-win_amd64.whl", hash = "sha256:813fbb8b6aea2fc9659815e585e548fe706d6f663fa73dff59a1677d4595a037"},
- {file = "propcache-0.3.1-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:a444192f20f5ce8a5e52761a031b90f5ea6288b1eef42ad4c7e64fef33540b8f"},
- {file = "propcache-0.3.1-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:0fbe94666e62ebe36cd652f5fc012abfbc2342de99b523f8267a678e4dfdee3c"},
- {file = "propcache-0.3.1-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:f011f104db880f4e2166bcdcf7f58250f7a465bc6b068dc84c824a3d4a5c94dc"},
- {file = "propcache-0.3.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3e584b6d388aeb0001d6d5c2bd86b26304adde6d9bb9bfa9c4889805021b96de"},
- {file = "propcache-0.3.1-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8a17583515a04358b034e241f952f1715243482fc2c2945fd99a1b03a0bd77d6"},
- {file = "propcache-0.3.1-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5aed8d8308215089c0734a2af4f2e95eeb360660184ad3912686c181e500b2e7"},
- {file = "propcache-0.3.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6d8e309ff9a0503ef70dc9a0ebd3e69cf7b3894c9ae2ae81fc10943c37762458"},
- {file = "propcache-0.3.1-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b655032b202028a582d27aeedc2e813299f82cb232f969f87a4fde491a233f11"},
- {file = "propcache-0.3.1-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:9f64d91b751df77931336b5ff7bafbe8845c5770b06630e27acd5dbb71e1931c"},
- {file = "propcache-0.3.1-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:19a06db789a4bd896ee91ebc50d059e23b3639c25d58eb35be3ca1cbe967c3bf"},
- {file = "propcache-0.3.1-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:bef100c88d8692864651b5f98e871fb090bd65c8a41a1cb0ff2322db39c96c27"},
- {file = "propcache-0.3.1-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:87380fb1f3089d2a0b8b00f006ed12bd41bd858fabfa7330c954c70f50ed8757"},
- {file = "propcache-0.3.1-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:e474fc718e73ba5ec5180358aa07f6aded0ff5f2abe700e3115c37d75c947e18"},
- {file = "propcache-0.3.1-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:17d1c688a443355234f3c031349da69444be052613483f3e4158eef751abcd8a"},
- {file = "propcache-0.3.1-cp313-cp313t-win32.whl", hash = "sha256:359e81a949a7619802eb601d66d37072b79b79c2505e6d3fd8b945538411400d"},
- {file = "propcache-0.3.1-cp313-cp313t-win_amd64.whl", hash = "sha256:e7fb9a84c9abbf2b2683fa3e7b0d7da4d8ecf139a1c635732a8bda29c5214b0e"},
- {file = "propcache-0.3.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:ed5f6d2edbf349bd8d630e81f474d33d6ae5d07760c44d33cd808e2f5c8f4ae6"},
- {file = "propcache-0.3.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:668ddddc9f3075af019f784456267eb504cb77c2c4bd46cc8402d723b4d200bf"},
- {file = "propcache-0.3.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0c86e7ceea56376216eba345aa1fc6a8a6b27ac236181f840d1d7e6a1ea9ba5c"},
- {file = "propcache-0.3.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:83be47aa4e35b87c106fc0c84c0fc069d3f9b9b06d3c494cd404ec6747544894"},
- {file = "propcache-0.3.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:27c6ac6aa9fc7bc662f594ef380707494cb42c22786a558d95fcdedb9aa5d035"},
- {file = "propcache-0.3.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:64a956dff37080b352c1c40b2966b09defb014347043e740d420ca1eb7c9b908"},
- {file = "propcache-0.3.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:82de5da8c8893056603ac2d6a89eb8b4df49abf1a7c19d536984c8dd63f481d5"},
- {file = "propcache-0.3.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0c3c3a203c375b08fd06a20da3cf7aac293b834b6f4f4db71190e8422750cca5"},
- {file = "propcache-0.3.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:b303b194c2e6f171cfddf8b8ba30baefccf03d36a4d9cab7fd0bb68ba476a3d7"},
- {file = "propcache-0.3.1-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:916cd229b0150129d645ec51614d38129ee74c03293a9f3f17537be0029a9641"},
- {file = "propcache-0.3.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:a461959ead5b38e2581998700b26346b78cd98540b5524796c175722f18b0294"},
- {file = "propcache-0.3.1-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:069e7212890b0bcf9b2be0a03afb0c2d5161d91e1bf51569a64f629acc7defbf"},
- {file = "propcache-0.3.1-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:ef2e4e91fb3945769e14ce82ed53007195e616a63aa43b40fb7ebaaf907c8d4c"},
- {file = "propcache-0.3.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:8638f99dca15b9dff328fb6273e09f03d1c50d9b6512f3b65a4154588a7595fe"},
- {file = "propcache-0.3.1-cp39-cp39-win32.whl", hash = "sha256:6f173bbfe976105aaa890b712d1759de339d8a7cef2fc0a1714cc1a1e1c47f64"},
- {file = "propcache-0.3.1-cp39-cp39-win_amd64.whl", hash = "sha256:603f1fe4144420374f1a69b907494c3acbc867a581c2d49d4175b0de7cc64566"},
- {file = "propcache-0.3.1-py3-none-any.whl", hash = "sha256:9a8ecf38de50a7f518c21568c80f985e776397b902f1ce0b01f799aba1608b40"},
- {file = "propcache-0.3.1.tar.gz", hash = "sha256:40d980c33765359098837527e18eddefc9a24cea5b45e078a7f3bb5b032c6ecf"},
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+files = [
+ {file = "propcache-0.3.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:22d9962a358aedbb7a2e36187ff273adeaab9743373a272976d2e348d08c7770"},
+ {file = "propcache-0.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0d0fda578d1dc3f77b6b5a5dce3b9ad69a8250a891760a548df850a5e8da87f3"},
+ {file = "propcache-0.3.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:3def3da3ac3ce41562d85db655d18ebac740cb3fa4367f11a52b3da9d03a5cc3"},
+ {file = "propcache-0.3.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9bec58347a5a6cebf239daba9bda37dffec5b8d2ce004d9fe4edef3d2815137e"},
+ {file = "propcache-0.3.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:55ffda449a507e9fbd4aca1a7d9aa6753b07d6166140e5a18d2ac9bc49eac220"},
+ {file = "propcache-0.3.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:64a67fb39229a8a8491dd42f864e5e263155e729c2e7ff723d6e25f596b1e8cb"},
+ {file = "propcache-0.3.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9da1cf97b92b51253d5b68cf5a2b9e0dafca095e36b7f2da335e27dc6172a614"},
+ {file = "propcache-0.3.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5f559e127134b07425134b4065be45b166183fdcb433cb6c24c8e4149056ad50"},
+ {file = "propcache-0.3.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:aff2e4e06435d61f11a428360a932138d0ec288b0a31dd9bd78d200bd4a2b339"},
+ {file = "propcache-0.3.2-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:4927842833830942a5d0a56e6f4839bc484785b8e1ce8d287359794818633ba0"},
+ {file = "propcache-0.3.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:6107ddd08b02654a30fb8ad7a132021759d750a82578b94cd55ee2772b6ebea2"},
+ {file = "propcache-0.3.2-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:70bd8b9cd6b519e12859c99f3fc9a93f375ebd22a50296c3a295028bea73b9e7"},
+ {file = "propcache-0.3.2-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:2183111651d710d3097338dd1893fcf09c9f54e27ff1a8795495a16a469cc90b"},
+ {file = "propcache-0.3.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:fb075ad271405dcad8e2a7ffc9a750a3bf70e533bd86e89f0603e607b93aa64c"},
+ {file = "propcache-0.3.2-cp310-cp310-win32.whl", hash = "sha256:404d70768080d3d3bdb41d0771037da19d8340d50b08e104ca0e7f9ce55fce70"},
+ {file = "propcache-0.3.2-cp310-cp310-win_amd64.whl", hash = "sha256:7435d766f978b4ede777002e6b3b6641dd229cd1da8d3d3106a45770365f9ad9"},
+ {file = "propcache-0.3.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:0b8d2f607bd8f80ddc04088bc2a037fdd17884a6fcadc47a96e334d72f3717be"},
+ {file = "propcache-0.3.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:06766d8f34733416e2e34f46fea488ad5d60726bb9481d3cddf89a6fa2d9603f"},
+ {file = "propcache-0.3.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a2dc1f4a1df4fecf4e6f68013575ff4af84ef6f478fe5344317a65d38a8e6dc9"},
+ {file = "propcache-0.3.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:be29c4f4810c5789cf10ddf6af80b041c724e629fa51e308a7a0fb19ed1ef7bf"},
+ {file = "propcache-0.3.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:59d61f6970ecbd8ff2e9360304d5c8876a6abd4530cb752c06586849ac8a9dc9"},
+ {file = "propcache-0.3.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:62180e0b8dbb6b004baec00a7983e4cc52f5ada9cd11f48c3528d8cfa7b96a66"},
+ {file = "propcache-0.3.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c144ca294a204c470f18cf4c9d78887810d04a3e2fbb30eea903575a779159df"},
+ {file = "propcache-0.3.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c5c2a784234c28854878d68978265617aa6dc0780e53d44b4d67f3651a17a9a2"},
+ {file = "propcache-0.3.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:5745bc7acdafa978ca1642891b82c19238eadc78ba2aaa293c6863b304e552d7"},
+ {file = "propcache-0.3.2-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:c0075bf773d66fa8c9d41f66cc132ecc75e5bb9dd7cce3cfd14adc5ca184cb95"},
+ {file = "propcache-0.3.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:5f57aa0847730daceff0497f417c9de353c575d8da3579162cc74ac294c5369e"},
+ {file = "propcache-0.3.2-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:eef914c014bf72d18efb55619447e0aecd5fb7c2e3fa7441e2e5d6099bddff7e"},
+ {file = "propcache-0.3.2-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:2a4092e8549031e82facf3decdbc0883755d5bbcc62d3aea9d9e185549936dcf"},
+ {file = "propcache-0.3.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:85871b050f174bc0bfb437efbdb68aaf860611953ed12418e4361bc9c392749e"},
+ {file = "propcache-0.3.2-cp311-cp311-win32.whl", hash = "sha256:36c8d9b673ec57900c3554264e630d45980fd302458e4ac801802a7fd2ef7897"},
+ {file = "propcache-0.3.2-cp311-cp311-win_amd64.whl", hash = "sha256:e53af8cb6a781b02d2ea079b5b853ba9430fcbe18a8e3ce647d5982a3ff69f39"},
+ {file = "propcache-0.3.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:8de106b6c84506b31c27168582cd3cb3000a6412c16df14a8628e5871ff83c10"},
+ {file = "propcache-0.3.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:28710b0d3975117239c76600ea351934ac7b5ff56e60953474342608dbbb6154"},
+ {file = "propcache-0.3.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:ce26862344bdf836650ed2487c3d724b00fbfec4233a1013f597b78c1cb73615"},
+ {file = "propcache-0.3.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bca54bd347a253af2cf4544bbec232ab982f4868de0dd684246b67a51bc6b1db"},
+ {file = "propcache-0.3.2-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:55780d5e9a2ddc59711d727226bb1ba83a22dd32f64ee15594b9392b1f544eb1"},
+ {file = "propcache-0.3.2-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:035e631be25d6975ed87ab23153db6a73426a48db688070d925aa27e996fe93c"},
+ {file = "propcache-0.3.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ee6f22b6eaa39297c751d0e80c0d3a454f112f5c6481214fcf4c092074cecd67"},
+ {file = "propcache-0.3.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7ca3aee1aa955438c4dba34fc20a9f390e4c79967257d830f137bd5a8a32ed3b"},
+ {file = "propcache-0.3.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7a4f30862869fa2b68380d677cc1c5fcf1e0f2b9ea0cf665812895c75d0ca3b8"},
+ {file = "propcache-0.3.2-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:b77ec3c257d7816d9f3700013639db7491a434644c906a2578a11daf13176251"},
+ {file = "propcache-0.3.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:cab90ac9d3f14b2d5050928483d3d3b8fb6b4018893fc75710e6aa361ecb2474"},
+ {file = "propcache-0.3.2-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:0b504d29f3c47cf6b9e936c1852246c83d450e8e063d50562115a6be6d3a2535"},
+ {file = "propcache-0.3.2-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:ce2ac2675a6aa41ddb2a0c9cbff53780a617ac3d43e620f8fd77ba1c84dcfc06"},
+ {file = "propcache-0.3.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:62b4239611205294cc433845b914131b2a1f03500ff3c1ed093ed216b82621e1"},
+ {file = "propcache-0.3.2-cp312-cp312-win32.whl", hash = "sha256:df4a81b9b53449ebc90cc4deefb052c1dd934ba85012aa912c7ea7b7e38b60c1"},
+ {file = "propcache-0.3.2-cp312-cp312-win_amd64.whl", hash = "sha256:7046e79b989d7fe457bb755844019e10f693752d169076138abf17f31380800c"},
+ {file = "propcache-0.3.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ca592ed634a73ca002967458187109265e980422116c0a107cf93d81f95af945"},
+ {file = "propcache-0.3.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:9ecb0aad4020e275652ba3975740f241bd12a61f1a784df044cf7477a02bc252"},
+ {file = "propcache-0.3.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:7f08f1cc28bd2eade7a8a3d2954ccc673bb02062e3e7da09bc75d843386b342f"},
+ {file = "propcache-0.3.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d1a342c834734edb4be5ecb1e9fb48cb64b1e2320fccbd8c54bf8da8f2a84c33"},
+ {file = "propcache-0.3.2-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8a544caaae1ac73f1fecfae70ded3e93728831affebd017d53449e3ac052ac1e"},
+ {file = "propcache-0.3.2-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:310d11aa44635298397db47a3ebce7db99a4cc4b9bbdfcf6c98a60c8d5261cf1"},
+ {file = "propcache-0.3.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4c1396592321ac83157ac03a2023aa6cc4a3cc3cfdecb71090054c09e5a7cce3"},
+ {file = "propcache-0.3.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8cabf5b5902272565e78197edb682017d21cf3b550ba0460ee473753f28d23c1"},
+ {file = "propcache-0.3.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:0a2f2235ac46a7aa25bdeb03a9e7060f6ecbd213b1f9101c43b3090ffb971ef6"},
+ {file = "propcache-0.3.2-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:92b69e12e34869a6970fd2f3da91669899994b47c98f5d430b781c26f1d9f387"},
+ {file = "propcache-0.3.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:54e02207c79968ebbdffc169591009f4474dde3b4679e16634d34c9363ff56b4"},
+ {file = "propcache-0.3.2-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:4adfb44cb588001f68c5466579d3f1157ca07f7504fc91ec87862e2b8e556b88"},
+ {file = "propcache-0.3.2-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:fd3e6019dc1261cd0291ee8919dd91fbab7b169bb76aeef6c716833a3f65d206"},
+ {file = "propcache-0.3.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:4c181cad81158d71c41a2bce88edce078458e2dd5ffee7eddd6b05da85079f43"},
+ {file = "propcache-0.3.2-cp313-cp313-win32.whl", hash = "sha256:8a08154613f2249519e549de2330cf8e2071c2887309a7b07fb56098f5170a02"},
+ {file = "propcache-0.3.2-cp313-cp313-win_amd64.whl", hash = "sha256:e41671f1594fc4ab0a6dec1351864713cb3a279910ae8b58f884a88a0a632c05"},
+ {file = "propcache-0.3.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:9a3cf035bbaf035f109987d9d55dc90e4b0e36e04bbbb95af3055ef17194057b"},
+ {file = "propcache-0.3.2-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:156c03d07dc1323d8dacaa221fbe028c5c70d16709cdd63502778e6c3ccca1b0"},
+ {file = "propcache-0.3.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:74413c0ba02ba86f55cf60d18daab219f7e531620c15f1e23d95563f505efe7e"},
+ {file = "propcache-0.3.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f066b437bb3fa39c58ff97ab2ca351db465157d68ed0440abecb21715eb24b28"},
+ {file = "propcache-0.3.2-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f1304b085c83067914721e7e9d9917d41ad87696bf70f0bc7dee450e9c71ad0a"},
+ {file = "propcache-0.3.2-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ab50cef01b372763a13333b4e54021bdcb291fc9a8e2ccb9c2df98be51bcde6c"},
+ {file = "propcache-0.3.2-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fad3b2a085ec259ad2c2842666b2a0a49dea8463579c606426128925af1ed725"},
+ {file = "propcache-0.3.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:261fa020c1c14deafd54c76b014956e2f86991af198c51139faf41c4d5e83892"},
+ {file = "propcache-0.3.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:46d7f8aa79c927e5f987ee3a80205c987717d3659f035c85cf0c3680526bdb44"},
+ {file = "propcache-0.3.2-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:6d8f3f0eebf73e3c0ff0e7853f68be638b4043c65a70517bb575eff54edd8dbe"},
+ {file = "propcache-0.3.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:03c89c1b14a5452cf15403e291c0ccd7751d5b9736ecb2c5bab977ad6c5bcd81"},
+ {file = "propcache-0.3.2-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:0cc17efde71e12bbaad086d679ce575268d70bc123a5a71ea7ad76f70ba30bba"},
+ {file = "propcache-0.3.2-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:acdf05d00696bc0447e278bb53cb04ca72354e562cf88ea6f9107df8e7fd9770"},
+ {file = "propcache-0.3.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:4445542398bd0b5d32df908031cb1b30d43ac848e20470a878b770ec2dcc6330"},
+ {file = "propcache-0.3.2-cp313-cp313t-win32.whl", hash = "sha256:f86e5d7cd03afb3a1db8e9f9f6eff15794e79e791350ac48a8c924e6f439f394"},
+ {file = "propcache-0.3.2-cp313-cp313t-win_amd64.whl", hash = "sha256:9704bedf6e7cbe3c65eca4379a9b53ee6a83749f047808cbb5044d40d7d72198"},
+ {file = "propcache-0.3.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:a7fad897f14d92086d6b03fdd2eb844777b0c4d7ec5e3bac0fbae2ab0602bbe5"},
+ {file = "propcache-0.3.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:1f43837d4ca000243fd7fd6301947d7cb93360d03cd08369969450cc6b2ce3b4"},
+ {file = "propcache-0.3.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:261df2e9474a5949c46e962065d88eb9b96ce0f2bd30e9d3136bcde84befd8f2"},
+ {file = "propcache-0.3.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e514326b79e51f0a177daab1052bc164d9d9e54133797a3a58d24c9c87a3fe6d"},
+ {file = "propcache-0.3.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d4a996adb6904f85894570301939afeee65f072b4fd265ed7e569e8d9058e4ec"},
+ {file = "propcache-0.3.2-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:76cace5d6b2a54e55b137669b30f31aa15977eeed390c7cbfb1dafa8dfe9a701"},
+ {file = "propcache-0.3.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:31248e44b81d59d6addbb182c4720f90b44e1efdc19f58112a3c3a1615fb47ef"},
+ {file = "propcache-0.3.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:abb7fa19dbf88d3857363e0493b999b8011eea856b846305d8c0512dfdf8fbb1"},
+ {file = "propcache-0.3.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:d81ac3ae39d38588ad0549e321e6f773a4e7cc68e7751524a22885d5bbadf886"},
+ {file = "propcache-0.3.2-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:cc2782eb0f7a16462285b6f8394bbbd0e1ee5f928034e941ffc444012224171b"},
+ {file = "propcache-0.3.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:db429c19a6c7e8a1c320e6a13c99799450f411b02251fb1b75e6217cf4a14fcb"},
+ {file = "propcache-0.3.2-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:21d8759141a9e00a681d35a1f160892a36fb6caa715ba0b832f7747da48fb6ea"},
+ {file = "propcache-0.3.2-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:2ca6d378f09adb13837614ad2754fa8afaee330254f404299611bce41a8438cb"},
+ {file = "propcache-0.3.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:34a624af06c048946709f4278b4176470073deda88d91342665d95f7c6270fbe"},
+ {file = "propcache-0.3.2-cp39-cp39-win32.whl", hash = "sha256:4ba3fef1c30f306b1c274ce0b8baaa2c3cdd91f645c48f06394068f37d3837a1"},
+ {file = "propcache-0.3.2-cp39-cp39-win_amd64.whl", hash = "sha256:7a2368eed65fc69a7a7a40b27f22e85e7627b74216f0846b04ba5c116e191ec9"},
+ {file = "propcache-0.3.2-py3-none-any.whl", hash = "sha256:98f1ec44fb675f5052cccc8e609c46ed23a35a1cfd18545ad4e29002d858a43f"},
+ {file = "propcache-0.3.2.tar.gz", hash = "sha256:20d7d62e4e7ef05f221e0db2856b979540686342e7dd9973b815599c7057e168"},
+]
+
+[[package]]
+name = "proto-plus"
+version = "1.26.1"
+description = "Beautiful, Pythonic protocol buffers"
+optional = true
+python-versions = ">=3.7"
+groups = ["main"]
+markers = "extra == \"gcsfs\""
+files = [
+ {file = "proto_plus-1.26.1-py3-none-any.whl", hash = "sha256:13285478c2dcf2abb829db158e1047e2f1e8d63a077d94263c2b88b043c75a66"},
+ {file = "proto_plus-1.26.1.tar.gz", hash = "sha256:21a515a4c4c0088a773899e23c7bbade3d18f9c66c73edd4c7ee3816bc96a012"},
]
+[package.dependencies]
+protobuf = ">=3.19.0,<7.0.0"
+
+[package.extras]
+testing = ["google-api-core (>=1.31.5)"]
+
[[package]]
name = "protobuf"
-version = "6.30.1"
+version = "6.31.1"
description = ""
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"ray\" or extra == \"gcsfs\""
files = [
- {file = "protobuf-6.30.1-cp310-abi3-win32.whl", hash = "sha256:ba0706f948d0195f5cac504da156d88174e03218d9364ab40d903788c1903d7e"},
- {file = "protobuf-6.30.1-cp310-abi3-win_amd64.whl", hash = "sha256:ed484f9ddd47f0f1bf0648806cccdb4fe2fb6b19820f9b79a5adf5dcfd1b8c5f"},
- {file = "protobuf-6.30.1-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:aa4f7dfaed0d840b03d08d14bfdb41348feaee06a828a8c455698234135b4075"},
- {file = "protobuf-6.30.1-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:47cd320b7db63e8c9ac35f5596ea1c1e61491d8a8eb6d8b45edc44760b53a4f6"},
- {file = "protobuf-6.30.1-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:e3083660225fa94748ac2e407f09a899e6a28bf9c0e70c75def8d15706bf85fc"},
- {file = "protobuf-6.30.1-cp39-cp39-win32.whl", hash = "sha256:554d7e61cce2aa4c63ca27328f757a9f3867bce8ec213bf09096a8d16bcdcb6a"},
- {file = "protobuf-6.30.1-cp39-cp39-win_amd64.whl", hash = "sha256:b510f55ce60f84dc7febc619b47215b900466e3555ab8cb1ba42deb4496d6cc0"},
- {file = "protobuf-6.30.1-py3-none-any.whl", hash = "sha256:3c25e51e1359f1f5fa3b298faa6016e650d148f214db2e47671131b9063c53be"},
- {file = "protobuf-6.30.1.tar.gz", hash = "sha256:535fb4e44d0236893d5cf1263a0f706f1160b689a7ab962e9da8a9ce4050b780"},
+ {file = "protobuf-6.31.1-cp310-abi3-win32.whl", hash = "sha256:7fa17d5a29c2e04b7d90e5e32388b8bfd0e7107cd8e616feef7ed3fa6bdab5c9"},
+ {file = "protobuf-6.31.1-cp310-abi3-win_amd64.whl", hash = "sha256:426f59d2964864a1a366254fa703b8632dcec0790d8862d30034d8245e1cd447"},
+ {file = "protobuf-6.31.1-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:6f1227473dc43d44ed644425268eb7c2e488ae245d51c6866d19fe158e207402"},
+ {file = "protobuf-6.31.1-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:a40fc12b84c154884d7d4c4ebd675d5b3b5283e155f324049ae396b95ddebc39"},
+ {file = "protobuf-6.31.1-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:4ee898bf66f7a8b0bd21bce523814e6fbd8c6add948045ce958b73af7e8878c6"},
+ {file = "protobuf-6.31.1-cp39-cp39-win32.whl", hash = "sha256:0414e3aa5a5f3ff423828e1e6a6e907d6c65c1d5b7e6e975793d5590bdeecc16"},
+ {file = "protobuf-6.31.1-cp39-cp39-win_amd64.whl", hash = "sha256:8764cf4587791e7564051b35524b72844f845ad0bb011704c3736cce762d8fe9"},
+ {file = "protobuf-6.31.1-py3-none-any.whl", hash = "sha256:720a6c7e6b77288b85063569baae8536671b39f15cc22037ec7045658d80489e"},
+ {file = "protobuf-6.31.1.tar.gz", hash = "sha256:d8cac4c982f0b957a4dc73a80e2ea24fab08e679c0de9deb835f4a12d69aca9a"},
]
[[package]]
@@ -3867,7 +4111,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
+markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -3887,19 +4131,19 @@ files = [
[[package]]
name = "pyasn1-modules"
-version = "0.4.1"
+version = "0.4.2"
description = "A collection of ASN.1-based protocols modules"
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "pyasn1_modules-0.4.1-py3-none-any.whl", hash = "sha256:49bfa96b45a292b711e986f222502c1c9a5e1f4e568fc30e2574a6c7d07838fd"},
- {file = "pyasn1_modules-0.4.1.tar.gz", hash = "sha256:c28e2dbf9c06ad61c71a075c7e0f9fd0f1b0bb2d2ad4377f240d33ac2ab60a7c"},
+ {file = "pyasn1_modules-0.4.2-py3-none-any.whl", hash = "sha256:29253a9207ce32b64c3ac6600edc75368f98473906e8fd1043bd6b5b1de2c14a"},
+ {file = "pyasn1_modules-0.4.2.tar.gz", hash = "sha256:677091de870a80aae844b1ca6134f54652fa2c8c5a52aa396440ac3106e941e6"},
]
[package.dependencies]
-pyasn1 = ">=0.4.6,<0.7.0"
+pyasn1 = ">=0.6.1,<0.7.0"
[[package]]
name = "pycparser"
@@ -4104,14 +4348,14 @@ tests = ["coverage[toml] (==5.0.4)", "pytest (>=6.0.0,<7.0.0)"]
[[package]]
name = "pymdown-extensions"
-version = "10.14.3"
+version = "10.15"
description = "Extension pack for Python Markdown."
optional = false
python-versions = ">=3.8"
groups = ["docs"]
files = [
- {file = "pymdown_extensions-10.14.3-py3-none-any.whl", hash = "sha256:05e0bee73d64b9c71a4ae17c72abc2f700e8bc8403755a00580b49a4e9f189e9"},
- {file = "pymdown_extensions-10.14.3.tar.gz", hash = "sha256:41e576ce3f5d650be59e900e4ceff231e0aed2a88cf30acaee41e02f063a061b"},
+ {file = "pymdown_extensions-10.15-py3-none-any.whl", hash = "sha256:46e99bb272612b0de3b7e7caf6da8dd5f4ca5212c0b273feb9304e236c484e5f"},
+ {file = "pymdown_extensions-10.15.tar.gz", hash = "sha256:0e5994e32155f4b03504f939e501b981d306daf7ec2aa1cd2eb6bd300784f8f7"},
]
[package.dependencies]
@@ -4377,7 +4621,7 @@ description = "World timezone definitions, modern and historical"
optional = true
python-versions = "*"
groups = ["main"]
-markers = "extra == \"gcsfs\" or extra == \"pandas\" or extra == \"ray\""
+markers = "extra == \"pandas\" or extra == \"ray\""
files = [
{file = "pytz-2025.2-py2.py3-none-any.whl", hash = "sha256:5ddf76296dd8c44c26eb8f4b6f35488f3ccbf6fbbd7adee0b7262d43f0ec2f00"},
{file = "pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3"},
@@ -4476,14 +4720,14 @@ markers = {main = "extra == \"ray\" or extra == \"hf\""}
[[package]]
name = "pyyaml-env-tag"
-version = "0.1"
-description = "A custom YAML tag for referencing environment variables in YAML files. "
+version = "1.1"
+description = "A custom YAML tag for referencing environment variables in YAML files."
optional = false
-python-versions = ">=3.6"
+python-versions = ">=3.9"
groups = ["docs"]
files = [
- {file = "pyyaml_env_tag-0.1-py3-none-any.whl", hash = "sha256:af31106dec8a4d68c60207c1886031cbf839b68aa7abccdb19868200532c2069"},
- {file = "pyyaml_env_tag-0.1.tar.gz", hash = "sha256:70092675bda14fdec33b31ba77e7543de9ddc88f2e5b99160396572d11525bdb"},
+ {file = "pyyaml_env_tag-1.1-py3-none-any.whl", hash = "sha256:17109e1a528561e32f026364712fee1264bc2ea6715120891174ed1b980d2e04"},
+ {file = "pyyaml_env_tag-1.1.tar.gz", hash = "sha256:2eb38b75a2d21ee0475d6d97ec19c63287a7e140231e4214969d0eac923cd7ff"},
]
[package.dependencies]
@@ -4734,19 +4978,18 @@ rsa = ["oauthlib[signedtoken] (>=3.0.0)"]
[[package]]
name = "requirements-parser"
-version = "0.11.0"
+version = "0.13.0"
description = "This is a small Python module for parsing Pip requirement files."
optional = false
python-versions = "<4.0,>=3.8"
groups = ["dev"]
files = [
- {file = "requirements_parser-0.11.0-py3-none-any.whl", hash = "sha256:50379eb50311834386c2568263ae5225d7b9d0867fb55cf4ecc93959de2c2684"},
- {file = "requirements_parser-0.11.0.tar.gz", hash = "sha256:35f36dc969d14830bf459803da84f314dc3d17c802592e9e970f63d0359e5920"},
+ {file = "requirements_parser-0.13.0-py3-none-any.whl", hash = "sha256:2b3173faecf19ec5501971b7222d38f04cb45bb9d87d0ad629ca71e2e62ded14"},
+ {file = "requirements_parser-0.13.0.tar.gz", hash = "sha256:0843119ca2cb2331de4eb31b10d70462e39ace698fd660a915c247d2301a4418"},
]
[package.dependencies]
packaging = ">=23.2"
-types-setuptools = ">=69.1.0"
[[package]]
name = "responses"
@@ -4803,142 +5046,162 @@ typing-extensions = {version = ">=4.0.0,<5.0", markers = "python_version < \"3.1
[package.extras]
jupyter = ["ipywidgets (>=7.5.1,<9)"]
+[[package]]
+name = "roman-numerals-py"
+version = "3.1.0"
+description = "Manipulate well-formed Roman numerals"
+optional = false
+python-versions = ">=3.9"
+groups = ["dev"]
+markers = "python_version >= \"3.11\""
+files = [
+ {file = "roman_numerals_py-3.1.0-py3-none-any.whl", hash = "sha256:9da2ad2fb670bcf24e81070ceb3be72f6c11c440d73bd579fbeca1e9f330954c"},
+ {file = "roman_numerals_py-3.1.0.tar.gz", hash = "sha256:be4bf804f083a4ce001b5eb7e3c0862479d10f94c936f6c4e5f250aa5ff5bd2d"},
+]
+
+[package.extras]
+lint = ["mypy (==1.15.0)", "pyright (==1.1.394)", "ruff (==0.9.7)"]
+test = ["pytest (>=8)"]
+
[[package]]
name = "rpds-py"
-version = "0.24.0"
+version = "0.25.1"
description = "Python bindings to Rust's persistent data structures (rpds)"
optional = false
python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "rpds_py-0.24.0-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:006f4342fe729a368c6df36578d7a348c7c716be1da0a1a0f86e3021f8e98724"},
- {file = "rpds_py-0.24.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2d53747da70a4e4b17f559569d5f9506420966083a31c5fbd84e764461c4444b"},
- {file = "rpds_py-0.24.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e8acd55bd5b071156bae57b555f5d33697998752673b9de554dd82f5b5352727"},
- {file = "rpds_py-0.24.0-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7e80d375134ddb04231a53800503752093dbb65dad8dabacce2c84cccc78e964"},
- {file = "rpds_py-0.24.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:60748789e028d2a46fc1c70750454f83c6bdd0d05db50f5ae83e2db500b34da5"},
- {file = "rpds_py-0.24.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6e1daf5bf6c2be39654beae83ee6b9a12347cb5aced9a29eecf12a2d25fff664"},
- {file = "rpds_py-0.24.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1b221c2457d92a1fb3c97bee9095c874144d196f47c038462ae6e4a14436f7bc"},
- {file = "rpds_py-0.24.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:66420986c9afff67ef0c5d1e4cdc2d0e5262f53ad11e4f90e5e22448df485bf0"},
- {file = "rpds_py-0.24.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:43dba99f00f1d37b2a0265a259592d05fcc8e7c19d140fe51c6e6f16faabeb1f"},
- {file = "rpds_py-0.24.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:a88c0d17d039333a41d9bf4616bd062f0bd7aa0edeb6cafe00a2fc2a804e944f"},
- {file = "rpds_py-0.24.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:cc31e13ce212e14a539d430428cd365e74f8b2d534f8bc22dd4c9c55b277b875"},
- {file = "rpds_py-0.24.0-cp310-cp310-win32.whl", hash = "sha256:fc2c1e1b00f88317d9de6b2c2b39b012ebbfe35fe5e7bef980fd2a91f6100a07"},
- {file = "rpds_py-0.24.0-cp310-cp310-win_amd64.whl", hash = "sha256:c0145295ca415668420ad142ee42189f78d27af806fcf1f32a18e51d47dd2052"},
- {file = "rpds_py-0.24.0-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:2d3ee4615df36ab8eb16c2507b11e764dcc11fd350bbf4da16d09cda11fcedef"},
- {file = "rpds_py-0.24.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e13ae74a8a3a0c2f22f450f773e35f893484fcfacb00bb4344a7e0f4f48e1f97"},
- {file = "rpds_py-0.24.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cf86f72d705fc2ef776bb7dd9e5fbba79d7e1f3e258bf9377f8204ad0fc1c51e"},
- {file = "rpds_py-0.24.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:c43583ea8517ed2e780a345dd9960896afc1327e8cf3ac8239c167530397440d"},
- {file = "rpds_py-0.24.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4cd031e63bc5f05bdcda120646a0d32f6d729486d0067f09d79c8db5368f4586"},
- {file = "rpds_py-0.24.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:34d90ad8c045df9a4259c47d2e16a3f21fdb396665c94520dbfe8766e62187a4"},
- {file = "rpds_py-0.24.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e838bf2bb0b91ee67bf2b889a1a841e5ecac06dd7a2b1ef4e6151e2ce155c7ae"},
- {file = "rpds_py-0.24.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:04ecf5c1ff4d589987b4d9882872f80ba13da7d42427234fce8f22efb43133bc"},
- {file = "rpds_py-0.24.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:630d3d8ea77eabd6cbcd2ea712e1c5cecb5b558d39547ac988351195db433f6c"},
- {file = "rpds_py-0.24.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:ebcb786b9ff30b994d5969213a8430cbb984cdd7ea9fd6df06663194bd3c450c"},
- {file = "rpds_py-0.24.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:174e46569968ddbbeb8a806d9922f17cd2b524aa753b468f35b97ff9c19cb718"},
- {file = "rpds_py-0.24.0-cp311-cp311-win32.whl", hash = "sha256:5ef877fa3bbfb40b388a5ae1cb00636a624690dcb9a29a65267054c9ea86d88a"},
- {file = "rpds_py-0.24.0-cp311-cp311-win_amd64.whl", hash = "sha256:e274f62cbd274359eff63e5c7e7274c913e8e09620f6a57aae66744b3df046d6"},
- {file = "rpds_py-0.24.0-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:d8551e733626afec514b5d15befabea0dd70a343a9f23322860c4f16a9430205"},
- {file = "rpds_py-0.24.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0e374c0ce0ca82e5b67cd61fb964077d40ec177dd2c4eda67dba130de09085c7"},
- {file = "rpds_py-0.24.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d69d003296df4840bd445a5d15fa5b6ff6ac40496f956a221c4d1f6f7b4bc4d9"},
- {file = "rpds_py-0.24.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8212ff58ac6dfde49946bea57474a386cca3f7706fc72c25b772b9ca4af6b79e"},
- {file = "rpds_py-0.24.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:528927e63a70b4d5f3f5ccc1fa988a35456eb5d15f804d276709c33fc2f19bda"},
- {file = "rpds_py-0.24.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a824d2c7a703ba6daaca848f9c3d5cb93af0505be505de70e7e66829affd676e"},
- {file = "rpds_py-0.24.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:44d51febb7a114293ffd56c6cf4736cb31cd68c0fddd6aa303ed09ea5a48e029"},
- {file = "rpds_py-0.24.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:3fab5f4a2c64a8fb64fc13b3d139848817a64d467dd6ed60dcdd6b479e7febc9"},
- {file = "rpds_py-0.24.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9be4f99bee42ac107870c61dfdb294d912bf81c3c6d45538aad7aecab468b6b7"},
- {file = "rpds_py-0.24.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:564c96b6076a98215af52f55efa90d8419cc2ef45d99e314fddefe816bc24f91"},
- {file = "rpds_py-0.24.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:75a810b7664c17f24bf2ffd7f92416c00ec84b49bb68e6a0d93e542406336b56"},
- {file = "rpds_py-0.24.0-cp312-cp312-win32.whl", hash = "sha256:f6016bd950be4dcd047b7475fdf55fb1e1f59fc7403f387be0e8123e4a576d30"},
- {file = "rpds_py-0.24.0-cp312-cp312-win_amd64.whl", hash = "sha256:998c01b8e71cf051c28f5d6f1187abbdf5cf45fc0efce5da6c06447cba997034"},
- {file = "rpds_py-0.24.0-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:3d2d8e4508e15fc05b31285c4b00ddf2e0eb94259c2dc896771966a163122a0c"},
- {file = "rpds_py-0.24.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0f00c16e089282ad68a3820fd0c831c35d3194b7cdc31d6e469511d9bffc535c"},
- {file = "rpds_py-0.24.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:951cc481c0c395c4a08639a469d53b7d4afa252529a085418b82a6b43c45c240"},
- {file = "rpds_py-0.24.0-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:c9ca89938dff18828a328af41ffdf3902405a19f4131c88e22e776a8e228c5a8"},
- {file = "rpds_py-0.24.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ed0ef550042a8dbcd657dfb284a8ee00f0ba269d3f2286b0493b15a5694f9fe8"},
- {file = "rpds_py-0.24.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2b2356688e5d958c4d5cb964af865bea84db29971d3e563fb78e46e20fe1848b"},
- {file = "rpds_py-0.24.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:78884d155fd15d9f64f5d6124b486f3d3f7fd7cd71a78e9670a0f6f6ca06fb2d"},
- {file = "rpds_py-0.24.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:6a4a535013aeeef13c5532f802708cecae8d66c282babb5cd916379b72110cf7"},
- {file = "rpds_py-0.24.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:84e0566f15cf4d769dade9b366b7b87c959be472c92dffb70462dd0844d7cbad"},
- {file = "rpds_py-0.24.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:823e74ab6fbaa028ec89615ff6acb409e90ff45580c45920d4dfdddb069f2120"},
- {file = "rpds_py-0.24.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:c61a2cb0085c8783906b2f8b1f16a7e65777823c7f4d0a6aaffe26dc0d358dd9"},
- {file = "rpds_py-0.24.0-cp313-cp313-win32.whl", hash = "sha256:60d9b630c8025b9458a9d114e3af579a2c54bd32df601c4581bd054e85258143"},
- {file = "rpds_py-0.24.0-cp313-cp313-win_amd64.whl", hash = "sha256:6eea559077d29486c68218178ea946263b87f1c41ae7f996b1f30a983c476a5a"},
- {file = "rpds_py-0.24.0-cp313-cp313t-macosx_10_12_x86_64.whl", hash = "sha256:d09dc82af2d3c17e7dd17120b202a79b578d79f2b5424bda209d9966efeed114"},
- {file = "rpds_py-0.24.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:5fc13b44de6419d1e7a7e592a4885b323fbc2f46e1f22151e3a8ed3b8b920405"},
- {file = "rpds_py-0.24.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c347a20d79cedc0a7bd51c4d4b7dbc613ca4e65a756b5c3e57ec84bd43505b47"},
- {file = "rpds_py-0.24.0-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:20f2712bd1cc26a3cc16c5a1bfee9ed1abc33d4cdf1aabd297fe0eb724df4272"},
- {file = "rpds_py-0.24.0-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:aad911555286884be1e427ef0dc0ba3929e6821cbeca2194b13dc415a462c7fd"},
- {file = "rpds_py-0.24.0-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:0aeb3329c1721c43c58cae274d7d2ca85c1690d89485d9c63a006cb79a85771a"},
- {file = "rpds_py-0.24.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2a0f156e9509cee987283abd2296ec816225145a13ed0391df8f71bf1d789e2d"},
- {file = "rpds_py-0.24.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:aa6800adc8204ce898c8a424303969b7aa6a5e4ad2789c13f8648739830323b7"},
- {file = "rpds_py-0.24.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:a18fc371e900a21d7392517c6f60fe859e802547309e94313cd8181ad9db004d"},
- {file = "rpds_py-0.24.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:9168764133fd919f8dcca2ead66de0105f4ef5659cbb4fa044f7014bed9a1797"},
- {file = "rpds_py-0.24.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:5f6e3cec44ba05ee5cbdebe92d052f69b63ae792e7d05f1020ac5e964394080c"},
- {file = "rpds_py-0.24.0-cp313-cp313t-win32.whl", hash = "sha256:8ebc7e65ca4b111d928b669713865f021b7773350eeac4a31d3e70144297baba"},
- {file = "rpds_py-0.24.0-cp313-cp313t-win_amd64.whl", hash = "sha256:675269d407a257b8c00a6b58205b72eec8231656506c56fd429d924ca00bb350"},
- {file = "rpds_py-0.24.0-cp39-cp39-macosx_10_12_x86_64.whl", hash = "sha256:a36b452abbf29f68527cf52e181fced56685731c86b52e852053e38d8b60bc8d"},
- {file = "rpds_py-0.24.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:8b3b397eefecec8e8e39fa65c630ef70a24b09141a6f9fc17b3c3a50bed6b50e"},
- {file = "rpds_py-0.24.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cdabcd3beb2a6dca7027007473d8ef1c3b053347c76f685f5f060a00327b8b65"},
- {file = "rpds_py-0.24.0-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:5db385bacd0c43f24be92b60c857cf760b7f10d8234f4bd4be67b5b20a7c0b6b"},
- {file = "rpds_py-0.24.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8097b3422d020ff1c44effc40ae58e67d93e60d540a65649d2cdaf9466030791"},
- {file = "rpds_py-0.24.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:493fe54318bed7d124ce272fc36adbf59d46729659b2c792e87c3b95649cdee9"},
- {file = "rpds_py-0.24.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8aa362811ccdc1f8dadcc916c6d47e554169ab79559319ae9fae7d7752d0d60c"},
- {file = "rpds_py-0.24.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:d8f9a6e7fd5434817526815f09ea27f2746c4a51ee11bb3439065f5fc754db58"},
- {file = "rpds_py-0.24.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:8205ee14463248d3349131bb8099efe15cd3ce83b8ef3ace63c7e976998e7124"},
- {file = "rpds_py-0.24.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:921ae54f9ecba3b6325df425cf72c074cd469dea843fb5743a26ca7fb2ccb149"},
- {file = "rpds_py-0.24.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:32bab0a56eac685828e00cc2f5d1200c548f8bc11f2e44abf311d6b548ce2e45"},
- {file = "rpds_py-0.24.0-cp39-cp39-win32.whl", hash = "sha256:f5c0ed12926dec1dfe7d645333ea59cf93f4d07750986a586f511c0bc61fe103"},
- {file = "rpds_py-0.24.0-cp39-cp39-win_amd64.whl", hash = "sha256:afc6e35f344490faa8276b5f2f7cbf71f88bc2cda4328e00553bd451728c571f"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:619ca56a5468f933d940e1bf431c6f4e13bef8e688698b067ae68eb4f9b30e3a"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:4b28e5122829181de1898c2c97f81c0b3246d49f585f22743a1246420bb8d399"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e8e5ab32cf9eb3647450bc74eb201b27c185d3857276162c101c0f8c6374e098"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:208b3a70a98cf3710e97cabdc308a51cd4f28aa6e7bb11de3d56cd8b74bab98d"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:bbc4362e06f950c62cad3d4abf1191021b2ffaf0b31ac230fbf0526453eee75e"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ebea2821cdb5f9fef44933617be76185b80150632736f3d76e54829ab4a3b4d1"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b9a4df06c35465ef4d81799999bba810c68d29972bf1c31db61bfdb81dd9d5bb"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:d3aa13bdf38630da298f2e0d77aca967b200b8cc1473ea05248f6c5e9c9bdb44"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:041f00419e1da7a03c46042453598479f45be3d787eb837af382bfc169c0db33"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-musllinux_1_2_i686.whl", hash = "sha256:d8754d872a5dfc3c5bf9c0e059e8107451364a30d9fd50f1f1a85c4fb9481164"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:896c41007931217a343eff197c34513c154267636c8056fb409eafd494c3dcdc"},
- {file = "rpds_py-0.24.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:92558d37d872e808944c3c96d0423b8604879a3d1c86fdad508d7ed91ea547d5"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-macosx_10_12_x86_64.whl", hash = "sha256:f9e0057a509e096e47c87f753136c9b10d7a91842d8042c2ee6866899a717c0d"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:d6e109a454412ab82979c5b1b3aee0604eca4bbf9a02693bb9df027af2bfa91a"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc1c892b1ec1f8cbd5da8de287577b455e388d9c328ad592eabbdcb6fc93bee5"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9c39438c55983d48f4bb3487734d040e22dad200dab22c41e331cee145e7a50d"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9d7e8ce990ae17dda686f7e82fd41a055c668e13ddcf058e7fb5e9da20b57793"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9ea7f4174d2e4194289cb0c4e172d83e79a6404297ff95f2875cf9ac9bced8ba"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bb2954155bb8f63bb19d56d80e5e5320b61d71084617ed89efedb861a684baea"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:04f2b712a2206e13800a8136b07aaedc23af3facab84918e7aa89e4be0260032"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:eda5c1e2a715a4cbbca2d6d304988460942551e4e5e3b7457b50943cd741626d"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-musllinux_1_2_i686.whl", hash = "sha256:9abc80fe8c1f87218db116016de575a7998ab1629078c90840e8d11ab423ee25"},
- {file = "rpds_py-0.24.0-pp311-pypy311_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:6a727fd083009bc83eb83d6950f0c32b3c94c8b80a9b667c87f4bd1274ca30ba"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:e0f3ef95795efcd3b2ec3fe0a5bcfb5dadf5e3996ea2117427e524d4fbf309c6"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:2c13777ecdbbba2077670285dd1fe50828c8742f6a4119dbef6f83ea13ad10fb"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:79e8d804c2ccd618417e96720ad5cd076a86fa3f8cb310ea386a3e6229bae7d1"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:fd822f019ccccd75c832deb7aa040bb02d70a92eb15a2f16c7987b7ad4ee8d83"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0047638c3aa0dbcd0ab99ed1e549bbf0e142c9ecc173b6492868432d8989a046"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a5b66d1b201cc71bc3081bc2f1fc36b0c1f268b773e03bbc39066651b9e18391"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dbcbb6db5582ea33ce46a5d20a5793134b5365110d84df4e30b9d37c6fd40ad3"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:63981feca3f110ed132fd217bf7768ee8ed738a55549883628ee3da75bb9cb78"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:3a55fc10fdcbf1a4bd3c018eea422c52cf08700cf99c28b5cb10fe97ab77a0d3"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-musllinux_1_2_i686.whl", hash = "sha256:c30ff468163a48535ee7e9bf21bd14c7a81147c0e58a36c1078289a8ca7af0bd"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:369d9c6d4c714e36d4a03957b4783217a3ccd1e222cdd67d464a3a479fc17796"},
- {file = "rpds_py-0.24.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:24795c099453e3721fda5d8ddd45f5dfcc8e5a547ce7b8e9da06fecc3832e26f"},
- {file = "rpds_py-0.24.0.tar.gz", hash = "sha256:772cc1b2cd963e7e17e6cc55fe0371fb9c704d63e44cacec7b9b7f523b78919e"},
+ {file = "rpds_py-0.25.1-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:f4ad628b5174d5315761b67f212774a32f5bad5e61396d38108bd801c0a8f5d9"},
+ {file = "rpds_py-0.25.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8c742af695f7525e559c16f1562cf2323db0e3f0fbdcabdf6865b095256b2d40"},
+ {file = "rpds_py-0.25.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:605ffe7769e24b1800b4d024d24034405d9404f0bc2f55b6db3362cd34145a6f"},
+ {file = "rpds_py-0.25.1-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ccc6f3ddef93243538be76f8e47045b4aad7a66a212cd3a0f23e34469473d36b"},
+ {file = "rpds_py-0.25.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f70316f760174ca04492b5ab01be631a8ae30cadab1d1081035136ba12738cfa"},
+ {file = "rpds_py-0.25.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e1dafef8df605fdb46edcc0bf1573dea0d6d7b01ba87f85cd04dc855b2b4479e"},
+ {file = "rpds_py-0.25.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0701942049095741a8aeb298a31b203e735d1c61f4423511d2b1a41dcd8a16da"},
+ {file = "rpds_py-0.25.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:e87798852ae0b37c88babb7f7bbbb3e3fecc562a1c340195b44c7e24d403e380"},
+ {file = "rpds_py-0.25.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3bcce0edc1488906c2d4c75c94c70a0417e83920dd4c88fec1078c94843a6ce9"},
+ {file = "rpds_py-0.25.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e2f6a2347d3440ae789505693a02836383426249d5293541cd712e07e7aecf54"},
+ {file = "rpds_py-0.25.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:4fd52d3455a0aa997734f3835cbc4c9f32571345143960e7d7ebfe7b5fbfa3b2"},
+ {file = "rpds_py-0.25.1-cp310-cp310-win32.whl", hash = "sha256:3f0b1798cae2bbbc9b9db44ee068c556d4737911ad53a4e5093d09d04b3bbc24"},
+ {file = "rpds_py-0.25.1-cp310-cp310-win_amd64.whl", hash = "sha256:3ebd879ab996537fc510a2be58c59915b5dd63bccb06d1ef514fee787e05984a"},
+ {file = "rpds_py-0.25.1-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:5f048bbf18b1f9120685c6d6bb70cc1a52c8cc11bdd04e643d28d3be0baf666d"},
+ {file = "rpds_py-0.25.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:4fbb0dbba559959fcb5d0735a0f87cdbca9e95dac87982e9b95c0f8f7ad10255"},
+ {file = "rpds_py-0.25.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d4ca54b9cf9d80b4016a67a0193ebe0bcf29f6b0a96f09db942087e294d3d4c2"},
+ {file = "rpds_py-0.25.1-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:1ee3e26eb83d39b886d2cb6e06ea701bba82ef30a0de044d34626ede51ec98b0"},
+ {file = "rpds_py-0.25.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:89706d0683c73a26f76a5315d893c051324d771196ae8b13e6ffa1ffaf5e574f"},
+ {file = "rpds_py-0.25.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c2013ee878c76269c7b557a9a9c042335d732e89d482606990b70a839635feb7"},
+ {file = "rpds_py-0.25.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:45e484db65e5380804afbec784522de84fa95e6bb92ef1bd3325d33d13efaebd"},
+ {file = "rpds_py-0.25.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:48d64155d02127c249695abb87d39f0faf410733428d499867606be138161d65"},
+ {file = "rpds_py-0.25.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:048893e902132fd6548a2e661fb38bf4896a89eea95ac5816cf443524a85556f"},
+ {file = "rpds_py-0.25.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:0317177b1e8691ab5879f4f33f4b6dc55ad3b344399e23df2e499de7b10a548d"},
+ {file = "rpds_py-0.25.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:bffcf57826d77a4151962bf1701374e0fc87f536e56ec46f1abdd6a903354042"},
+ {file = "rpds_py-0.25.1-cp311-cp311-win32.whl", hash = "sha256:cda776f1967cb304816173b30994faaf2fd5bcb37e73118a47964a02c348e1bc"},
+ {file = "rpds_py-0.25.1-cp311-cp311-win_amd64.whl", hash = "sha256:dc3c1ff0abc91444cd20ec643d0f805df9a3661fcacf9c95000329f3ddf268a4"},
+ {file = "rpds_py-0.25.1-cp311-cp311-win_arm64.whl", hash = "sha256:5a3ddb74b0985c4387719fc536faced33cadf2172769540c62e2a94b7b9be1c4"},
+ {file = "rpds_py-0.25.1-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:b5ffe453cde61f73fea9430223c81d29e2fbf412a6073951102146c84e19e34c"},
+ {file = "rpds_py-0.25.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:115874ae5e2fdcfc16b2aedc95b5eef4aebe91b28e7e21951eda8a5dc0d3461b"},
+ {file = "rpds_py-0.25.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a714bf6e5e81b0e570d01f56e0c89c6375101b8463999ead3a93a5d2a4af91fa"},
+ {file = "rpds_py-0.25.1-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:35634369325906bcd01577da4c19e3b9541a15e99f31e91a02d010816b49bfda"},
+ {file = "rpds_py-0.25.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d4cb2b3ddc16710548801c6fcc0cfcdeeff9dafbc983f77265877793f2660309"},
+ {file = "rpds_py-0.25.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9ceca1cf097ed77e1a51f1dbc8d174d10cb5931c188a4505ff9f3e119dfe519b"},
+ {file = "rpds_py-0.25.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2c2cd1a4b0c2b8c5e31ffff50d09f39906fe351389ba143c195566056c13a7ea"},
+ {file = "rpds_py-0.25.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:1de336a4b164c9188cb23f3703adb74a7623ab32d20090d0e9bf499a2203ad65"},
+ {file = "rpds_py-0.25.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9fca84a15333e925dd59ce01da0ffe2ffe0d6e5d29a9eeba2148916d1824948c"},
+ {file = "rpds_py-0.25.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:88ec04afe0c59fa64e2f6ea0dd9657e04fc83e38de90f6de201954b4d4eb59bd"},
+ {file = "rpds_py-0.25.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:a8bd2f19e312ce3e1d2c635618e8a8d8132892bb746a7cf74780a489f0f6cdcb"},
+ {file = "rpds_py-0.25.1-cp312-cp312-win32.whl", hash = "sha256:e5e2f7280d8d0d3ef06f3ec1b4fd598d386cc6f0721e54f09109a8132182fbfe"},
+ {file = "rpds_py-0.25.1-cp312-cp312-win_amd64.whl", hash = "sha256:db58483f71c5db67d643857404da360dce3573031586034b7d59f245144cc192"},
+ {file = "rpds_py-0.25.1-cp312-cp312-win_arm64.whl", hash = "sha256:6d50841c425d16faf3206ddbba44c21aa3310a0cebc3c1cdfc3e3f4f9f6f5728"},
+ {file = "rpds_py-0.25.1-cp313-cp313-macosx_10_12_x86_64.whl", hash = "sha256:659d87430a8c8c704d52d094f5ba6fa72ef13b4d385b7e542a08fc240cb4a559"},
+ {file = "rpds_py-0.25.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:68f6f060f0bbdfb0245267da014d3a6da9be127fe3e8cc4a68c6f833f8a23bb1"},
+ {file = "rpds_py-0.25.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:083a9513a33e0b92cf6e7a6366036c6bb43ea595332c1ab5c8ae329e4bcc0a9c"},
+ {file = "rpds_py-0.25.1-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:816568614ecb22b18a010c7a12559c19f6fe993526af88e95a76d5a60b8b75fb"},
+ {file = "rpds_py-0.25.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3c6564c0947a7f52e4792983f8e6cf9bac140438ebf81f527a21d944f2fd0a40"},
+ {file = "rpds_py-0.25.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5c4a128527fe415d73cf1f70a9a688d06130d5810be69f3b553bf7b45e8acf79"},
+ {file = "rpds_py-0.25.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a49e1d7a4978ed554f095430b89ecc23f42014a50ac385eb0c4d163ce213c325"},
+ {file = "rpds_py-0.25.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:d74ec9bc0e2feb81d3f16946b005748119c0f52a153f6db6a29e8cd68636f295"},
+ {file = "rpds_py-0.25.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:3af5b4cc10fa41e5bc64e5c198a1b2d2864337f8fcbb9a67e747e34002ce812b"},
+ {file = "rpds_py-0.25.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:79dc317a5f1c51fd9c6a0c4f48209c6b8526d0524a6904fc1076476e79b00f98"},
+ {file = "rpds_py-0.25.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:1521031351865e0181bc585147624d66b3b00a84109b57fcb7a779c3ec3772cd"},
+ {file = "rpds_py-0.25.1-cp313-cp313-win32.whl", hash = "sha256:5d473be2b13600b93a5675d78f59e63b51b1ba2d0476893415dfbb5477e65b31"},
+ {file = "rpds_py-0.25.1-cp313-cp313-win_amd64.whl", hash = "sha256:a7b74e92a3b212390bdce1d93da9f6488c3878c1d434c5e751cbc202c5e09500"},
+ {file = "rpds_py-0.25.1-cp313-cp313-win_arm64.whl", hash = "sha256:dd326a81afe332ede08eb39ab75b301d5676802cdffd3a8f287a5f0b694dc3f5"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-macosx_10_12_x86_64.whl", hash = "sha256:a58d1ed49a94d4183483a3ce0af22f20318d4a1434acee255d683ad90bf78129"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:f251bf23deb8332823aef1da169d5d89fa84c89f67bdfb566c49dea1fccfd50d"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8dbd586bfa270c1103ece2109314dd423df1fa3d9719928b5d09e4840cec0d72"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:6d273f136e912aa101a9274c3145dcbddbe4bac560e77e6d5b3c9f6e0ed06d34"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:666fa7b1bd0a3810a7f18f6d3a25ccd8866291fbbc3c9b912b917a6715874bb9"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:921954d7fbf3fccc7de8f717799304b14b6d9a45bbeec5a8d7408ccbf531faf5"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f3d86373ff19ca0441ebeb696ef64cb58b8b5cbacffcda5a0ec2f3911732a194"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:c8980cde3bb8575e7c956a530f2c217c1d6aac453474bf3ea0f9c89868b531b6"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:8eb8c84ecea987a2523e057c0d950bcb3f789696c0499290b8d7b3107a719d78"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:e43a005671a9ed5a650f3bc39e4dbccd6d4326b24fb5ea8be5f3a43a6f576c72"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:58f77c60956501a4a627749a6dcb78dac522f249dd96b5c9f1c6af29bfacfb66"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-win32.whl", hash = "sha256:2cb9e5b5e26fc02c8a4345048cd9998c2aca7c2712bd1b36da0c72ee969a3523"},
+ {file = "rpds_py-0.25.1-cp313-cp313t-win_amd64.whl", hash = "sha256:401ca1c4a20cc0510d3435d89c069fe0a9ae2ee6495135ac46bdd49ec0495763"},
+ {file = "rpds_py-0.25.1-cp39-cp39-macosx_10_12_x86_64.whl", hash = "sha256:ce4c8e485a3c59593f1a6f683cf0ea5ab1c1dc94d11eea5619e4fb5228b40fbd"},
+ {file = "rpds_py-0.25.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:d8222acdb51a22929c3b2ddb236b69c59c72af4019d2cba961e2f9add9b6e634"},
+ {file = "rpds_py-0.25.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4593c4eae9b27d22df41cde518b4b9e4464d139e4322e2127daa9b5b981b76be"},
+ {file = "rpds_py-0.25.1-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:bd035756830c712b64725a76327ce80e82ed12ebab361d3a1cdc0f51ea21acb0"},
+ {file = "rpds_py-0.25.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:114a07e85f32b125404f28f2ed0ba431685151c037a26032b213c882f26eb908"},
+ {file = "rpds_py-0.25.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:dec21e02e6cc932538b5203d3a8bd6aa1480c98c4914cb88eea064ecdbc6396a"},
+ {file = "rpds_py-0.25.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:09eab132f41bf792c7a0ea1578e55df3f3e7f61888e340779b06050a9a3f16e9"},
+ {file = "rpds_py-0.25.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:c98f126c4fc697b84c423e387337d5b07e4a61e9feac494362a59fd7a2d9ed80"},
+ {file = "rpds_py-0.25.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:0e6a327af8ebf6baba1c10fadd04964c1965d375d318f4435d5f3f9651550f4a"},
+ {file = "rpds_py-0.25.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:bc120d1132cff853ff617754196d0ac0ae63befe7c8498bd67731ba368abe451"},
+ {file = "rpds_py-0.25.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:140f61d9bed7839446bdd44852e30195c8e520f81329b4201ceead4d64eb3a9f"},
+ {file = "rpds_py-0.25.1-cp39-cp39-win32.whl", hash = "sha256:9c006f3aadeda131b438c3092124bd196b66312f0caa5823ef09585a669cf449"},
+ {file = "rpds_py-0.25.1-cp39-cp39-win_amd64.whl", hash = "sha256:a61d0b2c7c9a0ae45732a77844917b427ff16ad5464b4d4f5e4adb955f582890"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:b24bf3cd93d5b6ecfbedec73b15f143596c88ee249fa98cefa9a9dc9d92c6f28"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:0eb90e94f43e5085623932b68840b6f379f26db7b5c2e6bcef3179bd83c9330f"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d50e4864498a9ab639d6d8854b25e80642bd362ff104312d9770b05d66e5fb13"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7c9409b47ba0650544b0bb3c188243b83654dfe55dcc173a86832314e1a6a35d"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:796ad874c89127c91970652a4ee8b00d56368b7e00d3477f4415fe78164c8000"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:85608eb70a659bf4c1142b2781083d4b7c0c4e2c90eff11856a9754e965b2540"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c4feb9211d15d9160bc85fa72fed46432cdc143eb9cf6d5ca377335a921ac37b"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:ccfa689b9246c48947d31dd9d8b16d89a0ecc8e0e26ea5253068efb6c542b76e"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:3c5b317ecbd8226887994852e85de562f7177add602514d4ac40f87de3ae45a8"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-musllinux_1_2_i686.whl", hash = "sha256:454601988aab2c6e8fd49e7634c65476b2b919647626208e376afcd22019eeb8"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:1c0c434a53714358532d13539272db75a5ed9df75a4a090a753ac7173ec14e11"},
+ {file = "rpds_py-0.25.1-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:f73ce1512e04fbe2bc97836e89830d6b4314c171587a99688082d090f934d20a"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-macosx_10_12_x86_64.whl", hash = "sha256:ee86d81551ec68a5c25373c5643d343150cc54672b5e9a0cafc93c1870a53954"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:89c24300cd4a8e4a51e55c31a8ff3918e6651b241ee8876a42cc2b2a078533ba"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:771c16060ff4e79584dc48902a91ba79fd93eade3aa3a12d6d2a4aadaf7d542b"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:785ffacd0ee61c3e60bdfde93baa6d7c10d86f15655bd706c89da08068dc5038"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2a40046a529cc15cef88ac5ab589f83f739e2d332cb4d7399072242400ed68c9"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:85fc223d9c76cabe5d0bff82214459189720dc135db45f9f66aa7cffbf9ff6c1"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b0be9965f93c222fb9b4cc254235b3b2b215796c03ef5ee64f995b1b69af0762"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:8378fa4a940f3fb509c081e06cb7f7f2adae8cf46ef258b0e0ed7519facd573e"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:33358883a4490287e67a2c391dfaea4d9359860281db3292b6886bf0be3d8692"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-musllinux_1_2_i686.whl", hash = "sha256:1d1fadd539298e70cac2f2cb36f5b8a65f742b9b9f1014dd4ea1f7785e2470bf"},
+ {file = "rpds_py-0.25.1-pp311-pypy311_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:9a46c2fb2545e21181445515960006e85d22025bd2fe6db23e76daec6eb689fe"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:50f2c501a89c9a5f4e454b126193c5495b9fb441a75b298c60591d8a2eb92e1b"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:7d779b325cc8238227c47fbc53964c8cc9a941d5dbae87aa007a1f08f2f77b23"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:036ded36bedb727beeabc16dc1dad7cb154b3fa444e936a03b67a86dc6a5066e"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:245550f5a1ac98504147cba96ffec8fabc22b610742e9150138e5d60774686d7"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ff7c23ba0a88cb7b104281a99476cccadf29de2a0ef5ce864959a52675b1ca83"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e37caa8cdb3b7cf24786451a0bdb853f6347b8b92005eeb64225ae1db54d1c2b"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9f2f48ab00181600ee266a095fe815134eb456163f7d6699f525dee471f312cf"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:9e5fc7484fa7dce57e25063b0ec9638ff02a908304f861d81ea49273e43838c1"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-musllinux_1_2_aarch64.whl", hash = "sha256:d3c10228d6cf6fe2b63d2e7985e94f6916fa46940df46b70449e9ff9297bd3d1"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-musllinux_1_2_i686.whl", hash = "sha256:5d9e40f32745db28c1ef7aad23f6fc458dc1e29945bd6781060f0d15628b8ddf"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-musllinux_1_2_x86_64.whl", hash = "sha256:35a8d1a24b5936b35c5003313bc177403d8bdef0f8b24f28b1c4a255f94ea992"},
+ {file = "rpds_py-0.25.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:6099263f526efff9cf3883dfef505518730f7a7a93049b1d90d42e50a22b4793"},
+ {file = "rpds_py-0.25.1.tar.gz", hash = "sha256:8960b6dac09b62dac26e75d7e2c4a22efb835d827a7278c34f72b2b84fa160e3"},
]
markers = {main = "extra == \"ray\""}
[[package]]
name = "rsa"
-version = "4.9"
+version = "4.9.1"
description = "Pure-Python RSA implementation"
optional = true
-python-versions = ">=3.6,<4"
+python-versions = "<4,>=3.6"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "rsa-4.9-py3-none-any.whl", hash = "sha256:90260d9058e514786967344d0ef75fa8727eed8a7d2e43ce9f4bcf1b536174f7"},
- {file = "rsa-4.9.tar.gz", hash = "sha256:e38464a49c6c85d7f1351b0126661487a7e0a14a50f1675ec50eb34d4f20ef21"},
+ {file = "rsa-4.9.1-py3-none-any.whl", hash = "sha256:68635866661c6836b8d39430f97a996acbd61bfa49406748ea243539fe239762"},
+ {file = "rsa-4.9.1.tar.gz", hash = "sha256:e7bdbfdb5497da4c07dfd35530e1a902659db6ff241e39d9953cad06ebd0ae75"},
]
[package.dependencies]
@@ -4946,21 +5209,21 @@ pyasn1 = ">=0.1.3"
[[package]]
name = "s3fs"
-version = "2025.3.0"
+version = "2025.5.1"
description = "Convenient Filesystem interface over S3"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"s3fs\""
files = [
- {file = "s3fs-2025.3.0-py3-none-any.whl", hash = "sha256:88d803615baa04945156ca0e1498009b7acd3132c07198bd81b3e874846e0aa2"},
- {file = "s3fs-2025.3.0.tar.gz", hash = "sha256:446dd539eb0d0678209723cb7ad1bedbb172185b0d34675b09be1ad81843a644"},
+ {file = "s3fs-2025.5.1-py3-none-any.whl", hash = "sha256:7475e7c40a3a112f17144907ffae50782ab6c03487fe0b45a9c3942bb7a5c606"},
+ {file = "s3fs-2025.5.1.tar.gz", hash = "sha256:84beffa231b8ed94f8d667e93387b38351e1c4447aedea5c2c19dd88b7fcb658"},
]
[package.dependencies]
aiobotocore = ">=2.5.4,<3.0.0"
aiohttp = "<4.0.0a0 || >4.0.0a0,<4.0.0a1 || >4.0.0a1"
-fsspec = "==2025.3.0.*"
+fsspec = "2025.5.1"
[package.extras]
awscli = ["aiobotocore[awscli] (>=2.5.4,<3.0.0)"]
@@ -4977,7 +5240,7 @@ files = [
{file = "s3transfer-0.11.3-py3-none-any.whl", hash = "sha256:ca855bdeb885174b5ffa95b9913622459d4ad8e331fc98eb01e6d5eb6a30655d"},
{file = "s3transfer-0.11.3.tar.gz", hash = "sha256:edae4977e3a122445660c7c114bba949f9d191bae3b34a096f18a1c8c354527a"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.36.0,<2.0a.0"
@@ -4987,16 +5250,15 @@ crt = ["botocore[crt] (>=1.36.0,<2.0a.0)"]
[[package]]
name = "setuptools"
-version = "78.1.1"
+version = "80.9.0"
description = "Easily download, build, install, upgrade, and uninstall Python packages"
optional = false
python-versions = ">=3.9"
-groups = ["main", "dev"]
+groups = ["dev"]
files = [
- {file = "setuptools-78.1.1-py3-none-any.whl", hash = "sha256:c3a9c4211ff4c309edb8b8c4f1cbfa7ae324c4ba9f91ff254e3d305b9fd54561"},
- {file = "setuptools-78.1.1.tar.gz", hash = "sha256:fcc17fd9cd898242f6b4adfaca46137a9edef687f43e6f78469692a5e70d851d"},
+ {file = "setuptools-80.9.0-py3-none-any.whl", hash = "sha256:062d34222ad13e0cc312a4c02d73f059e86a4acbfbdea8f8f76b28c99f306922"},
+ {file = "setuptools-80.9.0.tar.gz", hash = "sha256:f36b47402ecde768dbfafc46e8e4207b4360c654f1f3bb84475f0a28628fb19c"},
]
-markers = {main = "extra == \"gcsfs\""}
[package.extras]
check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\"", "ruff (>=0.8.0) ; sys_platform != \"cygwin\""]
@@ -5021,14 +5283,14 @@ files = [
[[package]]
name = "snowballstemmer"
-version = "2.2.0"
-description = "This package provides 29 stemmers for 28 languages generated from Snowball algorithms."
+version = "3.0.1"
+description = "This package provides 32 stemmers for 30 languages generated from Snowball algorithms."
optional = false
-python-versions = "*"
+python-versions = "!=3.0.*,!=3.1.*,!=3.2.*"
groups = ["dev"]
files = [
- {file = "snowballstemmer-2.2.0-py2.py3-none-any.whl", hash = "sha256:c8e1716e83cc398ae16824e5572ae04e0d9fc2c6b985fb0f900f5f0c96ecba1a"},
- {file = "snowballstemmer-2.2.0.tar.gz", hash = "sha256:09b16deb8547d3412ad7b590689584cd0fe25ec8db3be37788be3810cbf19cb1"},
+ {file = "snowballstemmer-3.0.1-py3-none-any.whl", hash = "sha256:6cd7b3897da8d6c9ffb968a6781fa6532dce9c3618a4b127d920dab764a19064"},
+ {file = "snowballstemmer-3.0.1.tar.gz", hash = "sha256:6d5eeeec8e9f84d4d56b847692bacf79bc2c8e90c7f80ca4444ff8b6f2e52895"},
]
[[package]]
@@ -5050,6 +5312,7 @@ description = "Python documentation generator"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
+markers = "python_version < \"3.10\""
files = [
{file = "sphinx-7.4.7-py3-none-any.whl", hash = "sha256:c2419e2135d11f1951cd994d6eb18a1835bd8fdd8429f9ca375dc1f3281bd239"},
{file = "sphinx-7.4.7.tar.gz", hash = "sha256:242f92a7ea7e6c5b406fdc2615413890ba9f699114a9c09192d7dfead2ee9cfe"},
@@ -5080,6 +5343,80 @@ docs = ["sphinxcontrib-websupport"]
lint = ["flake8 (>=6.0)", "importlib-metadata (>=6.0)", "mypy (==1.10.1)", "pytest (>=6.0)", "ruff (==0.5.2)", "sphinx-lint (>=0.9)", "tomli (>=2)", "types-docutils (==0.21.0.20240711)", "types-requests (>=2.30.0)"]
test = ["cython (>=3.0)", "defusedxml (>=0.7.1)", "pytest (>=8.0)", "setuptools (>=70.0)", "typing_extensions (>=4.9)"]
+[[package]]
+name = "sphinx"
+version = "8.1.3"
+description = "Python documentation generator"
+optional = false
+python-versions = ">=3.10"
+groups = ["dev"]
+markers = "python_version == \"3.10\""
+files = [
+ {file = "sphinx-8.1.3-py3-none-any.whl", hash = "sha256:09719015511837b76bf6e03e42eb7595ac8c2e41eeb9c29c5b755c6b677992a2"},
+ {file = "sphinx-8.1.3.tar.gz", hash = "sha256:43c1911eecb0d3e161ad78611bc905d1ad0e523e4ddc202a58a821773dc4c927"},
+]
+
+[package.dependencies]
+alabaster = ">=0.7.14"
+babel = ">=2.13"
+colorama = {version = ">=0.4.6", markers = "sys_platform == \"win32\""}
+docutils = ">=0.20,<0.22"
+imagesize = ">=1.3"
+Jinja2 = ">=3.1"
+packaging = ">=23.0"
+Pygments = ">=2.17"
+requests = ">=2.30.0"
+snowballstemmer = ">=2.2"
+sphinxcontrib-applehelp = ">=1.0.7"
+sphinxcontrib-devhelp = ">=1.0.6"
+sphinxcontrib-htmlhelp = ">=2.0.6"
+sphinxcontrib-jsmath = ">=1.0.1"
+sphinxcontrib-qthelp = ">=1.0.6"
+sphinxcontrib-serializinghtml = ">=1.1.9"
+tomli = {version = ">=2", markers = "python_version < \"3.11\""}
+
+[package.extras]
+docs = ["sphinxcontrib-websupport"]
+lint = ["flake8 (>=6.0)", "mypy (==1.11.1)", "pyright (==1.1.384)", "pytest (>=6.0)", "ruff (==0.6.9)", "sphinx-lint (>=0.9)", "tomli (>=2)", "types-Pillow (==10.2.0.20240822)", "types-Pygments (==2.18.0.20240506)", "types-colorama (==0.4.15.20240311)", "types-defusedxml (==0.7.0.20240218)", "types-docutils (==0.21.0.20241005)", "types-requests (==2.32.0.20240914)", "types-urllib3 (==1.26.25.14)"]
+test = ["cython (>=3.0)", "defusedxml (>=0.7.1)", "pytest (>=8.0)", "setuptools (>=70.0)", "typing_extensions (>=4.9)"]
+
+[[package]]
+name = "sphinx"
+version = "8.2.3"
+description = "Python documentation generator"
+optional = false
+python-versions = ">=3.11"
+groups = ["dev"]
+markers = "python_version >= \"3.11\""
+files = [
+ {file = "sphinx-8.2.3-py3-none-any.whl", hash = "sha256:4405915165f13521d875a8c29c8970800a0141c14cc5416a38feca4ea5d9b9c3"},
+ {file = "sphinx-8.2.3.tar.gz", hash = "sha256:398ad29dee7f63a75888314e9424d40f52ce5a6a87ae88e7071e80af296ec348"},
+]
+
+[package.dependencies]
+alabaster = ">=0.7.14"
+babel = ">=2.13"
+colorama = {version = ">=0.4.6", markers = "sys_platform == \"win32\""}
+docutils = ">=0.20,<0.22"
+imagesize = ">=1.3"
+Jinja2 = ">=3.1"
+packaging = ">=23.0"
+Pygments = ">=2.17"
+requests = ">=2.30.0"
+roman-numerals-py = ">=1.0.0"
+snowballstemmer = ">=2.2"
+sphinxcontrib-applehelp = ">=1.0.7"
+sphinxcontrib-devhelp = ">=1.0.6"
+sphinxcontrib-htmlhelp = ">=2.0.6"
+sphinxcontrib-jsmath = ">=1.0.1"
+sphinxcontrib-qthelp = ">=1.0.6"
+sphinxcontrib-serializinghtml = ">=1.1.9"
+
+[package.extras]
+docs = ["sphinxcontrib-websupport"]
+lint = ["betterproto (==2.0.0b6)", "mypy (==1.15.0)", "pypi-attestations (==0.0.21)", "pyright (==1.1.395)", "pytest (>=8.0)", "ruff (==0.9.9)", "sphinx-lint (>=0.9)", "types-Pillow (==10.2.0.20240822)", "types-Pygments (==2.19.0.20250219)", "types-colorama (==0.4.15.20240311)", "types-defusedxml (==0.7.0.20240218)", "types-docutils (==0.21.0.20241128)", "types-requests (==2.32.0.20241016)", "types-urllib3 (==1.26.25.14)"]
+test = ["cython (>=3.0)", "defusedxml (>=0.7.1)", "pytest (>=8.0)", "pytest-xdist[psutil] (>=3.4)", "setuptools (>=70.0)", "typing_extensions (>=4.9)"]
+
[[package]]
name = "sphinxcontrib-applehelp"
version = "2.0.0"
@@ -5294,14 +5631,14 @@ python-dateutil = ">=2.6.0"
[[package]]
name = "sympy"
-version = "1.13.3"
+version = "1.14.0"
description = "Computer algebra system (CAS) in Python"
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "sympy-1.13.3-py3-none-any.whl", hash = "sha256:54612cf55a62755ee71824ce692986f23c88ffa77207b30c1368eda4a7060f73"},
- {file = "sympy-1.13.3.tar.gz", hash = "sha256:b27fd2c6530e0ab39e275fc9b683895367e51d5da91baa8d3d64db2565fec4d9"},
+ {file = "sympy-1.14.0-py3-none-any.whl", hash = "sha256:e091cc3e99d2141a0ba2847328f5479b05d94a6635cb96148ccb3f34671bd8f5"},
+ {file = "sympy-1.14.0.tar.gz", hash = "sha256:d3d3fe8df1e5a0b42f0e7bdf50541697dbe7d23746e894990c030e2b05e72517"},
]
[package.dependencies]
@@ -5411,7 +5748,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"daft\" or extra == \"hf\""
+markers = "extra == \"hf\" or extra == \"daft\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -5427,21 +5764,6 @@ notebook = ["ipywidgets (>=6)"]
slack = ["slack-sdk"]
telegram = ["requests"]
-[[package]]
-name = "types-setuptools"
-version = "76.0.0.20250313"
-description = "Typing stubs for setuptools"
-optional = false
-python-versions = ">=3.9"
-groups = ["dev"]
-files = [
- {file = "types_setuptools-76.0.0.20250313-py3-none-any.whl", hash = "sha256:bf454b2a49b8cfd7ebcf5844d4dd5fe4c8666782df1e3663c5866fd51a47460e"},
- {file = "types_setuptools-76.0.0.20250313.tar.gz", hash = "sha256:b2be66f550f95f3cad2a7d46177b273c7e9c80df7d257fa57addbbcfc8126a9e"},
-]
-
-[package.dependencies]
-setuptools = "*"
-
[[package]]
name = "typing-extensions"
version = "4.13.2"
@@ -5457,14 +5779,14 @@ markers = {docs = "python_version <= \"3.10\""}
[[package]]
name = "typing-inspection"
-version = "0.4.0"
+version = "0.4.1"
description = "Runtime typing introspection tools"
optional = false
python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "typing_inspection-0.4.0-py3-none-any.whl", hash = "sha256:50e72559fcd2a6367a19f7a7e610e6afcb9fac940c650290eed893d61386832f"},
- {file = "typing_inspection-0.4.0.tar.gz", hash = "sha256:9765c87de36671694a67904bf2c96e395be9c6439bb6c87b5142569dcdd65122"},
+ {file = "typing_inspection-0.4.1-py3-none-any.whl", hash = "sha256:389055682238f53b04f7badcb49b989835495a96700ced5dab2d8feae4b26f51"},
+ {file = "typing_inspection-0.4.1.tar.gz", hash = "sha256:6ae134cc0203c33377d43188d4064e9b357dba58cff3185f22924610e70a9d28"},
]
[package.dependencies]
@@ -5503,15 +5825,15 @@ socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"]
[[package]]
name = "urllib3"
-version = "2.3.0"
+version = "2.4.0"
description = "HTTP library with thread-safe connection pooling, file post, and more."
optional = false
python-versions = ">=3.9"
groups = ["main", "dev", "docs"]
markers = "python_version >= \"3.10\""
files = [
- {file = "urllib3-2.3.0-py3-none-any.whl", hash = "sha256:1cee9ad369867bfdbbb48b7dd50374c0967a0bb7710050facf0dd6911440e3df"},
- {file = "urllib3-2.3.0.tar.gz", hash = "sha256:f8c5449b3cf0861679ce7e0503c7b44b5ec981bec0d1d3795a07f1ba96f0204d"},
+ {file = "urllib3-2.4.0-py3-none-any.whl", hash = "sha256:4e16665048960a0900c702d4a66415956a584919c03361cac9f1df5c5dd7e813"},
+ {file = "urllib3-2.4.0.tar.gz", hash = "sha256:414bc6535b787febd7567804cc015fee39daab8ad86268f1310a9250697de466"},
]
[package.extras]
@@ -5522,14 +5844,14 @@ zstd = ["zstandard (>=0.18.0)"]
[[package]]
name = "virtualenv"
-version = "20.29.3"
+version = "20.31.2"
description = "Virtual Python Environment builder"
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "virtualenv-20.29.3-py3-none-any.whl", hash = "sha256:3e3d00f5807e83b234dfb6122bf37cfadf4be216c53a49ac059d02414f819170"},
- {file = "virtualenv-20.29.3.tar.gz", hash = "sha256:95e39403fcf3940ac45bc717597dba16110b74506131845d9b687d5e73d947ac"},
+ {file = "virtualenv-20.31.2-py3-none-any.whl", hash = "sha256:36efd0d9650ee985f0cad72065001e66d49a6f24eb44d98980f630686243cf11"},
+ {file = "virtualenv-20.31.2.tar.gz", hash = "sha256:e10c0a9d02835e592521be48b332b6caee6887f332c111aa79a09b9e79efc2af"},
]
[package.dependencies]
@@ -5539,7 +5861,7 @@ platformdirs = ">=3.9.1,<5"
[package.extras]
docs = ["furo (>=2023.7.26)", "proselint (>=0.13)", "sphinx (>=7.1.2,!=7.3)", "sphinx-argparse (>=0.4)", "sphinxcontrib-towncrier (>=0.2.1a0)", "towncrier (>=23.6)"]
-test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.4)", "pytest-env (>=0.8.2)", "pytest-freezer (>=0.4.8) ; platform_python_implementation == \"PyPy\" or platform_python_implementation == \"CPython\" and sys_platform == \"win32\" and python_version >= \"3.13\"", "pytest-mock (>=3.11.1)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=68)", "time-machine (>=2.10) ; platform_python_implementation == \"CPython\""]
+test = ["covdefaults (>=2.3)", "coverage (>=7.2.7)", "coverage-enable-subprocess (>=1)", "flaky (>=3.7)", "packaging (>=23.1)", "pytest (>=7.4)", "pytest-env (>=0.8.2)", "pytest-freezer (>=0.4.8) ; platform_python_implementation == \"PyPy\" or platform_python_implementation == \"GraalVM\" or platform_python_implementation == \"CPython\" and sys_platform == \"win32\" and python_version >= \"3.13\"", "pytest-mock (>=3.11.1)", "pytest-randomly (>=3.12)", "pytest-timeout (>=2.1)", "setuptools (>=68)", "time-machine (>=2.10) ; platform_python_implementation == \"CPython\""]
[[package]]
name = "watchdog"
@@ -5706,112 +6028,134 @@ files = [
[[package]]
name = "yarl"
-version = "1.18.3"
+version = "1.20.1"
description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
-files = [
- {file = "yarl-1.18.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7df647e8edd71f000a5208fe6ff8c382a1de8edfbccdbbfe649d263de07d8c34"},
- {file = "yarl-1.18.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c69697d3adff5aa4f874b19c0e4ed65180ceed6318ec856ebc423aa5850d84f7"},
- {file = "yarl-1.18.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:602d98f2c2d929f8e697ed274fbadc09902c4025c5a9963bf4e9edfc3ab6f7ed"},
- {file = "yarl-1.18.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c654d5207c78e0bd6d749f6dae1dcbbfde3403ad3a4b11f3c5544d9906969dde"},
- {file = "yarl-1.18.3-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5094d9206c64181d0f6e76ebd8fb2f8fe274950a63890ee9e0ebfd58bf9d787b"},
- {file = "yarl-1.18.3-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:35098b24e0327fc4ebdc8ffe336cee0a87a700c24ffed13161af80124b7dc8e5"},
- {file = "yarl-1.18.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3236da9272872443f81fedc389bace88408f64f89f75d1bdb2256069a8730ccc"},
- {file = "yarl-1.18.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e2c08cc9b16f4f4bc522771d96734c7901e7ebef70c6c5c35dd0f10845270bcd"},
- {file = "yarl-1.18.3-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:80316a8bd5109320d38eef8833ccf5f89608c9107d02d2a7f985f98ed6876990"},
- {file = "yarl-1.18.3-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:c1e1cc06da1491e6734f0ea1e6294ce00792193c463350626571c287c9a704db"},
- {file = "yarl-1.18.3-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:fea09ca13323376a2fdfb353a5fa2e59f90cd18d7ca4eaa1fd31f0a8b4f91e62"},
- {file = "yarl-1.18.3-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:e3b9fd71836999aad54084906f8663dffcd2a7fb5cdafd6c37713b2e72be1760"},
- {file = "yarl-1.18.3-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:757e81cae69244257d125ff31663249b3013b5dc0a8520d73694aed497fb195b"},
- {file = "yarl-1.18.3-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:b1771de9944d875f1b98a745bc547e684b863abf8f8287da8466cf470ef52690"},
- {file = "yarl-1.18.3-cp310-cp310-win32.whl", hash = "sha256:8874027a53e3aea659a6d62751800cf6e63314c160fd607489ba5c2edd753cf6"},
- {file = "yarl-1.18.3-cp310-cp310-win_amd64.whl", hash = "sha256:93b2e109287f93db79210f86deb6b9bbb81ac32fc97236b16f7433db7fc437d8"},
- {file = "yarl-1.18.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:8503ad47387b8ebd39cbbbdf0bf113e17330ffd339ba1144074da24c545f0069"},
- {file = "yarl-1.18.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:02ddb6756f8f4517a2d5e99d8b2f272488e18dd0bfbc802f31c16c6c20f22193"},
- {file = "yarl-1.18.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:67a283dd2882ac98cc6318384f565bffc751ab564605959df4752d42483ad889"},
- {file = "yarl-1.18.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d980e0325b6eddc81331d3f4551e2a333999fb176fd153e075c6d1c2530aa8a8"},
- {file = "yarl-1.18.3-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b643562c12680b01e17239be267bc306bbc6aac1f34f6444d1bded0c5ce438ca"},
- {file = "yarl-1.18.3-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c017a3b6df3a1bd45b9fa49a0f54005e53fbcad16633870104b66fa1a30a29d8"},
- {file = "yarl-1.18.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75674776d96d7b851b6498f17824ba17849d790a44d282929c42dbb77d4f17ae"},
- {file = "yarl-1.18.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ccaa3a4b521b780a7e771cc336a2dba389a0861592bbce09a476190bb0c8b4b3"},
- {file = "yarl-1.18.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:2d06d3005e668744e11ed80812e61efd77d70bb7f03e33c1598c301eea20efbb"},
- {file = "yarl-1.18.3-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:9d41beda9dc97ca9ab0b9888cb71f7539124bc05df02c0cff6e5acc5a19dcc6e"},
- {file = "yarl-1.18.3-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:ba23302c0c61a9999784e73809427c9dbedd79f66a13d84ad1b1943802eaaf59"},
- {file = "yarl-1.18.3-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:6748dbf9bfa5ba1afcc7556b71cda0d7ce5f24768043a02a58846e4a443d808d"},
- {file = "yarl-1.18.3-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:0b0cad37311123211dc91eadcb322ef4d4a66008d3e1bdc404808992260e1a0e"},
- {file = "yarl-1.18.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:0fb2171a4486bb075316ee754c6d8382ea6eb8b399d4ec62fde2b591f879778a"},
- {file = "yarl-1.18.3-cp311-cp311-win32.whl", hash = "sha256:61b1a825a13bef4a5f10b1885245377d3cd0bf87cba068e1d9a88c2ae36880e1"},
- {file = "yarl-1.18.3-cp311-cp311-win_amd64.whl", hash = "sha256:b9d60031cf568c627d028239693fd718025719c02c9f55df0a53e587aab951b5"},
- {file = "yarl-1.18.3-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:1dd4bdd05407ced96fed3d7f25dbbf88d2ffb045a0db60dbc247f5b3c5c25d50"},
- {file = "yarl-1.18.3-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:7c33dd1931a95e5d9a772d0ac5e44cac8957eaf58e3c8da8c1414de7dd27c576"},
- {file = "yarl-1.18.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:25b411eddcfd56a2f0cd6a384e9f4f7aa3efee14b188de13048c25b5e91f1640"},
- {file = "yarl-1.18.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:436c4fc0a4d66b2badc6c5fc5ef4e47bb10e4fd9bf0c79524ac719a01f3607c2"},
- {file = "yarl-1.18.3-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e35ef8683211db69ffe129a25d5634319a677570ab6b2eba4afa860f54eeaf75"},
- {file = "yarl-1.18.3-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:84b2deecba4a3f1a398df819151eb72d29bfeb3b69abb145a00ddc8d30094512"},
- {file = "yarl-1.18.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:00e5a1fea0fd4f5bfa7440a47eff01d9822a65b4488f7cff83155a0f31a2ecba"},
- {file = "yarl-1.18.3-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d0e883008013c0e4aef84dcfe2a0b172c4d23c2669412cf5b3371003941f72bb"},
- {file = "yarl-1.18.3-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:5a3f356548e34a70b0172d8890006c37be92995f62d95a07b4a42e90fba54272"},
- {file = "yarl-1.18.3-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:ccd17349166b1bee6e529b4add61727d3f55edb7babbe4069b5764c9587a8cc6"},
- {file = "yarl-1.18.3-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:b958ddd075ddba5b09bb0be8a6d9906d2ce933aee81100db289badbeb966f54e"},
- {file = "yarl-1.18.3-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:c7d79f7d9aabd6011004e33b22bc13056a3e3fb54794d138af57f5ee9d9032cb"},
- {file = "yarl-1.18.3-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:4891ed92157e5430874dad17b15eb1fda57627710756c27422200c52d8a4e393"},
- {file = "yarl-1.18.3-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ce1af883b94304f493698b00d0f006d56aea98aeb49d75ec7d98cd4a777e9285"},
- {file = "yarl-1.18.3-cp312-cp312-win32.whl", hash = "sha256:f91c4803173928a25e1a55b943c81f55b8872f0018be83e3ad4938adffb77dd2"},
- {file = "yarl-1.18.3-cp312-cp312-win_amd64.whl", hash = "sha256:7e2ee16578af3b52ac2f334c3b1f92262f47e02cc6193c598502bd46f5cd1477"},
- {file = "yarl-1.18.3-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:90adb47ad432332d4f0bc28f83a5963f426ce9a1a8809f5e584e704b82685dcb"},
- {file = "yarl-1.18.3-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:913829534200eb0f789d45349e55203a091f45c37a2674678744ae52fae23efa"},
- {file = "yarl-1.18.3-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:ef9f7768395923c3039055c14334ba4d926f3baf7b776c923c93d80195624782"},
- {file = "yarl-1.18.3-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:88a19f62ff30117e706ebc9090b8ecc79aeb77d0b1f5ec10d2d27a12bc9f66d0"},
- {file = "yarl-1.18.3-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e17c9361d46a4d5addf777c6dd5eab0715a7684c2f11b88c67ac37edfba6c482"},
- {file = "yarl-1.18.3-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1a74a13a4c857a84a845505fd2d68e54826a2cd01935a96efb1e9d86c728e186"},
- {file = "yarl-1.18.3-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:41f7ce59d6ee7741af71d82020346af364949314ed3d87553763a2df1829cc58"},
- {file = "yarl-1.18.3-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f52a265001d830bc425f82ca9eabda94a64a4d753b07d623a9f2863fde532b53"},
- {file = "yarl-1.18.3-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:82123d0c954dc58db301f5021a01854a85bf1f3bb7d12ae0c01afc414a882ca2"},
- {file = "yarl-1.18.3-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:2ec9bbba33b2d00999af4631a3397d1fd78290c48e2a3e52d8dd72db3a067ac8"},
- {file = "yarl-1.18.3-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:fbd6748e8ab9b41171bb95c6142faf068f5ef1511935a0aa07025438dd9a9bc1"},
- {file = "yarl-1.18.3-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:877d209b6aebeb5b16c42cbb377f5f94d9e556626b1bfff66d7b0d115be88d0a"},
- {file = "yarl-1.18.3-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:b464c4ab4bfcb41e3bfd3f1c26600d038376c2de3297760dfe064d2cb7ea8e10"},
- {file = "yarl-1.18.3-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8d39d351e7faf01483cc7ff7c0213c412e38e5a340238826be7e0e4da450fdc8"},
- {file = "yarl-1.18.3-cp313-cp313-win32.whl", hash = "sha256:61ee62ead9b68b9123ec24bc866cbef297dd266175d53296e2db5e7f797f902d"},
- {file = "yarl-1.18.3-cp313-cp313-win_amd64.whl", hash = "sha256:578e281c393af575879990861823ef19d66e2b1d0098414855dd367e234f5b3c"},
- {file = "yarl-1.18.3-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:61e5e68cb65ac8f547f6b5ef933f510134a6bf31bb178be428994b0cb46c2a04"},
- {file = "yarl-1.18.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:fe57328fbc1bfd0bd0514470ac692630f3901c0ee39052ae47acd1d90a436719"},
- {file = "yarl-1.18.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:a440a2a624683108a1b454705ecd7afc1c3438a08e890a1513d468671d90a04e"},
- {file = "yarl-1.18.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09c7907c8548bcd6ab860e5f513e727c53b4a714f459b084f6580b49fa1b9cee"},
- {file = "yarl-1.18.3-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b4f6450109834af88cb4cc5ecddfc5380ebb9c228695afc11915a0bf82116789"},
- {file = "yarl-1.18.3-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a9ca04806f3be0ac6d558fffc2fdf8fcef767e0489d2684a21912cc4ed0cd1b8"},
- {file = "yarl-1.18.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:77a6e85b90a7641d2e07184df5557132a337f136250caafc9ccaa4a2a998ca2c"},
- {file = "yarl-1.18.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6333c5a377c8e2f5fae35e7b8f145c617b02c939d04110c76f29ee3676b5f9a5"},
- {file = "yarl-1.18.3-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:0b3c92fa08759dbf12b3a59579a4096ba9af8dd344d9a813fc7f5070d86bbab1"},
- {file = "yarl-1.18.3-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:4ac515b860c36becb81bb84b667466885096b5fc85596948548b667da3bf9f24"},
- {file = "yarl-1.18.3-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:045b8482ce9483ada4f3f23b3774f4e1bf4f23a2d5c912ed5170f68efb053318"},
- {file = "yarl-1.18.3-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:a4bb030cf46a434ec0225bddbebd4b89e6471814ca851abb8696170adb163985"},
- {file = "yarl-1.18.3-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:54d6921f07555713b9300bee9c50fb46e57e2e639027089b1d795ecd9f7fa910"},
- {file = "yarl-1.18.3-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1d407181cfa6e70077df3377938c08012d18893f9f20e92f7d2f314a437c30b1"},
- {file = "yarl-1.18.3-cp39-cp39-win32.whl", hash = "sha256:ac36703a585e0929b032fbaab0707b75dc12703766d0b53486eabd5139ebadd5"},
- {file = "yarl-1.18.3-cp39-cp39-win_amd64.whl", hash = "sha256:ba87babd629f8af77f557b61e49e7c7cac36f22f871156b91e10a6e9d4f829e9"},
- {file = "yarl-1.18.3-py3-none-any.whl", hash = "sha256:b57f4f58099328dfb26c6a771d09fb20dbbae81d20cfb66141251ea063bd101b"},
- {file = "yarl-1.18.3.tar.gz", hash = "sha256:ac1801c45cbf77b6c99242eeff4fffb5e4e73a800b5c4ad4fc0be5def634d2e1"},
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+files = [
+ {file = "yarl-1.20.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6032e6da6abd41e4acda34d75a816012717000fa6839f37124a47fcefc49bec4"},
+ {file = "yarl-1.20.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2c7b34d804b8cf9b214f05015c4fee2ebe7ed05cf581e7192c06555c71f4446a"},
+ {file = "yarl-1.20.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0c869f2651cc77465f6cd01d938d91a11d9ea5d798738c1dc077f3de0b5e5fed"},
+ {file = "yarl-1.20.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:62915e6688eb4d180d93840cda4110995ad50c459bf931b8b3775b37c264af1e"},
+ {file = "yarl-1.20.1-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:41ebd28167bc6af8abb97fec1a399f412eec5fd61a3ccbe2305a18b84fb4ca73"},
+ {file = "yarl-1.20.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:21242b4288a6d56f04ea193adde174b7e347ac46ce6bc84989ff7c1b1ecea84e"},
+ {file = "yarl-1.20.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bea21cdae6c7eb02ba02a475f37463abfe0a01f5d7200121b03e605d6a0439f8"},
+ {file = "yarl-1.20.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1f8a891e4a22a89f5dde7862994485e19db246b70bb288d3ce73a34422e55b23"},
+ {file = "yarl-1.20.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:dd803820d44c8853a109a34e3660e5a61beae12970da479cf44aa2954019bf70"},
+ {file = "yarl-1.20.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:b982fa7f74c80d5c0c7b5b38f908971e513380a10fecea528091405f519b9ebb"},
+ {file = "yarl-1.20.1-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:33f29ecfe0330c570d997bcf1afd304377f2e48f61447f37e846a6058a4d33b2"},
+ {file = "yarl-1.20.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:835ab2cfc74d5eb4a6a528c57f05688099da41cf4957cf08cad38647e4a83b30"},
+ {file = "yarl-1.20.1-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:46b5e0ccf1943a9a6e766b2c2b8c732c55b34e28be57d8daa2b3c1d1d4009309"},
+ {file = "yarl-1.20.1-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:df47c55f7d74127d1b11251fe6397d84afdde0d53b90bedb46a23c0e534f9d24"},
+ {file = "yarl-1.20.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:76d12524d05841276b0e22573f28d5fbcb67589836772ae9244d90dd7d66aa13"},
+ {file = "yarl-1.20.1-cp310-cp310-win32.whl", hash = "sha256:6c4fbf6b02d70e512d7ade4b1f998f237137f1417ab07ec06358ea04f69134f8"},
+ {file = "yarl-1.20.1-cp310-cp310-win_amd64.whl", hash = "sha256:aef6c4d69554d44b7f9d923245f8ad9a707d971e6209d51279196d8e8fe1ae16"},
+ {file = "yarl-1.20.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:47ee6188fea634bdfaeb2cc420f5b3b17332e6225ce88149a17c413c77ff269e"},
+ {file = "yarl-1.20.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:d0f6500f69e8402d513e5eedb77a4e1818691e8f45e6b687147963514d84b44b"},
+ {file = "yarl-1.20.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:7a8900a42fcdaad568de58887c7b2f602962356908eedb7628eaf6021a6e435b"},
+ {file = "yarl-1.20.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bad6d131fda8ef508b36be3ece16d0902e80b88ea7200f030a0f6c11d9e508d4"},
+ {file = "yarl-1.20.1-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:df018d92fe22aaebb679a7f89fe0c0f368ec497e3dda6cb81a567610f04501f1"},
+ {file = "yarl-1.20.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8f969afbb0a9b63c18d0feecf0db09d164b7a44a053e78a7d05f5df163e43833"},
+ {file = "yarl-1.20.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:812303eb4aa98e302886ccda58d6b099e3576b1b9276161469c25803a8db277d"},
+ {file = "yarl-1.20.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:98c4a7d166635147924aa0bf9bfe8d8abad6fffa6102de9c99ea04a1376f91e8"},
+ {file = "yarl-1.20.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:12e768f966538e81e6e7550f9086a6236b16e26cd964cf4df35349970f3551cf"},
+ {file = "yarl-1.20.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:fe41919b9d899661c5c28a8b4b0acf704510b88f27f0934ac7a7bebdd8938d5e"},
+ {file = "yarl-1.20.1-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:8601bc010d1d7780592f3fc1bdc6c72e2b6466ea34569778422943e1a1f3c389"},
+ {file = "yarl-1.20.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:daadbdc1f2a9033a2399c42646fbd46da7992e868a5fe9513860122d7fe7a73f"},
+ {file = "yarl-1.20.1-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:03aa1e041727cb438ca762628109ef1333498b122e4c76dd858d186a37cec845"},
+ {file = "yarl-1.20.1-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:642980ef5e0fa1de5fa96d905c7e00cb2c47cb468bfcac5a18c58e27dbf8d8d1"},
+ {file = "yarl-1.20.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:86971e2795584fe8c002356d3b97ef6c61862720eeff03db2a7c86b678d85b3e"},
+ {file = "yarl-1.20.1-cp311-cp311-win32.whl", hash = "sha256:597f40615b8d25812f14562699e287f0dcc035d25eb74da72cae043bb884d773"},
+ {file = "yarl-1.20.1-cp311-cp311-win_amd64.whl", hash = "sha256:26ef53a9e726e61e9cd1cda6b478f17e350fb5800b4bd1cd9fe81c4d91cfeb2e"},
+ {file = "yarl-1.20.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:bdcc4cd244e58593a4379fe60fdee5ac0331f8eb70320a24d591a3be197b94a9"},
+ {file = "yarl-1.20.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:b29a2c385a5f5b9c7d9347e5812b6f7ab267193c62d282a540b4fc528c8a9d2a"},
+ {file = "yarl-1.20.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:1112ae8154186dfe2de4732197f59c05a83dc814849a5ced892b708033f40dc2"},
+ {file = "yarl-1.20.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:90bbd29c4fe234233f7fa2b9b121fb63c321830e5d05b45153a2ca68f7d310ee"},
+ {file = "yarl-1.20.1-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:680e19c7ce3710ac4cd964e90dad99bf9b5029372ba0c7cbfcd55e54d90ea819"},
+ {file = "yarl-1.20.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4a979218c1fdb4246a05efc2cc23859d47c89af463a90b99b7c56094daf25a16"},
+ {file = "yarl-1.20.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:255b468adf57b4a7b65d8aad5b5138dce6a0752c139965711bdcb81bc370e1b6"},
+ {file = "yarl-1.20.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a97d67108e79cfe22e2b430d80d7571ae57d19f17cda8bb967057ca8a7bf5bfd"},
+ {file = "yarl-1.20.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8570d998db4ddbfb9a590b185a0a33dbf8aafb831d07a5257b4ec9948df9cb0a"},
+ {file = "yarl-1.20.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:97c75596019baae7c71ccf1d8cc4738bc08134060d0adfcbe5642f778d1dca38"},
+ {file = "yarl-1.20.1-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:1c48912653e63aef91ff988c5432832692ac5a1d8f0fb8a33091520b5bbe19ef"},
+ {file = "yarl-1.20.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:4c3ae28f3ae1563c50f3d37f064ddb1511ecc1d5584e88c6b7c63cf7702a6d5f"},
+ {file = "yarl-1.20.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:c5e9642f27036283550f5f57dc6156c51084b458570b9d0d96100c8bebb186a8"},
+ {file = "yarl-1.20.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:2c26b0c49220d5799f7b22c6838409ee9bc58ee5c95361a4d7831f03cc225b5a"},
+ {file = "yarl-1.20.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:564ab3d517e3d01c408c67f2e5247aad4019dcf1969982aba3974b4093279004"},
+ {file = "yarl-1.20.1-cp312-cp312-win32.whl", hash = "sha256:daea0d313868da1cf2fac6b2d3a25c6e3a9e879483244be38c8e6a41f1d876a5"},
+ {file = "yarl-1.20.1-cp312-cp312-win_amd64.whl", hash = "sha256:48ea7d7f9be0487339828a4de0360d7ce0efc06524a48e1810f945c45b813698"},
+ {file = "yarl-1.20.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:0b5ff0fbb7c9f1b1b5ab53330acbfc5247893069e7716840c8e7d5bb7355038a"},
+ {file = "yarl-1.20.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:14f326acd845c2b2e2eb38fb1346c94f7f3b01a4f5c788f8144f9b630bfff9a3"},
+ {file = "yarl-1.20.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:f60e4ad5db23f0b96e49c018596707c3ae89f5d0bd97f0ad3684bcbad899f1e7"},
+ {file = "yarl-1.20.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:49bdd1b8e00ce57e68ba51916e4bb04461746e794e7c4d4bbc42ba2f18297691"},
+ {file = "yarl-1.20.1-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:66252d780b45189975abfed839616e8fd2dbacbdc262105ad7742c6ae58f3e31"},
+ {file = "yarl-1.20.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:59174e7332f5d153d8f7452a102b103e2e74035ad085f404df2e40e663a22b28"},
+ {file = "yarl-1.20.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e3968ec7d92a0c0f9ac34d5ecfd03869ec0cab0697c91a45db3fbbd95fe1b653"},
+ {file = "yarl-1.20.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d1a4fbb50e14396ba3d375f68bfe02215d8e7bc3ec49da8341fe3157f59d2ff5"},
+ {file = "yarl-1.20.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:11a62c839c3a8eac2410e951301309426f368388ff2f33799052787035793b02"},
+ {file = "yarl-1.20.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:041eaa14f73ff5a8986b4388ac6bb43a77f2ea09bf1913df7a35d4646db69e53"},
+ {file = "yarl-1.20.1-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:377fae2fef158e8fd9d60b4c8751387b8d1fb121d3d0b8e9b0be07d1b41e83dc"},
+ {file = "yarl-1.20.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:1c92f4390e407513f619d49319023664643d3339bd5e5a56a3bebe01bc67ec04"},
+ {file = "yarl-1.20.1-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:d25ddcf954df1754ab0f86bb696af765c5bfaba39b74095f27eececa049ef9a4"},
+ {file = "yarl-1.20.1-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:909313577e9619dcff8c31a0ea2aa0a2a828341d92673015456b3ae492e7317b"},
+ {file = "yarl-1.20.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:793fd0580cb9664548c6b83c63b43c477212c0260891ddf86809e1c06c8b08f1"},
+ {file = "yarl-1.20.1-cp313-cp313-win32.whl", hash = "sha256:468f6e40285de5a5b3c44981ca3a319a4b208ccc07d526b20b12aeedcfa654b7"},
+ {file = "yarl-1.20.1-cp313-cp313-win_amd64.whl", hash = "sha256:495b4ef2fea40596bfc0affe3837411d6aa3371abcf31aac0ccc4bdd64d4ef5c"},
+ {file = "yarl-1.20.1-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:f60233b98423aab21d249a30eb27c389c14929f47be8430efa7dbd91493a729d"},
+ {file = "yarl-1.20.1-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:6f3eff4cc3f03d650d8755c6eefc844edde99d641d0dcf4da3ab27141a5f8ddf"},
+ {file = "yarl-1.20.1-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:69ff8439d8ba832d6bed88af2c2b3445977eba9a4588b787b32945871c2444e3"},
+ {file = "yarl-1.20.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3cf34efa60eb81dd2645a2e13e00bb98b76c35ab5061a3989c7a70f78c85006d"},
+ {file = "yarl-1.20.1-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:8e0fe9364ad0fddab2688ce72cb7a8e61ea42eff3c7caeeb83874a5d479c896c"},
+ {file = "yarl-1.20.1-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8f64fbf81878ba914562c672024089e3401974a39767747691c65080a67b18c1"},
+ {file = "yarl-1.20.1-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f6342d643bf9a1de97e512e45e4b9560a043347e779a173250824f8b254bd5ce"},
+ {file = "yarl-1.20.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:56dac5f452ed25eef0f6e3c6a066c6ab68971d96a9fb441791cad0efba6140d3"},
+ {file = "yarl-1.20.1-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c7d7f497126d65e2cad8dc5f97d34c27b19199b6414a40cb36b52f41b79014be"},
+ {file = "yarl-1.20.1-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:67e708dfb8e78d8a19169818eeb5c7a80717562de9051bf2413aca8e3696bf16"},
+ {file = "yarl-1.20.1-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:595c07bc79af2494365cc96ddeb772f76272364ef7c80fb892ef9d0649586513"},
+ {file = "yarl-1.20.1-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:7bdd2f80f4a7df852ab9ab49484a4dee8030023aa536df41f2d922fd57bf023f"},
+ {file = "yarl-1.20.1-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:c03bfebc4ae8d862f853a9757199677ab74ec25424d0ebd68a0027e9c639a390"},
+ {file = "yarl-1.20.1-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:344d1103e9c1523f32a5ed704d576172d2cabed3122ea90b1d4e11fe17c66458"},
+ {file = "yarl-1.20.1-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:88cab98aa4e13e1ade8c141daeedd300a4603b7132819c484841bb7af3edce9e"},
+ {file = "yarl-1.20.1-cp313-cp313t-win32.whl", hash = "sha256:b121ff6a7cbd4abc28985b6028235491941b9fe8fe226e6fdc539c977ea1739d"},
+ {file = "yarl-1.20.1-cp313-cp313t-win_amd64.whl", hash = "sha256:541d050a355bbbc27e55d906bc91cb6fe42f96c01413dd0f4ed5a5240513874f"},
+ {file = "yarl-1.20.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:e42ba79e2efb6845ebab49c7bf20306c4edf74a0b20fc6b2ccdd1a219d12fad3"},
+ {file = "yarl-1.20.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:41493b9b7c312ac448b7f0a42a089dffe1d6e6e981a2d76205801a023ed26a2b"},
+ {file = "yarl-1.20.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f5a5928ff5eb13408c62a968ac90d43f8322fd56d87008b8f9dabf3c0f6ee983"},
+ {file = "yarl-1.20.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:30c41ad5d717b3961b2dd785593b67d386b73feca30522048d37298fee981805"},
+ {file = "yarl-1.20.1-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:59febc3969b0781682b469d4aca1a5cab7505a4f7b85acf6db01fa500fa3f6ba"},
+ {file = "yarl-1.20.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d2b6fb3622b7e5bf7a6e5b679a69326b4279e805ed1699d749739a61d242449e"},
+ {file = "yarl-1.20.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:749d73611db8d26a6281086f859ea7ec08f9c4c56cec864e52028c8b328db723"},
+ {file = "yarl-1.20.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9427925776096e664c39e131447aa20ec738bdd77c049c48ea5200db2237e000"},
+ {file = "yarl-1.20.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ff70f32aa316393eaf8222d518ce9118148eddb8a53073c2403863b41033eed5"},
+ {file = "yarl-1.20.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:c7ddf7a09f38667aea38801da8b8d6bfe81df767d9dfc8c88eb45827b195cd1c"},
+ {file = "yarl-1.20.1-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:57edc88517d7fc62b174fcfb2e939fbc486a68315d648d7e74d07fac42cec240"},
+ {file = "yarl-1.20.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:dab096ce479d5894d62c26ff4f699ec9072269d514b4edd630a393223f45a0ee"},
+ {file = "yarl-1.20.1-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:14a85f3bd2d7bb255be7183e5d7d6e70add151a98edf56a770d6140f5d5f4010"},
+ {file = "yarl-1.20.1-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:2c89b5c792685dd9cd3fa9761c1b9f46fc240c2a3265483acc1565769996a3f8"},
+ {file = "yarl-1.20.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:69e9b141de5511021942a6866990aea6d111c9042235de90e08f94cf972ca03d"},
+ {file = "yarl-1.20.1-cp39-cp39-win32.whl", hash = "sha256:b5f307337819cdfdbb40193cad84978a029f847b0a357fbe49f712063cfc4f06"},
+ {file = "yarl-1.20.1-cp39-cp39-win_amd64.whl", hash = "sha256:eae7bfe2069f9c1c5b05fc7fe5d612e5bbc089a39309904ee8b829e322dcad00"},
+ {file = "yarl-1.20.1-py3-none-any.whl", hash = "sha256:83b8eb083fe4683c6115795d9fc1cfaf2cbbefb19b3a1cb68f6527460f483a77"},
+ {file = "yarl-1.20.1.tar.gz", hash = "sha256:d017a4997ee50c91fd5466cef416231bb82177b93b029906cefc542ce14c35ac"},
]
[package.dependencies]
idna = ">=2.0"
multidict = ">=4.0"
-propcache = ">=0.2.0"
+propcache = ">=0.2.1"
[[package]]
name = "zipp"
-version = "3.21.0"
+version = "3.23.0"
description = "Backport of pathlib-compatible object wrapper for zip files"
optional = false
python-versions = ">=3.9"
groups = ["main", "dev", "docs"]
files = [
- {file = "zipp-3.21.0-py3-none-any.whl", hash = "sha256:ac1bbe05fd2991f160ebce24ffbac5f6d11d83dc90891255885223d42b3cd931"},
- {file = "zipp-3.21.0.tar.gz", hash = "sha256:2c9958f6430a2040341a52eb608ed6dd93ef4392e02ffe219417c1b28b5dd1f4"},
+ {file = "zipp-3.23.0-py3-none-any.whl", hash = "sha256:071652d6115ed432f5ce1d34c336c0adfd6a884660d1e9712a256d3d3bd4b14e"},
+ {file = "zipp-3.23.0.tar.gz", hash = "sha256:a07157588a12518c9d4034df3fbbee09c814741a33ff63c05fa29d26a2404166"},
]
markers = {main = "extra == \"daft\" and python_version < \"3.10\"", dev = "python_full_version < \"3.10.2\"", docs = "python_version < \"3.10\""}
@@ -5820,7 +6164,7 @@ check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"
cover = ["pytest-cov"]
doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"]
enabler = ["pytest-enabler (>=2.2)"]
-test = ["big-O", "importlib-resources ; python_version < \"3.9\"", "jaraco.functools", "jaraco.itertools", "jaraco.test", "more-itertools", "pytest (>=6,!=8.1.*)", "pytest-ignore-flaky"]
+test = ["big-O", "jaraco.functools", "jaraco.itertools", "jaraco.test", "more_itertools", "pytest (>=6,!=8.1.*)", "pytest-ignore-flaky"]
type = ["pytest-mypy"]
[[package]]
@@ -5943,7 +6287,7 @@ daft = ["getdaft"]
duckdb = ["duckdb", "pyarrow"]
dynamodb = ["boto3"]
gcsfs = ["gcsfs"]
-glue = ["boto3", "mypy-boto3-glue"]
+glue = ["boto3"]
hf = ["huggingface-hub"]
hive = ["thrift"]
hive-kerberos = ["kerberos", "thrift", "thrift-sasl"]
@@ -5962,4 +6306,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "7825a9ae7c96f9b76a4ef4727b45e9ebdd01c9de27a0a98579dd881af6396e3b"
+content-hash = "7e2fa5ddc3b2389fc07541d6f2d4b4136cec8bef32dbe1dec13199818ef88212"
diff --git a/pyiceberg/catalog/dynamodb.py b/pyiceberg/catalog/dynamodb.py
index 63466b0142..7879e5a2d3 100644
--- a/pyiceberg/catalog/dynamodb.py
+++ b/pyiceberg/catalog/dynamodb.py
@@ -66,6 +66,8 @@
if TYPE_CHECKING:
import pyarrow as pa
+ from mypy_boto3_dynamodb.client import DynamoDBClient
+
DYNAMODB_CLIENT = "dynamodb"
@@ -94,18 +96,28 @@
class DynamoDbCatalog(MetastoreCatalog):
- def __init__(self, name: str, **properties: str):
+ def __init__(self, name: str, client: Optional["DynamoDBClient"] = None, **properties: str):
+ """Dynamodb catalog.
+
+ Args:
+ name: Name to identify the catalog.
+ client: An optional boto3 dynamodb client.
+ properties: Properties for dynamodb client construction and configuration.
+ """
super().__init__(name, **properties)
+ if client is not None:
+ self.dynamodb = client
+ else:
+ session = boto3.Session(
+ profile_name=properties.get(DYNAMODB_PROFILE_NAME),
+ region_name=get_first_property_value(properties, DYNAMODB_REGION, AWS_REGION),
+ botocore_session=properties.get(BOTOCORE_SESSION),
+ aws_access_key_id=get_first_property_value(properties, DYNAMODB_ACCESS_KEY_ID, AWS_ACCESS_KEY_ID),
+ aws_secret_access_key=get_first_property_value(properties, DYNAMODB_SECRET_ACCESS_KEY, AWS_SECRET_ACCESS_KEY),
+ aws_session_token=get_first_property_value(properties, DYNAMODB_SESSION_TOKEN, AWS_SESSION_TOKEN),
+ )
+ self.dynamodb = session.client(DYNAMODB_CLIENT)
- session = boto3.Session(
- profile_name=properties.get(DYNAMODB_PROFILE_NAME),
- region_name=get_first_property_value(properties, DYNAMODB_REGION, AWS_REGION),
- botocore_session=properties.get(BOTOCORE_SESSION),
- aws_access_key_id=get_first_property_value(properties, DYNAMODB_ACCESS_KEY_ID, AWS_ACCESS_KEY_ID),
- aws_secret_access_key=get_first_property_value(properties, DYNAMODB_SECRET_ACCESS_KEY, AWS_SECRET_ACCESS_KEY),
- aws_session_token=get_first_property_value(properties, DYNAMODB_SESSION_TOKEN, AWS_SESSION_TOKEN),
- )
- self.dynamodb = session.client(DYNAMODB_CLIENT)
self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
self._ensure_catalog_table_exists_or_create()
diff --git a/pyiceberg/catalog/glue.py b/pyiceberg/catalog/glue.py
index 01bb0e9b05..8212b9c2ec 100644
--- a/pyiceberg/catalog/glue.py
+++ b/pyiceberg/catalog/glue.py
@@ -30,15 +30,6 @@
import boto3
from botocore.config import Config
-from mypy_boto3_glue.client import GlueClient
-from mypy_boto3_glue.type_defs import (
- ColumnTypeDef,
- DatabaseInputTypeDef,
- DatabaseTypeDef,
- StorageDescriptorTypeDef,
- TableInputTypeDef,
- TableTypeDef,
-)
from pyiceberg.catalog import (
BOTOCORE_SESSION,
@@ -101,6 +92,15 @@
if TYPE_CHECKING:
import pyarrow as pa
+ from mypy_boto3_glue.client import GlueClient
+ from mypy_boto3_glue.type_defs import (
+ ColumnTypeDef,
+ DatabaseInputTypeDef,
+ DatabaseTypeDef,
+ StorageDescriptorTypeDef,
+ TableInputTypeDef,
+ TableTypeDef,
+ )
# There is a unique Glue metastore in each AWS account and each AWS region. By default, GlueCatalog chooses the Glue
@@ -140,7 +140,7 @@
def _construct_parameters(
- metadata_location: str, glue_table: Optional[TableTypeDef] = None, prev_metadata_location: Optional[str] = None
+ metadata_location: str, glue_table: Optional["TableTypeDef"] = None, prev_metadata_location: Optional[str] = None
) -> Properties:
new_parameters = glue_table.get("Parameters", {}) if glue_table else {}
new_parameters.update({TABLE_TYPE: ICEBERG.upper(), METADATA_LOCATION: metadata_location})
@@ -190,15 +190,15 @@ def primitive(self, primitive: PrimitiveType) -> str:
return GLUE_PRIMITIVE_TYPES[primitive_type]
-def _to_columns(metadata: TableMetadata) -> List[ColumnTypeDef]:
- results: Dict[str, ColumnTypeDef] = {}
+def _to_columns(metadata: TableMetadata) -> List["ColumnTypeDef"]:
+ results: Dict[str, "ColumnTypeDef"] = {}
def _append_to_results(field: NestedField, is_current: bool) -> None:
if field.name in results:
return
results[field.name] = cast(
- ColumnTypeDef,
+ "ColumnTypeDef",
{
"Name": field.name,
"Type": visit(field.field_type, _IcebergSchemaToGlueType()),
@@ -230,10 +230,10 @@ def _construct_table_input(
metadata_location: str,
properties: Properties,
metadata: TableMetadata,
- glue_table: Optional[TableTypeDef] = None,
+ glue_table: Optional["TableTypeDef"] = None,
prev_metadata_location: Optional[str] = None,
-) -> TableInputTypeDef:
- table_input: TableInputTypeDef = {
+) -> "TableInputTypeDef":
+ table_input: "TableInputTypeDef" = {
"Name": table_name,
"TableType": EXTERNAL_TABLE,
"Parameters": _construct_parameters(metadata_location, glue_table, prev_metadata_location),
@@ -249,8 +249,8 @@ def _construct_table_input(
return table_input
-def _construct_rename_table_input(to_table_name: str, glue_table: TableTypeDef) -> TableInputTypeDef:
- rename_table_input: TableInputTypeDef = {"Name": to_table_name}
+def _construct_rename_table_input(to_table_name: str, glue_table: "TableTypeDef") -> "TableInputTypeDef":
+ rename_table_input: "TableInputTypeDef" = {"Name": to_table_name}
# use the same Glue info to create the new table, pointing to the old metadata
assert glue_table["TableType"]
rename_table_input["TableType"] = glue_table["TableType"]
@@ -264,7 +264,7 @@ def _construct_rename_table_input(to_table_name: str, glue_table: TableTypeDef)
# It turns out the output of StorageDescriptor is not the same as the input type
# because the Column can have a different type, but for now it seems to work, so
# silence the type error.
- rename_table_input["StorageDescriptor"] = cast(StorageDescriptorTypeDef, glue_table["StorageDescriptor"])
+ rename_table_input["StorageDescriptor"] = cast("StorageDescriptorTypeDef", glue_table["StorageDescriptor"])
if "Description" in glue_table:
rename_table_input["Description"] = glue_table["Description"]
@@ -272,8 +272,8 @@ def _construct_rename_table_input(to_table_name: str, glue_table: TableTypeDef)
return rename_table_input
-def _construct_database_input(database_name: str, properties: Properties) -> DatabaseInputTypeDef:
- database_input: DatabaseInputTypeDef = {"Name": database_name}
+def _construct_database_input(database_name: str, properties: Properties) -> "DatabaseInputTypeDef":
+ database_input: "DatabaseInputTypeDef" = {"Name": database_name}
parameters = {}
for k, v in properties.items():
if k == "Description":
@@ -286,7 +286,7 @@ def _construct_database_input(database_name: str, properties: Properties) -> Dat
return database_input
-def _register_glue_catalog_id_with_glue_client(glue: GlueClient, glue_catalog_id: str) -> None:
+def _register_glue_catalog_id_with_glue_client(glue: "GlueClient", glue_catalog_id: str) -> None:
"""
Register the Glue Catalog ID (AWS Account ID) as a parameter on all Glue client methods.
@@ -303,9 +303,9 @@ def add_glue_catalog_id(params: Dict[str, str], **kwargs: Any) -> None:
class GlueCatalog(MetastoreCatalog):
- glue: GlueClient
+ glue: "GlueClient"
- def __init__(self, name: str, client: Optional[GlueClient] = None, **properties: Any):
+ def __init__(self, name: str, client: Optional["GlueClient"] = None, **properties: Any):
"""Glue Catalog.
You either need to provide a boto3 glue client, or one will be constructed from the properties.
@@ -317,7 +317,7 @@ def __init__(self, name: str, client: Optional[GlueClient] = None, **properties:
"""
super().__init__(name, **properties)
- if client:
+ if client is not None:
self.glue = client
else:
retry_mode_prop_value = get_first_property_value(properties, GLUE_RETRY_MODE)
@@ -344,7 +344,7 @@ def __init__(self, name: str, client: Optional[GlueClient] = None, **properties:
if glue_catalog_id := properties.get(GLUE_ID):
_register_glue_catalog_id_with_glue_client(self.glue, glue_catalog_id)
- def _convert_glue_to_iceberg(self, glue_table: TableTypeDef) -> Table:
+ def _convert_glue_to_iceberg(self, glue_table: "TableTypeDef") -> Table:
properties: Properties = glue_table["Parameters"]
assert glue_table["DatabaseName"]
@@ -380,7 +380,7 @@ def _convert_glue_to_iceberg(self, glue_table: TableTypeDef) -> Table:
catalog=self,
)
- def _create_glue_table(self, database_name: str, table_name: str, table_input: TableInputTypeDef) -> None:
+ def _create_glue_table(self, database_name: str, table_name: str, table_input: "TableInputTypeDef") -> None:
try:
self.glue.create_table(DatabaseName=database_name, TableInput=table_input)
except self.glue.exceptions.AlreadyExistsException as e:
@@ -388,7 +388,7 @@ def _create_glue_table(self, database_name: str, table_name: str, table_input: T
except self.glue.exceptions.EntityNotFoundException as e:
raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
- def _update_glue_table(self, database_name: str, table_name: str, table_input: TableInputTypeDef, version_id: str) -> None:
+ def _update_glue_table(self, database_name: str, table_name: str, table_input: "TableInputTypeDef", version_id: str) -> None:
try:
self.glue.update_table(
DatabaseName=database_name,
@@ -403,7 +403,7 @@ def _update_glue_table(self, database_name: str, table_name: str, table_input: T
f"Cannot commit {database_name}.{table_name} because Glue detected concurrent update to table version {version_id}"
) from e
- def _get_glue_table(self, database_name: str, table_name: str) -> TableTypeDef:
+ def _get_glue_table(self, database_name: str, table_name: str) -> "TableTypeDef":
try:
load_table_response = self.glue.get_table(DatabaseName=database_name, Name=table_name)
return load_table_response["Table"]
@@ -496,7 +496,7 @@ def commit_table(
table_identifier = table.name()
database_name, table_name = self.identifier_to_database_and_table(table_identifier, NoSuchTableError)
- current_glue_table: Optional[TableTypeDef]
+ current_glue_table: Optional["TableTypeDef"]
glue_table_version_id: Optional[str]
current_table: Optional[Table]
try:
@@ -702,7 +702,7 @@ def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid.
"""
database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
- table_list: List[TableTypeDef] = []
+ table_list: List["TableTypeDef"] = []
next_token: Optional[str] = None
try:
while True:
@@ -730,7 +730,7 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi
if namespace:
return []
- database_list: List[DatabaseTypeDef] = []
+ database_list: List["DatabaseTypeDef"] = []
next_token: Optional[str] = None
while True:
@@ -806,5 +806,5 @@ def view_exists(self, identifier: Union[str, Identifier]) -> bool:
raise NotImplementedError
@staticmethod
- def __is_iceberg_table(table: TableTypeDef) -> bool:
+ def __is_iceberg_table(table: "TableTypeDef") -> bool:
return table.get("Parameters", {}).get(TABLE_TYPE, "").lower() == ICEBERG
diff --git a/pyiceberg/catalog/rest/__init__.py b/pyiceberg/catalog/rest/__init__.py
index 4ad3395378..5ad7faef89 100644
--- a/pyiceberg/catalog/rest/__init__.py
+++ b/pyiceberg/catalog/rest/__init__.py
@@ -32,6 +32,7 @@
from pyiceberg import __version__
from pyiceberg.catalog import (
+ BOTOCORE_SESSION,
TOKEN,
URI,
WAREHOUSE_LOCATION,
@@ -53,6 +54,7 @@
TableAlreadyExistsError,
UnauthorizedError,
)
+from pyiceberg.io import AWS_ACCESS_KEY_ID, AWS_REGION, AWS_SECRET_ACCESS_KEY, AWS_SESSION_TOKEN
from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec, assign_fresh_partition_spec_ids
from pyiceberg.schema import Schema, assign_fresh_schema_ids
from pyiceberg.table import (
@@ -72,7 +74,7 @@
from pyiceberg.typedef import EMPTY_DICT, UTF8, IcebergBaseModel, Identifier, Properties
from pyiceberg.types import transform_dict_value_to_str
from pyiceberg.utils.deprecated import deprecation_message
-from pyiceberg.utils.properties import get_header_properties, property_as_bool
+from pyiceberg.utils.properties import get_first_property_value, get_header_properties, property_as_bool
if TYPE_CHECKING:
import pyarrow as pa
@@ -390,11 +392,17 @@ class SigV4Adapter(HTTPAdapter):
def __init__(self, **properties: str):
super().__init__()
self._properties = properties
+ self._boto_session = boto3.Session(
+ region_name=get_first_property_value(self._properties, AWS_REGION),
+ botocore_session=self._properties.get(BOTOCORE_SESSION),
+ aws_access_key_id=get_first_property_value(self._properties, AWS_ACCESS_KEY_ID),
+ aws_secret_access_key=get_first_property_value(self._properties, AWS_SECRET_ACCESS_KEY),
+ aws_session_token=get_first_property_value(self._properties, AWS_SESSION_TOKEN),
+ )
def add_headers(self, request: PreparedRequest, **kwargs: Any) -> None: # pylint: disable=W0613
- boto_session = boto3.Session()
- credentials = boto_session.get_credentials().get_frozen_credentials()
- region = self._properties.get(SIGV4_REGION, boto_session.region_name)
+ credentials = self._boto_session.get_credentials().get_frozen_credentials()
+ region = self._properties.get(SIGV4_REGION, self._boto_session.region_name)
service = self._properties.get(SIGV4_SERVICE, "execute-api")
url = str(request.url).split("?")[0]
diff --git a/pyproject.toml b/pyproject.toml
index 2dfcbcefcd..d759a95f3e 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -67,11 +67,10 @@ pandas = { version = ">=1.0.0,<3.0.0", optional = true }
duckdb = { version = ">=0.5.0,<2.0.0", optional = true }
ray = [
{ version = "==2.10.0", python = "<3.9", optional = true },
- { version = ">=2.10.0,<3.0.0", python = ">=3.9", optional = true },
+ { version = ">=2.10.0,<=2.44.0", python = ">=3.9", optional = true },
]
python-snappy = { version = ">=0.6.0,<1.0.0", optional = true }
thrift = { version = ">=0.13.0,<1.0.0", optional = true }
-mypy-boto3-glue = { version = ">=1.28.18", optional = true }
boto3 = { version = ">=1.24.59", optional = true }
s3fs = { version = ">=2023.1.0", optional = true }
adlfs = { version = ">=2023.1.0", optional = true }
@@ -102,6 +101,8 @@ cython = "3.1.2"
deptry = ">=0.14,<0.24"
datafusion = ">=44,<48"
docutils = "!=0.21.post1" # https://github.com/python-poetry/poetry/issues/9248#issuecomment-2026240520
+mypy-boto3-glue = ">=1.28.18"
+mypy-boto3-dynamodb = ">=1.28.18"
[tool.poetry.group.docs.dependencies]
# for mkdocs
@@ -217,6 +218,10 @@ ignore_missing_imports = true
module = "mypy_boto3_glue.*"
ignore_missing_imports = true
+[[tool.mypy.overrides]]
+module = "mypy_boto3_dynamodb.*"
+ignore_missing_imports = true
+
[[tool.mypy.overrides]]
module = "moto"
ignore_missing_imports = true
@@ -299,7 +304,7 @@ snappy = ["python-snappy"]
hive = ["thrift"]
hive-kerberos = ["thrift", "thrift_sasl", "kerberos"]
s3fs = ["s3fs"]
-glue = ["boto3", "mypy-boto3-glue"]
+glue = ["boto3"]
adlfs = ["adlfs"]
dynamodb = ["boto3"]
zstandard = ["zstandard"]
diff --git a/tests/catalog/test_dynamodb.py b/tests/catalog/test_dynamodb.py
index 7ab875af90..c7c39a600d 100644
--- a/tests/catalog/test_dynamodb.py
+++ b/tests/catalog/test_dynamodb.py
@@ -626,3 +626,11 @@ def test_table_exists(
assert test_catalog.table_exists(identifier) is True
# Act and Assert for an non-existing table
assert test_catalog.table_exists(("non", "exist")) is False
+
+
+@mock_aws
+def test_dynamodb_client_override() -> None:
+ catalog_name = "glue"
+ test_client = boto3.client("dynamodb", region_name="us-west-2")
+ test_catalog = DynamoDbCatalog(catalog_name, test_client)
+ assert test_catalog.dynamodb is test_client
From 8d3c147afcaab5ed214165a7cf16404a393a132e Mon Sep 17 00:00:00 2001
From: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
Date: Thu, 12 Jun 2025 12:37:31 -0400
Subject: [PATCH 052/112] chore: remove non-test asserts (#2082)
Baby PR to remove `assert` statements out of any non-test code.
---
pyiceberg/catalog/dynamodb.py | 4 +++-
pyiceberg/catalog/glue.py | 15 +++++++++++----
pyiceberg/cli/console.py | 2 --
pyiceberg/schema.py | 4 ++--
pyiceberg/utils/config.py | 3 ++-
5 files changed, 18 insertions(+), 10 deletions(-)
diff --git a/pyiceberg/catalog/dynamodb.py b/pyiceberg/catalog/dynamodb.py
index 7879e5a2d3..29df2582a7 100644
--- a/pyiceberg/catalog/dynamodb.py
+++ b/pyiceberg/catalog/dynamodb.py
@@ -836,7 +836,9 @@ def _convert_dynamo_item_to_regular_dict(dynamo_json: Dict[str, Any]) -> Dict[st
raise ValueError("Only S and N data types are supported.")
values = list(val_dict.values())
- assert len(values) == 1
+ if len(values) != 1:
+ raise ValueError(f"Expecting only 1 value: {values}")
+
column_value = values[0]
regular_json[column_name] = column_value
diff --git a/pyiceberg/catalog/glue.py b/pyiceberg/catalog/glue.py
index 8212b9c2ec..adfcc910e6 100644
--- a/pyiceberg/catalog/glue.py
+++ b/pyiceberg/catalog/glue.py
@@ -252,7 +252,9 @@ def _construct_table_input(
def _construct_rename_table_input(to_table_name: str, glue_table: "TableTypeDef") -> "TableInputTypeDef":
rename_table_input: "TableInputTypeDef" = {"Name": to_table_name}
# use the same Glue info to create the new table, pointing to the old metadata
- assert glue_table["TableType"]
+ if not glue_table["TableType"]:
+ raise ValueError("Glue table type is missing, cannot rename table")
+
rename_table_input["TableType"] = glue_table["TableType"]
if "Owner" in glue_table:
rename_table_input["Owner"] = glue_table["Owner"]
@@ -347,9 +349,14 @@ def __init__(self, name: str, client: Optional["GlueClient"] = None, **propertie
def _convert_glue_to_iceberg(self, glue_table: "TableTypeDef") -> Table:
properties: Properties = glue_table["Parameters"]
- assert glue_table["DatabaseName"]
- assert glue_table["Parameters"]
- database_name = glue_table["DatabaseName"]
+ database_name = glue_table.get("DatabaseName", None)
+ if database_name is None:
+ raise ValueError("Glue table is missing DatabaseName property")
+
+ parameters = glue_table.get("Parameters", None)
+ if parameters is None:
+ raise ValueError("Glue table is missing Parameters property")
+
table_name = glue_table["Name"]
if TABLE_TYPE not in properties:
diff --git a/pyiceberg/cli/console.py b/pyiceberg/cli/console.py
index 83e67a3cbb..25a536d2a6 100644
--- a/pyiceberg/cli/console.py
+++ b/pyiceberg/cli/console.py
@@ -300,7 +300,6 @@ def get_namespace(ctx: Context, identifier: str, property_name: str) -> None:
identifier_tuple = Catalog.identifier_to_tuple(identifier)
namespace_properties = catalog.load_namespace_properties(identifier_tuple)
- assert namespace_properties
if property_name:
if property_value := namespace_properties.get(property_name):
@@ -322,7 +321,6 @@ def get_table(ctx: Context, identifier: str, property_name: str) -> None:
identifier_tuple = Catalog.identifier_to_tuple(identifier)
metadata = catalog.load_table(identifier_tuple).metadata
- assert metadata
if property_name:
if property_value := metadata.properties.get(property_name):
diff --git a/pyiceberg/schema.py b/pyiceberg/schema.py
index 6aa1f88852..6333ace6e2 100644
--- a/pyiceberg/schema.py
+++ b/pyiceberg/schema.py
@@ -1362,8 +1362,8 @@ def make_compatible_name(name: str) -> str:
def _valid_avro_name(name: str) -> bool:
- length = len(name)
- assert length > 0, ValueError("Can not validate empty avro name")
+ if not len(name):
+ raise ValueError("Can not validate empty avro name")
first = name[0]
if not (first.isalpha() or first == "_"):
return False
diff --git a/pyiceberg/utils/config.py b/pyiceberg/utils/config.py
index 0c162777d6..0bfaefdbc6 100644
--- a/pyiceberg/utils/config.py
+++ b/pyiceberg/utils/config.py
@@ -154,7 +154,8 @@ def get_catalog_config(self, catalog_name: str) -> Optional[RecursiveDict]:
raise ValueError(f"Catalog configurations needs to be an object: {catalog_name}")
if catalog_name_lower in catalogs:
catalog_conf = catalogs[catalog_name_lower]
- assert isinstance(catalog_conf, dict), f"Configuration path catalogs.{catalog_name_lower} needs to be an object"
+ if not isinstance(catalog_conf, dict):
+ raise ValueError(f"Configuration path catalogs.{catalog_name_lower} needs to be an object")
return catalog_conf
return None
From 315a4eb4bc9d7a4db214d6fee710063d200b9e45 Mon Sep 17 00:00:00 2001
From: Drew Gallardo
Date: Thu, 12 Jun 2025 14:47:17 -0700
Subject: [PATCH 053/112] fix: glue drop_namespace to check non-iceberg tables
(#2083)
# Rationale for this change
This PR is adding a check for all tables in the GlueCatalog instead of
just iceberg tables.
GlueCatalog allows users to store other table formats under the same
database. Using the catalog's `list_tables()` call filters out
non-Iceberg table types, which avoids the proper Iceberg
`NamespaceNotEmptyError` and instead returns a generic Glue SDK error.
This change aligns with the behavior in the java implementation:
-
https://github.com/apache/iceberg/blob/main/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java#L538-L571
# Are these changes tested?
Tested locally with Glue Catalog
# Are there any user-facing changes?
Yes new exception message when a `drop_namespace()` call is made against
a database with non-iceberg tables.
---
pyiceberg/catalog/glue.py | 14 ++++++++++----
tests/catalog/test_glue.py | 12 ++++++++++++
2 files changed, 22 insertions(+), 4 deletions(-)
diff --git a/pyiceberg/catalog/glue.py b/pyiceberg/catalog/glue.py
index adfcc910e6..25e7d3f73f 100644
--- a/pyiceberg/catalog/glue.py
+++ b/pyiceberg/catalog/glue.py
@@ -687,13 +687,19 @@ def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
"""
database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
try:
- table_list = self.list_tables(namespace=database_name)
- except NoSuchNamespaceError as e:
+ table_list_response = self.glue.get_tables(DatabaseName=database_name)
+ table_list = table_list_response["TableList"]
+ except self.glue.exceptions.EntityNotFoundException as e:
raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
if len(table_list) > 0:
- raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
-
+ first_table = table_list[0]
+ if self.__is_iceberg_table(first_table):
+ raise NamespaceNotEmptyError(f"Cannot drop namespace {database_name} because it still contains Iceberg tables")
+ else:
+ raise NamespaceNotEmptyError(
+ f"Cannot drop namespace {database_name} because it still contains non-Iceberg tables"
+ )
self.glue.delete_database(Name=database_name)
def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
diff --git a/tests/catalog/test_glue.py b/tests/catalog/test_glue.py
index df1734eb68..399c9e8bb1 100644
--- a/tests/catalog/test_glue.py
+++ b/tests/catalog/test_glue.py
@@ -548,6 +548,18 @@ def test_drop_non_empty_namespace(
test_catalog.drop_namespace(database_name)
+@mock_aws
+def test_drop_namespace_that_contains_non_iceberg_tables(
+ _bucket_initialize: None, moto_endpoint_url: str, table_schema_nested: Schema, database_name: str, table_name: str
+) -> None:
+ test_catalog = GlueCatalog("glue", **{"s3.endpoint": moto_endpoint_url, "warehouse": f"s3://{BUCKET_NAME}/"})
+ test_catalog.create_namespace(namespace=database_name)
+ test_catalog.glue.create_table(DatabaseName=database_name, TableInput={"Name": "hive_table"})
+
+ with pytest.raises(NamespaceNotEmptyError):
+ test_catalog.drop_namespace(database_name)
+
+
@mock_aws
def test_drop_non_exist_namespace(_bucket_initialize: None, moto_endpoint_url: str, database_name: str) -> None:
test_catalog = GlueCatalog("glue", **{"s3.endpoint": moto_endpoint_url})
From 6f960b268a14e173bc17019e40431fddd55b322c Mon Sep 17 00:00:00 2001
From: Brian Phillips <149107659+b-phi@users.noreply.github.com>
Date: Fri, 13 Jun 2025 09:26:25 -0400
Subject: [PATCH 054/112] Add v3 structs (#2065)
Closes https://github.com/apache/iceberg-python/issues/1982
# Rationale for this change
Adds v3 structs for data files, manifest entries, and manifest lists.
Necessary for related v3 work.
# Are these changes tested?
No, these are just schemas.
# Are there any user-facing changes?
No
---
pyiceberg/manifest.py | 148 ++++++++++++++++++++++++++++++++++++++++++
1 file changed, 148 insertions(+)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 26f11792c0..d7c923404e 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -289,6 +289,129 @@ def __repr__(self) -> str:
doc="ID representing sort order for this file",
),
),
+ 3: StructType(
+ NestedField(
+ field_id=134,
+ name="content",
+ field_type=IntegerType(),
+ required=True,
+ doc="File format name: avro, orc, or parquet",
+ initial_default=DataFileContent.DATA,
+ ),
+ NestedField(field_id=100, name="file_path", field_type=StringType(), required=True, doc="Location URI with FS scheme"),
+ NestedField(
+ field_id=101,
+ name="file_format",
+ field_type=StringType(),
+ required=True,
+ doc="File format name: avro, orc, or parquet",
+ ),
+ NestedField(
+ field_id=102,
+ name="partition",
+ field_type=StructType(),
+ required=True,
+ doc="Partition data tuple, schema based on the partition spec",
+ ),
+ NestedField(field_id=103, name="record_count", field_type=LongType(), required=True, doc="Number of records in the file"),
+ NestedField(
+ field_id=104, name="file_size_in_bytes", field_type=LongType(), required=True, doc="Total file size in bytes"
+ ),
+ NestedField(
+ field_id=108,
+ name="column_sizes",
+ field_type=MapType(key_id=117, key_type=IntegerType(), value_id=118, value_type=LongType()),
+ required=False,
+ doc="Map of column id to total size on disk",
+ ),
+ NestedField(
+ field_id=109,
+ name="value_counts",
+ field_type=MapType(key_id=119, key_type=IntegerType(), value_id=120, value_type=LongType()),
+ required=False,
+ doc="Map of column id to total count, including null and NaN",
+ ),
+ NestedField(
+ field_id=110,
+ name="null_value_counts",
+ field_type=MapType(key_id=121, key_type=IntegerType(), value_id=122, value_type=LongType()),
+ required=False,
+ doc="Map of column id to null value count",
+ ),
+ NestedField(
+ field_id=137,
+ name="nan_value_counts",
+ field_type=MapType(key_id=138, key_type=IntegerType(), value_id=139, value_type=LongType()),
+ required=False,
+ doc="Map of column id to number of NaN values in the column",
+ ),
+ NestedField(
+ field_id=125,
+ name="lower_bounds",
+ field_type=MapType(key_id=126, key_type=IntegerType(), value_id=127, value_type=BinaryType()),
+ required=False,
+ doc="Map of column id to lower bound",
+ ),
+ NestedField(
+ field_id=128,
+ name="upper_bounds",
+ field_type=MapType(key_id=129, key_type=IntegerType(), value_id=130, value_type=BinaryType()),
+ required=False,
+ doc="Map of column id to upper bound",
+ ),
+ NestedField(
+ field_id=131, name="key_metadata", field_type=BinaryType(), required=False, doc="Encryption key metadata blob"
+ ),
+ NestedField(
+ field_id=132,
+ name="split_offsets",
+ field_type=ListType(element_id=133, element_type=LongType(), element_required=True),
+ required=False,
+ doc="Splittable offsets",
+ ),
+ NestedField(
+ field_id=135,
+ name="equality_ids",
+ field_type=ListType(element_id=136, element_type=LongType(), element_required=True),
+ required=False,
+ doc="Field ids used to determine row equality in equality delete files.",
+ ),
+ NestedField(
+ field_id=140,
+ name="sort_order_id",
+ field_type=IntegerType(),
+ required=False,
+ doc="ID representing sort order for this file",
+ ),
+ NestedField(
+ field_id=142,
+ name="first_row_id",
+ field_type=LongType(),
+ required=False,
+ doc="The _row_id for the first row in the data file.",
+ ),
+ NestedField(
+ field_id=143,
+ name="referenced_data_file",
+ field_type=StringType(),
+ required=False,
+ doc="Fully qualified location (URI with FS scheme) of a data file that all deletes reference",
+ ),
+ NestedField(
+ field_id=144,
+ name="content_offset",
+ field_type=LongType(),
+ required=False,
+ doc="The offset in the file where the content starts.",
+ ),
+ NestedField(
+ field_id=145,
+ name="content_size_in_bytes",
+ field_type=LongType(),
+ required=False,
+ doc="The length of a referenced content stored in the file; required if content_offset is present",
+ ),
+ ),
}
@@ -434,6 +557,13 @@ def __eq__(self, other: Any) -> bool:
NestedField(4, "file_sequence_number", LongType(), required=False),
NestedField(2, "data_file", DATA_FILE_TYPE[2], required=True),
),
+ 3: Schema(
+ NestedField(0, "status", IntegerType(), required=True),
+ NestedField(1, "snapshot_id", LongType(), required=False),
+ NestedField(3, "sequence_number", LongType(), required=False),
+ NestedField(4, "file_sequence_number", LongType(), required=False),
+ NestedField(2, "data_file", DATA_FILE_TYPE[3], required=True),
+ ),
}
MANIFEST_ENTRY_SCHEMAS_STRUCT = {format_version: schema.as_struct() for format_version, schema in MANIFEST_ENTRY_SCHEMAS.items()}
@@ -604,6 +734,24 @@ def construct_partition_summaries(spec: PartitionSpec, schema: Schema, partition
NestedField(507, "partitions", ListType(508, PARTITION_FIELD_SUMMARY_TYPE, element_required=True), required=False),
NestedField(519, "key_metadata", BinaryType(), required=False),
),
+ 3: Schema(
+ NestedField(500, "manifest_path", StringType(), required=True, doc="Location URI with FS scheme"),
+ NestedField(501, "manifest_length", LongType(), required=True),
+ NestedField(502, "partition_spec_id", IntegerType(), required=True),
+ NestedField(517, "content", IntegerType(), required=True, initial_default=ManifestContent.DATA),
+ NestedField(515, "sequence_number", LongType(), required=True, initial_default=0),
+ NestedField(516, "min_sequence_number", LongType(), required=True, initial_default=0),
+ NestedField(503, "added_snapshot_id", LongType(), required=True),
+ NestedField(504, "added_files_count", IntegerType(), required=True),
+ NestedField(505, "existing_files_count", IntegerType(), required=True),
+ NestedField(506, "deleted_files_count", IntegerType(), required=True),
+ NestedField(512, "added_rows_count", LongType(), required=True),
+ NestedField(513, "existing_rows_count", LongType(), required=True),
+ NestedField(514, "deleted_rows_count", LongType(), required=True),
+ NestedField(507, "partitions", ListType(508, PARTITION_FIELD_SUMMARY_TYPE, element_required=True), required=False),
+ NestedField(519, "key_metadata", BinaryType(), required=False),
+ NestedField(520, "first_row_id", LongType(), required=False),
+ ),
}
MANIFEST_LIST_FILE_STRUCTS = {format_version: schema.as_struct() for format_version, schema in MANIFEST_LIST_FILE_SCHEMAS.items()}
From e016922161e253e81de9e17a9995e09c159b146b Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Fri, 13 Jun 2025 16:04:57 +0200
Subject: [PATCH 055/112] Add Avro compression (#1976)
# Rationale for this change
PyIceberg did not compress the Avro. This will make gzip/deflate the
same as in Java.
# Are these changes tested?
Existing round-trip tests with FastAvro and Spark. Some tests are
extended to both write compressed and uncompressed data.
# Are there any user-facing changes?
Smaller and faster manifest files :)
---
pyiceberg/avro/codecs/__init__.py | 13 +++-
pyiceberg/avro/file.py | 47 ++++++++++--
pyiceberg/manifest.py | 76 +++++++++++++++-----
pyiceberg/table/__init__.py | 3 +
pyiceberg/table/update/snapshot.py | 14 ++++
tests/integration/test_rest_manifest.py | 5 +-
tests/integration/test_writes/test_writes.py | 26 +++++++
tests/utils/test_manifest.py | 12 +++-
8 files changed, 167 insertions(+), 29 deletions(-)
diff --git a/pyiceberg/avro/codecs/__init__.py b/pyiceberg/avro/codecs/__init__.py
index 22e2f71cf8..ce592ccc5a 100644
--- a/pyiceberg/avro/codecs/__init__.py
+++ b/pyiceberg/avro/codecs/__init__.py
@@ -26,7 +26,9 @@
from __future__ import annotations
-from typing import Dict, Optional, Type
+from typing import Dict, Literal, Optional, Type
+
+from typing_extensions import TypeAlias
from pyiceberg.avro.codecs.bzip2 import BZip2Codec
from pyiceberg.avro.codecs.codec import Codec
@@ -34,10 +36,17 @@
from pyiceberg.avro.codecs.snappy_codec import SnappyCodec
from pyiceberg.avro.codecs.zstandard_codec import ZStandardCodec
-KNOWN_CODECS: Dict[str, Optional[Type[Codec]]] = {
+AvroCompressionCodec: TypeAlias = Literal["null", "bzip2", "snappy", "zstandard", "deflate"]
+
+AVRO_CODEC_KEY = "avro.codec"
+
+KNOWN_CODECS: Dict[AvroCompressionCodec, Optional[Type[Codec]]] = {
"null": None,
"bzip2": BZip2Codec,
"snappy": SnappyCodec,
"zstandard": ZStandardCodec,
"deflate": DeflateCodec,
}
+
+# Map to convert the naming from Iceberg to Avro
+CODEC_MAPPING_ICEBERG_TO_AVRO: Dict[str, str] = {"gzip": "deflate", "zstd": "zstandard"}
diff --git a/pyiceberg/avro/file.py b/pyiceberg/avro/file.py
index 9db585308d..82b042a412 100644
--- a/pyiceberg/avro/file.py
+++ b/pyiceberg/avro/file.py
@@ -35,7 +35,7 @@
TypeVar,
)
-from pyiceberg.avro.codecs import KNOWN_CODECS
+from pyiceberg.avro.codecs import AVRO_CODEC_KEY, CODEC_MAPPING_ICEBERG_TO_AVRO, KNOWN_CODECS
from pyiceberg.avro.codecs.codec import Codec
from pyiceberg.avro.decoder import BinaryDecoder, new_decoder
from pyiceberg.avro.encoder import BinaryEncoder
@@ -69,7 +69,6 @@
NestedField(field_id=300, name="sync", field_type=FixedType(length=SYNC_SIZE), required=True),
)
-_CODEC_KEY = "avro.codec"
_SCHEMA_KEY = "avro.schema"
@@ -92,11 +91,13 @@ def compression_codec(self) -> Optional[Type[Codec]]:
In the case of a null codec, we return a None indicating that we
don't need to compress/decompress.
"""
- codec_name = self.meta.get(_CODEC_KEY, "null")
+ from pyiceberg.table import TableProperties
+
+ codec_name = self.meta.get(AVRO_CODEC_KEY, TableProperties.WRITE_AVRO_COMPRESSION_DEFAULT)
if codec_name not in KNOWN_CODECS:
raise ValueError(f"Unsupported codec: {codec_name}")
- return KNOWN_CODECS[codec_name]
+ return KNOWN_CODECS[codec_name] # type: ignore
def get_schema(self) -> Schema:
if _SCHEMA_KEY in self.meta:
@@ -276,11 +277,36 @@ def __exit__(
self.output_stream.close()
def _write_header(self) -> None:
+ from pyiceberg.table import TableProperties
+
+ codec_name = self.metadata.get(AVRO_CODEC_KEY, TableProperties.WRITE_AVRO_COMPRESSION_DEFAULT)
+ if avro_codec_name := CODEC_MAPPING_ICEBERG_TO_AVRO.get(codec_name):
+ codec_name = avro_codec_name
+
json_schema = json.dumps(AvroSchemaConversion().iceberg_to_avro(self.file_schema, schema_name=self.schema_name))
- meta = {**self.metadata, _SCHEMA_KEY: json_schema, _CODEC_KEY: "null"}
+
+ meta = {**self.metadata, _SCHEMA_KEY: json_schema, AVRO_CODEC_KEY: codec_name}
header = AvroFileHeader(MAGIC, meta, self.sync_bytes)
construct_writer(META_SCHEMA).write(self.encoder, header)
+ def compression_codec(self) -> Optional[Type[Codec]]:
+ """Get the file's compression codec algorithm from the file's metadata.
+
+ In the case of a null codec, we return a None indicating that we
+ don't need to compress/decompress.
+ """
+ from pyiceberg.table import TableProperties
+
+ codec_name = self.metadata.get(AVRO_CODEC_KEY, TableProperties.WRITE_AVRO_COMPRESSION_DEFAULT)
+
+ if avro_codec_name := CODEC_MAPPING_ICEBERG_TO_AVRO.get(codec_name):
+ codec_name = avro_codec_name
+
+ if codec_name not in KNOWN_CODECS:
+ raise ValueError(f"Unsupported codec: {codec_name}")
+
+ return KNOWN_CODECS[codec_name] # type: ignore
+
def write_block(self, objects: List[D]) -> None:
in_memory = io.BytesIO()
block_content_encoder = BinaryEncoder(output_stream=in_memory)
@@ -289,6 +315,13 @@ def write_block(self, objects: List[D]) -> None:
block_content = in_memory.getvalue()
self.encoder.write_int(len(objects))
- self.encoder.write_int(len(block_content))
- self.encoder.write(block_content)
+
+ if codec := self.compression_codec():
+ content, content_length = codec.compress(block_content)
+ self.encoder.write_int(content_length)
+ self.encoder.write(content)
+ else:
+ self.encoder.write_int(len(block_content))
+ self.encoder.write(block_content)
+
self.encoder.write(self.sync_bytes)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index d7c923404e..5d8608b19a 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -37,6 +37,7 @@
from cachetools.keys import hashkey
from pydantic_core import to_json
+from pyiceberg.avro.codecs import AVRO_CODEC_KEY, AvroCompressionCodec
from pyiceberg.avro.file import AvroFile, AvroOutputFile
from pyiceberg.conversions import to_bytes
from pyiceberg.exceptions import ValidationError
@@ -950,9 +951,16 @@ class ManifestWriter(ABC):
_deleted_rows: int
_min_sequence_number: Optional[int]
_partitions: List[Record]
- _reused_entry_wrapper: ManifestEntry
+ _compression: AvroCompressionCodec
- def __init__(self, spec: PartitionSpec, schema: Schema, output_file: OutputFile, snapshot_id: int) -> None:
+ def __init__(
+ self,
+ spec: PartitionSpec,
+ schema: Schema,
+ output_file: OutputFile,
+ snapshot_id: int,
+ avro_compression: AvroCompressionCodec,
+ ) -> None:
self.closed = False
self._spec = spec
self._schema = schema
@@ -967,6 +975,7 @@ def __init__(self, spec: PartitionSpec, schema: Schema, output_file: OutputFile,
self._deleted_rows = 0
self._min_sequence_number = None
self._partitions = []
+ self._compression = avro_compression
def __enter__(self) -> ManifestWriter:
"""Open the writer."""
@@ -1002,6 +1011,7 @@ def _meta(self) -> Dict[str, str]:
"partition-spec": to_json(self._spec.fields).decode("utf-8"),
"partition-spec-id": str(self._spec.spec_id),
"format-version": str(self.version),
+ AVRO_CODEC_KEY: self._compression,
}
def _with_partition(self, format_version: TableVersion) -> Schema:
@@ -1113,13 +1123,15 @@ def existing(self, entry: ManifestEntry) -> ManifestWriter:
class ManifestWriterV1(ManifestWriter):
- def __init__(self, spec: PartitionSpec, schema: Schema, output_file: OutputFile, snapshot_id: int):
- super().__init__(
- spec,
- schema,
- output_file,
- snapshot_id,
- )
+ def __init__(
+ self,
+ spec: PartitionSpec,
+ schema: Schema,
+ output_file: OutputFile,
+ snapshot_id: int,
+ avro_compression: AvroCompressionCodec,
+ ):
+ super().__init__(spec, schema, output_file, snapshot_id, avro_compression)
def content(self) -> ManifestContent:
return ManifestContent.DATA
@@ -1133,8 +1145,15 @@ def prepare_entry(self, entry: ManifestEntry) -> ManifestEntry:
class ManifestWriterV2(ManifestWriter):
- def __init__(self, spec: PartitionSpec, schema: Schema, output_file: OutputFile, snapshot_id: int):
- super().__init__(spec, schema, output_file, snapshot_id)
+ def __init__(
+ self,
+ spec: PartitionSpec,
+ schema: Schema,
+ output_file: OutputFile,
+ snapshot_id: int,
+ avro_compression: AvroCompressionCodec,
+ ):
+ super().__init__(spec, schema, output_file, snapshot_id, avro_compression)
def content(self) -> ManifestContent:
return ManifestContent.DATA
@@ -1160,12 +1179,17 @@ def prepare_entry(self, entry: ManifestEntry) -> ManifestEntry:
def write_manifest(
- format_version: TableVersion, spec: PartitionSpec, schema: Schema, output_file: OutputFile, snapshot_id: int
+ format_version: TableVersion,
+ spec: PartitionSpec,
+ schema: Schema,
+ output_file: OutputFile,
+ snapshot_id: int,
+ avro_compression: AvroCompressionCodec,
) -> ManifestWriter:
if format_version == 1:
- return ManifestWriterV1(spec, schema, output_file, snapshot_id)
+ return ManifestWriterV1(spec, schema, output_file, snapshot_id, avro_compression)
elif format_version == 2:
- return ManifestWriterV2(spec, schema, output_file, snapshot_id)
+ return ManifestWriterV2(spec, schema, output_file, snapshot_id, avro_compression)
else:
raise ValueError(f"Cannot write manifest for table version: {format_version}")
@@ -1215,7 +1239,13 @@ def add_manifests(self, manifest_files: List[ManifestFile]) -> ManifestListWrite
class ManifestListWriterV1(ManifestListWriter):
- def __init__(self, output_file: OutputFile, snapshot_id: int, parent_snapshot_id: Optional[int]):
+ def __init__(
+ self,
+ output_file: OutputFile,
+ snapshot_id: int,
+ parent_snapshot_id: Optional[int],
+ compression: AvroCompressionCodec,
+ ):
super().__init__(
format_version=1,
output_file=output_file,
@@ -1223,6 +1253,7 @@ def __init__(self, output_file: OutputFile, snapshot_id: int, parent_snapshot_id
"snapshot-id": str(snapshot_id),
"parent-snapshot-id": str(parent_snapshot_id) if parent_snapshot_id is not None else "null",
"format-version": "1",
+ AVRO_CODEC_KEY: compression,
},
)
@@ -1236,7 +1267,14 @@ class ManifestListWriterV2(ManifestListWriter):
_commit_snapshot_id: int
_sequence_number: int
- def __init__(self, output_file: OutputFile, snapshot_id: int, parent_snapshot_id: Optional[int], sequence_number: int):
+ def __init__(
+ self,
+ output_file: OutputFile,
+ snapshot_id: int,
+ parent_snapshot_id: Optional[int],
+ sequence_number: int,
+ compression: AvroCompressionCodec,
+ ):
super().__init__(
format_version=2,
output_file=output_file,
@@ -1245,6 +1283,7 @@ def __init__(self, output_file: OutputFile, snapshot_id: int, parent_snapshot_id
"parent-snapshot-id": str(parent_snapshot_id) if parent_snapshot_id is not None else "null",
"sequence-number": str(sequence_number),
"format-version": "2",
+ AVRO_CODEC_KEY: compression,
},
)
self._commit_snapshot_id = snapshot_id
@@ -1279,12 +1318,13 @@ def write_manifest_list(
snapshot_id: int,
parent_snapshot_id: Optional[int],
sequence_number: Optional[int],
+ avro_compression: AvroCompressionCodec,
) -> ManifestListWriter:
if format_version == 1:
- return ManifestListWriterV1(output_file, snapshot_id, parent_snapshot_id)
+ return ManifestListWriterV1(output_file, snapshot_id, parent_snapshot_id, avro_compression)
elif format_version == 2:
if sequence_number is None:
raise ValueError(f"Sequence-number is required for V2 tables: {sequence_number}")
- return ManifestListWriterV2(output_file, snapshot_id, parent_snapshot_id, sequence_number)
+ return ManifestListWriterV2(output_file, snapshot_id, parent_snapshot_id, sequence_number, avro_compression)
else:
raise ValueError(f"Cannot write manifest list for table version: {format_version}")
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 1dc20f3d80..dabc2ee4b8 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -188,6 +188,9 @@ class TableProperties:
WRITE_TARGET_FILE_SIZE_BYTES = "write.target-file-size-bytes"
WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT = 512 * 1024 * 1024 # 512 MB
+ WRITE_AVRO_COMPRESSION = "write.avro.compression-codec"
+ WRITE_AVRO_COMPRESSION_DEFAULT = "gzip"
+
DEFAULT_WRITE_METRICS_MODE = "write.metadata.metrics.default"
DEFAULT_WRITE_METRICS_MODE_DEFAULT = "truncate(16)"
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 87f0481f2e..599c7fa0f3 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -28,6 +28,7 @@
from sortedcontainers import SortedList
+from pyiceberg.avro.codecs import AvroCompressionCodec
from pyiceberg.expressions import (
AlwaysFalse,
BooleanExpression,
@@ -105,6 +106,7 @@ class _SnapshotProducer(UpdateTableMetadata[U], Generic[U]):
_added_data_files: List[DataFile]
_manifest_num_counter: itertools.count[int]
_deleted_data_files: Set[DataFile]
+ _compression: AvroCompressionCodec
def __init__(
self,
@@ -127,6 +129,11 @@ def __init__(
self._deleted_data_files = set()
self.snapshot_properties = snapshot_properties
self._manifest_num_counter = itertools.count(0)
+ from pyiceberg.table import TableProperties
+
+ self._compression = self._transaction.table_metadata.properties.get( # type: ignore
+ TableProperties.WRITE_AVRO_COMPRESSION, TableProperties.WRITE_AVRO_COMPRESSION_DEFAULT
+ )
def append_data_file(self, data_file: DataFile) -> _SnapshotProducer[U]:
self._added_data_files.append(data_file)
@@ -155,6 +162,7 @@ def _write_added_manifest() -> List[ManifestFile]:
schema=self._transaction.table_metadata.schema(),
output_file=self.new_manifest_output(),
snapshot_id=self._snapshot_id,
+ avro_compression=self._compression,
) as writer:
for data_file in self._added_data_files:
writer.add(
@@ -185,6 +193,7 @@ def _write_delete_manifest() -> List[ManifestFile]:
schema=self._transaction.table_metadata.schema(),
output_file=self.new_manifest_output(),
snapshot_id=self._snapshot_id,
+ avro_compression=self._compression,
) as writer:
for entry in entries:
writer.add_entry(entry)
@@ -250,12 +259,14 @@ def _commit(self) -> UpdatesAndRequirements:
)
location_provider = self._transaction._table.location_provider()
manifest_list_file_path = location_provider.new_metadata_location(file_name)
+
with write_manifest_list(
format_version=self._transaction.table_metadata.format_version,
output_file=self._io.new_output(manifest_list_file_path),
snapshot_id=self._snapshot_id,
parent_snapshot_id=self._parent_snapshot_id,
sequence_number=next_sequence_number,
+ avro_compression=self._compression,
) as writer:
writer.add_manifests(new_manifests)
@@ -292,6 +303,7 @@ def new_manifest_writer(self, spec: PartitionSpec) -> ManifestWriter:
schema=self._transaction.table_metadata.schema(),
output_file=self.new_manifest_output(),
snapshot_id=self._snapshot_id,
+ avro_compression=self._compression,
)
def new_manifest_output(self) -> OutputFile:
@@ -417,6 +429,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
schema=self._transaction.table_metadata.schema(),
output_file=self.new_manifest_output(),
snapshot_id=self._snapshot_id,
+ avro_compression=self._compression,
) as writer:
for existing_entry in existing_entries:
writer.add_entry(existing_entry)
@@ -704,6 +717,7 @@ def _existing_manifests(self) -> List[ManifestFile]:
schema=self._transaction.table_metadata.schema(),
output_file=self.new_manifest_output(),
snapshot_id=self._snapshot_id,
+ avro_compression=self._compression,
) as writer:
[
writer.add_entry(
diff --git a/tests/integration/test_rest_manifest.py b/tests/integration/test_rest_manifest.py
index dda0bbfe3b..8dd9510ac8 100644
--- a/tests/integration/test_rest_manifest.py
+++ b/tests/integration/test_rest_manifest.py
@@ -25,6 +25,7 @@
import pytest
from fastavro import reader
+from pyiceberg.avro.codecs import AvroCompressionCodec
from pyiceberg.catalog import Catalog, load_catalog
from pyiceberg.io.pyarrow import PyArrowFileIO
from pyiceberg.manifest import DataFile, write_manifest
@@ -77,7 +78,8 @@ def table_test_all_types(catalog: Catalog) -> Table:
@pytest.mark.integration
-def test_write_sample_manifest(table_test_all_types: Table) -> None:
+@pytest.mark.parametrize("compression", ["null", "deflate"])
+def test_write_sample_manifest(table_test_all_types: Table, compression: AvroCompressionCodec) -> None:
test_snapshot = table_test_all_types.current_snapshot()
if test_snapshot is None:
raise ValueError("Table has no current snapshot, check the docker environment")
@@ -120,6 +122,7 @@ def test_write_sample_manifest(table_test_all_types: Table) -> None:
schema=test_schema,
output_file=output,
snapshot_id=test_snapshot.snapshot_id,
+ avro_compression=compression,
) as manifest_writer:
# For simplicity, try one entry first
manifest_writer.add_entry(test_manifest_entries[0])
diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py
index 150d2b750c..493b163b95 100644
--- a/tests/integration/test_writes/test_writes.py
+++ b/tests/integration/test_writes/test_writes.py
@@ -25,6 +25,7 @@
from typing import Any, Dict
from urllib.parse import urlparse
+import fastavro
import pandas as pd
import pandas.testing
import pyarrow as pa
@@ -1841,3 +1842,28 @@ def test_read_write_decimals(session_catalog: Catalog) -> None:
tbl.append(arrow_table)
assert tbl.scan().to_arrow() == arrow_table
+
+
+@pytest.mark.integration
+def test_avro_compression_codecs(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_avro_compression_codecs"
+ tbl = _create_table(session_catalog, identifier, schema=arrow_table_with_null.schema, data=[arrow_table_with_null])
+
+ current_snapshot = tbl.current_snapshot()
+ assert current_snapshot is not None
+
+ with tbl.io.new_input(current_snapshot.manifest_list).open() as f:
+ reader = fastavro.reader(f)
+ assert reader.codec == "deflate"
+
+ with tbl.transaction() as tx:
+ tx.set_properties(**{TableProperties.WRITE_AVRO_COMPRESSION: "null"}) # type: ignore
+
+ tbl.append(arrow_table_with_null)
+
+ current_snapshot = tbl.current_snapshot()
+ assert current_snapshot is not None
+
+ with tbl.io.new_input(current_snapshot.manifest_list).open() as f:
+ reader = fastavro.reader(f)
+ assert reader.codec == "null"
diff --git a/tests/utils/test_manifest.py b/tests/utils/test_manifest.py
index 5740587958..d92f87a464 100644
--- a/tests/utils/test_manifest.py
+++ b/tests/utils/test_manifest.py
@@ -22,6 +22,7 @@
import fastavro
import pytest
+from pyiceberg.avro.codecs import AvroCompressionCodec
from pyiceberg.io import load_file_io
from pyiceberg.io.pyarrow import PyArrowFileIO
from pyiceberg.manifest import (
@@ -351,13 +352,18 @@ def test_write_empty_manifest() -> None:
schema=test_schema,
output_file=io.new_output(tmp_avro_file),
snapshot_id=8744736658442914487,
+ avro_compression="deflate",
) as _:
pass
@pytest.mark.parametrize("format_version", [1, 2])
+@pytest.mark.parametrize("compression", ["null", "deflate"])
def test_write_manifest(
- generated_manifest_file_file_v1: str, generated_manifest_file_file_v2: str, format_version: TableVersion
+ generated_manifest_file_file_v1: str,
+ generated_manifest_file_file_v2: str,
+ format_version: TableVersion,
+ compression: AvroCompressionCodec,
) -> None:
io = load_file_io()
snapshot = Snapshot(
@@ -387,6 +393,7 @@ def test_write_manifest(
schema=test_schema,
output_file=output,
snapshot_id=8744736658442914487,
+ avro_compression=compression,
) as writer:
for entry in manifest_entries:
writer.add_entry(entry)
@@ -527,11 +534,13 @@ def test_write_manifest(
@pytest.mark.parametrize("format_version", [1, 2])
@pytest.mark.parametrize("parent_snapshot_id", [19, None])
+@pytest.mark.parametrize("compression", ["null", "deflate"])
def test_write_manifest_list(
generated_manifest_file_file_v1: str,
generated_manifest_file_file_v2: str,
format_version: TableVersion,
parent_snapshot_id: Optional[int],
+ compression: AvroCompressionCodec,
) -> None:
io = load_file_io()
@@ -554,6 +563,7 @@ def test_write_manifest_list(
snapshot_id=25,
parent_snapshot_id=parent_snapshot_id,
sequence_number=0,
+ avro_compression=compression,
) as writer:
writer.add_manifests(demo_manifest_list)
new_manifest_list = list(read_manifest_list(io.new_input(path)))
From 58b0cafb14be290399821e6a9eb7b28d1be76aac Mon Sep 17 00:00:00 2001
From: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
Date: Sat, 14 Jun 2025 06:47:40 -0400
Subject: [PATCH 056/112] REST: Pass `purgeRequest` to catalog (#2086)
---
pyiceberg/catalog/rest/__init__.py | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)
diff --git a/pyiceberg/catalog/rest/__init__.py b/pyiceberg/catalog/rest/__init__.py
index 5ad7faef89..3f59a196ea 100644
--- a/pyiceberg/catalog/rest/__init__.py
+++ b/pyiceberg/catalog/rest/__init__.py
@@ -95,7 +95,7 @@ class Endpoints:
register_table = "namespaces/{namespace}/register"
load_table: str = "namespaces/{namespace}/tables/{table}"
update_table: str = "namespaces/{namespace}/tables/{table}"
- drop_table: str = "namespaces/{namespace}/tables/{table}?purgeRequested={purge}"
+ drop_table: str = "namespaces/{namespace}/tables/{table}"
table_exists: str = "namespaces/{namespace}/tables/{table}"
get_token: str = "oauth/tokens"
rename_table: str = "tables/rename"
@@ -617,7 +617,8 @@ def load_table(self, identifier: Union[str, Identifier]) -> Table:
@retry(**_RETRY_ARGS)
def drop_table(self, identifier: Union[str, Identifier], purge_requested: bool = False) -> None:
response = self._session.delete(
- self.url(Endpoints.drop_table, prefixed=True, purge=purge_requested, **self._split_identifier_for_path(identifier)),
+ self.url(Endpoints.drop_table, prefixed=True, **self._split_identifier_for_path(identifier)),
+ params={"purgeRequested": purge_requested},
)
try:
response.raise_for_status()
From 7761e26d27c6b8f38ba418dfa869474ce29f7da0 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sat, 14 Jun 2025 19:37:09 +0200
Subject: [PATCH 057/112] Bump Ruff to the latest version (#2073)
Makes the code look nicer
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
.pre-commit-config.yaml | 2 +-
tests/catalog/test_rest.py | 48 +--
tests/expressions/test_visitors.py | 480 ++++++++++++++---------------
tests/table/test_init.py | 18 +-
tests/table/test_upsert.py | 4 +-
tests/test_types.py | 6 +-
tests/utils/test_manifest.py | 6 +-
7 files changed, 282 insertions(+), 282 deletions(-)
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 66f830e2b8..63697b098f 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -27,7 +27,7 @@ repos:
- id: check-yaml
- id: check-ast
- repo: https://github.com/astral-sh/ruff-pre-commit
- rev: v0.8.6
+ rev: v0.11.13
hooks:
- id: ruff
args: [ --fix, --exit-non-zero-on-fix ]
diff --git a/tests/catalog/test_rest.py b/tests/catalog/test_rest.py
index 1ad6f57d36..ed91dd15a1 100644
--- a/tests/catalog/test_rest.py
+++ b/tests/catalog/test_rest.py
@@ -323,19 +323,19 @@ def test_properties_sets_headers(requests_mock: Mocker) -> None:
**{"header.Content-Type": "application/vnd.api+json", "header.Customized-Header": "some/value"},
)
- assert (
- catalog._session.headers.get("Content-type") == "application/json"
- ), "Expected 'Content-Type' default header not to be overwritten"
- assert (
- requests_mock.last_request.headers["Content-type"] == "application/json"
- ), "Config request did not include expected 'Content-Type' header"
+ assert catalog._session.headers.get("Content-type") == "application/json", (
+ "Expected 'Content-Type' default header not to be overwritten"
+ )
+ assert requests_mock.last_request.headers["Content-type"] == "application/json", (
+ "Config request did not include expected 'Content-Type' header"
+ )
- assert (
- catalog._session.headers.get("Customized-Header") == "some/value"
- ), "Expected 'Customized-Header' header to be 'some/value'"
- assert (
- requests_mock.last_request.headers["Customized-Header"] == "some/value"
- ), "Config request did not include expected 'Customized-Header' header"
+ assert catalog._session.headers.get("Customized-Header") == "some/value", (
+ "Expected 'Customized-Header' header to be 'some/value'"
+ )
+ assert requests_mock.last_request.headers["Customized-Header"] == "some/value", (
+ "Config request did not include expected 'Customized-Header' header"
+ )
def test_config_sets_headers(requests_mock: Mocker) -> None:
@@ -352,19 +352,19 @@ def test_config_sets_headers(requests_mock: Mocker) -> None:
catalog = RestCatalog("rest", uri=TEST_URI, warehouse="s3://some-bucket")
catalog.create_namespace(namespace)
- assert (
- catalog._session.headers.get("Content-type") == "application/json"
- ), "Expected 'Content-Type' default header not to be overwritten"
- assert (
- requests_mock.last_request.headers["Content-type"] == "application/json"
- ), "Create namespace request did not include expected 'Content-Type' header"
+ assert catalog._session.headers.get("Content-type") == "application/json", (
+ "Expected 'Content-Type' default header not to be overwritten"
+ )
+ assert requests_mock.last_request.headers["Content-type"] == "application/json", (
+ "Create namespace request did not include expected 'Content-Type' header"
+ )
- assert (
- catalog._session.headers.get("Customized-Header") == "some/value"
- ), "Expected 'Customized-Header' header to be 'some/value'"
- assert (
- requests_mock.last_request.headers["Customized-Header"] == "some/value"
- ), "Create namespace request did not include expected 'Customized-Header' header"
+ assert catalog._session.headers.get("Customized-Header") == "some/value", (
+ "Expected 'Customized-Header' header to be 'some/value'"
+ )
+ assert requests_mock.last_request.headers["Customized-Header"] == "some/value", (
+ "Create namespace request did not include expected 'Customized-Header' header"
+ )
@pytest.mark.filterwarnings(
diff --git a/tests/expressions/test_visitors.py b/tests/expressions/test_visitors.py
index 9e0d667b10..273bd24c9b 100644
--- a/tests/expressions/test_visitors.py
+++ b/tests/expressions/test_visitors.py
@@ -947,95 +947,95 @@ def manifest() -> ManifestFile:
def test_all_nulls(schema: Schema, manifest: ManifestFile) -> None:
- assert not _ManifestEvalVisitor(schema, NotNull(Reference("all_nulls_missing_nan")), case_sensitive=True).eval(
- manifest
- ), "Should skip: all nulls column with non-floating type contains all null"
+ assert not _ManifestEvalVisitor(schema, NotNull(Reference("all_nulls_missing_nan")), case_sensitive=True).eval(manifest), (
+ "Should skip: all nulls column with non-floating type contains all null"
+ )
- assert _ManifestEvalVisitor(schema, NotNull(Reference("all_nulls_missing_nan_float")), case_sensitive=True).eval(
- manifest
- ), "Should read: no NaN information may indicate presence of NaN value"
+ assert _ManifestEvalVisitor(schema, NotNull(Reference("all_nulls_missing_nan_float")), case_sensitive=True).eval(manifest), (
+ "Should read: no NaN information may indicate presence of NaN value"
+ )
- assert _ManifestEvalVisitor(schema, NotNull(Reference("some_nulls")), case_sensitive=True).eval(
- manifest
- ), "Should read: column with some nulls contains a non-null value"
+ assert _ManifestEvalVisitor(schema, NotNull(Reference("some_nulls")), case_sensitive=True).eval(manifest), (
+ "Should read: column with some nulls contains a non-null value"
+ )
- assert _ManifestEvalVisitor(schema, NotNull(Reference("no_nulls")), case_sensitive=True).eval(
- manifest
- ), "Should read: non-null column contains a non-null value"
+ assert _ManifestEvalVisitor(schema, NotNull(Reference("no_nulls")), case_sensitive=True).eval(manifest), (
+ "Should read: non-null column contains a non-null value"
+ )
def test_no_nulls(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, IsNull(Reference("all_nulls_missing_nan")), case_sensitive=True).eval(
- manifest
- ), "Should read: at least one null value in all null column"
+ assert _ManifestEvalVisitor(schema, IsNull(Reference("all_nulls_missing_nan")), case_sensitive=True).eval(manifest), (
+ "Should read: at least one null value in all null column"
+ )
- assert _ManifestEvalVisitor(schema, IsNull(Reference("some_nulls")), case_sensitive=True).eval(
- manifest
- ), "Should read: column with some nulls contains a null value"
+ assert _ManifestEvalVisitor(schema, IsNull(Reference("some_nulls")), case_sensitive=True).eval(manifest), (
+ "Should read: column with some nulls contains a null value"
+ )
- assert not _ManifestEvalVisitor(schema, IsNull(Reference("no_nulls")), case_sensitive=True).eval(
- manifest
- ), "Should skip: non-null column contains no null values"
+ assert not _ManifestEvalVisitor(schema, IsNull(Reference("no_nulls")), case_sensitive=True).eval(manifest), (
+ "Should skip: non-null column contains no null values"
+ )
- assert _ManifestEvalVisitor(schema, IsNull(Reference("both_nan_and_null")), case_sensitive=True).eval(
- manifest
- ), "Should read: both_nan_and_null column contains no null values"
+ assert _ManifestEvalVisitor(schema, IsNull(Reference("both_nan_and_null")), case_sensitive=True).eval(manifest), (
+ "Should read: both_nan_and_null column contains no null values"
+ )
def test_is_nan(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, IsNaN(Reference("float")), case_sensitive=True).eval(
- manifest
- ), "Should read: no information on if there are nan value in float column"
+ assert _ManifestEvalVisitor(schema, IsNaN(Reference("float")), case_sensitive=True).eval(manifest), (
+ "Should read: no information on if there are nan value in float column"
+ )
- assert _ManifestEvalVisitor(schema, IsNaN(Reference("all_nulls_double")), case_sensitive=True).eval(
- manifest
- ), "Should read: no NaN information may indicate presence of NaN value"
+ assert _ManifestEvalVisitor(schema, IsNaN(Reference("all_nulls_double")), case_sensitive=True).eval(manifest), (
+ "Should read: no NaN information may indicate presence of NaN value"
+ )
- assert _ManifestEvalVisitor(schema, IsNaN(Reference("all_nulls_missing_nan_float")), case_sensitive=True).eval(
- manifest
- ), "Should read: no NaN information may indicate presence of NaN value"
+ assert _ManifestEvalVisitor(schema, IsNaN(Reference("all_nulls_missing_nan_float")), case_sensitive=True).eval(manifest), (
+ "Should read: no NaN information may indicate presence of NaN value"
+ )
- assert not _ManifestEvalVisitor(schema, IsNaN(Reference("all_nulls_no_nans")), case_sensitive=True).eval(
- manifest
- ), "Should skip: no nan column doesn't contain nan value"
+ assert not _ManifestEvalVisitor(schema, IsNaN(Reference("all_nulls_no_nans")), case_sensitive=True).eval(manifest), (
+ "Should skip: no nan column doesn't contain nan value"
+ )
- assert _ManifestEvalVisitor(schema, IsNaN(Reference("all_nans")), case_sensitive=True).eval(
- manifest
- ), "Should read: all_nans column contains nan value"
+ assert _ManifestEvalVisitor(schema, IsNaN(Reference("all_nans")), case_sensitive=True).eval(manifest), (
+ "Should read: all_nans column contains nan value"
+ )
- assert _ManifestEvalVisitor(schema, IsNaN(Reference("both_nan_and_null")), case_sensitive=True).eval(
- manifest
- ), "Should read: both_nan_and_null column contains nan value"
+ assert _ManifestEvalVisitor(schema, IsNaN(Reference("both_nan_and_null")), case_sensitive=True).eval(manifest), (
+ "Should read: both_nan_and_null column contains nan value"
+ )
- assert not _ManifestEvalVisitor(schema, IsNaN(Reference("no_nan_or_null")), case_sensitive=True).eval(
- manifest
- ), "Should skip: no_nan_or_null column doesn't contain nan value"
+ assert not _ManifestEvalVisitor(schema, IsNaN(Reference("no_nan_or_null")), case_sensitive=True).eval(manifest), (
+ "Should skip: no_nan_or_null column doesn't contain nan value"
+ )
def test_not_nan(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, NotNaN(Reference("float")), case_sensitive=True).eval(
- manifest
- ), "Should read: no information on if there are nan value in float column"
+ assert _ManifestEvalVisitor(schema, NotNaN(Reference("float")), case_sensitive=True).eval(manifest), (
+ "Should read: no information on if there are nan value in float column"
+ )
- assert _ManifestEvalVisitor(schema, NotNaN(Reference("all_nulls_double")), case_sensitive=True).eval(
- manifest
- ), "Should read: all null column contains non nan value"
+ assert _ManifestEvalVisitor(schema, NotNaN(Reference("all_nulls_double")), case_sensitive=True).eval(manifest), (
+ "Should read: all null column contains non nan value"
+ )
- assert _ManifestEvalVisitor(schema, NotNaN(Reference("all_nulls_no_nans")), case_sensitive=True).eval(
- manifest
- ), "Should read: no_nans column contains non nan value"
+ assert _ManifestEvalVisitor(schema, NotNaN(Reference("all_nulls_no_nans")), case_sensitive=True).eval(manifest), (
+ "Should read: no_nans column contains non nan value"
+ )
- assert not _ManifestEvalVisitor(schema, NotNaN(Reference("all_nans")), case_sensitive=True).eval(
- manifest
- ), "Should skip: all nans column doesn't contain non nan value"
+ assert not _ManifestEvalVisitor(schema, NotNaN(Reference("all_nans")), case_sensitive=True).eval(manifest), (
+ "Should skip: all nans column doesn't contain non nan value"
+ )
- assert _ManifestEvalVisitor(schema, NotNaN(Reference("both_nan_and_null")), case_sensitive=True).eval(
- manifest
- ), "Should read: both_nan_and_null nans column contains non nan value"
+ assert _ManifestEvalVisitor(schema, NotNaN(Reference("both_nan_and_null")), case_sensitive=True).eval(manifest), (
+ "Should read: both_nan_and_null nans column contains non nan value"
+ )
- assert _ManifestEvalVisitor(schema, NotNaN(Reference("no_nan_or_null")), case_sensitive=True).eval(
- manifest
- ), "Should read: no_nan_or_null column contains non nan value"
+ assert _ManifestEvalVisitor(schema, NotNaN(Reference("no_nan_or_null")), case_sensitive=True).eval(manifest), (
+ "Should read: no_nan_or_null column contains non nan value"
+ )
def test_missing_stats(schema: Schema, manifest_no_stats: ManifestFile) -> None:
@@ -1053,15 +1053,15 @@ def test_missing_stats(schema: Schema, manifest_no_stats: ManifestFile) -> None:
]
for expr in expressions:
- assert _ManifestEvalVisitor(schema, expr, case_sensitive=True).eval(
- manifest_no_stats
- ), f"Should read when missing stats for expr: {expr}"
+ assert _ManifestEvalVisitor(schema, expr, case_sensitive=True).eval(manifest_no_stats), (
+ f"Should read when missing stats for expr: {expr}"
+ )
def test_not(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, Not(LessThan(Reference("id"), INT_MIN_VALUE - 25)), case_sensitive=True).eval(
- manifest
- ), "Should read: not(false)"
+ assert _ManifestEvalVisitor(schema, Not(LessThan(Reference("id"), INT_MIN_VALUE - 25)), case_sensitive=True).eval(manifest), (
+ "Should read: not(false)"
+ )
assert not _ManifestEvalVisitor(schema, Not(GreaterThan(Reference("id"), INT_MIN_VALUE - 25)), case_sensitive=True).eval(
manifest
@@ -1118,21 +1118,21 @@ def test_or(schema: Schema, manifest: ManifestFile) -> None:
def test_integer_lt(schema: Schema, manifest: ManifestFile) -> None:
- assert not _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MIN_VALUE - 25), case_sensitive=True).eval(
- manifest
- ), "Should not read: id range below lower bound (5 < 30)"
+ assert not _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MIN_VALUE - 25), case_sensitive=True).eval(manifest), (
+ "Should not read: id range below lower bound (5 < 30)"
+ )
- assert not _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should not read: id range below lower bound (30 is not < 30)"
+ assert not _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(manifest), (
+ "Should not read: id range below lower bound (30 is not < 30)"
+ )
- assert _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MIN_VALUE + 1), case_sensitive=True).eval(
- manifest
- ), "Should read: one possible id"
+ assert _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MIN_VALUE + 1), case_sensitive=True).eval(manifest), (
+ "Should read: one possible id"
+ )
- assert _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: may possible ids"
+ assert _ManifestEvalVisitor(schema, LessThan(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: may possible ids"
+ )
def test_integer_lt_eq(schema: Schema, manifest: ManifestFile) -> None:
@@ -1144,13 +1144,13 @@ def test_integer_lt_eq(schema: Schema, manifest: ManifestFile) -> None:
manifest
), "Should not read: id range below lower bound (29 < 30)"
- assert _ManifestEvalVisitor(schema, LessThanOrEqual(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: one possible id"
+ assert _ManifestEvalVisitor(schema, LessThanOrEqual(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: one possible id"
+ )
- assert _ManifestEvalVisitor(schema, LessThanOrEqual(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: many possible ids"
+ assert _ManifestEvalVisitor(schema, LessThanOrEqual(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: many possible ids"
+ )
def test_integer_gt(schema: Schema, manifest: ManifestFile) -> None:
@@ -1158,17 +1158,17 @@ def test_integer_gt(schema: Schema, manifest: ManifestFile) -> None:
manifest
), "Should not read: id range above upper bound (85 < 79)"
- assert not _ManifestEvalVisitor(schema, GreaterThan(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should not read: id range above upper bound (79 is not > 79)"
+ assert not _ManifestEvalVisitor(schema, GreaterThan(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(manifest), (
+ "Should not read: id range above upper bound (79 is not > 79)"
+ )
- assert _ManifestEvalVisitor(schema, GreaterThan(Reference("id"), INT_MAX_VALUE - 1), case_sensitive=True).eval(
- manifest
- ), "Should read: one possible id"
+ assert _ManifestEvalVisitor(schema, GreaterThan(Reference("id"), INT_MAX_VALUE - 1), case_sensitive=True).eval(manifest), (
+ "Should read: one possible id"
+ )
- assert _ManifestEvalVisitor(schema, GreaterThan(Reference("id"), INT_MAX_VALUE - 4), case_sensitive=True).eval(
- manifest
- ), "Should read: may possible ids"
+ assert _ManifestEvalVisitor(schema, GreaterThan(Reference("id"), INT_MAX_VALUE - 4), case_sensitive=True).eval(manifest), (
+ "Should read: may possible ids"
+ )
def test_integer_gt_eq(schema: Schema, manifest: ManifestFile) -> None:
@@ -1180,133 +1180,133 @@ def test_integer_gt_eq(schema: Schema, manifest: ManifestFile) -> None:
manifest
), "Should not read: id range above upper bound (80 > 79)"
- assert _ManifestEvalVisitor(schema, GreaterThanOrEqual(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: one possible id"
+ assert _ManifestEvalVisitor(schema, GreaterThanOrEqual(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: one possible id"
+ )
- assert _ManifestEvalVisitor(schema, GreaterThanOrEqual(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: may possible ids"
+ assert _ManifestEvalVisitor(schema, GreaterThanOrEqual(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: may possible ids"
+ )
def test_integer_eq(schema: Schema, manifest: ManifestFile) -> None:
- assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MIN_VALUE - 25), case_sensitive=True).eval(
- manifest
- ), "Should not read: id below lower bound"
+ assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MIN_VALUE - 25), case_sensitive=True).eval(manifest), (
+ "Should not read: id below lower bound"
+ )
- assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MIN_VALUE - 1), case_sensitive=True).eval(
- manifest
- ), "Should not read: id below lower bound"
+ assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MIN_VALUE - 1), case_sensitive=True).eval(manifest), (
+ "Should not read: id below lower bound"
+ )
- assert _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: id equal to lower bound"
+ assert _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: id equal to lower bound"
+ )
- assert _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE - 4), case_sensitive=True).eval(
- manifest
- ), "Should read: id between lower and upper bounds"
+ assert _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE - 4), case_sensitive=True).eval(manifest), (
+ "Should read: id between lower and upper bounds"
+ )
- assert _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: id equal to upper bound"
+ assert _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: id equal to upper bound"
+ )
- assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE + 1), case_sensitive=True).eval(
- manifest
- ), "Should not read: id above upper bound"
+ assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE + 1), case_sensitive=True).eval(manifest), (
+ "Should not read: id above upper bound"
+ )
- assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE + 6), case_sensitive=True).eval(
- manifest
- ), "Should not read: id above upper bound"
+ assert not _ManifestEvalVisitor(schema, EqualTo(Reference("id"), INT_MAX_VALUE + 6), case_sensitive=True).eval(manifest), (
+ "Should not read: id above upper bound"
+ )
def test_integer_not_eq(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MIN_VALUE - 25), case_sensitive=True).eval(
- manifest
- ), "Should read: id below lower bound"
+ assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MIN_VALUE - 25), case_sensitive=True).eval(manifest), (
+ "Should read: id below lower bound"
+ )
- assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MIN_VALUE - 1), case_sensitive=True).eval(
- manifest
- ), "Should read: id below lower bound"
+ assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MIN_VALUE - 1), case_sensitive=True).eval(manifest), (
+ "Should read: id below lower bound"
+ )
- assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: id equal to lower bound"
+ assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MIN_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: id equal to lower bound"
+ )
- assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE - 4), case_sensitive=True).eval(
- manifest
- ), "Should read: id between lower and upper bounds"
+ assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE - 4), case_sensitive=True).eval(manifest), (
+ "Should read: id between lower and upper bounds"
+ )
- assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(
- manifest
- ), "Should read: id equal to upper bound"
+ assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE), case_sensitive=True).eval(manifest), (
+ "Should read: id equal to upper bound"
+ )
- assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE + 1), case_sensitive=True).eval(
- manifest
- ), "Should read: id above upper bound"
+ assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE + 1), case_sensitive=True).eval(manifest), (
+ "Should read: id above upper bound"
+ )
- assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE + 6), case_sensitive=True).eval(
- manifest
- ), "Should read: id above upper bound"
+ assert _ManifestEvalVisitor(schema, NotEqualTo(Reference("id"), INT_MAX_VALUE + 6), case_sensitive=True).eval(manifest), (
+ "Should read: id above upper bound"
+ )
def test_integer_not_eq_rewritten(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MIN_VALUE - 25)), case_sensitive=True).eval(
- manifest
- ), "Should read: id below lower bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MIN_VALUE - 25)), case_sensitive=True).eval(manifest), (
+ "Should read: id below lower bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MIN_VALUE - 1)), case_sensitive=True).eval(
- manifest
- ), "Should read: id below lower bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MIN_VALUE - 1)), case_sensitive=True).eval(manifest), (
+ "Should read: id below lower bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MIN_VALUE)), case_sensitive=True).eval(
- manifest
- ), "Should read: id equal to lower bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MIN_VALUE)), case_sensitive=True).eval(manifest), (
+ "Should read: id equal to lower bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE - 4)), case_sensitive=True).eval(
- manifest
- ), "Should read: id between lower and upper bounds"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE - 4)), case_sensitive=True).eval(manifest), (
+ "Should read: id between lower and upper bounds"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE)), case_sensitive=True).eval(
- manifest
- ), "Should read: id equal to upper bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE)), case_sensitive=True).eval(manifest), (
+ "Should read: id equal to upper bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE + 1)), case_sensitive=True).eval(
- manifest
- ), "Should read: id above upper bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE + 1)), case_sensitive=True).eval(manifest), (
+ "Should read: id above upper bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE + 6)), case_sensitive=True).eval(
- manifest
- ), "Should read: id above upper bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("id"), INT_MAX_VALUE + 6)), case_sensitive=True).eval(manifest), (
+ "Should read: id above upper bound"
+ )
def test_integer_not_eq_rewritten_case_insensitive(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MIN_VALUE - 25)), case_sensitive=False).eval(
- manifest
- ), "Should read: id below lower bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MIN_VALUE - 25)), case_sensitive=False).eval(manifest), (
+ "Should read: id below lower bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MIN_VALUE - 1)), case_sensitive=False).eval(
- manifest
- ), "Should read: id below lower bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MIN_VALUE - 1)), case_sensitive=False).eval(manifest), (
+ "Should read: id below lower bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MIN_VALUE)), case_sensitive=False).eval(
- manifest
- ), "Should read: id equal to lower bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MIN_VALUE)), case_sensitive=False).eval(manifest), (
+ "Should read: id equal to lower bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE - 4)), case_sensitive=False).eval(
- manifest
- ), "Should read: id between lower and upper bounds"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE - 4)), case_sensitive=False).eval(manifest), (
+ "Should read: id between lower and upper bounds"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE)), case_sensitive=False).eval(
- manifest
- ), "Should read: id equal to upper bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE)), case_sensitive=False).eval(manifest), (
+ "Should read: id equal to upper bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE + 1)), case_sensitive=False).eval(
- manifest
- ), "Should read: id above upper bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE + 1)), case_sensitive=False).eval(manifest), (
+ "Should read: id above upper bound"
+ )
- assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE + 6)), case_sensitive=False).eval(
- manifest
- ), "Should read: id above upper bound"
+ assert _ManifestEvalVisitor(schema, Not(EqualTo(Reference("ID"), INT_MAX_VALUE + 6)), case_sensitive=False).eval(manifest), (
+ "Should read: id above upper bound"
+ )
def test_integer_in(schema: Schema, manifest: ManifestFile) -> None:
@@ -1342,13 +1342,13 @@ def test_integer_in(schema: Schema, manifest: ManifestFile) -> None:
manifest
), "Should skip: in on all nulls column"
- assert _ManifestEvalVisitor(schema, In(Reference("some_nulls"), ("abc", "def")), case_sensitive=True).eval(
- manifest
- ), "Should read: in on some nulls column"
+ assert _ManifestEvalVisitor(schema, In(Reference("some_nulls"), ("abc", "def")), case_sensitive=True).eval(manifest), (
+ "Should read: in on some nulls column"
+ )
- assert _ManifestEvalVisitor(schema, In(Reference("no_nulls"), ("abc", "def")), case_sensitive=True).eval(
- manifest
- ), "Should read: in on no nulls column"
+ assert _ManifestEvalVisitor(schema, In(Reference("no_nulls"), ("abc", "def")), case_sensitive=True).eval(manifest), (
+ "Should read: in on no nulls column"
+ )
def test_integer_not_in(schema: Schema, manifest: ManifestFile) -> None:
@@ -1384,73 +1384,73 @@ def test_integer_not_in(schema: Schema, manifest: ManifestFile) -> None:
manifest
), "Should read: notIn on no nulls column"
- assert _ManifestEvalVisitor(schema, NotIn(Reference("some_nulls"), ("abc", "def")), case_sensitive=True).eval(
- manifest
- ), "Should read: in on some nulls column"
+ assert _ManifestEvalVisitor(schema, NotIn(Reference("some_nulls"), ("abc", "def")), case_sensitive=True).eval(manifest), (
+ "Should read: in on some nulls column"
+ )
- assert _ManifestEvalVisitor(schema, NotIn(Reference("no_nulls"), ("abc", "def")), case_sensitive=True).eval(
- manifest
- ), "Should read: in on no nulls column"
+ assert _ManifestEvalVisitor(schema, NotIn(Reference("no_nulls"), ("abc", "def")), case_sensitive=True).eval(manifest), (
+ "Should read: in on no nulls column"
+ )
def test_string_starts_with(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "a"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "a"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "aa"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "aa"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "dddd"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "dddd"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "z"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "z"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, StartsWith(Reference("no_nulls"), "a"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, StartsWith(Reference("no_nulls"), "a"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert not _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "zzzz"), case_sensitive=False).eval(
- manifest
- ), "Should skip: range doesn't match"
+ assert not _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "zzzz"), case_sensitive=False).eval(manifest), (
+ "Should skip: range doesn't match"
+ )
- assert not _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "1"), case_sensitive=False).eval(
- manifest
- ), "Should skip: range doesn't match"
+ assert not _ManifestEvalVisitor(schema, StartsWith(Reference("some_nulls"), "1"), case_sensitive=False).eval(manifest), (
+ "Should skip: range doesn't match"
+ )
def test_string_not_starts_with(schema: Schema, manifest: ManifestFile) -> None:
- assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "a"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "a"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "aa"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "aa"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "dddd"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "dddd"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "z"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "z"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("no_nulls"), "a"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("no_nulls"), "a"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "zzzz"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "zzzz"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
- assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "1"), case_sensitive=False).eval(
- manifest
- ), "Should read: range matches"
+ assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("some_nulls"), "1"), case_sensitive=False).eval(manifest), (
+ "Should read: range matches"
+ )
assert _ManifestEvalVisitor(schema, NotStartsWith(Reference("all_same_value_or_null"), "a"), case_sensitive=False).eval(
manifest
diff --git a/tests/table/test_init.py b/tests/table/test_init.py
index dbac84bd81..6165dadec4 100644
--- a/tests/table/test_init.py
+++ b/tests/table/test_init.py
@@ -511,15 +511,15 @@ def test_update_column(table_v1: Table, table_v2: Table) -> None:
assert new_schema3.find_field("z").required is False, "failed to update existing field required"
# assert the above two updates also works with union_by_name
- assert (
- table.update_schema().union_by_name(new_schema)._apply() == new_schema
- ), "failed to update existing field doc with union_by_name"
- assert (
- table.update_schema().union_by_name(new_schema2)._apply() == new_schema2
- ), "failed to remove existing field doc with union_by_name"
- assert (
- table.update_schema().union_by_name(new_schema3)._apply() == new_schema3
- ), "failed to update existing field required with union_by_name"
+ assert table.update_schema().union_by_name(new_schema)._apply() == new_schema, (
+ "failed to update existing field doc with union_by_name"
+ )
+ assert table.update_schema().union_by_name(new_schema2)._apply() == new_schema2, (
+ "failed to remove existing field doc with union_by_name"
+ )
+ assert table.update_schema().union_by_name(new_schema3)._apply() == new_schema3, (
+ "failed to update existing field required with union_by_name"
+ )
def test_add_primitive_type_column(table_v2: Table) -> None:
diff --git a/tests/table/test_upsert.py b/tests/table/test_upsert.py
index 9fecbbb7bb..cc6e008b1e 100644
--- a/tests/table/test_upsert.py
+++ b/tests/table/test_upsert.py
@@ -83,7 +83,7 @@ def gen_source_dataset(start_row: int, end_row: int, composite_key: bool, add_du
)
sql = f"""
- with t as (SELECT unnest(range({start_row},{end_row+1})) as order_id)
+ with t as (SELECT unnest(range({start_row},{end_row + 1})) as order_id)
SELECT t.order_id {additional_columns}
, date '2021-01-01' as order_date, 'B' as order_type
from t
@@ -101,7 +101,7 @@ def gen_target_iceberg_table(
additional_columns = ", t.order_id + 1000 as order_line_id" if composite_key else ""
df = ctx.sql(f"""
- with t as (SELECT unnest(range({start_row},{end_row+1})) as order_id)
+ with t as (SELECT unnest(range({start_row},{end_row + 1})) as order_id)
SELECT t.order_id {additional_columns}
, date '2021-01-01' as order_date, 'A' as order_type
from t
diff --git a/tests/test_types.py b/tests/test_types.py
index 586022e5a4..2527f0e822 100644
--- a/tests/test_types.py
+++ b/tests/test_types.py
@@ -260,9 +260,9 @@ def test_nested_field_primitive_type_as_str() -> None:
type_str,
required=True,
)
- assert isinstance(
- field_var.field_type, type_class
- ), f"Expected {type_class.__name__}, got {field_var.field_type.__class__.__name__}"
+ assert isinstance(field_var.field_type, type_class), (
+ f"Expected {type_class.__name__}, got {field_var.field_type.__class__.__name__}"
+ )
# Test that passing 'bool' raises a ValueError, as it should be 'boolean'
with pytest.raises(ValueError) as exc_info:
diff --git a/tests/utils/test_manifest.py b/tests/utils/test_manifest.py
index d92f87a464..c838c02ac6 100644
--- a/tests/utils/test_manifest.py
+++ b/tests/utils/test_manifest.py
@@ -631,9 +631,9 @@ def test_write_manifest_list(
def test_file_format_case_insensitive(raw_file_format: str, expected_file_format: FileFormat) -> None:
if expected_file_format:
parsed_file_format = FileFormat(raw_file_format)
- assert (
- parsed_file_format == expected_file_format
- ), f"File format {raw_file_format}: {parsed_file_format} != {expected_file_format}"
+ assert parsed_file_format == expected_file_format, (
+ f"File format {raw_file_format}: {parsed_file_format} != {expected_file_format}"
+ )
else:
with pytest.raises(ValueError):
_ = FileFormat(raw_file_format)
From 4c0b549615446d2270eace72f19ba34380609246 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sat, 14 Jun 2025 19:37:18 +0200
Subject: [PATCH 058/112] Bump mypy in pre-commit (#2074)
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
.pre-commit-config.yaml | 2 +-
pyiceberg/expressions/literals.py | 2 +-
pyiceberg/io/fsspec.py | 2 +-
pyiceberg/manifest.py | 2 +-
pyiceberg/transforms.py | 8 ++++----
5 files changed, 8 insertions(+), 8 deletions(-)
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 63697b098f..afc1449321 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -33,7 +33,7 @@ repos:
args: [ --fix, --exit-non-zero-on-fix ]
- id: ruff-format
- repo: https://github.com/pre-commit/mirrors-mypy
- rev: v1.14.1
+ rev: v1.16.0
hooks:
- id: mypy
args:
diff --git a/pyiceberg/expressions/literals.py b/pyiceberg/expressions/literals.py
index 81e613d55a..921e24e29f 100644
--- a/pyiceberg/expressions/literals.py
+++ b/pyiceberg/expressions/literals.py
@@ -262,7 +262,7 @@ def __init__(self, value: bool) -> None:
super().__init__(value, bool)
@singledispatchmethod
- def to(self, type_var: IcebergType) -> Literal[bool]: # type: ignore
+ def to(self, type_var: IcebergType) -> Literal[bool]:
raise TypeError(f"Cannot convert BooleanLiteral into {type_var}")
@to.register(BooleanType)
diff --git a/pyiceberg/io/fsspec.py b/pyiceberg/io/fsspec.py
index 1cba30a038..cc80725d14 100644
--- a/pyiceberg/io/fsspec.py
+++ b/pyiceberg/io/fsspec.py
@@ -91,7 +91,7 @@
def s3v4_rest_signer(properties: Properties, request: "AWSRequest", **_: Any) -> "AWSRequest":
- signer_url = properties.get(S3_SIGNER_URI, properties["uri"]).rstrip("/")
+ signer_url = properties.get(S3_SIGNER_URI, properties["uri"]).rstrip("/") # type: ignore
signer_endpoint = properties.get(S3_SIGNER_ENDPOINT, S3_SIGNER_ENDPOINT_DEFAULT)
signer_headers = {}
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 5d8608b19a..9e8a5ae439 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -916,7 +916,7 @@ def _inherit_from_manifest(entry: ManifestEntry, manifest: ManifestFile) -> Mani
"""
# Inherit sequence numbers.
# The snapshot_id is required in V1, inherit with V2 when null
- if entry.snapshot_id is None:
+ if entry.snapshot_id is None and manifest.added_snapshot_id is not None:
entry.snapshot_id = manifest.added_snapshot_id
# in v1 tables, the sequence number is not persisted and can be safely defaulted to 0
diff --git a/pyiceberg/transforms.py b/pyiceberg/transforms.py
index 8dda85a48b..4c461e9ed7 100644
--- a/pyiceberg/transforms.py
+++ b/pyiceberg/transforms.py
@@ -1049,11 +1049,11 @@ def _truncate_number(
raise ValueError(f"Expected a numeric literal, got: {type(boundary)}")
if isinstance(pred, BoundLessThan):
- return LessThanOrEqual(Reference(name), _transform_literal(transform, boundary.decrement())) # type: ignore
+ return LessThanOrEqual(Reference(name), _transform_literal(transform, boundary.decrement()))
elif isinstance(pred, BoundLessThanOrEqual):
return LessThanOrEqual(Reference(name), _transform_literal(transform, boundary))
elif isinstance(pred, BoundGreaterThan):
- return GreaterThanOrEqual(Reference(name), _transform_literal(transform, boundary.increment())) # type: ignore
+ return GreaterThanOrEqual(Reference(name), _transform_literal(transform, boundary.increment()))
elif isinstance(pred, BoundGreaterThanOrEqual):
return GreaterThanOrEqual(Reference(name), _transform_literal(transform, boundary))
elif isinstance(pred, BoundEqualTo):
@@ -1073,11 +1073,11 @@ def _truncate_number_strict(
if isinstance(pred, BoundLessThan):
return LessThan(Reference(name), _transform_literal(transform, boundary))
elif isinstance(pred, BoundLessThanOrEqual):
- return LessThan(Reference(name), _transform_literal(transform, boundary.increment())) # type: ignore
+ return LessThan(Reference(name), _transform_literal(transform, boundary.increment()))
elif isinstance(pred, BoundGreaterThan):
return GreaterThan(Reference(name), _transform_literal(transform, boundary))
elif isinstance(pred, BoundGreaterThanOrEqual):
- return GreaterThan(Reference(name), _transform_literal(transform, boundary.decrement())) # type: ignore
+ return GreaterThan(Reference(name), _transform_literal(transform, boundary.decrement()))
elif isinstance(pred, BoundNotEqualTo):
return NotEqualTo(Reference(name), _transform_literal(transform, boundary))
elif isinstance(pred, BoundEqualTo):
From 36fd7b2452b502df0c60af31d829ead741c60f4b Mon Sep 17 00:00:00 2001
From: Anton-Tarazi <98493912+Anton-Tarazi@users.noreply.github.com>
Date: Sat, 14 Jun 2025 14:00:54 -0400
Subject: [PATCH 059/112] feat: support listing known catalogs (#2088)
Adds a new function `pyiceberg.catalog.list_catalogs() -> List[str]` to
list all known catalogs.
# Rationale for this change
In creating a pyiceberg-backed API, one must either duplicate the
config-parsing logic or access private data in order to know what
catalogs are available in the env. I believe that this logic should be
part of the library.
# Are these changes tested?
Yes
# Are there any user-facing changes?
Yes, knew public function in `pyiceberg.catalog`
---
pyiceberg/catalog/__init__.py | 4 ++++
pyiceberg/utils/config.py | 6 ++++++
tests/utils/test_config.py | 9 +++++++++
3 files changed, 19 insertions(+)
diff --git a/pyiceberg/catalog/__init__.py b/pyiceberg/catalog/__init__.py
index cf649ba7d6..d7d4c9d050 100644
--- a/pyiceberg/catalog/__init__.py
+++ b/pyiceberg/catalog/__init__.py
@@ -262,6 +262,10 @@ def load_catalog(name: Optional[str] = None, **properties: Optional[str]) -> Cat
raise ValueError(f"Could not initialize catalog with the following properties: {properties}")
+def list_catalogs() -> List[str]:
+ return _ENV_CONFIG.get_known_catalogs()
+
+
def delete_files(io: FileIO, files_to_delete: Set[str], file_type: str) -> None:
"""Delete files.
diff --git a/pyiceberg/utils/config.py b/pyiceberg/utils/config.py
index 0bfaefdbc6..78f121a402 100644
--- a/pyiceberg/utils/config.py
+++ b/pyiceberg/utils/config.py
@@ -159,6 +159,12 @@ def get_catalog_config(self, catalog_name: str) -> Optional[RecursiveDict]:
return catalog_conf
return None
+ def get_known_catalogs(self) -> List[str]:
+ catalogs = self.config.get(CATALOG, {})
+ if not isinstance(catalogs, dict):
+ raise ValueError("Catalog configurations needs to be an object")
+ return list(catalogs.keys())
+
def get_int(self, key: str) -> Optional[int]:
if (val := self.config.get(key)) is not None:
try:
diff --git a/tests/utils/test_config.py b/tests/utils/test_config.py
index 89247d8fca..24a867f812 100644
--- a/tests/utils/test_config.py
+++ b/tests/utils/test_config.py
@@ -56,6 +56,15 @@ def test_fix_nested_objects_from_environment_variables() -> None:
}
+@mock.patch.dict(os.environ, EXAMPLE_ENV)
+@mock.patch.dict(os.environ, {"PYICEBERG_CATALOG__DEVELOPMENT__URI": "https://dev.service.io/api"})
+def test_list_all_known_catalogs() -> None:
+ assert Config().get_known_catalogs() == [
+ "production",
+ "development",
+ ]
+
+
def test_from_configuration_files(tmp_path_factory: pytest.TempPathFactory) -> None:
config_path = str(tmp_path_factory.mktemp("config"))
with open(f"{config_path}/.pyiceberg.yaml", "w", encoding=UTF8) as file:
From 4e2bf97ee0e5ed7e3077110f257baad491241da5 Mon Sep 17 00:00:00 2001
From: d-lcw
Date: Sun, 15 Jun 2025 02:08:53 +0800
Subject: [PATCH 060/112] fix: add metadata_properties to _construct_parameters
when update hive table (#2013)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Closes: https://github.com/apache/iceberg-python/issues/2010
# Rationale for this change
This change adds metadata_properties to the _construct_parameters
function to ensure metadata properties are included in the parameters.
I'm not entirely confident about the changes, so please let me know if
my understanding is correct—if so, I’ll proceed to add tests.
Thanks you!
# Are these changes tested?
Not yet.
# Are there any user-facing changes?
Not sure.
---
pyiceberg/catalog/hive.py | 12 ++++++++++--
tests/catalog/test_hive.py | 16 ++++++++++++++--
tests/integration/test_reads.py | 21 +++++++++++++++++++++
3 files changed, 45 insertions(+), 4 deletions(-)
diff --git a/pyiceberg/catalog/hive.py b/pyiceberg/catalog/hive.py
index 5a9387577b..0fd6b9d79e 100644
--- a/pyiceberg/catalog/hive.py
+++ b/pyiceberg/catalog/hive.py
@@ -211,11 +211,18 @@ def _construct_hive_storage_descriptor(
DEFAULT_PROPERTIES = {TableProperties.PARQUET_COMPRESSION: TableProperties.PARQUET_COMPRESSION_DEFAULT}
-def _construct_parameters(metadata_location: str, previous_metadata_location: Optional[str] = None) -> Dict[str, Any]:
+def _construct_parameters(
+ metadata_location: str, previous_metadata_location: Optional[str] = None, metadata_properties: Optional[Properties] = None
+) -> Dict[str, Any]:
properties = {PROP_EXTERNAL: "TRUE", PROP_TABLE_TYPE: "ICEBERG", PROP_METADATA_LOCATION: metadata_location}
if previous_metadata_location:
properties[PROP_PREVIOUS_METADATA_LOCATION] = previous_metadata_location
+ if metadata_properties:
+ for key, value in metadata_properties.items():
+ if key not in properties:
+ properties[key] = str(value)
+
return properties
@@ -360,7 +367,7 @@ def _convert_iceberg_into_hive(self, table: Table) -> HiveTable:
property_as_bool(self.properties, HIVE2_COMPATIBLE, HIVE2_COMPATIBLE_DEFAULT),
),
tableType=EXTERNAL_TABLE,
- parameters=_construct_parameters(table.metadata_location),
+ parameters=_construct_parameters(metadata_location=table.metadata_location, metadata_properties=table.properties),
)
def _create_hive_table(self, open_client: Client, hive_table: HiveTable) -> None:
@@ -541,6 +548,7 @@ def commit_table(
hive_table.parameters = _construct_parameters(
metadata_location=updated_staged_table.metadata_location,
previous_metadata_location=current_table.metadata_location,
+ metadata_properties=updated_staged_table.properties,
)
open_client.alter_table_with_environment_context(
dbname=database_name,
diff --git a/tests/catalog/test_hive.py b/tests/catalog/test_hive.py
index 497ff99924..fef0d6acc6 100644
--- a/tests/catalog/test_hive.py
+++ b/tests/catalog/test_hive.py
@@ -342,7 +342,13 @@ def test_create_table(
storedAsSubDirectories=None,
),
partitionKeys=None,
- parameters={"EXTERNAL": "TRUE", "table_type": "ICEBERG", "metadata_location": metadata_location},
+ parameters={
+ "EXTERNAL": "TRUE",
+ "table_type": "ICEBERG",
+ "metadata_location": metadata_location,
+ "write.parquet.compression-codec": "zstd",
+ "owner": "javaberg",
+ },
viewOriginalText=None,
viewExpandedText=None,
tableType="EXTERNAL_TABLE",
@@ -517,7 +523,13 @@ def test_create_table_with_given_location_removes_trailing_slash(
storedAsSubDirectories=None,
),
partitionKeys=None,
- parameters={"EXTERNAL": "TRUE", "table_type": "ICEBERG", "metadata_location": metadata_location},
+ parameters={
+ "EXTERNAL": "TRUE",
+ "table_type": "ICEBERG",
+ "metadata_location": metadata_location,
+ "write.parquet.compression-codec": "zstd",
+ "owner": "javaberg",
+ },
viewOriginalText=None,
viewExpandedText=None,
tableType="EXTERNAL_TABLE",
diff --git a/tests/integration/test_reads.py b/tests/integration/test_reads.py
index 5ac5162f8e..b417a43616 100644
--- a/tests/integration/test_reads.py
+++ b/tests/integration/test_reads.py
@@ -112,6 +112,27 @@ def test_table_properties(catalog: Catalog) -> None:
assert "None type is not a supported value in properties: property_name" in str(exc_info.value)
+@pytest.mark.integration
+@pytest.mark.parametrize("catalog", [pytest.lazy_fixture("session_catalog_hive")])
+def test_hive_properties(catalog: Catalog) -> None:
+ table = create_table(catalog)
+ table.transaction().set_properties({"abc": "def", "p1": "123"}).commit_transaction()
+
+ hive_client: _HiveClient = _HiveClient(catalog.properties["uri"])
+
+ with hive_client as open_client:
+ hive_table = open_client.get_table(*TABLE_NAME)
+ assert hive_table.parameters.get("abc") == "def"
+ assert hive_table.parameters.get("p1") == "123"
+ assert hive_table.parameters.get("not_exist_parameter") is None
+
+ table.transaction().remove_properties("abc").commit_transaction()
+
+ with hive_client as open_client:
+ hive_table = open_client.get_table(*TABLE_NAME)
+ assert hive_table.parameters.get("abc") is None
+
+
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [pytest.lazy_fixture("session_catalog_hive"), pytest.lazy_fixture("session_catalog")])
def test_table_properties_dict(catalog: Catalog) -> None:
From e0ceff1bf38b169a4bcf2018c12db251a6a70bcf Mon Sep 17 00:00:00 2001
From: Kevin Liu
Date: Sat, 14 Jun 2025 14:16:30 -0400
Subject: [PATCH 061/112] change catalog default warehouse location to not use
hive-style warehouse location (#2059)
Closes #2052
# Rationale for this change
Aligns catalog behavior with the java reference implementation.
[HiveCatalog](https://github.com/apache/iceberg/blob/7537c3c3a2a6491abcf0c3ef58cc4d5dc6ac4bae/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java#L697-L736),
[DynamoDbCatalog](https://github.com/apache/iceberg/blob/7537c3c3a2a6491abcf0c3ef58cc4d5dc6ac4bae/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java#L185),
and
[GlueCatalog](https://github.com/apache/iceberg/blob/7537c3c3a2a6491abcf0c3ef58cc4d5dc6ac4bae/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java#L291)
all use `.db` suffix in warehouse location
[JdbcCatalog](https://github.com/apache/iceberg/blob/7537c3c3a2a6491abcf0c3ef58cc4d5dc6ac4bae/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java#L268),
[HadoopCatalog](https://github.com/apache/iceberg/blob/7537c3c3a2a6491abcf0c3ef58cc4d5dc6ac4bae/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java#L233-L245),
and
[InMemoryCatalog](https://github.com/apache/iceberg/blob/7537c3c3a2a6491abcf0c3ef58cc4d5dc6ac4bae/core/src/main/java/org/apache/iceberg/inmemory/InMemoryCatalog.java#L106-L117)
do not use `.db` suffix in warehouse location
# Are these changes tested?
Yes tests for sql catalog are modified to remove `.db`
# Are there any user-facing changes?
---
pyiceberg/catalog/__init__.py | 14 ++++++++++++++
pyiceberg/catalog/dynamodb.py | 4 ++++
pyiceberg/catalog/glue.py | 4 ++++
pyiceberg/catalog/hive.py | 4 ++++
tests/catalog/test_sql.py | 30 +++++++++++++++---------------
tests/cli/test_console.py | 4 ++--
6 files changed, 43 insertions(+), 17 deletions(-)
diff --git a/pyiceberg/catalog/__init__.py b/pyiceberg/catalog/__init__.py
index d7d4c9d050..81d3a34eaa 100644
--- a/pyiceberg/catalog/__init__.py
+++ b/pyiceberg/catalog/__init__.py
@@ -927,6 +927,20 @@ def _resolve_table_location(self, location: Optional[str], database_name: str, t
return location.rstrip("/")
def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+ """Return the default warehouse location using the convention of `warehousePath/databaseName/tableName`."""
+ database_properties = self.load_namespace_properties(database_name)
+ if database_location := database_properties.get(LOCATION):
+ database_location = database_location.rstrip("/")
+ return f"{database_location}/{table_name}"
+
+ if warehouse_path := self.properties.get(WAREHOUSE_LOCATION):
+ warehouse_path = warehouse_path.rstrip("/")
+ return f"{warehouse_path}/{database_name}/{table_name}"
+
+ raise ValueError("No default path is set, please specify a location when creating a table")
+
+ def _get_hive_style_warehouse_location(self, database_name: str, table_name: str) -> str:
+ """Return the default warehouse location following the Hive convention of `warehousePath/databaseName.db/tableName`."""
database_properties = self.load_namespace_properties(database_name)
if database_location := database_properties.get(LOCATION):
database_location = database_location.rstrip("/")
diff --git a/pyiceberg/catalog/dynamodb.py b/pyiceberg/catalog/dynamodb.py
index 29df2582a7..3b37762638 100644
--- a/pyiceberg/catalog/dynamodb.py
+++ b/pyiceberg/catalog/dynamodb.py
@@ -664,6 +664,10 @@ def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[st
catalog=self,
)
+ def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+ """Override the default warehouse location to follow Hive-style conventions."""
+ return self._get_hive_style_warehouse_location(database_name, table_name)
+
def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
current_timestamp_ms = str(round(time() * 1000))
diff --git a/pyiceberg/catalog/glue.py b/pyiceberg/catalog/glue.py
index 25e7d3f73f..88ad8aa433 100644
--- a/pyiceberg/catalog/glue.py
+++ b/pyiceberg/catalog/glue.py
@@ -821,3 +821,7 @@ def view_exists(self, identifier: Union[str, Identifier]) -> bool:
@staticmethod
def __is_iceberg_table(table: "TableTypeDef") -> bool:
return table.get("Parameters", {}).get(TABLE_TYPE, "").lower() == ICEBERG
+
+ def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+ """Override the default warehouse location to follow Hive-style conventions."""
+ return self._get_hive_style_warehouse_location(database_name, table_name)
diff --git a/pyiceberg/catalog/hive.py b/pyiceberg/catalog/hive.py
index 0fd6b9d79e..54e5100e95 100644
--- a/pyiceberg/catalog/hive.py
+++ b/pyiceberg/catalog/hive.py
@@ -798,3 +798,7 @@ def update_namespace_properties(
def drop_view(self, identifier: Union[str, Identifier]) -> None:
raise NotImplementedError
+
+ def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+ """Override the default warehouse location to follow Hive-style conventions."""
+ return self._get_hive_style_warehouse_location(database_name, table_name)
diff --git a/tests/catalog/test_sql.py b/tests/catalog/test_sql.py
index 8c3047b2ca..235951484f 100644
--- a/tests/catalog/test_sql.py
+++ b/tests/catalog/test_sql.py
@@ -72,7 +72,7 @@ def catalog_name() -> str:
@pytest.fixture(name="random_table_identifier")
def fixture_random_table_identifier(warehouse: Path, database_name: str, table_name: str) -> Identifier:
- os.makedirs(f"{warehouse}/{database_name}.db/{table_name}/metadata/", exist_ok=True)
+ os.makedirs(f"{warehouse}/{database_name}/{table_name}/metadata/", exist_ok=True)
return database_name, table_name
@@ -80,13 +80,13 @@ def fixture_random_table_identifier(warehouse: Path, database_name: str, table_n
def fixture_another_random_table_identifier(warehouse: Path, database_name: str, table_name: str) -> Identifier:
database_name = database_name + "_new"
table_name = table_name + "_new"
- os.makedirs(f"{warehouse}/{database_name}.db/{table_name}/metadata/", exist_ok=True)
+ os.makedirs(f"{warehouse}/{database_name}/{table_name}/metadata/", exist_ok=True)
return database_name, table_name
@pytest.fixture(name="random_hierarchical_identifier")
def fixture_random_hierarchical_identifier(warehouse: Path, hierarchical_namespace_name: str, table_name: str) -> Identifier:
- os.makedirs(f"{warehouse}/{hierarchical_namespace_name}.db/{table_name}/metadata/", exist_ok=True)
+ os.makedirs(f"{warehouse}/{hierarchical_namespace_name}/{table_name}/metadata/", exist_ok=True)
return Catalog.identifier_to_tuple(".".join((hierarchical_namespace_name, table_name)))
@@ -96,7 +96,7 @@ def fixture_another_random_hierarchical_identifier(
) -> Identifier:
hierarchical_namespace_name = hierarchical_namespace_name + "_new"
table_name = table_name + "_new"
- os.makedirs(f"{warehouse}/{hierarchical_namespace_name}.db/{table_name}/metadata/", exist_ok=True)
+ os.makedirs(f"{warehouse}/{hierarchical_namespace_name}/{table_name}/metadata/", exist_ok=True)
return Catalog.identifier_to_tuple(".".join((hierarchical_namespace_name, table_name)))
@@ -115,7 +115,7 @@ def catalog_memory(catalog_name: str, warehouse: Path) -> Generator[SqlCatalog,
@pytest.fixture(scope="module")
def catalog_sqlite(catalog_name: str, warehouse: Path) -> Generator[SqlCatalog, None, None]:
props = {
- "uri": f"sqlite:////{warehouse}/sql-catalog.db",
+ "uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
}
catalog = SqlCatalog(catalog_name, **props)
@@ -126,7 +126,7 @@ def catalog_sqlite(catalog_name: str, warehouse: Path) -> Generator[SqlCatalog,
@pytest.fixture(scope="module")
def catalog_uri(warehouse: Path) -> str:
- return f"sqlite:////{warehouse}/sql-catalog.db"
+ return f"sqlite:////{warehouse}/sql-catalog"
@pytest.fixture(scope="module")
@@ -137,7 +137,7 @@ def alchemy_engine(catalog_uri: str) -> Engine:
@pytest.fixture(scope="module")
def catalog_sqlite_without_rowcount(catalog_name: str, warehouse: Path) -> Generator[SqlCatalog, None, None]:
props = {
- "uri": f"sqlite:////{warehouse}/sql-catalog.db",
+ "uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
}
catalog = SqlCatalog(catalog_name, **props)
@@ -150,7 +150,7 @@ def catalog_sqlite_without_rowcount(catalog_name: str, warehouse: Path) -> Gener
@pytest.fixture(scope="module")
def catalog_sqlite_fsspec(catalog_name: str, warehouse: Path) -> Generator[SqlCatalog, None, None]:
props = {
- "uri": f"sqlite:////{warehouse}/sql-catalog.db",
+ "uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
PY_IO_IMPL: FSSPEC_FILE_IO,
}
@@ -176,7 +176,7 @@ def test_creation_with_echo_parameter(catalog_name: str, warehouse: Path) -> Non
for echo_param, expected_echo_value in test_cases:
props = {
- "uri": f"sqlite:////{warehouse}/sql-catalog.db",
+ "uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
}
# None is for default value
@@ -199,7 +199,7 @@ def test_creation_with_pool_pre_ping_parameter(catalog_name: str, warehouse: Pat
for pool_pre_ping_param, expected_pool_pre_ping_value in test_cases:
props = {
- "uri": f"sqlite:////{warehouse}/sql-catalog.db",
+ "uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
}
# None is for default value
@@ -219,7 +219,7 @@ def test_creation_from_impl(catalog_name: str, warehouse: Path) -> None:
catalog_name,
**{
"py-catalog-impl": "pyiceberg.catalog.sql.SqlCatalog",
- "uri": f"sqlite:////{warehouse}/sql-catalog.db",
+ "uri": f"sqlite:////{warehouse}/sql-catalog",
"warehouse": f"file://{warehouse}",
},
),
@@ -493,7 +493,7 @@ def test_create_table_with_given_location_removes_trailing_slash(
identifier_tuple = Catalog.identifier_to_tuple(table_identifier)
namespace = Catalog.namespace_from(table_identifier)
table_name = Catalog.table_name_from(identifier_tuple)
- location = f"file://{warehouse}/{catalog.name}.db/{table_name}-given"
+ location = f"file://{warehouse}/{catalog.name}/{table_name}-given"
catalog.create_namespace(namespace)
catalog.create_table(table_identifier, table_schema_nested, location=f"{location}/")
table = catalog.load_table(table_identifier)
@@ -1235,7 +1235,7 @@ def test_load_namespace_properties(catalog: SqlCatalog, namespace: str) -> None:
warehouse_location = "/test/location"
test_properties = {
"comment": "this is a test description",
- "location": f"{warehouse_location}/{namespace}.db",
+ "location": f"{warehouse_location}/{namespace}",
"test_property1": "1",
"test_property2": "2",
"test_property3": "3",
@@ -1286,7 +1286,7 @@ def test_update_namespace_properties(catalog: SqlCatalog, namespace: str) -> Non
warehouse_location = "/test/location"
test_properties = {
"comment": "this is a test description",
- "location": f"{warehouse_location}/{namespace}.db",
+ "location": f"{warehouse_location}/{namespace}",
"test_property1": "1",
"test_property2": "2",
"test_property3": "3",
@@ -1306,7 +1306,7 @@ def test_update_namespace_properties(catalog: SqlCatalog, namespace: str) -> Non
"comment": "updated test description",
"test_property4": "4",
"test_property5": "5",
- "location": f"{warehouse_location}/{namespace}.db",
+ "location": f"{warehouse_location}/{namespace}",
}
diff --git a/tests/cli/test_console.py b/tests/cli/test_console.py
index 70e04071ad..fe375eb276 100644
--- a/tests/cli/test_console.py
+++ b/tests/cli/test_console.py
@@ -271,7 +271,7 @@ def test_location(catalog: InMemoryCatalog) -> None:
runner = CliRunner()
result = runner.invoke(run, ["location", "default.my_table"])
assert result.exit_code == 0
- assert result.output == f"""{catalog._warehouse_location}/default.db/my_table\n"""
+ assert result.output == f"""{catalog._warehouse_location}/default/my_table\n"""
def test_location_does_not_exists(catalog: InMemoryCatalog) -> None:
@@ -700,7 +700,7 @@ def test_json_location(catalog: InMemoryCatalog) -> None:
runner = CliRunner()
result = runner.invoke(run, ["--output=json", "location", "default.my_table"])
assert result.exit_code == 0
- assert result.output == f'"{catalog._warehouse_location}/default.db/my_table"\n'
+ assert result.output == f'"{catalog._warehouse_location}/default/my_table"\n'
def test_json_location_does_not_exists(catalog: InMemoryCatalog) -> None:
From 9e88d26f874a98246ee8141aed4baf70b3fa930b Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sun, 15 Jun 2025 00:09:52 +0200
Subject: [PATCH 062/112] Bump PyIceberg to 0.9.1 (#2092)
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
dev/Dockerfile | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/dev/Dockerfile b/dev/Dockerfile
index b5b72e5617..9c542d9cd1 100644
--- a/dev/Dockerfile
+++ b/dev/Dockerfile
@@ -40,7 +40,7 @@ WORKDIR ${SPARK_HOME}
ENV SPARK_VERSION=3.5.4
ENV ICEBERG_SPARK_RUNTIME_VERSION=3.5_2.12
ENV ICEBERG_VERSION=1.9.0
-ENV PYICEBERG_VERSION=0.9.0
+ENV PYICEBERG_VERSION=0.9.1
RUN curl --retry 5 -s -C - https://archive.apache.org/dist/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop3.tgz -o spark-${SPARK_VERSION}-bin-hadoop3.tgz \
&& tar xzf spark-${SPARK_VERSION}-bin-hadoop3.tgz --directory /opt/spark --strip-components 1 \
From 875768fffe56862c9bff5ed242d9224ed81c3f56 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sun, 15 Jun 2025 00:11:22 +0200
Subject: [PATCH 063/112] Bump tests to Spark 3.5.6 (#2091)
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
dev/Dockerfile | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/dev/Dockerfile b/dev/Dockerfile
index 9c542d9cd1..c96130e597 100644
--- a/dev/Dockerfile
+++ b/dev/Dockerfile
@@ -37,7 +37,7 @@ RUN mkdir -p ${HADOOP_HOME} && mkdir -p ${SPARK_HOME} && mkdir -p /home/iceberg/
WORKDIR ${SPARK_HOME}
# Remember to also update `tests/conftest`'s spark setting
-ENV SPARK_VERSION=3.5.4
+ENV SPARK_VERSION=3.5.6
ENV ICEBERG_SPARK_RUNTIME_VERSION=3.5_2.12
ENV ICEBERG_VERSION=1.9.0
ENV PYICEBERG_VERSION=0.9.1
From 10ed70708c0ea1acd1e8549bc07fb8dffbc598c7 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sun, 15 Jun 2025 01:57:26 +0200
Subject: [PATCH 064/112] Bump Iceberg-Java to 1.9.1 (#2093)
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
Co-authored-by: Kevin Liu
---
dev/Dockerfile | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/dev/Dockerfile b/dev/Dockerfile
index c96130e597..c6ce71b2a2 100644
--- a/dev/Dockerfile
+++ b/dev/Dockerfile
@@ -39,7 +39,7 @@ WORKDIR ${SPARK_HOME}
# Remember to also update `tests/conftest`'s spark setting
ENV SPARK_VERSION=3.5.6
ENV ICEBERG_SPARK_RUNTIME_VERSION=3.5_2.12
-ENV ICEBERG_VERSION=1.9.0
+ENV ICEBERG_VERSION=1.9.1
ENV PYICEBERG_VERSION=0.9.1
RUN curl --retry 5 -s -C - https://archive.apache.org/dist/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop3.tgz -o spark-${SPARK_VERSION}-bin-hadoop3.tgz \
From e05c5f37db505b400258b4233df1986b91d72ae1 Mon Sep 17 00:00:00 2001
From: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
Date: Sun, 15 Jun 2025 16:58:48 -0400
Subject: [PATCH 065/112] maint: use `URI` constant instead of 'uri' strings
(#2094)
---
pyiceberg/catalog/__init__.py | 2 +-
pyiceberg/catalog/hive.py | 5 +++--
pyiceberg/catalog/memory.py | 5 +++--
pyiceberg/catalog/sql.py | 3 ++-
pyiceberg/cli/console.py | 4 ++--
pyiceberg/io/fsspec.py | 4 ++--
6 files changed, 13 insertions(+), 10 deletions(-)
diff --git a/pyiceberg/catalog/__init__.py b/pyiceberg/catalog/__init__.py
index 81d3a34eaa..4da116434e 100644
--- a/pyiceberg/catalog/__init__.py
+++ b/pyiceberg/catalog/__init__.py
@@ -195,7 +195,7 @@ def infer_catalog_type(name: str, catalog_properties: RecursiveDict) -> Optional
Raises:
ValueError: Raises a ValueError in case properties are missing, or the wrong type.
"""
- if uri := catalog_properties.get("uri"):
+ if uri := catalog_properties.get(URI):
if isinstance(uri, str):
if uri.startswith("http"):
return CatalogType.REST
diff --git a/pyiceberg/catalog/hive.py b/pyiceberg/catalog/hive.py
index 54e5100e95..05dd408fc2 100644
--- a/pyiceberg/catalog/hive.py
+++ b/pyiceberg/catalog/hive.py
@@ -63,6 +63,7 @@
LOCATION,
METADATA_LOCATION,
TABLE_TYPE,
+ URI,
MetastoreCatalog,
PropertiesUpdateSummary,
)
@@ -307,7 +308,7 @@ def __init__(self, name: str, **properties: str):
@staticmethod
def _create_hive_client(properties: Dict[str, str]) -> _HiveClient:
last_exception = None
- for uri in properties["uri"].split(","):
+ for uri in properties[URI].split(","):
try:
return _HiveClient(
uri,
@@ -319,7 +320,7 @@ def _create_hive_client(properties: Dict[str, str]) -> _HiveClient:
if last_exception is not None:
raise last_exception
else:
- raise ValueError(f"Unable to connect to hive using uri: {properties['uri']}")
+ raise ValueError(f"Unable to connect to hive using uri: {properties[URI]}")
def _convert_hive_into_iceberg(self, table: HiveTable) -> Table:
properties: Dict[str, str] = table.parameters
diff --git a/pyiceberg/catalog/memory.py b/pyiceberg/catalog/memory.py
index 7d6053baaf..024d14fba6 100644
--- a/pyiceberg/catalog/memory.py
+++ b/pyiceberg/catalog/memory.py
@@ -15,6 +15,7 @@
# specific language governing permissions and limitations
# under the License.
+from pyiceberg.catalog import URI
from pyiceberg.catalog.sql import SqlCatalog
@@ -27,6 +28,6 @@ class InMemoryCatalog(SqlCatalog):
def __init__(self, name: str, warehouse: str = "file:///tmp/iceberg/warehouse", **kwargs: str) -> None:
self._warehouse_location = warehouse
- if "uri" not in kwargs:
- kwargs["uri"] = "sqlite:///:memory:"
+ if URI not in kwargs:
+ kwargs[URI] = "sqlite:///:memory:"
super().__init__(name=name, warehouse=warehouse, **kwargs)
diff --git a/pyiceberg/catalog/sql.py b/pyiceberg/catalog/sql.py
index b4b06e3a46..880a4db481 100644
--- a/pyiceberg/catalog/sql.py
+++ b/pyiceberg/catalog/sql.py
@@ -44,6 +44,7 @@
from pyiceberg.catalog import (
METADATA_LOCATION,
+ URI,
Catalog,
MetastoreCatalog,
PropertiesUpdateSummary,
@@ -119,7 +120,7 @@ class SqlCatalog(MetastoreCatalog):
def __init__(self, name: str, **properties: str):
super().__init__(name, **properties)
- if not (uri_prop := self.properties.get("uri")):
+ if not (uri_prop := self.properties.get(URI)):
raise NoSuchPropertyException("SQL connection URI is required")
echo_str = str(self.properties.get("echo", DEFAULT_ECHO_VALUE)).lower()
diff --git a/pyiceberg/cli/console.py b/pyiceberg/cli/console.py
index 25a536d2a6..6be4df12cc 100644
--- a/pyiceberg/cli/console.py
+++ b/pyiceberg/cli/console.py
@@ -29,7 +29,7 @@
from click import Context
from pyiceberg import __version__
-from pyiceberg.catalog import Catalog, load_catalog
+from pyiceberg.catalog import URI, Catalog, load_catalog
from pyiceberg.cli.output import ConsoleOutput, JsonOutput, Output
from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchPropertyException, NoSuchTableError
from pyiceberg.table import TableProperties
@@ -75,7 +75,7 @@ def run(
if ugi:
properties["ugi"] = ugi
if uri:
- properties["uri"] = uri
+ properties[URI] = uri
if credential:
properties["credential"] = credential
diff --git a/pyiceberg/io/fsspec.py b/pyiceberg/io/fsspec.py
index cc80725d14..6febff0ae6 100644
--- a/pyiceberg/io/fsspec.py
+++ b/pyiceberg/io/fsspec.py
@@ -36,7 +36,7 @@
from fsspec.implementations.local import LocalFileSystem
from requests import HTTPError
-from pyiceberg.catalog import TOKEN
+from pyiceberg.catalog import TOKEN, URI
from pyiceberg.exceptions import SignError
from pyiceberg.io import (
ADLS_ACCOUNT_HOST,
@@ -91,7 +91,7 @@
def s3v4_rest_signer(properties: Properties, request: "AWSRequest", **_: Any) -> "AWSRequest":
- signer_url = properties.get(S3_SIGNER_URI, properties["uri"]).rstrip("/") # type: ignore
+ signer_url = properties.get(S3_SIGNER_URI, properties[URI]).rstrip("/") # type: ignore
signer_endpoint = properties.get(S3_SIGNER_ENDPOINT, S3_SIGNER_ENDPOINT_DEFAULT)
signer_headers = {}
From 0e4f84982ebe883447b1bf382fae2f85233cb3f8 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:42:09 -0400
Subject: [PATCH 066/112] Build: Bump pypa/cibuildwheel from 2.23.3 to 3.0.0
(#2099)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [pypa/cibuildwheel](https://github.com/pypa/cibuildwheel) from
2.23.3 to 3.0.0.
Release notes
Sourced from pypa/cibuildwheel's
releases.
v3.0.0
See @henryiii's release post
for more info on new features!
-
🌟 Adds the ability to build
wheels for iOS! Set the platform
option to ios on a Mac with the iOS toolchain to try it
out! (#2286,
#2363,
#2432)
-
🌟 Adds support for the GraalPy interpreter! Enable for your project
using the enable
option. (#1538,
#2411,
#2414)
-
✨ Adds CPython 3.14 support, under the enable
option cpython-prerelease. This version of cibuildwheel
uses 3.14.0b2. (#2390)
While CPython is in beta, the ABI can change, so your wheels
might not be compatible with the final release. For this reason, we
don't recommend distributing wheels until RC1, at which point 3.14 will
be available in cibuildwheel without the flag. (#2390)
-
✨ Adds the test-sources
option, and changes the working directory for tests. (#2062,
#2284,
#2437)
- If this option is set, cibuildwheel will copy the files and folders
specified in
test-sources into the temporary directory we
run from. This is required for iOS builds, but also useful for other
platforms, as it allows you to avoid placeholders.
- If this option is not set, behaviour matches v2.x - cibuildwheel
will run the tests from a temporary directory, and you can use the
{project} placeholder in the test-command to
refer to the project directory. (#2420)
-
✨ Adds dependency-versions
inline syntax (#2122)
-
✨ Improves support for Pyodide builds and adds the experimental pyodide-version
option, which allows you to specify the version of Pyodide to use for
builds. (#2002)
-
✨ Add pyodide-prerelease enable
option, with an early build of 0.28 (Python 3.13). (#2431)
-
✨ Adds the test-environment
option, which allows you to set environment variables for the test
command. (#2388)
-
✨ Adds the xbuild-tools
option, which allows you to specify tools safe for cross-compilation.
Currently only used on iOS; will be useful for Android in the future.
(#2317)
-
🛠 The default manylinux
image has changed from manylinux2014 to
manylinux_2_28. (#2330)
-
🛠 EOL images manylinux1, manylinux2010,
manylinux_2_24 and musllinux_1_1 can no longer
be specified by their shortname. The full OCI name can still be used for
these images, if you wish. (#2316)
-
🛠 Invokes build rather than pip wheel to
build wheels by default. You can control this via the build-frontend
option. You might notice that you can see your build log output now! (#2321)
-
🛠 Build verbosity settings have been reworked to have consistent
meanings between build backends when non-zero. (#2339)
-
🛠 Removed the CIBW_PRERELEASE_PYTHONS and
CIBW_FREE_THREADED_SUPPORT options - these have been folded
into the enable
option instead. (#2095)
-
🛠 Build environments no longer have setuptools and wheel
preinstalled. (#2329)
-
🛠 Use the standard Schema line for the integrated JSONSchema. (#2433)
-
⚠️ Dropped support for building Python 3.6 and 3.7 wheels. If you
need to build wheels for these versions, use cibuildwheel v2.23.3 or
earlier. (#2282)
-
⚠️ The minimum Python version required to run cibuildwheel is now
Python 3.11. You can still build wheels for Python 3.8 and newer. (#1912)
-
⚠️ 32-bit Linux wheels no longer built by default - the arch
was removed from "auto". It now requires explicit
"auto32". Note that modern manylinux images (like
the new default, manylinux_2_28) do not have 32-bit
versions. (#2458)
-
⚠️ PyPy wheels no longer built by default, due to a change to our
options system. To continue building PyPy wheels, you'll now need to set
the enable
option to pypy or pypy-eol. (#2095)
-
⚠️ Dropped official support for Appveyor. If it was working for you
before, it will probably continue to do so, but we can't be sure,
because our CI doesn't run there anymore. (#2386)
-
📚 A reorganisation of the docs, and numerous updates. (#2280)
-
📚 Use Python 3.14 color output in docs CLI output. (#2407)
-
📚 Docs now primarily use the pyproject.toml name of options, rather
than the environment variable name. (#2389)
-
📚 README table now matches docs and auto-updates. (#2427,
#2428)
v3.0.0rc3
Not yet released, but available for testing.
Note - when using a beta version, be sure to check the latest docs, rather
than the stable version, which is still on v2.X.
If you've used previous versions of the beta:
- ⚠️ Previous betas of v3.0 changed the working directory for tests.
This has been rolled back to the v2.x behaviour, so you might need to
change configs if you adapted to the beta 1 or 2 behaviour. See [issue
#2406](pypa/cibuildwheel#2406)
for more information.
- ⚠️ GraalPy shipped with the identifier
gp242-* in
previous betas, this has been changed to gp311_242-* to be
consistent with other interpreters, and to fix a bug with GraalPy and
project requires-python detection. If you were using GraalPy, you might
need to update your config to use the new identifier.
- ⚠️
test-sources now uses project directory
instead of the package directory (matching the docs).
- ⚠️ 32-bit linux builds were removed from
"auto" (the default), now require
"auto32" or explicit archs, as modern manylinux
images (including our new default) do not support them.
... (truncated)
Changelog
Sourced from pypa/cibuildwheel's
changelog.
v3.0.0
11 June 2025
See @henryiii's release post
for more info on new features!
-
🌟 Adds the ability to build
wheels for iOS! Set the platform
option to ios on a Mac with the iOS toolchain to try it
out! (#2286,
#2363,
#2432)
-
🌟 Adds support for the GraalPy interpreter! Enable for your project
using the enable
option. (#1538,
#2411,
#2414)
-
✨ Adds CPython 3.14 support, under the enable
option cpython-prerelease. This version of cibuildwheel
uses 3.14.0b2. (#2390)
While CPython is in beta, the ABI can change, so your wheels
might not be compatible with the final release. For this reason, we
don't recommend distributing wheels until RC1, at which point 3.14 will
be available in cibuildwheel without the flag. (#2390)
-
✨ Adds the test-sources
option, and changes the working directory for tests. (#2062,
#2284,
#2437)
- If this option is set, cibuildwheel will copy the files and folders
specified in
test-sources into the temporary directory we
run from. This is required for iOS builds, but also useful for other
platforms, as it allows you to avoid placeholders.
- If this option is not set, behaviour matches v2.x - cibuildwheel
will run the tests from a temporary directory, and you can use the
{project} placeholder in the test-command to
refer to the project directory. (#2420)
-
✨ Adds dependency-versions
inline syntax (#2122)
-
✨ Improves support for Pyodide builds and adds the experimental pyodide-version
option, which allows you to specify the version of Pyodide to use for
builds. (#2002)
-
✨ Add pyodide-prerelease enable
option, with an early build of 0.28 (Python 3.13). (#2431)
-
✨ Adds the test-environment
option, which allows you to set environment variables for the test
command. (#2388)
-
✨ Adds the xbuild-tools
option, which allows you to specify tools safe for cross-compilation.
Currently only used on iOS; will be useful for Android in the future.
(#2317)
-
🛠 The default manylinux
image has changed from manylinux2014 to
manylinux_2_28. (#2330)
-
🛠 EOL images manylinux1, manylinux2010,
manylinux_2_24 and musllinux_1_1 can no longer
be specified by their shortname. The full OCI name can still be used for
these images, if you wish. (#2316)
-
🛠 Invokes build rather than pip wheel to
build wheels by default. You can control this via the build-frontend
option. You might notice that you can see your build log output now! (#2321)
-
🛠 Build verbosity settings have been reworked to have consistent
meanings between build backends when non-zero. (#2339)
-
🛠 Removed the CIBW_PRERELEASE_PYTHONS and
CIBW_FREE_THREADED_SUPPORT options - these have been folded
into the enable
option instead. (#2095)
-
🛠 Build environments no longer have setuptools and wheel
preinstalled. (#2329)
-
🛠 Use the standard Schema line for the integrated JSONSchema. (#2433)
-
⚠️ Dropped support for building Python 3.6 and 3.7 wheels. If you
need to build wheels for these versions, use cibuildwheel v2.23.3 or
earlier. (#2282)
-
⚠️ The minimum Python version required to run cibuildwheel is now
Python 3.11. You can still build wheels for Python 3.8 and newer. (#1912)
-
⚠️ 32-bit Linux wheels no longer built by default - the arch
was removed from "auto". It now requires explicit
"auto32". Note that modern manylinux images (like
the new default, manylinux_2_28) do not have 32-bit
versions. (#2458)
-
⚠️ PyPy wheels no longer built by default, due to a change to our
options system. To continue building PyPy wheels, you'll now need to set
the enable
option to pypy or pypy-eol. (#2095)
-
⚠️ Dropped official support for Appveyor. If it was working for you
before, it will probably continue to do so, but we can't be sure,
because our CI doesn't run there anymore. (#2386)
-
📚 A reorganisation of the docs, and numerous updates. (#2280)
-
📚 Use Python 3.14 color output in docs CLI output. (#2407)
-
📚 Docs now primarily use the pyproject.toml name of options, rather
than the environment variable name. (#2389)
-
📚 README table now matches docs and auto-updates. (#2427,
#2428)
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
.github/workflows/pypi-build-artifacts.yml | 2 +-
.github/workflows/svn-build-artifacts.yml | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/.github/workflows/pypi-build-artifacts.yml b/.github/workflows/pypi-build-artifacts.yml
index c0430daa08..6c6c108475 100644
--- a/.github/workflows/pypi-build-artifacts.yml
+++ b/.github/workflows/pypi-build-artifacts.yml
@@ -62,7 +62,7 @@ jobs:
if: startsWith(matrix.os, 'ubuntu')
- name: Build wheels
- uses: pypa/cibuildwheel@v2.23.3
+ uses: pypa/cibuildwheel@v3.0.0
with:
output-dir: wheelhouse
config-file: "pyproject.toml"
diff --git a/.github/workflows/svn-build-artifacts.yml b/.github/workflows/svn-build-artifacts.yml
index dd0be8cb7f..cfd9434133 100644
--- a/.github/workflows/svn-build-artifacts.yml
+++ b/.github/workflows/svn-build-artifacts.yml
@@ -57,7 +57,7 @@ jobs:
if: startsWith(matrix.os, 'ubuntu')
- name: Build wheels
- uses: pypa/cibuildwheel@v2.23.3
+ uses: pypa/cibuildwheel@v3.0.0
with:
output-dir: wheelhouse
config-file: "pyproject.toml"
From 23ebc71a310aed900238ef54366d426122683b08 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:42:37 -0400
Subject: [PATCH 067/112] Build: Bump moto from 5.1.5 to 5.1.6 (#2100)
Bumps [moto](https://github.com/getmoto/moto) from 5.1.5 to 5.1.6.
Changelog
Sourced from moto's
changelog.
5.1.6
Docker Digest for 5.1.6:
sha256:0afd091e9d26ba1660ad54708ff1e853ffa1f437f057c91163b47d35155a852b
New Methods:
* Athena:
* get_query_statistics()
* AppSync:
* create_api()
* create_channel_namespace()
* delete_api()
* delete_channel_namespace()
* get_api()
* list_apis()
* list_channel_namespaces()
* CloudDirectory:
* apply_schema()
* create_schema()
* delete_schema()
* list_development_schema_arns()
* list_published_schema_arns()
* publish_schema()
* DirectConnect:
* describe_tags()
* tag_resource()
* untag_resource()
* EC2:
* modify_security_group_rules()
* Glue:
* delete_resource_policy()
* SecurityHub:
* describe_organization_configuration()
* enable_organization_admin_account()
* get_administrator_account()
* update_organization_configuration()
* StepFunctions:
* create_activity()
* delete_activity()
* describe_activity()
* list_activities()
Miscellaneous:
* CodeDeploy: get_deployment_group() now returns the
deploymentGroupId-attribute
... (truncated)
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 53 ++++++++++++++++++-----------------------------------
1 file changed, 18 insertions(+), 35 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index bbd207d7bb..bd6d015cb6 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -58,7 +58,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -71,7 +71,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohttp-3.12.12-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6f25e9d274d6abbb15254f76f100c3984d6b9ad6e66263cc60a465dd5c7e48f5"},
{file = "aiohttp-3.12.12-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b8ec3c1a1c13d24941b5b913607e57b9364e4c0ea69d5363181467492c4b2ba6"},
@@ -201,7 +201,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -267,7 +267,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -284,7 +284,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -487,7 +487,7 @@ files = [
{file = "boto3-1.37.3-py3-none-any.whl", hash = "sha256:2063b40af99fd02f6228ff52397b552ff3353831edaf8d25cc04801827ab9794"},
{file = "boto3-1.37.3.tar.gz", hash = "sha256:21f3ce0ef111297e63a6eb998a25197b8c10982970c320d4c6e8db08be2157be"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.3,<1.38.0"
@@ -508,7 +508,7 @@ files = [
{file = "botocore-1.37.3-py3-none-any.whl", hash = "sha256:d01bd3bf4c80e61fa88d636ad9f5c9f60a551d71549b481386c6b4efe0bb2b2e"},
{file = "botocore-1.37.3.tar.gz", hash = "sha256:fe8403eb55a88faf9b0f9da6615e5bee7be056d75e17af66c3c8f0a3b0648da4"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -801,7 +801,6 @@ description = "Composable command line interface toolkit"
optional = false
python-versions = ">=3.7"
groups = ["main", "dev", "docs"]
-markers = "python_version < \"3.10\""
files = [
{file = "click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2"},
{file = "click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a"},
@@ -810,22 +809,6 @@ files = [
[package.dependencies]
colorama = {version = "*", markers = "platform_system == \"Windows\""}
-[[package]]
-name = "click"
-version = "8.2.1"
-description = "Composable command line interface toolkit"
-optional = false
-python-versions = ">=3.10"
-groups = ["main", "dev", "docs"]
-markers = "python_version >= \"3.10\""
-files = [
- {file = "click-8.2.1-py3-none-any.whl", hash = "sha256:61a3265b914e850b85317d0b3109c7f8cd35a670f963866005d6ef1d5175a12b"},
- {file = "click-8.2.1.tar.gz", hash = "sha256:27c491cc05d968d271d5a1db13e3b5a184636d9d930f148c50b038f0d0646202"},
-]
-
-[package.dependencies]
-colorama = {version = "*", markers = "platform_system == \"Windows\""}
-
[[package]]
name = "colorama"
version = "0.4.6"
@@ -1489,7 +1472,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cc4df77d638aa2ed703b878dd093725b72a824c3c546c076e8fdf276f78ee84a"},
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:716a9973a2cc963160394f701964fe25012600f3d311f60c790400b00e568b61"},
@@ -2263,7 +2246,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -2860,14 +2843,14 @@ type = ["mypy (==1.14.1)"]
[[package]]
name = "moto"
-version = "5.1.5"
+version = "5.1.6"
description = "A library that allows you to easily mock out tests based on AWS infrastructure"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "moto-5.1.5-py3-none-any.whl", hash = "sha256:866ae85eb5efe11a78f991127531878fd7f49177eb4a6680f47060430eb8932d"},
- {file = "moto-5.1.5.tar.gz", hash = "sha256:42b362ea9a16181e8e7b615ac212c294b882f020e9ae02f01230f167926df84e"},
+ {file = "moto-5.1.6-py3-none-any.whl", hash = "sha256:e4a3092bc8fe9139caa77cd34cdcbad804de4d9671e2270ea3b4d53f5c645047"},
+ {file = "moto-5.1.6.tar.gz", hash = "sha256:baf7afa9d4a92f07277b29cf466d0738f25db2ed2ee12afcb1dc3f2c540beebd"},
]
[package.dependencies]
@@ -3058,7 +3041,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "multidict-6.4.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:8adee3ac041145ffe4488ea73fa0a622b464cc25340d98be76924d0cda8545ff"},
{file = "multidict-6.4.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b61e98c3e2a861035aaccd207da585bdcacef65fe01d7a0d07478efac005e028"},
@@ -3784,7 +3767,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:22d9962a358aedbb7a2e36187ff273adeaab9743373a272976d2e348d08c7770"},
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0d0fda578d1dc3f77b6b5a5dce3b9ad69a8250a891760a548df850a5e8da87f3"},
@@ -4111,7 +4094,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
+markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -5240,7 +5223,7 @@ files = [
{file = "s3transfer-0.11.3-py3-none-any.whl", hash = "sha256:ca855bdeb885174b5ffa95b9913622459d4ad8e331fc98eb01e6d5eb6a30655d"},
{file = "s3transfer-0.11.3.tar.gz", hash = "sha256:edae4977e3a122445660c7c114bba949f9d191bae3b34a096f18a1c8c354527a"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.36.0,<2.0a.0"
@@ -5748,7 +5731,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"hf\" or extra == \"daft\""
+markers = "extra == \"daft\" or extra == \"hf\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -6033,7 +6016,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6032e6da6abd41e4acda34d75a816012717000fa6839f37124a47fcefc49bec4"},
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2c7b34d804b8cf9b214f05015c4fee2ebe7ed05cf581e7192c06555c71f4446a"},
From 26dac9d8841ff1fd004b3deb3fc78eb10682f117 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:42:58 -0400
Subject: [PATCH 068/112] Build: Bump cachetools from 5.5.2 to 6.1.0 (#2101)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [cachetools](https://github.com/tkem/cachetools) from 5.5.2 to
6.1.0.
Changelog
Sourced from cachetools's
changelog.
v6.1.0 (2025-06-16)
v6.0.0 (2025-05-23)
-
Require Python 3.9 or later (breaking change).
-
Remove MRUCache and the @func.mru_cache
decorator (breaking
change).
-
Add an optional condition parameter to the
@cached and
@cachedmethod decorators, which, when used with a
threading.Condition instance, should improve cache
stampede <https://en.wikipedia.org/wiki/Cache_stampede>_
issues in massively
parallel environments. Note that this will inflict some performance
penalty, and therefore has to be enabled explicitly.
-
Convert the cachetools.func decorators to use a
threading.Condition instance to deal with cache
stampede <https://en.wikipedia.org/wiki/Cache_stampede>_
issues. Note that
this may result in a noticable performance degradation,
depending
on your actual use case.
-
Deprecate support for cache(self) returning
None to suppress
caching with the @cachedmethod decorator.
-
Improve documentation.
-
Update CI environment.
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 186 ++++++++++++++--------------------------------------
1 file changed, 50 insertions(+), 136 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index bd6d015cb6..7268d20246 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -550,14 +550,14 @@ virtualenv = ["virtualenv (>=20.0.35)"]
[[package]]
name = "cachetools"
-version = "5.5.2"
+version = "6.1.0"
description = "Extensible memoizing collections and decorators"
optional = false
-python-versions = ">=3.7"
+python-versions = ">=3.9"
groups = ["main"]
files = [
- {file = "cachetools-5.5.2-py3-none-any.whl", hash = "sha256:d26a22bcc62eb95c3beabd9f1ee5e820d3d2704fe2967cbe350e20c8ffcd3f0a"},
- {file = "cachetools-5.5.2.tar.gz", hash = "sha256:1a661caa9175d26759571b2e19580f9d6393969e5dfca11fdb1f947a23e640d4"},
+ {file = "cachetools-6.1.0-py3-none-any.whl", hash = "sha256:1c7bb3cf9193deaf3508b7c5f2a79986c13ea38965c5adcff1f84519cf39163e"},
+ {file = "cachetools-6.1.0.tar.gz", hash = "sha256:b4c4f404392848db3ce7aac34950d17be4d864da4b8b66911008e430bc544587"},
]
[[package]]
@@ -1704,76 +1704,65 @@ dev = ["flake8", "markdown", "twine", "wheel"]
[[package]]
name = "google-api-core"
-version = "2.25.0"
+version = "1.16.0"
description = "Google API client core library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_api_core-2.25.0-py3-none-any.whl", hash = "sha256:1db79d1281dcf9f3d10023283299ba38f3dc9f639ec41085968fd23e5bcf512e"},
- {file = "google_api_core-2.25.0.tar.gz", hash = "sha256:9b548e688702f82a34ed8409fb8a6961166f0b7795032f0be8f48308dff4333a"},
+ {file = "google-api-core-1.16.0.tar.gz", hash = "sha256:92e962a087f1c4b8d1c5c88ade1c1dfd550047dcffb320c57ef6a534a20403e2"},
+ {file = "google_api_core-1.16.0-py2.py3-none-any.whl", hash = "sha256:859f7392676761f2b160c6ee030c3422135ada4458f0948c5690a6a7c8d86294"},
]
[package.dependencies]
-google-auth = ">=2.14.1,<3.0.0"
-googleapis-common-protos = ">=1.56.2,<2.0.0"
-proto-plus = [
- {version = ">=1.22.3,<2.0.0"},
- {version = ">=1.25.0,<2.0.0", markers = "python_version >= \"3.13\""},
-]
-protobuf = ">=3.19.5,<3.20.0 || >3.20.0,<3.20.1 || >3.20.1,<4.21.0 || >4.21.0,<4.21.1 || >4.21.1,<4.21.2 || >4.21.2,<4.21.3 || >4.21.3,<4.21.4 || >4.21.4,<4.21.5 || >4.21.5,<7.0.0"
-requests = ">=2.18.0,<3.0.0"
+google-auth = ">=0.4.0,<2.0dev"
+googleapis-common-protos = ">=1.6.0,<2.0dev"
+protobuf = ">=3.4.0"
+pytz = "*"
+requests = ">=2.18.0,<3.0.0dev"
+setuptools = ">=34.0.0"
+six = ">=1.10.0"
[package.extras]
-async-rest = ["google-auth[aiohttp] (>=2.35.0,<3.0.0)"]
-grpc = ["grpcio (>=1.33.2,<2.0.0)", "grpcio (>=1.49.1,<2.0.0) ; python_version >= \"3.11\"", "grpcio-status (>=1.33.2,<2.0.0)", "grpcio-status (>=1.49.1,<2.0.0) ; python_version >= \"3.11\""]
-grpcgcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
-grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
+grpc = ["grpcio (>=1.8.2,<2.0dev)"]
+grpcgcp = ["grpcio-gcp (>=0.2.2)"]
+grpcio-gcp = ["grpcio-gcp (>=0.2.2)"]
[[package]]
name = "google-auth"
-version = "2.40.3"
+version = "1.6.3"
description = "Google Authentication Library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_auth-2.40.3-py2.py3-none-any.whl", hash = "sha256:1370d4593e86213563547f97a92752fc658456fe4514c809544f330fed45a7ca"},
- {file = "google_auth-2.40.3.tar.gz", hash = "sha256:500c3a29adedeb36ea9cf24b8d10858e152f2412e3ca37829b3fa18e33d63b77"},
+ {file = "google-auth-1.6.3.tar.gz", hash = "sha256:0f7c6a64927d34c1a474da92cfc59e552a5d3b940d3266606c6a28b72888b9e4"},
+ {file = "google_auth-1.6.3-py2.py3-none-any.whl", hash = "sha256:20705f6803fd2c4d1cc2dcb0df09d4dfcb9a7d51fd59e94a3a28231fd93119ed"},
]
[package.dependencies]
-cachetools = ">=2.0.0,<6.0"
+cachetools = ">=2.0.0"
pyasn1-modules = ">=0.2.1"
-rsa = ">=3.1.4,<5"
-
-[package.extras]
-aiohttp = ["aiohttp (>=3.6.2,<4.0.0)", "requests (>=2.20.0,<3.0.0)"]
-enterprise-cert = ["cryptography", "pyopenssl"]
-pyjwt = ["cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "pyjwt (>=2.0)"]
-pyopenssl = ["cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "pyopenssl (>=20.0.0)"]
-reauth = ["pyu2f (>=0.1.5)"]
-requests = ["requests (>=2.20.0,<3.0.0)"]
-testing = ["aiohttp (<3.10.0)", "aiohttp (>=3.6.2,<4.0.0)", "aioresponses", "cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "flask", "freezegun", "grpcio", "mock", "oauth2client", "packaging", "pyjwt (>=2.0)", "pyopenssl (<24.3.0)", "pyopenssl (>=20.0.0)", "pytest", "pytest-asyncio", "pytest-cov", "pytest-localserver", "pyu2f (>=0.1.5)", "requests (>=2.20.0,<3.0.0)", "responses", "urllib3"]
-urllib3 = ["packaging", "urllib3"]
+rsa = ">=3.1.4"
+six = ">=1.9.0"
[[package]]
name = "google-auth-oauthlib"
-version = "1.2.2"
+version = "0.5.3"
description = "Google Authentication Library"
optional = true
python-versions = ">=3.6"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_auth_oauthlib-1.2.2-py3-none-any.whl", hash = "sha256:fd619506f4b3908b5df17b65f39ca8d66ea56986e5472eb5978fd8f3786f00a2"},
- {file = "google_auth_oauthlib-1.2.2.tar.gz", hash = "sha256:11046fb8d3348b296302dd939ace8af0a724042e8029c1b872d87fabc9f41684"},
+ {file = "google-auth-oauthlib-0.5.3.tar.gz", hash = "sha256:307d21918d61a0741882ad1fd001c67e68ad81206451d05fc4d26f79de56fc90"},
+ {file = "google_auth_oauthlib-0.5.3-py2.py3-none-any.whl", hash = "sha256:9e8ff4ed2b21c174a2d6cc2172c698dbf0b1f686509774c663a83c495091fe09"},
]
[package.dependencies]
-google-auth = ">=2.15.0"
+google-auth = ">=1.0.0"
requests-oauthlib = ">=0.7.0"
[package.extras]
@@ -1781,115 +1770,58 @@ tool = ["click (>=6.0.0)"]
[[package]]
name = "google-cloud-core"
-version = "2.4.3"
+version = "1.4.0"
description = "Google Cloud API client core library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_cloud_core-2.4.3-py2.py3-none-any.whl", hash = "sha256:5130f9f4c14b4fafdff75c79448f9495cfade0d8775facf1b09c3bf67e027f6e"},
- {file = "google_cloud_core-2.4.3.tar.gz", hash = "sha256:1fab62d7102844b278fe6dead3af32408b1df3eb06f5c7e8634cbd40edc4da53"},
+ {file = "google-cloud-core-1.4.0.tar.gz", hash = "sha256:07a024a26c4eb14ee3df7e6e5021c04f8f7e9f0e83d3d47863229f3635f871ce"},
+ {file = "google_cloud_core-1.4.0-py2.py3-none-any.whl", hash = "sha256:e84d4a05c58fc905b9340118e257cd043e9c02a6d31168182204afe4afd521eb"},
]
[package.dependencies]
-google-api-core = ">=1.31.6,<2.0.dev0 || >2.3.0,<3.0.0dev"
-google-auth = ">=1.25.0,<3.0dev"
+google-api-core = ">=1.16.0,<2.0.0dev"
[package.extras]
-grpc = ["grpcio (>=1.38.0,<2.0dev)", "grpcio-status (>=1.38.0,<2.0.dev0)"]
+grpc = ["grpcio (>=1.8.2,<2.0dev)"]
[[package]]
name = "google-cloud-storage"
-version = "3.1.0"
+version = "1.23.0"
description = "Google Cloud Storage API client library"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_cloud_storage-3.1.0-py2.py3-none-any.whl", hash = "sha256:eaf36966b68660a9633f03b067e4a10ce09f1377cae3ff9f2c699f69a81c66c6"},
- {file = "google_cloud_storage-3.1.0.tar.gz", hash = "sha256:944273179897c7c8a07ee15f2e6466a02da0c7c4b9ecceac2a26017cb2972049"},
+ {file = "google-cloud-storage-1.23.0.tar.gz", hash = "sha256:c66e876ae9547884fa42566a2ebfec51d280f488d7a058af9611ba90c78bed78"},
+ {file = "google_cloud_storage-1.23.0-py2.py3-none-any.whl", hash = "sha256:9f59c100d3940e38567c48d54cf1a2e7591a2f38e9693dfc11a242d5e54a1626"},
]
[package.dependencies]
-google-api-core = ">=2.15.0,<3.0.0dev"
-google-auth = ">=2.26.1,<3.0dev"
-google-cloud-core = ">=2.4.2,<3.0dev"
-google-crc32c = ">=1.0,<2.0dev"
-google-resumable-media = ">=2.7.2"
-requests = ">=2.18.0,<3.0.0dev"
-
-[package.extras]
-protobuf = ["protobuf (<6.0.0dev)"]
-tracing = ["opentelemetry-api (>=1.1.0)"]
-
-[[package]]
-name = "google-crc32c"
-version = "1.7.1"
-description = "A python wrapper of the C library 'Google CRC32C'"
-optional = true
-python-versions = ">=3.9"
-groups = ["main"]
-markers = "extra == \"gcsfs\""
-files = [
- {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:b07d48faf8292b4db7c3d64ab86f950c2e94e93a11fd47271c28ba458e4a0d76"},
- {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:7cc81b3a2fbd932a4313eb53cc7d9dde424088ca3a0337160f35d91826880c1d"},
- {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:1c67ca0a1f5b56162951a9dae987988679a7db682d6f97ce0f6381ebf0fbea4c"},
- {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc5319db92daa516b653600794d5b9f9439a9a121f3e162f94b0e1891c7933cb"},
- {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dcdf5a64adb747610140572ed18d011896e3b9ae5195f2514b7ff678c80f1603"},
- {file = "google_crc32c-1.7.1-cp310-cp310-win_amd64.whl", hash = "sha256:754561c6c66e89d55754106739e22fdaa93fafa8da7221b29c8b8e8270c6ec8a"},
- {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:6fbab4b935989e2c3610371963ba1b86afb09537fd0c633049be82afe153ac06"},
- {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:ed66cbe1ed9cbaaad9392b5259b3eba4a9e565420d734e6238813c428c3336c9"},
- {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ee6547b657621b6cbed3562ea7826c3e11cab01cd33b74e1f677690652883e77"},
- {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d68e17bad8f7dd9a49181a1f5a8f4b251c6dbc8cc96fb79f1d321dfd57d66f53"},
- {file = "google_crc32c-1.7.1-cp311-cp311-win_amd64.whl", hash = "sha256:6335de12921f06e1f774d0dd1fbea6bf610abe0887a1638f64d694013138be5d"},
- {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:2d73a68a653c57281401871dd4aeebbb6af3191dcac751a76ce430df4d403194"},
- {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:22beacf83baaf59f9d3ab2bbb4db0fb018da8e5aebdce07ef9f09fce8220285e"},
- {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:19eafa0e4af11b0a4eb3974483d55d2d77ad1911e6cf6f832e1574f6781fd337"},
- {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b6d86616faaea68101195c6bdc40c494e4d76f41e07a37ffdef270879c15fb65"},
- {file = "google_crc32c-1.7.1-cp312-cp312-win_amd64.whl", hash = "sha256:b7491bdc0c7564fcf48c0179d2048ab2f7c7ba36b84ccd3a3e1c3f7a72d3bba6"},
- {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:df8b38bdaf1629d62d51be8bdd04888f37c451564c2042d36e5812da9eff3c35"},
- {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:e42e20a83a29aa2709a0cf271c7f8aefaa23b7ab52e53b322585297bb94d4638"},
- {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:905a385140bf492ac300026717af339790921f411c0dfd9aa5a9e69a08ed32eb"},
- {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b211ddaf20f7ebeec5c333448582c224a7c90a9d98826fbab82c0ddc11348e6"},
- {file = "google_crc32c-1.7.1-cp313-cp313-win_amd64.whl", hash = "sha256:0f99eaa09a9a7e642a61e06742856eec8b19fc0037832e03f941fe7cf0c8e4db"},
- {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:32d1da0d74ec5634a05f53ef7df18fc646666a25efaaca9fc7dcfd4caf1d98c3"},
- {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e10554d4abc5238823112c2ad7e4560f96c7bf3820b202660373d769d9e6e4c9"},
- {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:9fc196f0b8d8bd2789352c6a522db03f89e83a0ed6b64315923c396d7a932315"},
- {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:bb5e35dcd8552f76eed9461a23de1030920a3c953c1982f324be8f97946e7127"},
- {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f2226b6a8da04f1d9e61d3e357f2460b9551c5e6950071437e122c958a18ae14"},
- {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1f2b3522222746fff0e04a9bd0a23ea003ba3cccc8cf21385c564deb1f223242"},
- {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3bda0fcb632d390e3ea8b6b07bf6b4f4a66c9d02dcd6fbf7ba00a197c143f582"},
- {file = "google_crc32c-1.7.1-cp39-cp39-win_amd64.whl", hash = "sha256:713121af19f1a617054c41f952294764e0c5443d5a5d9034b2cd60f5dd7e0349"},
- {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a8e9afc74168b0b2232fb32dd202c93e46b7d5e4bf03e66ba5dc273bb3559589"},
- {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa8136cc14dd27f34a3221c0f16fd42d8a40e4778273e61a3c19aedaa44daf6b"},
- {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:85fef7fae11494e747c9fd1359a527e5970fc9603c90764843caabd3a16a0a48"},
- {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6efb97eb4369d52593ad6f75e7e10d053cf00c48983f7a973105bc70b0ac4d82"},
- {file = "google_crc32c-1.7.1.tar.gz", hash = "sha256:2bff2305f98846f3e825dbeec9ee406f89da7962accdb29356e4eadc251bd472"},
-]
-
-[package.extras]
-testing = ["pytest"]
+google-auth = ">=1.2.0"
+google-cloud-core = ">=1.0.3,<2.0dev"
+google-resumable-media = ">=0.5.0,<0.6dev"
[[package]]
name = "google-resumable-media"
-version = "2.7.2"
+version = "0.5.1"
description = "Utilities for Google Media Downloads and Resumable Uploads"
optional = true
-python-versions = ">=3.7"
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google_resumable_media-2.7.2-py2.py3-none-any.whl", hash = "sha256:3ce7551e9fe6d99e9a126101d2536612bb73486721951e9562fee0f90c6ababa"},
- {file = "google_resumable_media-2.7.2.tar.gz", hash = "sha256:5280aed4629f2b60b847b0d42f9857fd4935c11af266744df33d8074cae92fe0"},
+ {file = "google-resumable-media-0.5.1.tar.gz", hash = "sha256:97155236971970382b738921f978a6f86a7b5a0b0311703d991e065d3cb55773"},
+ {file = "google_resumable_media-0.5.1-py2.py3-none-any.whl", hash = "sha256:cdc64378dc9a7a7bf963a8d0c944c99b549dc0c195a9acbf1fcd465f380b9002"},
]
[package.dependencies]
-google-crc32c = ">=1.0,<2.0dev"
+six = "*"
[package.extras]
-aiohttp = ["aiohttp (>=3.6.2,<4.0.0dev)", "google-auth (>=1.22.0,<2.0dev)"]
requests = ["requests (>=2.18.0,<3.0.0dev)"]
[[package]]
@@ -3869,25 +3801,6 @@ files = [
{file = "propcache-0.3.2.tar.gz", hash = "sha256:20d7d62e4e7ef05f221e0db2856b979540686342e7dd9973b815599c7057e168"},
]
-[[package]]
-name = "proto-plus"
-version = "1.26.1"
-description = "Beautiful, Pythonic protocol buffers"
-optional = true
-python-versions = ">=3.7"
-groups = ["main"]
-markers = "extra == \"gcsfs\""
-files = [
- {file = "proto_plus-1.26.1-py3-none-any.whl", hash = "sha256:13285478c2dcf2abb829db158e1047e2f1e8d63a077d94263c2b88b043c75a66"},
- {file = "proto_plus-1.26.1.tar.gz", hash = "sha256:21a515a4c4c0088a773899e23c7bbade3d18f9c66c73edd4c7ee3816bc96a012"},
-]
-
-[package.dependencies]
-protobuf = ">=3.19.0,<7.0.0"
-
-[package.extras]
-testing = ["google-api-core (>=1.31.5)"]
-
[[package]]
name = "protobuf"
version = "6.31.1"
@@ -4604,7 +4517,7 @@ description = "World timezone definitions, modern and historical"
optional = true
python-versions = "*"
groups = ["main"]
-markers = "extra == \"pandas\" or extra == \"ray\""
+markers = "extra == \"gcsfs\" or extra == \"pandas\" or extra == \"ray\""
files = [
{file = "pytz-2025.2-py2.py3-none-any.whl", hash = "sha256:5ddf76296dd8c44c26eb8f4b6f35488f3ccbf6fbbd7adee0b7262d43f0ec2f00"},
{file = "pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3"},
@@ -5237,11 +5150,12 @@ version = "80.9.0"
description = "Easily download, build, install, upgrade, and uninstall Python packages"
optional = false
python-versions = ">=3.9"
-groups = ["dev"]
+groups = ["main", "dev"]
files = [
{file = "setuptools-80.9.0-py3-none-any.whl", hash = "sha256:062d34222ad13e0cc312a4c02d73f059e86a4acbfbdea8f8f76b28c99f306922"},
{file = "setuptools-80.9.0.tar.gz", hash = "sha256:f36b47402ecde768dbfafc46e8e4207b4360c654f1f3bb84475f0a28628fb19c"},
]
+markers = {main = "extra == \"gcsfs\""}
[package.extras]
check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\"", "ruff (>=0.8.0) ; sys_platform != \"cygwin\""]
From bffe41f2dd11becd31c85441df4728eeca37b0ff Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:43:21 -0400
Subject: [PATCH 069/112] Build: Bump huggingface-hub from 0.32.5 to 0.33.0
(#2103)
Bumps [huggingface-hub](https://github.com/huggingface/huggingface_hub)
from 0.32.5 to 0.33.0.
Release notes
Sourced from huggingface-hub's
releases.
[v0.32.6] [Upload large folder] fix for wrongly saved
upload_mode/remote_oid
- Fix for wrongly saved upload_mode/remote_oid #3113
Full Changelog: https://github.com/huggingface/huggingface_hub/compare/v0.32.5...v0.32.6
Commits
d5dff4e
Release: v0.33.0
9d1e603
Release: v0.33.0.rc0
a0429e7
ADd Groq provider (#3157)
4cc3143
Added Hindi translation for git_vs_http.md in concepts section (#3156)
85853e3
Inject env var in headers + better type annotations (#3142)
8809f44
fix table question answering (#3154)
c329d56
[Inference Providers] Raise warning if provider's status is in error
mode (#3...
987dadc
[MCP] reinject JSON parse & runtime tool errors back into the chat
history (#...
b4c01ed
Fix inference search (#3022)
2855402
[Internal] prepare for 0.33.0 release (#3138)
- Additional commits viewable in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 7268d20246..60c4d56b3c 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1966,15 +1966,15 @@ tests = ["pytest"]
[[package]]
name = "huggingface-hub"
-version = "0.32.5"
+version = "0.33.0"
description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
optional = true
python-versions = ">=3.8.0"
groups = ["main"]
markers = "extra == \"hf\""
files = [
- {file = "huggingface_hub-0.32.5-py3-none-any.whl", hash = "sha256:6df8d5f42034a1b61daac60eed04acf348d337a4bd83aa448d4235cfb003e379"},
- {file = "huggingface_hub-0.32.5.tar.gz", hash = "sha256:8328f848218e3212647cec77eab9fdfc2590e8117d979b925439bc01042a20de"},
+ {file = "huggingface_hub-0.33.0-py3-none-any.whl", hash = "sha256:e8668875b40c68f9929150d99727d39e5ebb8a05a98e4191b908dc7ded9074b3"},
+ {file = "huggingface_hub-0.33.0.tar.gz", hash = "sha256:aa31f70d29439d00ff7a33837c03f1f9dd83971ce4e29ad664d63ffb17d3bb97"},
]
[package.dependencies]
From 58b432f8fd56939976eca3c9ead11cdc9f8f1e6f Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:43:32 -0400
Subject: [PATCH 070/112] Build: Bump typing-extensions from 4.13.2 to 4.14.0
(#2104)
Bumps [typing-extensions](https://github.com/python/typing_extensions)
from 4.13.2 to 4.14.0.
Release notes
Sourced from typing-extensions's
releases.
4.14.0
This release adds several new features, including experimental
support for inline typed dictionaries (PEP 764) and sentinels (PEP 661), and support for
changes in Python 3.14. In addition, Python 3.8 is no longer
supported.
Changes since 4.14.0rc1:
- Remove
__or__ and __ror__ methods from
typing_extensions.Sentinel
on Python versions <3.10. PEP 604 was introduced in Python 3.10, and
typing_extensions does not generally attempt to backport
PEP-604 methods
to prior versions.
- Further update
typing_extensions.evaluate_forward_ref
with changes in Python 3.14.
Changes included in 4.14.0rc1:
- Drop support for Python 3.8 (including PyPy-3.8). Patch by Victorien Plot.
- Do not attempt to re-export names that have been removed from
typing,
anticipating the removal of typing.no_type_check_decorator
in Python 3.15.
Patch by Jelle Zijlstra.
- Update
typing_extensions.Format,
typing_extensions.evaluate_forward_ref, and
typing_extensions.TypedDict to align
with changes in Python 3.14. Patches by Jelle Zijlstra.
- Fix tests for Python 3.14 and 3.15. Patches by Jelle Zijlstra.
New features:
- Add support for inline typed dictionaries (PEP 764).
Patch by Victorien Plot.
- Add
typing_extensions.Reader and
typing_extensions.Writer. Patch by
Sebastian Rittau.
- Add support for sentinels (PEP 661). Patch by
Victorien Plot.
4.14.0rc1
Major changes:
- Drop support for Python 3.8 (including PyPy-3.8). Patch by Victorien Plot.
- Do not attempt to re-export names that have been removed from
typing,
anticipating the removal of typing.no_type_check_decorator
in Python 3.15.
Patch by Jelle Zijlstra.
- Update
typing_extensions.Format,
typing_extensions.evaluate_forward_ref, and
typing_extensions.TypedDict to align
with changes in Python 3.14. Patches by Jelle Zijlstra.
- Fix tests for Python 3.14 and 3.15. Patches by Jelle Zijlstra.
New features:
- Add support for inline typed dictionaries (PEP 764).
Patch by Victorien Plot.
- Add
typing_extensions.Reader and
typing_extensions.Writer. Patch by
Sebastian Rittau.
- Add support for sentinels (PEP 661). Patch by
... (truncated)
Changelog
Sourced from typing-extensions's
changelog.
Release 4.14.0 (June 2, 2025)
Changes since 4.14.0rc1:
- Remove
__or__ and __ror__ methods from
typing_extensions.Sentinel
on Python versions <3.10. PEP 604 was introduced in Python 3.10, and
typing_extensions does not generally attempt to backport
PEP-604 methods
to prior versions.
- Further update
typing_extensions.evaluate_forward_ref
with changes in Python 3.14.
Release 4.14.0rc1 (May 24, 2025)
- Drop support for Python 3.8 (including PyPy-3.8). Patch by Victorien Plot.
- Do not attempt to re-export names that have been removed from
typing,
anticipating the removal of typing.no_type_check_decorator
in Python 3.15.
Patch by Jelle Zijlstra.
- Update
typing_extensions.Format,
typing_extensions.evaluate_forward_ref, and
typing_extensions.TypedDict to align
with changes in Python 3.14. Patches by Jelle Zijlstra.
- Fix tests for Python 3.14 and 3.15. Patches by Jelle Zijlstra.
New features:
- Add support for inline typed dictionaries (PEP 764).
Patch by Victorien Plot.
- Add
typing_extensions.Reader and
typing_extensions.Writer. Patch by
Sebastian Rittau.
- Add support for sentinels (PEP 661). Patch by
Victorien Plot.
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 12 ++++++------
pyproject.toml | 2 +-
2 files changed, 7 insertions(+), 7 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 60c4d56b3c..71c7943180 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -5663,14 +5663,14 @@ telegram = ["requests"]
[[package]]
name = "typing-extensions"
-version = "4.13.2"
-description = "Backported and Experimental Type Hints for Python 3.8+"
+version = "4.14.0"
+description = "Backported and Experimental Type Hints for Python 3.9+"
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main", "dev", "docs"]
files = [
- {file = "typing_extensions-4.13.2-py3-none-any.whl", hash = "sha256:a439e7c04b49fec3e5d3e2beaa21755cadbbdc391694e28ccdd36ca4a1408f8c"},
- {file = "typing_extensions-4.13.2.tar.gz", hash = "sha256:e6c81219bd689f51865d9e372991c540bda33a0379d5573cddb9a3a23f7caaef"},
+ {file = "typing_extensions-4.14.0-py3-none-any.whl", hash = "sha256:a1514509136dd0b477638fc68d6a91497af5076466ad0fa6c338e44e359944af"},
+ {file = "typing_extensions-4.14.0.tar.gz", hash = "sha256:8676b788e32f02ab42d9e7c61324048ae4c6d844a399eebace3d4979d75ceef4"},
]
markers = {docs = "python_version <= \"3.10\""}
@@ -6203,4 +6203,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "7e2fa5ddc3b2389fc07541d6f2d4b4136cec8bef32dbe1dec13199818ef88212"
+content-hash = "c2f45d4d591caedd7d513922884de881cf4ef30a8b431a5ceb6bb9e56711a669"
diff --git a/pyproject.toml b/pyproject.toml
index d759a95f3e..4e479e9d0e 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -94,7 +94,7 @@ fastavro = "1.11.1"
coverage = { version = "^7.4.2", extras = ["toml"] }
requests-mock = "1.12.1"
moto = { version = "^5.0.2", extras = ["server"] }
-typing-extensions = "4.13.2"
+typing-extensions = "4.14.0"
pytest-mock = "3.14.1"
pyspark = "3.5.6"
cython = "3.1.2"
From 283523d7bb172f0cb65629b0492560ba4c0fae91 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:43:51 -0400
Subject: [PATCH 071/112] Build: Bump duckdb from 1.3.0 to 1.3.1 (#2105)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
[//]: # (dependabot-start)
⚠️ **Dependabot is rebasing this PR** ⚠️
Rebasing might not happen immediately, so don't worry if this takes some
time.
Note: if you make any changes to this PR yourself, they will take
precedence over the rebase.
---
[//]: # (dependabot-end)
Bumps [duckdb](https://github.com/duckdb/duckdb) from 1.3.0 to 1.3.1.
Release notes
Sourced from duckdb's
releases.
v1.3.1 Bugfix Release
This is a bug fix release for various issues discovered after we
released 1.3.0 "Ossivalis". There are no new major features,
just bug fixes. Database files created by DuckDB versions all the way
back to v0.9 can be read by this version.
What's Changed
... (truncated)
Commits
cf42228
Make sure distance is always an int when doing version bumps in setup.py
(#17...
89638a5
Make sure distance is always an int when doing version bumps in
setup.py
7b87023
stringstream: Wrap as duckdb::stringstream, and imbue with
locale::classic on...
f79594b
duckdb_base_std:: plus compile time test on discontinued
functions (#17866)
e61a68d
Move from duckdb/wrapped/std to duckdb/original/std, and review
comments
00fa790
Add urllib3 dependency and improve fixture download reliability
4ea0812
duckdb_wrapped::std: tools/shell
4552235
Use fixed StringUtil::CharacterIsSpace instead of locale-dependent
std::isspace
6d6324a
arrow_output_version option to produce arrow depending on a format
version. (...
c3248fb
Have to setup the format due to the change on the default
- Additional commits viewable in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 74 ++++++++++++++++++++++++++---------------------------
1 file changed, 37 insertions(+), 37 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 71c7943180..9872ba62d1 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -1290,49 +1290,49 @@ dates = ["pytz (>=2019.1)"]
[[package]]
name = "duckdb"
-version = "1.3.0"
+version = "1.3.1"
description = "DuckDB in-process database"
optional = true
python-versions = ">=3.7.0"
groups = ["main"]
markers = "extra == \"duckdb\""
files = [
- {file = "duckdb-1.3.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:fc65c1e97aa010359c43c0342ea423e6efa3cb8c8e3f133b0765451ce674e3db"},
- {file = "duckdb-1.3.0-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:8fc91b629646679e33806342510335ccbbeaf2b823186f0ae829fd48e7a63c66"},
- {file = "duckdb-1.3.0-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:1a69b970553fd015c557238d427ef00be3c8ed58c3bc3641aef987e33f8bf614"},
- {file = "duckdb-1.3.0-cp310-cp310-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:1003e84c07b84680cee6d06e4795b6e861892474704f7972058594a52c7473cf"},
- {file = "duckdb-1.3.0-cp310-cp310-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:992239b54ca6f015ad0ed0d80f3492c065313c4641df0a226183b8860cb7f5b0"},
- {file = "duckdb-1.3.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:0ba1c5af59e8147216149b814b1970b8f7e3c240494a9688171390db3c504b29"},
- {file = "duckdb-1.3.0-cp310-cp310-win_amd64.whl", hash = "sha256:57b794ca28e22b23bd170506cb1d4704a3608e67f0fe33273db9777b69bdf26a"},
- {file = "duckdb-1.3.0-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:60a58b85929754abb21db1e739d2f53eaef63e6015e62ba58eae3425030e7935"},
- {file = "duckdb-1.3.0-cp311-cp311-macosx_12_0_universal2.whl", hash = "sha256:1d46b5a20f078b1b2284243e02a1fde7e12cbb8d205fce62e4700bcfe6a09881"},
- {file = "duckdb-1.3.0-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:0044e5ffb2d46308099640a92f99980a44e12bb68642aa9e6b08acbf300d64a1"},
- {file = "duckdb-1.3.0-cp311-cp311-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:5cb813de2ca2f5e7c77392a67bdcaa174bfd69ebbfdfc983024af270c77a0447"},
- {file = "duckdb-1.3.0-cp311-cp311-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:7a0c993eb6df2b30b189ad747f3aea1b0b87b78ab7f80c6e7c57117b6e8dbfb0"},
- {file = "duckdb-1.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:6728e209570d36ece66dd7249e5d6055326321137cd807f26300733283930cd4"},
- {file = "duckdb-1.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:7e652b7c8dbdb91a94fd7d543d3e115d24a25aa0791a373a852e20cb7bb21154"},
- {file = "duckdb-1.3.0-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:f24038fe9b83dcbaeafb1ed76ec3b3f38943c1c8d27ab464ad384db8a6658b61"},
- {file = "duckdb-1.3.0-cp312-cp312-macosx_12_0_universal2.whl", hash = "sha256:956c85842841bef68f4a5388c6b225b933151a7c06d568390fc895fc44607913"},
- {file = "duckdb-1.3.0-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:efe883d822ed56fcfbb6a7b397c13f6a0d2eaeb3bc4ef4510f84fadb3dfe416d"},
- {file = "duckdb-1.3.0-cp312-cp312-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:3872a3a1b80ffba5264ea236a3754d0c41d3c7b01bdf8cdcb1c180fc1b8dc8e2"},
- {file = "duckdb-1.3.0-cp312-cp312-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:30bf45ad78a5a997f378863e036e917b481d18d685e5c977cd0a3faf2e31fbaf"},
- {file = "duckdb-1.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:85cbd8e1d65df8a0780023baf5045d3033fabd154799bc9ea6d9ab5728f41eb3"},
- {file = "duckdb-1.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:8754c40dac0f26d9fb0363bbb5df02f7a61ce6a6728d5efc02c3bc925d7c89c3"},
- {file = "duckdb-1.3.0-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:176b9818d940c52ac7f31c64a98cf172d7c19d2a006017c9c4e9c06c246e36bf"},
- {file = "duckdb-1.3.0-cp313-cp313-macosx_12_0_universal2.whl", hash = "sha256:03981f7e8793f07a4a9a2ba387640e71d0a99ebcaf8693ab09f96d59e628b713"},
- {file = "duckdb-1.3.0-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:a177d55a38a62fdf79b59a0eaa32531a1dbb443265f6d67f64992cc1e82b755c"},
- {file = "duckdb-1.3.0-cp313-cp313-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b1c30e3749823147d5578bc3f01f35d1a0433a1c768908d946056ec8d6e1757e"},
- {file = "duckdb-1.3.0-cp313-cp313-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:5855f3a564baf22eeeab70c120b51f5a11914f1f1634f03382daeb6b1dea4c62"},
- {file = "duckdb-1.3.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:9b1fac15a48056f7c2739cf8800873063ba2f691e91a9b2fc167658a401ca76a"},
- {file = "duckdb-1.3.0-cp313-cp313-win_amd64.whl", hash = "sha256:fbdfc1c0b83b90f780ae74038187ee696bb56ab727a289752372d7ec42dda65b"},
- {file = "duckdb-1.3.0-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:5f6b5d725546ad30abc125a6813734b493fea694bc3123e991c480744573c2f1"},
- {file = "duckdb-1.3.0-cp39-cp39-macosx_12_0_universal2.whl", hash = "sha256:fcbcc9b956b06cf5ee94629438ecab88de89b08b5620fcda93665c222ab18cd4"},
- {file = "duckdb-1.3.0-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:2d32f2d44105e1705d8a0fb6d6d246fd69aff82c80ad23293266244b66b69012"},
- {file = "duckdb-1.3.0-cp39-cp39-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:0aa7a5c0dcb780850e6da1227fb1d552af8e1a5091e02667ab6ace61ab49ce6c"},
- {file = "duckdb-1.3.0-cp39-cp39-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:7cb254fd5405f3edbd7d962ba39c72e4ab90b37cb4d0e34846089796c8078419"},
- {file = "duckdb-1.3.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:a7d337b58c59fd2cd9faae531b05d940f8d92bdc2e14cb6e9a5a37675ad2742d"},
- {file = "duckdb-1.3.0-cp39-cp39-win_amd64.whl", hash = "sha256:b3cea3a345755c7dbcb58403dbab8befd499c82f0d27f893a4c1d4b8cf56ec54"},
- {file = "duckdb-1.3.0.tar.gz", hash = "sha256:09aaa4b1dca24f4d1f231e7ae66b6413e317b7e04e2753541d42df6c8113fac7"},
+ {file = "duckdb-1.3.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:8321ecd3c6be22660ac7b48d1770781b2a9d22e3f961ad0bb9f851d4e109806c"},
+ {file = "duckdb-1.3.1-cp310-cp310-macosx_12_0_universal2.whl", hash = "sha256:ccccc9dc9cb2269430fed29a2be8ff65a84d7b9e427548e02b5a8e1e1aacfa6d"},
+ {file = "duckdb-1.3.1-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:f8a1ca3bbf84275ba4e0da2bccf6d43cb277a19af6f88fb86f98c33a98cce02e"},
+ {file = "duckdb-1.3.1-cp310-cp310-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:3ed9a942ba1167a51c0eb9f23c567051a51da4cbf920b3ac83fe63b010c4334c"},
+ {file = "duckdb-1.3.1-cp310-cp310-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:26944ff2c09749077ee63e5fec634da431b0b8eb7dd0d30c24fa7fe89ce70b66"},
+ {file = "duckdb-1.3.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:1ac996ac099f5d15468e33a93caf078da0fdace48c8a2c9af41e7bec766602f3"},
+ {file = "duckdb-1.3.1-cp310-cp310-win_amd64.whl", hash = "sha256:57a2324f8206a52f5fd2b44f34c3746bed8bcd5e98b05b298e04fafbf30e5079"},
+ {file = "duckdb-1.3.1-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:376193078285b243910b1239a927e271d12d9bf6358a6937d1f7af253cfef2b6"},
+ {file = "duckdb-1.3.1-cp311-cp311-macosx_12_0_universal2.whl", hash = "sha256:d690576e8b4479b1e0c58cd8179f600f67af237ad31186fb10e867a02d4d66ff"},
+ {file = "duckdb-1.3.1-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:833b3c0208c238aac0d9287fcaca93ea54b82deabd8d162a469bd9adb42a0453"},
+ {file = "duckdb-1.3.1-cp311-cp311-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:8bdd53e62917298208b7182d5fd1686a4caddc573dc1a95a58ca054105b23b38"},
+ {file = "duckdb-1.3.1-cp311-cp311-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:591c9ca1b8dc591548bf56b2f18e26ca2339d7b95613009f6ba00af855210029"},
+ {file = "duckdb-1.3.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:18f21142546edb5f935963f8f012b6569b978f398d48709da276b245ee4f5f4d"},
+ {file = "duckdb-1.3.1-cp311-cp311-win_amd64.whl", hash = "sha256:59121f0a8220b72050046a816e85e7464eb78e395f64118161b1115855284f87"},
+ {file = "duckdb-1.3.1-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:73f389f9c713325a6994dd9e04a7fa23bd73e8387883f8086946a9d3a1dd70e1"},
+ {file = "duckdb-1.3.1-cp312-cp312-macosx_12_0_universal2.whl", hash = "sha256:87c99569274b453d8f9963e43fea74bc86901773fac945c1fe612c133a91e506"},
+ {file = "duckdb-1.3.1-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:21da268355dfdf859b3d4db22180f7d5dd85a60517e077cb4158768cd5f0ee44"},
+ {file = "duckdb-1.3.1-cp312-cp312-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:77902954d15ba4aff92e82df700643b995c057f2d7d39af7ed226d8cceb9c2af"},
+ {file = "duckdb-1.3.1-cp312-cp312-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:67b1a3c9e2c3474991da97edfec0a89f382fef698d7f64b2d8d09006eaeeea24"},
+ {file = "duckdb-1.3.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:f1d076b12f0d2a7f9090ad9e4057ac41af3e4785969e5997afd44922c7b141e0"},
+ {file = "duckdb-1.3.1-cp312-cp312-win_amd64.whl", hash = "sha256:bf7d6884bfb67aef67aebb0bd2460ea1137c55b3fd8794a3530c653dbe0d4019"},
+ {file = "duckdb-1.3.1-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:72bbc8479c5d88e839a92c458c94c622f917ff0122853323728d6e25b0c3d4e1"},
+ {file = "duckdb-1.3.1-cp313-cp313-macosx_12_0_universal2.whl", hash = "sha256:937de83df6bbe4bee5830ce80f568d4c0ebf3ef5eb809db3343d2161e4f6e42b"},
+ {file = "duckdb-1.3.1-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:21440dd37f073944badd495c299c6d085cd133633450467ec420c71897ac1d5b"},
+ {file = "duckdb-1.3.1-cp313-cp313-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:663610b591ea6964f140441c81b718e745704cf098c540e905b200b9079e2a5c"},
+ {file = "duckdb-1.3.1-cp313-cp313-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:8793b5abb365bbbf64ba3065f3a37951fe04f2d4506b0e24f3f8ecd08b3af4ba"},
+ {file = "duckdb-1.3.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:27d775a5af405d1c228561830c8ccbe4e2832dafb4012f16c05fde1cde206dee"},
+ {file = "duckdb-1.3.1-cp313-cp313-win_amd64.whl", hash = "sha256:3eb045a9bf92da890d890cde2f676b3bda61b9de3b7dc46cbaaf75875b41e4b0"},
+ {file = "duckdb-1.3.1-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:fb86f0506d9e402187820030c71026b44138908feff2963ef4203f25d89296c0"},
+ {file = "duckdb-1.3.1-cp39-cp39-macosx_12_0_universal2.whl", hash = "sha256:b1d21f66e89100d7ae8353800d9525e5e24d19299b1fb099564d1106336dfdba"},
+ {file = "duckdb-1.3.1-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:a4d0019672cb3c1f2bbfceefdd23d14113472df3e70c5796866d6ba19a2c1575"},
+ {file = "duckdb-1.3.1-cp39-cp39-manylinux_2_27_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ebfd06a746b0f8fb1e83bc79bfa29b4b4ebe8c095d1c790b88f4d60d58e0ebbd"},
+ {file = "duckdb-1.3.1-cp39-cp39-manylinux_2_27_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:13d20cb8248f36b999bd1cbdd63d9066d7beb40ce9e6c2040ef2d7c6947f2152"},
+ {file = "duckdb-1.3.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:265979d57193fbeaf13b732a02ca9fadba76c694f2d63d87a7f136357f8c2dca"},
+ {file = "duckdb-1.3.1-cp39-cp39-win_amd64.whl", hash = "sha256:63e6757065ca24d327a9a8ebd7e0400ab3c73cd7f5876e75e9f49f3453aff793"},
+ {file = "duckdb-1.3.1.tar.gz", hash = "sha256:8e101990a879533b1d33f003df2eb2a3c4bc7bdf976bd7ef7c32342047935327"},
]
[[package]]
From 854aeebfeb830ea9807e98a61f3a25c72d3a6dfd Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:44:01 -0400
Subject: [PATCH 072/112] Build: Bump boto3 from 1.37.3 to 1.38.27 (#2108)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
[//]: # (dependabot-start)
⚠️ **Dependabot is rebasing this PR** ⚠️
Rebasing might not happen immediately, so don't worry if this takes some
time.
Note: if you make any changes to this PR yourself, they will take
precedence over the rebase.
---
[//]: # (dependabot-end)
Bumps [boto3](https://github.com/boto/boto3) from 1.37.3 to 1.38.27.
Commits
ccacc66
Merge branch 'release-1.38.27'
531a41c
Bumping version to 1.38.27
47419ec
Add changelog entries from botocore
173e2b4
Merge branch 'release-1.38.26'
62b7d27
Merge branch 'release-1.38.26' into develop
99ba763
Bumping version to 1.38.26
1133430
Add changelog entries from botocore
3797b9c
Merge branch 'release-1.38.25'
f84d4e8
Merge branch 'release-1.38.25' into develop
42157c8
Bumping version to 1.38.25
- Additional commits viewable in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 47 ++++++++++++++++++++++++-----------------------
1 file changed, 24 insertions(+), 23 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 9872ba62d1..abfcc5f797 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -27,29 +27,30 @@ tests = ["arrow", "dask[dataframe]", "docker", "pytest", "pytest-mock"]
[[package]]
name = "aiobotocore"
-version = "2.22.0"
+version = "2.23.0"
description = "Async client for aws services using botocore and aiohttp"
optional = true
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"s3fs\""
files = [
- {file = "aiobotocore-2.22.0-py3-none-any.whl", hash = "sha256:b4e6306f79df9d81daff1f9d63189a2dbee4b77ce3ab937304834e35eaaeeccf"},
- {file = "aiobotocore-2.22.0.tar.gz", hash = "sha256:11091477266b75c2b5d28421c1f2bc9a87d175d0b8619cb830805e7a113a170b"},
+ {file = "aiobotocore-2.23.0-py3-none-any.whl", hash = "sha256:8202cebbf147804a083a02bc282fbfda873bfdd0065fd34b64784acb7757b66e"},
+ {file = "aiobotocore-2.23.0.tar.gz", hash = "sha256:0333931365a6c7053aee292fe6ef50c74690c4ae06bb019afdf706cb6f2f5e32"},
]
[package.dependencies]
aiohttp = ">=3.9.2,<4.0.0"
aioitertools = ">=0.5.1,<1.0.0"
-botocore = ">=1.37.2,<1.37.4"
+botocore = ">=1.38.23,<1.38.28"
jmespath = ">=0.7.1,<2.0.0"
multidict = ">=6.0.0,<7.0.0"
python-dateutil = ">=2.1,<3.0.0"
wrapt = ">=1.10.10,<2.0.0"
[package.extras]
-awscli = ["awscli (>=1.38.2,<1.38.4)"]
-boto3 = ["boto3 (>=1.37.2,<1.37.4)"]
+awscli = ["awscli (>=1.40.22,<1.40.27)"]
+boto3 = ["boto3 (>=1.38.23,<1.38.28)"]
+httpx = ["httpx (>=0.25.1,<0.29)"]
[[package]]
name = "aiohappyeyeballs"
@@ -478,35 +479,35 @@ files = [
[[package]]
name = "boto3"
-version = "1.37.3"
+version = "1.38.27"
description = "The AWS SDK for Python"
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "boto3-1.37.3-py3-none-any.whl", hash = "sha256:2063b40af99fd02f6228ff52397b552ff3353831edaf8d25cc04801827ab9794"},
- {file = "boto3-1.37.3.tar.gz", hash = "sha256:21f3ce0ef111297e63a6eb998a25197b8c10982970c320d4c6e8db08be2157be"},
+ {file = "boto3-1.38.27-py3-none-any.whl", hash = "sha256:95f5fe688795303a8a15e8b7e7f255cadab35eae459d00cc281a4fd77252ea80"},
+ {file = "boto3-1.38.27.tar.gz", hash = "sha256:94bd7fdd92d5701b362d4df100d21e28f8307a67ff56b6a8b0398119cf22f859"},
]
markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
-botocore = ">=1.37.3,<1.38.0"
+botocore = ">=1.38.27,<1.39.0"
jmespath = ">=0.7.1,<2.0.0"
-s3transfer = ">=0.11.0,<0.12.0"
+s3transfer = ">=0.13.0,<0.14.0"
[package.extras]
crt = ["botocore[crt] (>=1.21.0,<2.0a0)"]
[[package]]
name = "botocore"
-version = "1.37.3"
+version = "1.38.27"
description = "Low-level, data-driven core of boto 3."
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "botocore-1.37.3-py3-none-any.whl", hash = "sha256:d01bd3bf4c80e61fa88d636ad9f5c9f60a551d71549b481386c6b4efe0bb2b2e"},
- {file = "botocore-1.37.3.tar.gz", hash = "sha256:fe8403eb55a88faf9b0f9da6615e5bee7be056d75e17af66c3c8f0a3b0648da4"},
+ {file = "botocore-1.38.27-py3-none-any.whl", hash = "sha256:a785d5e9a5eda88ad6ab9ed8b87d1f2ac409d0226bba6ff801c55359e94d91a8"},
+ {file = "botocore-1.38.27.tar.gz", hash = "sha256:9788f7efe974328a38cbade64cc0b1e67d27944b899f88cb786ae362973133b6"},
]
markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
@@ -5127,22 +5128,22 @@ boto3 = ["aiobotocore[boto3] (>=2.5.4,<3.0.0)"]
[[package]]
name = "s3transfer"
-version = "0.11.3"
+version = "0.13.0"
description = "An Amazon S3 Transfer Manager"
optional = false
-python-versions = ">=3.8"
+python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "s3transfer-0.11.3-py3-none-any.whl", hash = "sha256:ca855bdeb885174b5ffa95b9913622459d4ad8e331fc98eb01e6d5eb6a30655d"},
- {file = "s3transfer-0.11.3.tar.gz", hash = "sha256:edae4977e3a122445660c7c114bba949f9d191bae3b34a096f18a1c8c354527a"},
+ {file = "s3transfer-0.13.0-py3-none-any.whl", hash = "sha256:0148ef34d6dd964d0d8cf4311b2b21c474693e57c2e069ec708ce043d2b527be"},
+ {file = "s3transfer-0.13.0.tar.gz", hash = "sha256:f5e6db74eb7776a37208001113ea7aa97695368242b364d73e91c981ac522177"},
]
markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
-botocore = ">=1.36.0,<2.0a.0"
+botocore = ">=1.37.4,<2.0a.0"
[package.extras]
-crt = ["botocore[crt] (>=1.36.0,<2.0a.0)"]
+crt = ["botocore[crt] (>=1.37.4,<2.0a.0)"]
[[package]]
name = "setuptools"
From c18279f3b811195615c80f1816910ebf2119a02d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 16 Jun 2025 23:59:58 -0400
Subject: [PATCH 073/112] Build: Bump pydantic from 2.11.5 to 2.11.7 (#2107)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [pydantic](https://github.com/pydantic/pydantic) from 2.11.5 to
2.11.7.
Release notes
Sourced from pydantic's
releases.
v2.11.7 2025-06-14
What's Changed
Fixes
Full Changelog: https://github.com/pydantic/pydantic/compare/v2.11.6...v2.11.7
v2.11.6 2025-06-13
v2.11.6 (2025-06-13)
What's Changed
Fixes
- Rebuild dataclass fields before schema generation by
@Viicos in #11949
- Always store the original field assignment on
FieldInfo
by @Viicos in #11946
Full Changelog: https://github.com/pydantic/pydantic/compare/v2.11.5...v2.11.6
Changelog
Sourced from pydantic's
changelog.
v2.11.7 (2025-06-14)
GitHub
release
What's Changed
Fixes
- Copy
FieldInfo instance if necessary during
FieldInfo build by @Viicos in #11898
v2.11.6 (2025-06-13)
GitHub
release
What's Changed
Fixes
- Rebuild dataclass fields before schema generation by
@Viicos in #11949
- Always store the original field assignment on
FieldInfo
by @Viicos in #11946
Commits
5f033e4
Prepare release v2.11.7
c3368b8
Copy FieldInfo instance if necessary during
FieldInfo build (#11980)
3987b23
Prepare release v2.11.6
dc7a9d2
Always store the original field assignment on
FieldInfo
c284c27
Rebuild dataclass fields before schema generation
- See full diff in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index abfcc5f797..8b668399c6 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -4057,14 +4057,14 @@ markers = {main = "(extra == \"zstandard\" or extra == \"adlfs\") and platform_p
[[package]]
name = "pydantic"
-version = "2.11.5"
+version = "2.11.7"
description = "Data validation using Python type hints"
optional = false
python-versions = ">=3.9"
groups = ["main", "dev"]
files = [
- {file = "pydantic-2.11.5-py3-none-any.whl", hash = "sha256:f9c26ba06f9747749ca1e5c94d6a85cb84254577553c8785576fd38fa64dc0f7"},
- {file = "pydantic-2.11.5.tar.gz", hash = "sha256:7f853db3d0ce78ce8bbb148c401c2cdd6431b3473c0cdff2755c7690952a7b7a"},
+ {file = "pydantic-2.11.7-py3-none-any.whl", hash = "sha256:dde5df002701f6de26248661f6835bbe296a47bf73990135c7d07ce741b9623b"},
+ {file = "pydantic-2.11.7.tar.gz", hash = "sha256:d989c3c6cb79469287b1569f7447a17848c998458d49ebe294e975b9baf0f0db"},
]
[package.dependencies]
From 753ba7062a85a8034268f4544f140da0af2aafa2 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 17 Jun 2025 00:00:14 -0400
Subject: [PATCH 074/112] Build: Bump coverage from 7.8.2 to 7.9.1 (#2106)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [coverage](https://github.com/nedbat/coveragepy) from 7.8.2 to
7.9.1.
Changelog
Sourced from coverage's
changelog.
Version 7.9.1 — 2025-06-13
-
The "no-ctracer" warning is not issued for Python
pre-release versions.
Coverage doesn't ship compiled wheels for those versions, so this was
far too
noisy.
-
On Python 3.14+, the "sysmon" core is now the default if
it's supported for
your configuration. Plugins and dynamic contexts are still not supported
with it.
.. _changes_7-9-0:
Version 7.9.0 — 2025-06-11
-
Added a [run] core configuration setting to specify the
measurement core,
which was previously only available through the COVERAGE_CORE
environment
variable. Finishes issue 1746_.
-
Fixed incorrect rendering of f-strings with doubled braces, closing
issue 1980_.
-
If the C tracer core can't be imported, a warning
("no-ctracer") is issued
with the reason.
-
The C tracer core extension module now conforms to PEP
489, closing issue 1977. Thanks, Adam
Turner <pull 1978_>_.
-
Fixed a "ValueError: min() arg is an empty sequence" error
caused by strange
empty modules, found by oss-fuzz_.
.. _issue 1746: nedbat/coveragepy#1746
.. _issue 1977: nedbat/coveragepy#1977
.. _pull 1978: nedbat/coveragepy#1978
.. _issue 1980: nedbat/coveragepy#1980
.. _PEP 489: https://peps.python.org/pep-0489
.. _oss-fuzz: https://google.github.io/oss-fuzz/
.. _changes_7-8-2:
Commits
4fa1b71
docs: sample HTML for 7.9.1
5cf757b
docs: prep for 7.9.1
bea6204
fix: better defaulting of core
7795441
fix: fewer warnings, default to sys.monitoring on 3.14+
18aa074
fix: some custom builds don't have file
8c5bf65
chore: make upgrade
128c4f4
build: bump version to 7.9.1
452d86f
docs: sample HTML for 7.9.0
a670927
docs: prep for 7.9.0
3b0cb87
build: windows 3.14 is fixed
- Additional commits viewable in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 136 ++++++++++++++++++++++++++--------------------------
1 file changed, 68 insertions(+), 68 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 8b668399c6..f4da5368b1 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -825,79 +825,79 @@ markers = {main = "platform_system == \"Windows\"", dev = "platform_system == \"
[[package]]
name = "coverage"
-version = "7.8.2"
+version = "7.9.1"
description = "Code coverage measurement for Python"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "coverage-7.8.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:bd8ec21e1443fd7a447881332f7ce9d35b8fbd2849e761bb290b584535636b0a"},
- {file = "coverage-7.8.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:4c26c2396674816deaeae7ded0e2b42c26537280f8fe313335858ffff35019be"},
- {file = "coverage-7.8.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1aec326ed237e5880bfe69ad41616d333712c7937bcefc1343145e972938f9b3"},
- {file = "coverage-7.8.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5e818796f71702d7a13e50c70de2a1924f729228580bcba1607cccf32eea46e6"},
- {file = "coverage-7.8.2-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:546e537d9e24efc765c9c891328f30f826e3e4808e31f5d0f87c4ba12bbd1622"},
- {file = "coverage-7.8.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:ab9b09a2349f58e73f8ebc06fac546dd623e23b063e5398343c5270072e3201c"},
- {file = "coverage-7.8.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:fd51355ab8a372d89fb0e6a31719e825cf8df8b6724bee942fb5b92c3f016ba3"},
- {file = "coverage-7.8.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:0774df1e093acb6c9e4d58bce7f86656aeed6c132a16e2337692c12786b32404"},
- {file = "coverage-7.8.2-cp310-cp310-win32.whl", hash = "sha256:00f2e2f2e37f47e5f54423aeefd6c32a7dbcedc033fcd3928a4f4948e8b96af7"},
- {file = "coverage-7.8.2-cp310-cp310-win_amd64.whl", hash = "sha256:145b07bea229821d51811bf15eeab346c236d523838eda395ea969d120d13347"},
- {file = "coverage-7.8.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b99058eef42e6a8dcd135afb068b3d53aff3921ce699e127602efff9956457a9"},
- {file = "coverage-7.8.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:5feb7f2c3e6ea94d3b877def0270dff0947b8d8c04cfa34a17be0a4dc1836879"},
- {file = "coverage-7.8.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:670a13249b957bb9050fab12d86acef7bf8f6a879b9d1a883799276e0d4c674a"},
- {file = "coverage-7.8.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0bdc8bf760459a4a4187b452213e04d039990211f98644c7292adf1e471162b5"},
- {file = "coverage-7.8.2-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:07a989c867986c2a75f158f03fdb413128aad29aca9d4dbce5fc755672d96f11"},
- {file = "coverage-7.8.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:2db10dedeb619a771ef0e2949ccba7b75e33905de959c2643a4607bef2f3fb3a"},
- {file = "coverage-7.8.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e6ea7dba4e92926b7b5f0990634b78ea02f208d04af520c73a7c876d5a8d36cb"},
- {file = "coverage-7.8.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ef2f22795a7aca99fc3c84393a55a53dd18ab8c93fb431004e4d8f0774150f54"},
- {file = "coverage-7.8.2-cp311-cp311-win32.whl", hash = "sha256:641988828bc18a6368fe72355df5f1703e44411adbe49bba5644b941ce6f2e3a"},
- {file = "coverage-7.8.2-cp311-cp311-win_amd64.whl", hash = "sha256:8ab4a51cb39dc1933ba627e0875046d150e88478dbe22ce145a68393e9652975"},
- {file = "coverage-7.8.2-cp311-cp311-win_arm64.whl", hash = "sha256:8966a821e2083c74d88cca5b7dcccc0a3a888a596a04c0b9668a891de3a0cc53"},
- {file = "coverage-7.8.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:e2f6fe3654468d061942591aef56686131335b7a8325684eda85dacdf311356c"},
- {file = "coverage-7.8.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:76090fab50610798cc05241bf83b603477c40ee87acd358b66196ab0ca44ffa1"},
- {file = "coverage-7.8.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2bd0a0a5054be160777a7920b731a0570284db5142abaaf81bcbb282b8d99279"},
- {file = "coverage-7.8.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:da23ce9a3d356d0affe9c7036030b5c8f14556bd970c9b224f9c8205505e3b99"},
- {file = "coverage-7.8.2-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c9392773cffeb8d7e042a7b15b82a414011e9d2b5fdbbd3f7e6a6b17d5e21b20"},
- {file = "coverage-7.8.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:876cbfd0b09ce09d81585d266c07a32657beb3eaec896f39484b631555be0fe2"},
- {file = "coverage-7.8.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:3da9b771c98977a13fbc3830f6caa85cae6c9c83911d24cb2d218e9394259c57"},
- {file = "coverage-7.8.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:9a990f6510b3292686713bfef26d0049cd63b9c7bb17e0864f133cbfd2e6167f"},
- {file = "coverage-7.8.2-cp312-cp312-win32.whl", hash = "sha256:bf8111cddd0f2b54d34e96613e7fbdd59a673f0cf5574b61134ae75b6f5a33b8"},
- {file = "coverage-7.8.2-cp312-cp312-win_amd64.whl", hash = "sha256:86a323a275e9e44cdf228af9b71c5030861d4d2610886ab920d9945672a81223"},
- {file = "coverage-7.8.2-cp312-cp312-win_arm64.whl", hash = "sha256:820157de3a589e992689ffcda8639fbabb313b323d26388d02e154164c57b07f"},
- {file = "coverage-7.8.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:ea561010914ec1c26ab4188aef8b1567272ef6de096312716f90e5baa79ef8ca"},
- {file = "coverage-7.8.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:cb86337a4fcdd0e598ff2caeb513ac604d2f3da6d53df2c8e368e07ee38e277d"},
- {file = "coverage-7.8.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:26a4636ddb666971345541b59899e969f3b301143dd86b0ddbb570bd591f1e85"},
- {file = "coverage-7.8.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5040536cf9b13fb033f76bcb5e1e5cb3b57c4807fef37db9e0ed129c6a094257"},
- {file = "coverage-7.8.2-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dc67994df9bcd7e0150a47ef41278b9e0a0ea187caba72414b71dc590b99a108"},
- {file = "coverage-7.8.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:6e6c86888fd076d9e0fe848af0a2142bf606044dc5ceee0aa9eddb56e26895a0"},
- {file = "coverage-7.8.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:684ca9f58119b8e26bef860db33524ae0365601492e86ba0b71d513f525e7050"},
- {file = "coverage-7.8.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8165584ddedb49204c4e18da083913bdf6a982bfb558632a79bdaadcdafd0d48"},
- {file = "coverage-7.8.2-cp313-cp313-win32.whl", hash = "sha256:34759ee2c65362163699cc917bdb2a54114dd06d19bab860725f94ef45a3d9b7"},
- {file = "coverage-7.8.2-cp313-cp313-win_amd64.whl", hash = "sha256:2f9bc608fbafaee40eb60a9a53dbfb90f53cc66d3d32c2849dc27cf5638a21e3"},
- {file = "coverage-7.8.2-cp313-cp313-win_arm64.whl", hash = "sha256:9fe449ee461a3b0c7105690419d0b0aba1232f4ff6d120a9e241e58a556733f7"},
- {file = "coverage-7.8.2-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:8369a7c8ef66bded2b6484053749ff220dbf83cba84f3398c84c51a6f748a008"},
- {file = "coverage-7.8.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:159b81df53a5fcbc7d45dae3adad554fdbde9829a994e15227b3f9d816d00b36"},
- {file = "coverage-7.8.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e6fcbbd35a96192d042c691c9e0c49ef54bd7ed865846a3c9d624c30bb67ce46"},
- {file = "coverage-7.8.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:05364b9cc82f138cc86128dc4e2e1251c2981a2218bfcd556fe6b0fbaa3501be"},
- {file = "coverage-7.8.2-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:46d532db4e5ff3979ce47d18e2fe8ecad283eeb7367726da0e5ef88e4fe64740"},
- {file = "coverage-7.8.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:4000a31c34932e7e4fa0381a3d6deb43dc0c8f458e3e7ea6502e6238e10be625"},
- {file = "coverage-7.8.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:43ff5033d657cd51f83015c3b7a443287250dc14e69910577c3e03bd2e06f27b"},
- {file = "coverage-7.8.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:94316e13f0981cbbba132c1f9f365cac1d26716aaac130866ca812006f662199"},
- {file = "coverage-7.8.2-cp313-cp313t-win32.whl", hash = "sha256:3f5673888d3676d0a745c3d0e16da338c5eea300cb1f4ada9c872981265e76d8"},
- {file = "coverage-7.8.2-cp313-cp313t-win_amd64.whl", hash = "sha256:2c08b05ee8d7861e45dc5a2cc4195c8c66dca5ac613144eb6ebeaff2d502e73d"},
- {file = "coverage-7.8.2-cp313-cp313t-win_arm64.whl", hash = "sha256:1e1448bb72b387755e1ff3ef1268a06617afd94188164960dba8d0245a46004b"},
- {file = "coverage-7.8.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:496948261eaac5ac9cf43f5d0a9f6eb7a6d4cb3bedb2c5d294138142f5c18f2a"},
- {file = "coverage-7.8.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:eacd2de0d30871eff893bab0b67840a96445edcb3c8fd915e6b11ac4b2f3fa6d"},
- {file = "coverage-7.8.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b039ffddc99ad65d5078ef300e0c7eed08c270dc26570440e3ef18beb816c1ca"},
- {file = "coverage-7.8.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0e49824808d4375ede9dd84e9961a59c47f9113039f1a525e6be170aa4f5c34d"},
- {file = "coverage-7.8.2-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b069938961dfad881dc2f8d02b47645cd2f455d3809ba92a8a687bf513839787"},
- {file = "coverage-7.8.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:de77c3ba8bb686d1c411e78ee1b97e6e0b963fb98b1637658dd9ad2c875cf9d7"},
- {file = "coverage-7.8.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:1676628065a498943bd3f64f099bb573e08cf1bc6088bbe33cf4424e0876f4b3"},
- {file = "coverage-7.8.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:8e1a26e7e50076e35f7afafde570ca2b4d7900a491174ca357d29dece5aacee7"},
- {file = "coverage-7.8.2-cp39-cp39-win32.whl", hash = "sha256:6782a12bf76fa61ad9350d5a6ef5f3f020b57f5e6305cbc663803f2ebd0f270a"},
- {file = "coverage-7.8.2-cp39-cp39-win_amd64.whl", hash = "sha256:1efa4166ba75ccefd647f2d78b64f53f14fb82622bc94c5a5cb0a622f50f1c9e"},
- {file = "coverage-7.8.2-pp39.pp310.pp311-none-any.whl", hash = "sha256:ec455eedf3ba0bbdf8f5a570012617eb305c63cb9f03428d39bf544cb2b94837"},
- {file = "coverage-7.8.2-py3-none-any.whl", hash = "sha256:726f32ee3713f7359696331a18daf0c3b3a70bb0ae71141b9d3c52be7c595e32"},
- {file = "coverage-7.8.2.tar.gz", hash = "sha256:a886d531373a1f6ff9fad2a2ba4a045b68467b779ae729ee0b3b10ac20033b27"},
+ {file = "coverage-7.9.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:cc94d7c5e8423920787c33d811c0be67b7be83c705f001f7180c7b186dcf10ca"},
+ {file = "coverage-7.9.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:16aa0830d0c08a2c40c264cef801db8bc4fc0e1892782e45bcacbd5889270509"},
+ {file = "coverage-7.9.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cf95981b126f23db63e9dbe4cf65bd71f9a6305696fa5e2262693bc4e2183f5b"},
+ {file = "coverage-7.9.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f05031cf21699785cd47cb7485f67df619e7bcdae38e0fde40d23d3d0210d3c3"},
+ {file = "coverage-7.9.1-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bb4fbcab8764dc072cb651a4bcda4d11fb5658a1d8d68842a862a6610bd8cfa3"},
+ {file = "coverage-7.9.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:0f16649a7330ec307942ed27d06ee7e7a38417144620bb3d6e9a18ded8a2d3e5"},
+ {file = "coverage-7.9.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:cea0a27a89e6432705fffc178064503508e3c0184b4f061700e771a09de58187"},
+ {file = "coverage-7.9.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:e980b53a959fa53b6f05343afbd1e6f44a23ed6c23c4b4c56c6662bbb40c82ce"},
+ {file = "coverage-7.9.1-cp310-cp310-win32.whl", hash = "sha256:70760b4c5560be6ca70d11f8988ee6542b003f982b32f83d5ac0b72476607b70"},
+ {file = "coverage-7.9.1-cp310-cp310-win_amd64.whl", hash = "sha256:a66e8f628b71f78c0e0342003d53b53101ba4e00ea8dabb799d9dba0abbbcebe"},
+ {file = "coverage-7.9.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:95c765060e65c692da2d2f51a9499c5e9f5cf5453aeaf1420e3fc847cc060582"},
+ {file = "coverage-7.9.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ba383dc6afd5ec5b7a0d0c23d38895db0e15bcba7fb0fa8901f245267ac30d86"},
+ {file = "coverage-7.9.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:37ae0383f13cbdcf1e5e7014489b0d71cc0106458878ccde52e8a12ced4298ed"},
+ {file = "coverage-7.9.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:69aa417a030bf11ec46149636314c24c8d60fadb12fc0ee8f10fda0d918c879d"},
+ {file = "coverage-7.9.1-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0a4be2a28656afe279b34d4f91c3e26eccf2f85500d4a4ff0b1f8b54bf807338"},
+ {file = "coverage-7.9.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:382e7ddd5289f140259b610e5f5c58f713d025cb2f66d0eb17e68d0a94278875"},
+ {file = "coverage-7.9.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e5532482344186c543c37bfad0ee6069e8ae4fc38d073b8bc836fc8f03c9e250"},
+ {file = "coverage-7.9.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:a39d18b3f50cc121d0ce3838d32d58bd1d15dab89c910358ebefc3665712256c"},
+ {file = "coverage-7.9.1-cp311-cp311-win32.whl", hash = "sha256:dd24bd8d77c98557880def750782df77ab2b6885a18483dc8588792247174b32"},
+ {file = "coverage-7.9.1-cp311-cp311-win_amd64.whl", hash = "sha256:6b55ad10a35a21b8015eabddc9ba31eb590f54adc9cd39bcf09ff5349fd52125"},
+ {file = "coverage-7.9.1-cp311-cp311-win_arm64.whl", hash = "sha256:6ad935f0016be24c0e97fc8c40c465f9c4b85cbbe6eac48934c0dc4d2568321e"},
+ {file = "coverage-7.9.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:a8de12b4b87c20de895f10567639c0797b621b22897b0af3ce4b4e204a743626"},
+ {file = "coverage-7.9.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:5add197315a054e92cee1b5f686a2bcba60c4c3e66ee3de77ace6c867bdee7cb"},
+ {file = "coverage-7.9.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:600a1d4106fe66f41e5d0136dfbc68fe7200a5cbe85610ddf094f8f22e1b0300"},
+ {file = "coverage-7.9.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2a876e4c3e5a2a1715a6608906aa5a2e0475b9c0f68343c2ada98110512ab1d8"},
+ {file = "coverage-7.9.1-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:81f34346dd63010453922c8e628a52ea2d2ccd73cb2487f7700ac531b247c8a5"},
+ {file = "coverage-7.9.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:888f8eee13f2377ce86d44f338968eedec3291876b0b8a7289247ba52cb984cd"},
+ {file = "coverage-7.9.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:9969ef1e69b8c8e1e70d591f91bbc37fc9a3621e447525d1602801a24ceda898"},
+ {file = "coverage-7.9.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:60c458224331ee3f1a5b472773e4a085cc27a86a0b48205409d364272d67140d"},
+ {file = "coverage-7.9.1-cp312-cp312-win32.whl", hash = "sha256:5f646a99a8c2b3ff4c6a6e081f78fad0dde275cd59f8f49dc4eab2e394332e74"},
+ {file = "coverage-7.9.1-cp312-cp312-win_amd64.whl", hash = "sha256:30f445f85c353090b83e552dcbbdad3ec84c7967e108c3ae54556ca69955563e"},
+ {file = "coverage-7.9.1-cp312-cp312-win_arm64.whl", hash = "sha256:af41da5dca398d3474129c58cb2b106a5d93bbb196be0d307ac82311ca234342"},
+ {file = "coverage-7.9.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:31324f18d5969feef7344a932c32428a2d1a3e50b15a6404e97cba1cc9b2c631"},
+ {file = "coverage-7.9.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0c804506d624e8a20fb3108764c52e0eef664e29d21692afa375e0dd98dc384f"},
+ {file = "coverage-7.9.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ef64c27bc40189f36fcc50c3fb8f16ccda73b6a0b80d9bd6e6ce4cffcd810bbd"},
+ {file = "coverage-7.9.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d4fe2348cc6ec372e25adec0219ee2334a68d2f5222e0cba9c0d613394e12d86"},
+ {file = "coverage-7.9.1-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:34ed2186fe52fcc24d4561041979a0dec69adae7bce2ae8d1c49eace13e55c43"},
+ {file = "coverage-7.9.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:25308bd3d00d5eedd5ae7d4357161f4df743e3c0240fa773ee1b0f75e6c7c0f1"},
+ {file = "coverage-7.9.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:73e9439310f65d55a5a1e0564b48e34f5369bee943d72c88378f2d576f5a5751"},
+ {file = "coverage-7.9.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:37ab6be0859141b53aa89412a82454b482c81cf750de4f29223d52268a86de67"},
+ {file = "coverage-7.9.1-cp313-cp313-win32.whl", hash = "sha256:64bdd969456e2d02a8b08aa047a92d269c7ac1f47e0c977675d550c9a0863643"},
+ {file = "coverage-7.9.1-cp313-cp313-win_amd64.whl", hash = "sha256:be9e3f68ca9edb897c2184ad0eee815c635565dbe7a0e7e814dc1f7cbab92c0a"},
+ {file = "coverage-7.9.1-cp313-cp313-win_arm64.whl", hash = "sha256:1c503289ffef1d5105d91bbb4d62cbe4b14bec4d13ca225f9c73cde9bb46207d"},
+ {file = "coverage-7.9.1-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:0b3496922cb5f4215bf5caaef4cf12364a26b0be82e9ed6d050f3352cf2d7ef0"},
+ {file = "coverage-7.9.1-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:9565c3ab1c93310569ec0d86b017f128f027cab0b622b7af288696d7ed43a16d"},
+ {file = "coverage-7.9.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2241ad5dbf79ae1d9c08fe52b36d03ca122fb9ac6bca0f34439e99f8327ac89f"},
+ {file = "coverage-7.9.1-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3bb5838701ca68b10ebc0937dbd0eb81974bac54447c55cd58dea5bca8451029"},
+ {file = "coverage-7.9.1-cp313-cp313t-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b30a25f814591a8c0c5372c11ac8967f669b97444c47fd794926e175c4047ece"},
+ {file = "coverage-7.9.1-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:2d04b16a6062516df97969f1ae7efd0de9c31eb6ebdceaa0d213b21c0ca1a683"},
+ {file = "coverage-7.9.1-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:7931b9e249edefb07cd6ae10c702788546341d5fe44db5b6108a25da4dca513f"},
+ {file = "coverage-7.9.1-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:52e92b01041151bf607ee858e5a56c62d4b70f4dac85b8c8cb7fb8a351ab2c10"},
+ {file = "coverage-7.9.1-cp313-cp313t-win32.whl", hash = "sha256:684e2110ed84fd1ca5f40e89aa44adf1729dc85444004111aa01866507adf363"},
+ {file = "coverage-7.9.1-cp313-cp313t-win_amd64.whl", hash = "sha256:437c576979e4db840539674e68c84b3cda82bc824dd138d56bead1435f1cb5d7"},
+ {file = "coverage-7.9.1-cp313-cp313t-win_arm64.whl", hash = "sha256:18a0912944d70aaf5f399e350445738a1a20b50fbea788f640751c2ed9208b6c"},
+ {file = "coverage-7.9.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6f424507f57878e424d9a95dc4ead3fbdd72fd201e404e861e465f28ea469951"},
+ {file = "coverage-7.9.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:535fde4001b2783ac80865d90e7cc7798b6b126f4cd8a8c54acfe76804e54e58"},
+ {file = "coverage-7.9.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:02532fd3290bb8fa6bec876520842428e2a6ed6c27014eca81b031c2d30e3f71"},
+ {file = "coverage-7.9.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:56f5eb308b17bca3bbff810f55ee26d51926d9f89ba92707ee41d3c061257e55"},
+ {file = "coverage-7.9.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bfa447506c1a52271f1b0de3f42ea0fa14676052549095e378d5bff1c505ff7b"},
+ {file = "coverage-7.9.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:9ca8e220006966b4a7b68e8984a6aee645a0384b0769e829ba60281fe61ec4f7"},
+ {file = "coverage-7.9.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:49f1d0788ba5b7ba65933f3a18864117c6506619f5ca80326b478f72acf3f385"},
+ {file = "coverage-7.9.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:68cd53aec6f45b8e4724c0950ce86eacb775c6be01ce6e3669fe4f3a21e768ed"},
+ {file = "coverage-7.9.1-cp39-cp39-win32.whl", hash = "sha256:95335095b6c7b1cc14c3f3f17d5452ce677e8490d101698562b2ffcacc304c8d"},
+ {file = "coverage-7.9.1-cp39-cp39-win_amd64.whl", hash = "sha256:e1b5191d1648acc439b24721caab2fd0c86679d8549ed2c84d5a7ec1bedcc244"},
+ {file = "coverage-7.9.1-pp39.pp310.pp311-none-any.whl", hash = "sha256:db0f04118d1db74db6c9e1cb1898532c7dcc220f1d2718f058601f7c3f499514"},
+ {file = "coverage-7.9.1-py3-none-any.whl", hash = "sha256:66b974b145aa189516b6bf2d8423e888b742517d37872f6ee4c5be0073bd9a3c"},
+ {file = "coverage-7.9.1.tar.gz", hash = "sha256:6cf43c78c4282708a28e466316935ec7489a9c487518a77fa68f716c67909cec"},
]
[package.dependencies]
From d4621fd11fc47def51d40447c9b410f0752e0ebf Mon Sep 17 00:00:00 2001
From: "Thomas Ds." <43245169+ThomasDsantos@users.noreply.github.com>
Date: Wed, 18 Jun 2025 16:49:50 +0200
Subject: [PATCH 075/112] Docs: fix manifest min count to merge description
(#2116)
# Rationale for this change
Fixing docs for
[commit.manifest.min-count-to-merge](https://py.iceberg.apache.org/configuration/#table-behavior-options)
py iceberg doc according to
https://iceberg.apache.org/docs/1.5.2/configuration/#table-behavior-properties
# Are these changes tested?
Nope, but just fixing typo :)
# Are there any user-facing changes?
Nope
---
mkdocs/docs/configuration.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/mkdocs/docs/configuration.md b/mkdocs/docs/configuration.md
index 4cc38db5dc..3037e0f5a2 100644
--- a/mkdocs/docs/configuration.md
+++ b/mkdocs/docs/configuration.md
@@ -75,7 +75,7 @@ Iceberg tables support table properties to configure table behavior.
| Key | Options | Default | Description |
| ------------------------------------ | ------------------- | ------------- | ----------------------------------------------------------- |
| `commit.manifest.target-size-bytes` | Size in bytes | 8388608 (8MB) | Target size when merging manifest files |
-| `commit.manifest.min-count-to-merge` | Number of manifests | 100 | Target size when merging manifest files |
+| `commit.manifest.min-count-to-merge` | Number of manifests | 100 | Minimum number of manifests to accumulate before merging |
| `commit.manifest-merge.enabled` | Boolean | False | Controls whether to automatically merge manifests on writes |
From 2a5fb97c30507961620213ec4a464dd28324a7c8 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Wed, 18 Jun 2025 17:01:53 +0200
Subject: [PATCH 076/112] Prefer `FileIO` over the PyArrow `FileSystem` (#2115)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
# Rationale for this change
This is problematic if you try to implement your own `FileIO`. Then
Streams are opened both through the FileIO and the FileSystem directly.
# Are these changes tested?
Yes, existing tests.
# Are there any user-facing changes?
No, but I think this makes the code esthetically also more pleasing by
removing complexity.
# Numbers
A while ago I did some inspection of the calls being made to S3, so just
to be sure that we don't alter anything, I've collected some stats using
a small "benchmark" locally:
```python
def test_fokko(session_catalog: RestCatalog):
parquet_file = "/Users/fokko.driesprong/Downloads/yellow_tripdata_2024-01.parquet"
from pyarrow import parquet as pq
df = pq.read_table(parquet_file)
try:
session_catalog.drop_table("default.taxi")
except Exception:
pass
tbl = session_catalog.create_table("default.taxi", schema=df.schema)
with tbl.update_spec() as tx:
tx.add_field("tpep_pickup_datetime", "hour")
tbl.append(df)
rounds = []
for _ in range(22):
start = round(time.time() * 1000)
assert len(tbl.scan().to_arrow()) == 2964624
stop = round(time.time() * 1000)
rounds.append(stop - start)
print(f"Took: {sum(rounds) / len(rounds)} ms on average")
```
Main:
Took: 1715.1818181818182 ms on average
```
> mc admin trace --stats minio
Call Count RPM Avg Time Min Time Max Time Avg TTFB Max TTFB Avg Size Rate /min Errors
s3.GetObject 77 (29.2%) 697.9 701µs 153µs 1.6ms 463µs 838µs ↑159B ↓712K ↑108K ↓485M 0
s3.HeadObject 73 (27.7%) 661.6 192µs 107µs 735µs 177µs 719µs ↑153B ↑99K 0
s3.CompleteMultipartUpload 37 (14.0%) 335.4 8.2ms 1.9ms 17.5ms 8.2ms 17.5ms ↑397B ↓507B ↑130K ↓166K 0
s3.NewMultipartUpload 37 (14.0%) 335.4 6.2ms 2.1ms 14.2ms 6.1ms 14.1ms ↑153B ↓437B ↑50K ↓143K 0
s3.PutObjectPart 37 (14.0%) 335.4 18.4ms 5.1ms 38.8ms 18.4ms 38.8ms ↑1.4M ↑469M 0
s3.PutObject 3 (1.1%) 27.2 5.4ms 3.4ms 8.8ms 5.3ms 8.8ms ↑2.8K ↑75K 0
```
Branch:
Took: 1723.1818181818182 ms on average
```
> mc admin trace --stats minio
Call Count RPM Avg Time Min Time Max Time Avg TTFB Max TTFB Avg Size Rate /min Errors
s3.GetObject 77 (29.2%) 696.3 927µs 171µs 4.5ms 610µs 3.5ms ↑159B ↓712K ↑108K ↓484M 0
s3.HeadObject 73 (27.7%) 660.1 222µs 109µs 1.2ms 205µs 1.2ms ↑153B ↑99K 0
s3.CompleteMultipartUpload 37 (14.0%) 334.6 4.4ms 1.2ms 14.2ms 4.4ms 14.2ms ↑397B ↓507B ↑130K ↓166K 0
s3.NewMultipartUpload 37 (14.0%) 334.6 4.3ms 1.2ms 15ms 4.3ms 15ms ↑153B ↓437B ↑50K ↓143K 0
s3.PutObjectPart 37 (14.0%) 334.6 14.5ms 2.6ms 30.7ms 14.5ms 30.7ms ↑1.4M ↑468M 0
s3.PutObject 3 (1.1%) 27.1 6.6ms 2.8ms 10.4ms 6.5ms 10.3ms ↑2.8K ↑75K 0
```
---
pyiceberg/io/__init__.py | 13 ----------
pyiceberg/io/pyarrow.py | 55 +++++++++-------------------------------
tests/io/test_pyarrow.py | 2 +-
3 files changed, 13 insertions(+), 57 deletions(-)
diff --git a/pyiceberg/io/__init__.py b/pyiceberg/io/__init__.py
index b6fa934fdd..ba9738070f 100644
--- a/pyiceberg/io/__init__.py
+++ b/pyiceberg/io/__init__.py
@@ -27,7 +27,6 @@
import importlib
import logging
-import os
import warnings
from abc import ABC, abstractmethod
from io import SEEK_SET
@@ -37,7 +36,6 @@
List,
Optional,
Protocol,
- Tuple,
Type,
Union,
runtime_checkable,
@@ -371,14 +369,3 @@ def load_file_io(properties: Properties = EMPTY_DICT, location: Optional[str] =
raise ModuleNotFoundError(
'Could not load a FileIO, please consider installing one: pip3 install "pyiceberg[pyarrow]", for more options refer to the docs.'
) from e
-
-
-def _parse_location(location: str) -> Tuple[str, str, str]:
- """Return the path without the scheme."""
- uri = urlparse(location)
- if not uri.scheme:
- return "file", uri.netloc, os.path.abspath(location)
- elif uri.scheme in ("hdfs", "viewfs"):
- return uri.scheme, uri.netloc, uri.path
- else:
- return uri.scheme, uri.netloc, f"{uri.netloc}{uri.path}"
diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py
index 1aaab32dbe..eeacaccfe2 100644
--- a/pyiceberg/io/pyarrow.py
+++ b/pyiceberg/io/pyarrow.py
@@ -69,7 +69,6 @@
FileInfo,
FileSystem,
FileType,
- FSSpecHandler,
)
from sortedcontainers import SortedList
@@ -117,7 +116,6 @@
InputStream,
OutputFile,
OutputStream,
- _parse_location,
)
from pyiceberg.manifest import (
DataFile,
@@ -309,9 +307,7 @@ def open(self, seekable: bool = True) -> InputStream:
input_file = self._filesystem.open_input_file(self._path)
else:
input_file = self._filesystem.open_input_stream(self._path, buffer_size=self._buffer_size)
- except FileNotFoundError:
- raise
- except PermissionError:
+ except (FileNotFoundError, PermissionError):
raise
except OSError as e:
if e.errno == 2 or "Path does not exist" in str(e):
@@ -916,27 +912,20 @@ def _get_file_format(file_format: FileFormat, **kwargs: Dict[str, Any]) -> ds.Fi
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]:
+def _read_deletes(io: FileIO, data_file: DataFile) -> Dict[str, pa.ChunkedArray]:
if data_file.file_format == FileFormat.PARQUET:
- 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()
+ with io.new_input(data_file.file_path).open() as fi:
+ delete_fragment = _get_file_format(
+ data_file.file_format, dictionary_columns=("file_path",), pre_buffer=True, buffer_size=ONE_MEGABYTE
+ ).make_fragment(fi)
+ table = ds.Scanner.from_fragment(fragment=delete_fragment).to_table()
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
}
elif data_file.file_format == FileFormat.PUFFIN:
- _, _, path = PyArrowFileIO.parse_location(data_file.file_path)
- with fs.open_input_file(path) as fi:
+ with io.new_input(data_file.file_path).open() as fi:
payload = fi.read()
return PuffinFile(payload).to_vector()
@@ -1383,7 +1372,7 @@ def _get_column_projection_values(
def _task_to_record_batches(
- fs: FileSystem,
+ io: FileIO,
task: FileScanTask,
bound_row_filter: BooleanExpression,
projected_schema: Schema,
@@ -1393,9 +1382,8 @@ def _task_to_record_batches(
name_mapping: Optional[NameMapping] = None,
partition_spec: Optional[PartitionSpec] = None,
) -> Iterator[pa.RecordBatch]:
- _, _, path = _parse_location(task.file.file_path)
arrow_format = ds.ParquetFileFormat(pre_buffer=True, buffer_size=(ONE_MEGABYTE * 8))
- with fs.open_input_file(path) as fin:
+ with io.new_input(task.file.file_path).open() as fin:
fragment = arrow_format.make_fragment(fin)
physical_schema = fragment.physical_schema
# In V1 and V2 table formats, we only support Timestamp 'us' in Iceberg Schema
@@ -1479,7 +1467,7 @@ def _read_all_delete_files(io: FileIO, tasks: Iterable[FileScanTask]) -> Dict[st
executor = ExecutorFactory.get_or_create()
deletes_per_files: Iterator[Dict[str, ChunkedArray]] = executor.map(
lambda args: _read_deletes(*args),
- [(_fs_from_file_path(io, delete_file.file_path), delete_file) for delete_file in unique_deletes],
+ [(io, delete_file) for delete_file in unique_deletes],
)
for delete in deletes_per_files:
for file, arr in delete.items():
@@ -1491,25 +1479,6 @@ def _read_all_delete_files(io: FileIO, tasks: Iterable[FileScanTask]) -> Dict[st
return deletes_per_file
-def _fs_from_file_path(io: FileIO, file_path: str) -> FileSystem:
- scheme, netloc, _ = _parse_location(file_path)
- if isinstance(io, PyArrowFileIO):
- return io.fs_by_scheme(scheme, netloc)
- else:
- try:
- from pyiceberg.io.fsspec import FsspecFileIO
-
- if isinstance(io, FsspecFileIO):
- from pyarrow.fs import PyFileSystem
-
- return PyFileSystem(FSSpecHandler(io.get_fs(scheme)))
- else:
- raise ValueError(f"Expected PyArrowFileIO or FsspecFileIO, got: {io}")
- except ModuleNotFoundError as e:
- # When FsSpec is not installed
- raise ValueError(f"Expected PyArrowFileIO or FsspecFileIO, got: {io}") from e
-
-
class ArrowScan:
_table_metadata: TableMetadata
_io: FileIO
@@ -1654,7 +1623,7 @@ def _record_batches_from_scan_tasks_and_deletes(
if self._limit is not None and total_row_count >= self._limit:
break
batches = _task_to_record_batches(
- _fs_from_file_path(self._io, task.file.file_path),
+ self._io,
task,
self._bound_row_filter,
self._projected_schema,
diff --git a/tests/io/test_pyarrow.py b/tests/io/test_pyarrow.py
index e90f3a46fc..9daefacdca 100644
--- a/tests/io/test_pyarrow.py
+++ b/tests/io/test_pyarrow.py
@@ -1539,7 +1539,7 @@ def deletes_file(tmp_path: str, example_task: FileScanTask) -> str:
def test_read_deletes(deletes_file: str, example_task: FileScanTask) -> None:
- deletes = _read_deletes(LocalFileSystem(), DataFile.from_args(file_path=deletes_file, file_format=FileFormat.PARQUET))
+ deletes = _read_deletes(PyArrowFileIO(), DataFile.from_args(file_path=deletes_file, file_format=FileFormat.PARQUET))
assert set(deletes.keys()) == {example_task.file.file_path}
assert list(deletes.values())[0] == pa.chunked_array([[1, 3, 5]])
From f1ef03dd13448d3f0412c99abcacae6897b2f87b Mon Sep 17 00:00:00 2001
From: Brad <61765732+ForeverAngry@users.noreply.github.com>
Date: Thu, 19 Jun 2025 15:49:57 -0400
Subject: [PATCH 077/112] Add `ExpireSnapshots` (#1880)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
## Summary
This PR Closes issue #516 by implementing support for the
`ExpireSnapshot` table metadata action.
## Rationale
The `ExpireSnapshot` action is a core part of Iceberg’s table
maintenance APIs. Adding support for this action in PyIceberg helps
ensure feature parity with other language implementations (e.g., Java)
and supports users who want to programmatically manage snapshot
retention using PyIceberg’s public API.
## Testing
- Unit tests have been added to cover the initial expected usage paths.
- Additional feedback on edge cases, missing test scenarios or
corrections to the setup test logic is greatly welcome during the review
process.
## User-facing changes
- This change introduces a new public API: `ExpireSnapshot`.
- No breaking changes or modifications to existing APIs were made.
---
---------
Co-authored-by: Fokko Driesprong
---
pyiceberg/table/__init__.py | 6 +-
pyiceberg/table/update/snapshot.py | 102 ++++++++++++
tests/table/test_expire_snapshots.py | 224 +++++++++++++++++++++++++++
3 files changed, 331 insertions(+), 1 deletion(-)
create mode 100644 tests/table/test_expire_snapshots.py
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index dabc2ee4b8..294a6bac34 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -115,7 +115,7 @@
update_table_metadata,
)
from pyiceberg.table.update.schema import UpdateSchema
-from pyiceberg.table.update.snapshot import ManageSnapshots, RewriteManifestsResult, UpdateSnapshot, _FastAppendFiles
+from pyiceberg.table.update.snapshot import ExpireSnapshots, ManageSnapshots, RewriteManifestsResult, UpdateSnapshot, _FastAppendFiles
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
from pyiceberg.transforms import IdentityTransform
@@ -1202,6 +1202,10 @@ def manage_snapshots(self) -> ManageSnapshots:
"""
return ManageSnapshots(transaction=Transaction(self, autocommit=True))
+ def expire_snapshots(self) -> ExpireSnapshots:
+ """Shorthand to run expire snapshots by id or by a timestamp."""
+ return ExpireSnapshots(transaction=Transaction(self, autocommit=True))
+
def update_statistics(self) -> UpdateStatistics:
"""
Shorthand to run statistics management operations like add statistics and remove statistics.
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 599c7fa0f3..264c5242fc 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -57,6 +57,7 @@
from pyiceberg.partitioning import (
PartitionSpec,
)
+from pyiceberg.table.refs import SnapshotRefType
from pyiceberg.table.snapshots import (
Operation,
Snapshot,
@@ -68,6 +69,7 @@
AddSnapshotUpdate,
AssertRefSnapshotId,
RemoveSnapshotRefUpdate,
+ RemoveSnapshotsUpdate,
SetSnapshotRefUpdate,
TableRequirement,
TableUpdate,
@@ -1018,3 +1020,103 @@ def remove_branch(self, branch_name: str) -> ManageSnapshots:
This for method chaining
"""
return self._remove_ref_snapshot(ref_name=branch_name)
+
+
+class ExpireSnapshots(UpdateTableMetadata["ExpireSnapshots"]):
+ """
+ Expire snapshots by ID.
+
+ Use table.expire_snapshots().().commit() to run a specific operation.
+ Use table.expire_snapshots().().().commit() to run multiple operations.
+ Pending changes are applied on commit.
+ """
+
+ _snapshot_ids_to_expire: Set[int] = set()
+ _updates: Tuple[TableUpdate, ...] = ()
+ _requirements: Tuple[TableRequirement, ...] = ()
+
+ def _commit(self) -> UpdatesAndRequirements:
+ """
+ Commit the staged updates and requirements.
+
+ This will remove the snapshots with the given IDs, but will always skip protected snapshots (branch/tag heads).
+
+ Returns:
+ Tuple of updates and requirements to be committed,
+ as required by the calling parent apply functions.
+ """
+ # Remove any protected snapshot IDs from the set to expire, just in case
+ protected_ids = self._get_protected_snapshot_ids()
+ self._snapshot_ids_to_expire -= protected_ids
+ update = RemoveSnapshotsUpdate(snapshot_ids=self._snapshot_ids_to_expire)
+ self._updates += (update,)
+ return self._updates, self._requirements
+
+ def _get_protected_snapshot_ids(self) -> Set[int]:
+ """
+ Get the IDs of protected snapshots.
+
+ These are the HEAD snapshots of all branches and all tagged snapshots. These ids are to be excluded from expiration.
+
+ Returns:
+ Set of protected snapshot IDs to exclude from expiration.
+ """
+ protected_ids: Set[int] = set()
+
+ for ref in self._transaction.table_metadata.refs.values():
+ if ref.snapshot_ref_type in [SnapshotRefType.TAG, SnapshotRefType.BRANCH]:
+ protected_ids.add(ref.snapshot_id)
+
+ return protected_ids
+
+ def expire_snapshot_by_id(self, snapshot_id: int) -> ExpireSnapshots:
+ """
+ Expire a snapshot by its ID.
+
+ This will mark the snapshot for expiration.
+
+ Args:
+ snapshot_id (int): The ID of the snapshot to expire.
+ Returns:
+ This for method chaining.
+ """
+ if self._transaction.table_metadata.snapshot_by_id(snapshot_id) is None:
+ raise ValueError(f"Snapshot with ID {snapshot_id} does not exist.")
+
+ if snapshot_id in self._get_protected_snapshot_ids():
+ raise ValueError(f"Snapshot with ID {snapshot_id} is protected and cannot be expired.")
+
+ self._snapshot_ids_to_expire.add(snapshot_id)
+
+ return self
+
+ def expire_snapshots_by_ids(self, snapshot_ids: List[int]) -> "ExpireSnapshots":
+ """
+ Expire multiple snapshots by their IDs.
+
+ This will mark the snapshots for expiration.
+
+ Args:
+ snapshot_ids (List[int]): List of snapshot IDs to expire.
+ Returns:
+ This for method chaining.
+ """
+ for snapshot_id in snapshot_ids:
+ self.expire_snapshot_by_id(snapshot_id)
+ return self
+
+ def expire_snapshots_older_than(self, timestamp_ms: int) -> "ExpireSnapshots":
+ """
+ Expire all unprotected snapshots with a timestamp older than a given value.
+
+ Args:
+ timestamp_ms (int): Only snapshots with timestamp_ms < this value will be expired.
+
+ Returns:
+ This for method chaining.
+ """
+ protected_ids = self._get_protected_snapshot_ids()
+ for snapshot in self._transaction.table_metadata.snapshots:
+ if snapshot.timestamp_ms < timestamp_ms and snapshot.snapshot_id not in protected_ids:
+ self._snapshot_ids_to_expire.add(snapshot.snapshot_id)
+ return self
diff --git a/tests/table/test_expire_snapshots.py b/tests/table/test_expire_snapshots.py
new file mode 100644
index 0000000000..82ecb9e493
--- /dev/null
+++ b/tests/table/test_expire_snapshots.py
@@ -0,0 +1,224 @@
+# 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.
+from unittest.mock import MagicMock
+from uuid import uuid4
+
+import pytest
+
+from pyiceberg.table import CommitTableResponse, Table
+
+
+def test_cannot_expire_protected_head_snapshot(table_v2: Table) -> None:
+ """Test that a HEAD (branch) snapshot cannot be expired."""
+ HEAD_SNAPSHOT = 3051729675574597004
+ KEEP_SNAPSHOT = 3055729675574597004
+
+ # Mock the catalog's commit_table method
+ table_v2.catalog = MagicMock()
+ # Simulate refs protecting HEAD_SNAPSHOT as a branch
+ table_v2.metadata = table_v2.metadata.model_copy(
+ update={
+ "refs": {
+ "main": MagicMock(snapshot_id=HEAD_SNAPSHOT, snapshot_ref_type="branch"),
+ "tag1": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="tag"),
+ }
+ }
+ )
+ # Assert fixture data
+ assert any(ref.snapshot_id == HEAD_SNAPSHOT for ref in table_v2.metadata.refs.values())
+
+ # Attempt to expire the HEAD snapshot and expect a ValueError
+ with pytest.raises(ValueError, match=f"Snapshot with ID {HEAD_SNAPSHOT} is protected and cannot be expired."):
+ table_v2.expire_snapshots().expire_snapshot_by_id(HEAD_SNAPSHOT).commit()
+
+ table_v2.catalog.commit_table.assert_not_called()
+
+
+def test_cannot_expire_tagged_snapshot(table_v2: Table) -> None:
+ """Test that a tagged snapshot cannot be expired."""
+ TAGGED_SNAPSHOT = 3051729675574597004
+ KEEP_SNAPSHOT = 3055729675574597004
+
+ table_v2.catalog = MagicMock()
+ # Simulate refs protecting TAGGED_SNAPSHOT as a tag
+ table_v2.metadata = table_v2.metadata.model_copy(
+ update={
+ "refs": {
+ "tag1": MagicMock(snapshot_id=TAGGED_SNAPSHOT, snapshot_ref_type="tag"),
+ "main": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="branch"),
+ }
+ }
+ )
+ assert any(ref.snapshot_id == TAGGED_SNAPSHOT for ref in table_v2.metadata.refs.values())
+
+ with pytest.raises(ValueError, match=f"Snapshot with ID {TAGGED_SNAPSHOT} is protected and cannot be expired."):
+ table_v2.expire_snapshots().expire_snapshot_by_id(TAGGED_SNAPSHOT).commit()
+
+ table_v2.catalog.commit_table.assert_not_called()
+
+
+def test_expire_unprotected_snapshot(table_v2: Table) -> None:
+ """Test that an unprotected snapshot can be expired."""
+ EXPIRE_SNAPSHOT = 3051729675574597004
+ KEEP_SNAPSHOT = 3055729675574597004
+
+ mock_response = CommitTableResponse(
+ metadata=table_v2.metadata.model_copy(update={"snapshots": [KEEP_SNAPSHOT]}),
+ metadata_location="mock://metadata/location",
+ uuid=uuid4(),
+ )
+ table_v2.catalog = MagicMock()
+ table_v2.catalog.commit_table.return_value = mock_response
+
+ # Remove any refs that protect the snapshot to be expired
+ table_v2.metadata = table_v2.metadata.model_copy(
+ update={
+ "refs": {
+ "main": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="branch"),
+ "tag1": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="tag"),
+ }
+ }
+ )
+
+ # Assert fixture data
+ assert all(ref.snapshot_id != EXPIRE_SNAPSHOT for ref in table_v2.metadata.refs.values())
+
+ # Expire the snapshot
+ table_v2.expire_snapshots().expire_snapshot_by_id(EXPIRE_SNAPSHOT).commit()
+
+ table_v2.catalog.commit_table.assert_called_once()
+ remaining_snapshots = table_v2.metadata.snapshots
+ assert EXPIRE_SNAPSHOT not in remaining_snapshots
+ assert len(table_v2.metadata.snapshots) == 1
+
+
+def test_expire_nonexistent_snapshot_raises(table_v2: Table) -> None:
+ """Test that trying to expire a non-existent snapshot raises an error."""
+ NONEXISTENT_SNAPSHOT = 9999999999999999999
+
+ table_v2.catalog = MagicMock()
+ table_v2.metadata = table_v2.metadata.model_copy(update={"refs": {}})
+
+ with pytest.raises(ValueError, match=f"Snapshot with ID {NONEXISTENT_SNAPSHOT} does not exist."):
+ table_v2.expire_snapshots().expire_snapshot_by_id(NONEXISTENT_SNAPSHOT).commit()
+
+ table_v2.catalog.commit_table.assert_not_called()
+
+
+def test_expire_snapshots_by_timestamp_skips_protected(table_v2: Table) -> None:
+ # Setup: two snapshots; both are old, but one is head/tag protected
+ HEAD_SNAPSHOT = 3051729675574597004
+ TAGGED_SNAPSHOT = 3055729675574597004
+
+ # Add snapshots to metadata for timestamp/protected test
+ from types import SimpleNamespace
+
+ table_v2.metadata = table_v2.metadata.model_copy(
+ update={
+ "refs": {
+ "main": MagicMock(snapshot_id=HEAD_SNAPSHOT, snapshot_ref_type="branch"),
+ "mytag": MagicMock(snapshot_id=TAGGED_SNAPSHOT, snapshot_ref_type="tag"),
+ },
+ "snapshots": [
+ SimpleNamespace(snapshot_id=HEAD_SNAPSHOT, timestamp_ms=1, parent_snapshot_id=None),
+ SimpleNamespace(snapshot_id=TAGGED_SNAPSHOT, timestamp_ms=1, parent_snapshot_id=None),
+ ],
+ }
+ )
+ table_v2.catalog = MagicMock()
+
+ # Attempt to expire all snapshots before a future timestamp (so both are candidates)
+ future_timestamp = 9999999999999 # Far in the future, after any real snapshot
+
+ # Mock the catalog's commit_table to return the current metadata (simulate no change)
+ mock_response = CommitTableResponse(
+ metadata=table_v2.metadata, # protected snapshots remain
+ metadata_location="mock://metadata/location",
+ uuid=uuid4(),
+ )
+ table_v2.catalog.commit_table.return_value = mock_response
+
+ table_v2.expire_snapshots().expire_snapshots_older_than(future_timestamp).commit()
+ # Update metadata to reflect the commit (as in other tests)
+ table_v2.metadata = mock_response.metadata
+
+ # Both protected snapshots should remain
+ remaining_ids = {s.snapshot_id for s in table_v2.metadata.snapshots}
+ assert HEAD_SNAPSHOT in remaining_ids
+ assert TAGGED_SNAPSHOT in remaining_ids
+
+ # No snapshots should have been expired (commit_table called, but with empty snapshot_ids)
+ args, kwargs = table_v2.catalog.commit_table.call_args
+ updates = args[2] if len(args) > 2 else ()
+ # Find RemoveSnapshotsUpdate in updates
+ remove_update = next((u for u in updates if getattr(u, "action", None) == "remove-snapshots"), None)
+ assert remove_update is not None
+ assert remove_update.snapshot_ids == []
+
+
+def test_expire_snapshots_by_ids(table_v2: Table) -> None:
+ """Test that multiple unprotected snapshots can be expired by IDs."""
+ EXPIRE_SNAPSHOT_1 = 3051729675574597004
+ EXPIRE_SNAPSHOT_2 = 3051729675574597005
+ KEEP_SNAPSHOT = 3055729675574597004
+
+ mock_response = CommitTableResponse(
+ metadata=table_v2.metadata.model_copy(update={"snapshots": [KEEP_SNAPSHOT]}),
+ metadata_location="mock://metadata/location",
+ uuid=uuid4(),
+ )
+ table_v2.catalog = MagicMock()
+ table_v2.catalog.commit_table.return_value = mock_response
+
+ # Remove any refs that protect the snapshots to be expired
+ table_v2.metadata = table_v2.metadata.model_copy(
+ update={
+ "refs": {
+ "main": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="branch"),
+ "tag1": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="tag"),
+ }
+ }
+ )
+
+ # Add snapshots to metadata for multi-id test
+ from types import SimpleNamespace
+
+ table_v2.metadata = table_v2.metadata.model_copy(
+ update={
+ "refs": {
+ "main": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="branch"),
+ "tag1": MagicMock(snapshot_id=KEEP_SNAPSHOT, snapshot_ref_type="tag"),
+ },
+ "snapshots": [
+ SimpleNamespace(snapshot_id=EXPIRE_SNAPSHOT_1, timestamp_ms=1, parent_snapshot_id=None),
+ SimpleNamespace(snapshot_id=EXPIRE_SNAPSHOT_2, timestamp_ms=1, parent_snapshot_id=None),
+ SimpleNamespace(snapshot_id=KEEP_SNAPSHOT, timestamp_ms=2, parent_snapshot_id=None),
+ ],
+ }
+ )
+
+ # Assert fixture data
+ assert all(ref.snapshot_id not in (EXPIRE_SNAPSHOT_1, EXPIRE_SNAPSHOT_2) for ref in table_v2.metadata.refs.values())
+
+ # Expire the snapshots
+ table_v2.expire_snapshots().expire_snapshots_by_ids([EXPIRE_SNAPSHOT_1, EXPIRE_SNAPSHOT_2]).commit()
+
+ table_v2.catalog.commit_table.assert_called_once()
+ remaining_snapshots = table_v2.metadata.snapshots
+ assert EXPIRE_SNAPSHOT_1 not in remaining_snapshots
+ assert EXPIRE_SNAPSHOT_2 not in remaining_snapshots
+ assert len(table_v2.metadata.snapshots) == 1
From d988323ad8a8de379ae26fcface65720b2c709d7 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Fri, 20 Jun 2025 16:36:27 +0200
Subject: [PATCH 078/112] Build: Bump third party deps (#2127)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
In https://github.com/apache/iceberg-python/pull/2125 we got a warning
from the google-auth package, and we're on `1.6.3` while `2.40.3` is out
there. So maybe good to bump everything 👍
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
poetry.lock | 871 ++++++++++++++++++++++++++++------------------------
1 file changed, 478 insertions(+), 393 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index f4da5368b1..887d029eff 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -59,7 +59,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -67,99 +67,99 @@ files = [
[[package]]
name = "aiohttp"
-version = "3.12.12"
+version = "3.12.13"
description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
-files = [
- {file = "aiohttp-3.12.12-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6f25e9d274d6abbb15254f76f100c3984d6b9ad6e66263cc60a465dd5c7e48f5"},
- {file = "aiohttp-3.12.12-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b8ec3c1a1c13d24941b5b913607e57b9364e4c0ea69d5363181467492c4b2ba6"},
- {file = "aiohttp-3.12.12-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:81ef2f9253c327c211cb7b06ea2edd90e637cf21c347b894d540466b8d304e08"},
- {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:28ded835c3663fd41c9ad44685811b11e34e6ac9a7516a30bfce13f6abba4496"},
- {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:a4b78ccf254fc10605b263996949a94ca3f50e4f9100e05137d6583e266b711e"},
- {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4f4a5af90d5232c41bb857568fe7d11ed84408653ec9da1ff999cc30258b9bd1"},
- {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:ffa5205c2f53f1120e93fdf2eca41b0f6344db131bc421246ee82c1e1038a14a"},
- {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f68301660f0d7a3eddfb84f959f78a8f9db98c76a49b5235508fa16edaad0f7c"},
- {file = "aiohttp-3.12.12-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:db874d3b0c92fdbb553751af9d2733b378c25cc83cd9dfba87f12fafd2dc9cd5"},
- {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:5e53cf9c201b45838a2d07b1f2d5f7fec9666db7979240002ce64f9b8a1e0cf2"},
- {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:8687cc5f32b4e328c233acd387d09a1b477007896b2f03c1c823a0fd05f63883"},
- {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:5ee537ad29de716a3d8dc46c609908de0c25ffeebf93cd94a03d64cdc07d66d0"},
- {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:411f821be5af6af11dc5bed6c6c1dc6b6b25b91737d968ec2756f9baa75e5f9b"},
- {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:f90319d94cf5f9786773237f24bd235a7b5959089f1af8ec1154580a3434b503"},
- {file = "aiohttp-3.12.12-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:73b148e606f34e9d513c451fd65efe1091772659ca5703338a396a99f60108ff"},
- {file = "aiohttp-3.12.12-cp310-cp310-win32.whl", hash = "sha256:d40e7bfd577fdc8a92b72f35dfbdd3ec90f1bc8a72a42037fefe34d4eca2d4a1"},
- {file = "aiohttp-3.12.12-cp310-cp310-win_amd64.whl", hash = "sha256:65c7804a2343893d6dea9fce69811aea0a9ac47f68312cf2e3ee1668cd9a387f"},
- {file = "aiohttp-3.12.12-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:38823fe0d8bc059b3eaedb263fe427d887c7032e72b4ef92c472953285f0e658"},
- {file = "aiohttp-3.12.12-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:10237f2c34711215d04ed21da63852ce023608299554080a45c576215d9df81c"},
- {file = "aiohttp-3.12.12-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:563ec477c0dc6d56fc7f943a3475b5acdb399c7686c30f5a98ada24bb7562c7a"},
- {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f3d05c46a61aca7c47df74afff818bc06a251ab95d95ff80b53665edfe1e0bdf"},
- {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:277c882916759b4a6b6dc7e2ceb124aad071b3c6456487808d9ab13e1b448d57"},
- {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:216abf74b324b0f4e67041dd4fb2819613909a825904f8a51701fbcd40c09cd7"},
- {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:65d6cefad286459b68e7f867b9586a821fb7f121057b88f02f536ef570992329"},
- {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:feaaaff61966b5f4b4eae0b79fc79427f49484e4cfa5ab7d138ecd933ab540a8"},
- {file = "aiohttp-3.12.12-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a05917780b7cad1755784b16cfaad806bc16029a93d15f063ca60185b7d9ba05"},
- {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:082c5ec6d262c1b2ee01c63f4fb9152c17f11692bf16f0f100ad94a7a287d456"},
- {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:b265a3a8b379b38696ac78bdef943bdc4f4a5d6bed1a3fb5c75c6bab1ecea422"},
- {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:2e0f2e208914ecbc4b2a3b7b4daa759d0c587d9a0b451bb0835ac47fae7fa735"},
- {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:9923b025845b72f64d167bca221113377c8ffabd0a351dc18fb839d401ee8e22"},
- {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:1ebb213445900527831fecc70e185bf142fdfe5f2a691075f22d63c65ee3c35a"},
- {file = "aiohttp-3.12.12-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:6fc369fb273a8328077d37798b77c1e65676709af5c182cb74bd169ca9defe81"},
- {file = "aiohttp-3.12.12-cp311-cp311-win32.whl", hash = "sha256:58ecd10fda6a44c311cd3742cfd2aea8c4c600338e9f27cb37434d9f5ca9ddaa"},
- {file = "aiohttp-3.12.12-cp311-cp311-win_amd64.whl", hash = "sha256:b0066e88f30be00badffb5ef8f2281532b9a9020863d873ae15f7c147770b6ec"},
- {file = "aiohttp-3.12.12-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:98451ce9ce229d092f278a74a7c2a06b3aa72984673c87796126d7ccade893e9"},
- {file = "aiohttp-3.12.12-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:adbac7286d89245e1aff42e948503fdc6edf6d5d65c8e305a67c40f6a8fb95f4"},
- {file = "aiohttp-3.12.12-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0728882115bfa85cbd8d0f664c8ccc0cfd5bd3789dd837596785450ae52fac31"},
- {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6bf3b9d9e767f9d0e09fb1a31516410fc741a62cc08754578c40abc497d09540"},
- {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:c944860e86b9f77a462321a440ccf6fa10f5719bb9d026f6b0b11307b1c96c7b"},
- {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3b1979e1f0c98c06fd0cd940988833b102fa3aa56751f6c40ffe85cabc51f6fd"},
- {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:120b7dd084e96cfdad85acea2ce1e7708c70a26db913eabb8d7b417c728f5d84"},
- {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e58f5ae79649ffa247081c2e8c85e31d29623cf2a3137dda985ae05c9478aae"},
- {file = "aiohttp-3.12.12-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9aa5f049e3e2745b0141f13e5a64e7c48b1a1427ed18bbb7957b348f282fee56"},
- {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:7163cc9cf3722d90f1822f8a38b211e3ae2fc651c63bb55449f03dc1b3ff1d44"},
- {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:ef97c4d035b721de6607f3980fa3e4ef0ec3aca76474b5789b7fac286a8c4e23"},
- {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:1c14448d6a86acadc3f7b2f4cc385d1fb390acb6f37dce27f86fe629410d92e3"},
- {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:a1b6df6255cfc493454c79221183d64007dd5080bcda100db29b7ff181b8832c"},
- {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:60fc7338dfb0626c2927bfbac4785de3ea2e2bbe3d328ba5f3ece123edda4977"},
- {file = "aiohttp-3.12.12-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:d2afc72207ef4c9d4ca9fcd00689a6a37ef2d625600c3d757b5c2b80c9d0cf9a"},
- {file = "aiohttp-3.12.12-cp312-cp312-win32.whl", hash = "sha256:8098a48f93b2cbcdb5778e7c9a0e0375363e40ad692348e6e65c3b70d593b27c"},
- {file = "aiohttp-3.12.12-cp312-cp312-win_amd64.whl", hash = "sha256:d1c1879b2e0fc337d7a1b63fe950553c2b9e93c071cf95928aeea1902d441403"},
- {file = "aiohttp-3.12.12-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ea5d604318234427929d486954e3199aded65f41593ac57aa0241ab93dda3d15"},
- {file = "aiohttp-3.12.12-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:e03ff38250b8b572dce6fcd7b6fb6ee398bb8a59e6aa199009c5322d721df4fc"},
- {file = "aiohttp-3.12.12-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:71125b1fc2b6a94bccc63bbece620906a4dead336d2051f8af9cbf04480bc5af"},
- {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:784a66f9f853a22c6b8c2bd0ff157f9b879700f468d6d72cfa99167df08c5c9c"},
- {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:a5be0b58670b54301404bd1840e4902570a1c3be00358e2700919cb1ea73c438"},
- {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ce8f13566fc7bf5a728275b434bc3bdea87a7ed3ad5f734102b02ca59d9b510f"},
- {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d736e57d1901683bc9be648aa308cb73e646252c74b4c639c35dcd401ed385ea"},
- {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2007eaa7aae9102f211c519d1ec196bd3cecb1944a095db19eeaf132b798738"},
- {file = "aiohttp-3.12.12-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2a813e61583cab6d5cdbaa34bc28863acdb92f9f46e11de1b3b9251a1e8238f6"},
- {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:e408293aa910b0aea48b86a28eace41d497a85ba16c20f619f0c604597ef996c"},
- {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:f3d31faf290f5a30acba46b388465b67c6dbe8655d183e9efe2f6a1d594e6d9d"},
- {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:0b84731697325b023902aa643bd1726d999f5bc7854bc28b17ff410a81151d4b"},
- {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:a324c6852b6e327811748446e56cc9bb6eaa58710557922183175816e82a4234"},
- {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:22fd867fbd72612dcf670c90486dbcbaf702cb807fb0b42bc0b7a142a573574a"},
- {file = "aiohttp-3.12.12-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:3e092f1a970223794a4bf620a26c0e4e4e8e36bccae9b0b5da35e6d8ee598a03"},
- {file = "aiohttp-3.12.12-cp313-cp313-win32.whl", hash = "sha256:7f5f5eb8717ef8ba15ab35fcde5a70ad28bbdc34157595d1cddd888a985f5aae"},
- {file = "aiohttp-3.12.12-cp313-cp313-win_amd64.whl", hash = "sha256:ace2499bdd03c329c054dc4b47361f2b19d5aa470f7db5c7e0e989336761b33c"},
- {file = "aiohttp-3.12.12-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:0d0b1c27c05a7d39a50e946ec5f94c3af4ffadd33fa5f20705df42fb0a72ca14"},
- {file = "aiohttp-3.12.12-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:e5928847e6f7b7434921fbabf73fa5609d1f2bf4c25d9d4522b1fcc3b51995cb"},
- {file = "aiohttp-3.12.12-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7678147c3c85a7ae61559b06411346272ed40a08f54bc05357079a63127c9718"},
- {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f50057f36f2a1d8e750b273bb966bec9f69ee1e0a20725ae081610501f25d555"},
- {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:e5e834f0f11ff5805d11f0f22b627c75eadfaf91377b457875e4e3affd0b924f"},
- {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f94b2e2dea19d09745ef02ed483192260750f18731876a5c76f1c254b841443a"},
- {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b434bfb49564dc1c318989a0ab1d3000d23e5cfd00d8295dc9d5a44324cdd42d"},
- {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8ed76bc80177ddb7c5c93e1a6440b115ed2c92a3063420ac55206fd0832a6459"},
- {file = "aiohttp-3.12.12-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e1282a9acd378f2aed8dc79c01e702b1d5fd260ad083926a88ec7e987c4e0ade"},
- {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:09a213c13fba321586edab1528b530799645b82bd64d79b779eb8d47ceea155a"},
- {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:72eae16a9233561d315e72ae78ed9fc65ab3db0196e56cb2d329c755d694f137"},
- {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:f25990c507dbbeefd5a6a17df32a4ace634f7b20a38211d1b9609410c7f67a24"},
- {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:3a2aa255417c8ccf1b39359cd0a3d63ae3b5ced83958dbebc4d9113327c0536a"},
- {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:a4c53b89b3f838e9c25f943d1257efff10b348cb56895f408ddbcb0ec953a2ad"},
- {file = "aiohttp-3.12.12-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:b5a49c2dcb32114455ad503e8354624d85ab311cbe032da03965882492a9cb98"},
- {file = "aiohttp-3.12.12-cp39-cp39-win32.whl", hash = "sha256:74fddc0ba8cea6b9c5bd732eb9d97853543586596b86391f8de5d4f6c2a0e068"},
- {file = "aiohttp-3.12.12-cp39-cp39-win_amd64.whl", hash = "sha256:ddf40ba4a1d0b4d232dc47d2b98ae7e937dcbc40bb5f2746bce0af490a64526f"},
- {file = "aiohttp-3.12.12.tar.gz", hash = "sha256:05875595d2483d96cb61fa9f64e75262d7ac6251a7e3c811d8e26f7d721760bd"},
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+files = [
+ {file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5421af8f22a98f640261ee48aae3a37f0c41371e99412d55eaf2f8a46d5dad29"},
+ {file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0fcda86f6cb318ba36ed8f1396a6a4a3fd8f856f84d426584392083d10da4de0"},
+ {file = "aiohttp-3.12.13-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:4cd71c9fb92aceb5a23c4c39d8ecc80389c178eba9feab77f19274843eb9412d"},
+ {file = "aiohttp-3.12.13-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:34ebf1aca12845066c963016655dac897651e1544f22a34c9b461ac3b4b1d3aa"},
+ {file = "aiohttp-3.12.13-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:893a4639694c5b7edd4bdd8141be296042b6806e27cc1d794e585c43010cc294"},
+ {file = "aiohttp-3.12.13-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:663d8ee3ffb3494502ebcccb49078faddbb84c1d870f9c1dd5a29e85d1f747ce"},
+ {file = "aiohttp-3.12.13-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f0f8f6a85a0006ae2709aa4ce05749ba2cdcb4b43d6c21a16c8517c16593aabe"},
+ {file = "aiohttp-3.12.13-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1582745eb63df267c92d8b61ca655a0ce62105ef62542c00a74590f306be8cb5"},
+ {file = "aiohttp-3.12.13-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d59227776ee2aa64226f7e086638baa645f4b044f2947dbf85c76ab11dcba073"},
+ {file = "aiohttp-3.12.13-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:06b07c418bde1c8e737d8fa67741072bd3f5b0fb66cf8c0655172188c17e5fa6"},
+ {file = "aiohttp-3.12.13-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:9445c1842680efac0f81d272fd8db7163acfcc2b1436e3f420f4c9a9c5a50795"},
+ {file = "aiohttp-3.12.13-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:09c4767af0b0b98c724f5d47f2bf33395c8986995b0a9dab0575ca81a554a8c0"},
+ {file = "aiohttp-3.12.13-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:f3854fbde7a465318ad8d3fc5bef8f059e6d0a87e71a0d3360bb56c0bf87b18a"},
+ {file = "aiohttp-3.12.13-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:2332b4c361c05ecd381edb99e2a33733f3db906739a83a483974b3df70a51b40"},
+ {file = "aiohttp-3.12.13-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:1561db63fa1b658cd94325d303933553ea7d89ae09ff21cc3bcd41b8521fbbb6"},
+ {file = "aiohttp-3.12.13-cp310-cp310-win32.whl", hash = "sha256:a0be857f0b35177ba09d7c472825d1b711d11c6d0e8a2052804e3b93166de1ad"},
+ {file = "aiohttp-3.12.13-cp310-cp310-win_amd64.whl", hash = "sha256:fcc30ad4fb5cb41a33953292d45f54ef4066746d625992aeac33b8c681173178"},
+ {file = "aiohttp-3.12.13-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:7c229b1437aa2576b99384e4be668af1db84b31a45305d02f61f5497cfa6f60c"},
+ {file = "aiohttp-3.12.13-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:04076d8c63471e51e3689c93940775dc3d12d855c0c80d18ac5a1c68f0904358"},
+ {file = "aiohttp-3.12.13-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:55683615813ce3601640cfaa1041174dc956d28ba0511c8cbd75273eb0587014"},
+ {file = "aiohttp-3.12.13-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:921bc91e602d7506d37643e77819cb0b840d4ebb5f8d6408423af3d3bf79a7b7"},
+ {file = "aiohttp-3.12.13-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:e72d17fe0974ddeae8ed86db297e23dba39c7ac36d84acdbb53df2e18505a013"},
+ {file = "aiohttp-3.12.13-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0653d15587909a52e024a261943cf1c5bdc69acb71f411b0dd5966d065a51a47"},
+ {file = "aiohttp-3.12.13-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a77b48997c66722c65e157c06c74332cdf9c7ad00494b85ec43f324e5c5a9b9a"},
+ {file = "aiohttp-3.12.13-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d6946bae55fd36cfb8e4092c921075cde029c71c7cb571d72f1079d1e4e013bc"},
+ {file = "aiohttp-3.12.13-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4f95db8c8b219bcf294a53742c7bda49b80ceb9d577c8e7aa075612b7f39ffb7"},
+ {file = "aiohttp-3.12.13-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:03d5eb3cfb4949ab4c74822fb3326cd9655c2b9fe22e4257e2100d44215b2e2b"},
+ {file = "aiohttp-3.12.13-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:6383dd0ffa15515283c26cbf41ac8e6705aab54b4cbb77bdb8935a713a89bee9"},
+ {file = "aiohttp-3.12.13-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:6548a411bc8219b45ba2577716493aa63b12803d1e5dc70508c539d0db8dbf5a"},
+ {file = "aiohttp-3.12.13-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:81b0fcbfe59a4ca41dc8f635c2a4a71e63f75168cc91026c61be665945739e2d"},
+ {file = "aiohttp-3.12.13-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:6a83797a0174e7995e5edce9dcecc517c642eb43bc3cba296d4512edf346eee2"},
+ {file = "aiohttp-3.12.13-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:a5734d8469a5633a4e9ffdf9983ff7cdb512524645c7a3d4bc8a3de45b935ac3"},
+ {file = "aiohttp-3.12.13-cp311-cp311-win32.whl", hash = "sha256:fef8d50dfa482925bb6b4c208b40d8e9fa54cecba923dc65b825a72eed9a5dbd"},
+ {file = "aiohttp-3.12.13-cp311-cp311-win_amd64.whl", hash = "sha256:9a27da9c3b5ed9d04c36ad2df65b38a96a37e9cfba6f1381b842d05d98e6afe9"},
+ {file = "aiohttp-3.12.13-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:0aa580cf80558557285b49452151b9c69f2fa3ad94c5c9e76e684719a8791b73"},
+ {file = "aiohttp-3.12.13-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:b103a7e414b57e6939cc4dece8e282cfb22043efd0c7298044f6594cf83ab347"},
+ {file = "aiohttp-3.12.13-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:78f64e748e9e741d2eccff9597d09fb3cd962210e5b5716047cbb646dc8fe06f"},
+ {file = "aiohttp-3.12.13-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:29c955989bf4c696d2ededc6b0ccb85a73623ae6e112439398935362bacfaaf6"},
+ {file = "aiohttp-3.12.13-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:d640191016763fab76072c87d8854a19e8e65d7a6fcfcbf017926bdbbb30a7e5"},
+ {file = "aiohttp-3.12.13-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4dc507481266b410dede95dd9f26c8d6f5a14315372cc48a6e43eac652237d9b"},
+ {file = "aiohttp-3.12.13-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8a94daa873465d518db073bd95d75f14302e0208a08e8c942b2f3f1c07288a75"},
+ {file = "aiohttp-3.12.13-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:177f52420cde4ce0bb9425a375d95577fe082cb5721ecb61da3049b55189e4e6"},
+ {file = "aiohttp-3.12.13-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0f7df1f620ec40f1a7fbcb99ea17d7326ea6996715e78f71a1c9a021e31b96b8"},
+ {file = "aiohttp-3.12.13-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:3062d4ad53b36e17796dce1c0d6da0ad27a015c321e663657ba1cc7659cfc710"},
+ {file = "aiohttp-3.12.13-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:8605e22d2a86b8e51ffb5253d9045ea73683d92d47c0b1438e11a359bdb94462"},
+ {file = "aiohttp-3.12.13-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:54fbbe6beafc2820de71ece2198458a711e224e116efefa01b7969f3e2b3ddae"},
+ {file = "aiohttp-3.12.13-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:050bd277dfc3768b606fd4eae79dd58ceda67d8b0b3c565656a89ae34525d15e"},
+ {file = "aiohttp-3.12.13-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:2637a60910b58f50f22379b6797466c3aa6ae28a6ab6404e09175ce4955b4e6a"},
+ {file = "aiohttp-3.12.13-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e986067357550d1aaa21cfe9897fa19e680110551518a5a7cf44e6c5638cb8b5"},
+ {file = "aiohttp-3.12.13-cp312-cp312-win32.whl", hash = "sha256:ac941a80aeea2aaae2875c9500861a3ba356f9ff17b9cb2dbfb5cbf91baaf5bf"},
+ {file = "aiohttp-3.12.13-cp312-cp312-win_amd64.whl", hash = "sha256:671f41e6146a749b6c81cb7fd07f5a8356d46febdaaaf07b0e774ff04830461e"},
+ {file = "aiohttp-3.12.13-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:d4a18e61f271127465bdb0e8ff36e8f02ac4a32a80d8927aa52371e93cd87938"},
+ {file = "aiohttp-3.12.13-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:532542cb48691179455fab429cdb0d558b5e5290b033b87478f2aa6af5d20ace"},
+ {file = "aiohttp-3.12.13-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:d7eea18b52f23c050ae9db5d01f3d264ab08f09e7356d6f68e3f3ac2de9dfabb"},
+ {file = "aiohttp-3.12.13-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ad7c8e5c25f2a26842a7c239de3f7b6bfb92304593ef997c04ac49fb703ff4d7"},
+ {file = "aiohttp-3.12.13-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:6af355b483e3fe9d7336d84539fef460120c2f6e50e06c658fe2907c69262d6b"},
+ {file = "aiohttp-3.12.13-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a95cf9f097498f35c88e3609f55bb47b28a5ef67f6888f4390b3d73e2bac6177"},
+ {file = "aiohttp-3.12.13-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b8ed8c38a1c584fe99a475a8f60eefc0b682ea413a84c6ce769bb19a7ff1c5ef"},
+ {file = "aiohttp-3.12.13-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7a0b9170d5d800126b5bc89d3053a2363406d6e327afb6afaeda2d19ee8bb103"},
+ {file = "aiohttp-3.12.13-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:372feeace612ef8eb41f05ae014a92121a512bd5067db8f25101dd88a8db11da"},
+ {file = "aiohttp-3.12.13-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:a946d3702f7965d81f7af7ea8fb03bb33fe53d311df48a46eeca17e9e0beed2d"},
+ {file = "aiohttp-3.12.13-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:a0c4725fae86555bbb1d4082129e21de7264f4ab14baf735278c974785cd2041"},
+ {file = "aiohttp-3.12.13-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:9b28ea2f708234f0a5c44eb6c7d9eb63a148ce3252ba0140d050b091b6e842d1"},
+ {file = "aiohttp-3.12.13-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:d4f5becd2a5791829f79608c6f3dc745388162376f310eb9c142c985f9441cc1"},
+ {file = "aiohttp-3.12.13-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:60f2ce6b944e97649051d5f5cc0f439360690b73909230e107fd45a359d3e911"},
+ {file = "aiohttp-3.12.13-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:69fc1909857401b67bf599c793f2183fbc4804717388b0b888f27f9929aa41f3"},
+ {file = "aiohttp-3.12.13-cp313-cp313-win32.whl", hash = "sha256:7d7e68787a2046b0e44ba5587aa723ce05d711e3a3665b6b7545328ac8e3c0dd"},
+ {file = "aiohttp-3.12.13-cp313-cp313-win_amd64.whl", hash = "sha256:5a178390ca90419bfd41419a809688c368e63c86bd725e1186dd97f6b89c2706"},
+ {file = "aiohttp-3.12.13-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:36f6c973e003dc9b0bb4e8492a643641ea8ef0e97ff7aaa5c0f53d68839357b4"},
+ {file = "aiohttp-3.12.13-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6cbfc73179bd67c229eb171e2e3745d2afd5c711ccd1e40a68b90427f282eab1"},
+ {file = "aiohttp-3.12.13-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:1e8b27b2d414f7e3205aa23bb4a692e935ef877e3a71f40d1884f6e04fd7fa74"},
+ {file = "aiohttp-3.12.13-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:eabded0c2b2ef56243289112c48556c395d70150ce4220d9008e6b4b3dd15690"},
+ {file = "aiohttp-3.12.13-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:003038e83f1a3ff97409999995ec02fe3008a1d675478949643281141f54751d"},
+ {file = "aiohttp-3.12.13-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1b6f46613031dbc92bdcaad9c4c22c7209236ec501f9c0c5f5f0b6a689bf50f3"},
+ {file = "aiohttp-3.12.13-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c332c6bb04650d59fb94ed96491f43812549a3ba6e7a16a218e612f99f04145e"},
+ {file = "aiohttp-3.12.13-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3fea41a2c931fb582cb15dc86a3037329e7b941df52b487a9f8b5aa960153cbd"},
+ {file = "aiohttp-3.12.13-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:846104f45d18fb390efd9b422b27d8f3cf8853f1218c537f36e71a385758c896"},
+ {file = "aiohttp-3.12.13-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:5d6c85ac7dd350f8da2520bac8205ce99df4435b399fa7f4dc4a70407073e390"},
+ {file = "aiohttp-3.12.13-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:5a1ecce0ed281bec7da8550da052a6b89552db14d0a0a45554156f085a912f48"},
+ {file = "aiohttp-3.12.13-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:5304d74867028cca8f64f1cc1215eb365388033c5a691ea7aa6b0dc47412f495"},
+ {file = "aiohttp-3.12.13-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:64d1f24ee95a2d1e094a4cd7a9b7d34d08db1bbcb8aa9fb717046b0a884ac294"},
+ {file = "aiohttp-3.12.13-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:119c79922a7001ca6a9e253228eb39b793ea994fd2eccb79481c64b5f9d2a055"},
+ {file = "aiohttp-3.12.13-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:bb18f00396d22e2f10cd8825d671d9f9a3ba968d708a559c02a627536b36d91c"},
+ {file = "aiohttp-3.12.13-cp39-cp39-win32.whl", hash = "sha256:0022de47ef63fd06b065d430ac79c6b0bd24cdae7feaf0e8c6bac23b805a23a8"},
+ {file = "aiohttp-3.12.13-cp39-cp39-win_amd64.whl", hash = "sha256:29e08111ccf81b2734ae03f1ad1cb03b9615e7d8f616764f22f71209c094f122"},
+ {file = "aiohttp-3.12.13.tar.gz", hash = "sha256:47e2da578528264a12e4e3dd8dd72a7289e5f812758fe086473fab037a10fcce"},
]
[package.dependencies]
@@ -202,7 +202,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "python_version == \"3.9\" and extra == \"ray\" or (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version < \"3.10\" or python_version >= \"3.10\" and (extra == \"ray\" or extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -268,7 +268,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -285,7 +285,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
+markers = {main = "python_version == \"3.9\" and extra == \"ray\" or (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version < \"3.10\" or python_version >= \"3.10\" and (extra == \"ray\" or extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -297,24 +297,24 @@ tests-mypy = ["mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" a
[[package]]
name = "aws-sam-translator"
-version = "1.98.0"
+version = "1.99.0"
description = "AWS SAM Translator is a library that transform SAM templates into AWS CloudFormation templates"
optional = false
python-versions = "!=4.0,<=4.0,>=3.8"
groups = ["dev"]
files = [
- {file = "aws_sam_translator-1.98.0-py3-none-any.whl", hash = "sha256:65e7afffdda2e6f715debc251ddae5deba079af41db5dd9ecd370d658b9d728e"},
- {file = "aws_sam_translator-1.98.0.tar.gz", hash = "sha256:fe9fdf51b593aca4cde29f555e272b00d90662315c8078e9f5f3448dd962c66b"},
+ {file = "aws_sam_translator-1.99.0-py3-none-any.whl", hash = "sha256:b1997e09da876342655eb568e66098280ffd137213009f0136b57f4e7694c98c"},
+ {file = "aws_sam_translator-1.99.0.tar.gz", hash = "sha256:be326054a7ee2f535fcd914db85e5d50bdf4054313c14888af69b6de3187cdf8"},
]
[package.dependencies]
-boto3 = ">=1.19.5,<2.dev0"
+boto3 = ">=1.34.0,<2.0.0"
jsonschema = ">=3.2,<5"
pydantic = ">=1.8,<1.10.15 || >1.10.15,<1.10.17 || >1.10.17,<3"
typing_extensions = ">=4.4"
[package.extras]
-dev = ["black (==24.3.0)", "boto3 (>=1.23,<2)", "boto3-stubs[appconfig,serverlessrepo] (>=1.19.5,<2.dev0)", "coverage (>=5.3,<8)", "dateparser (>=1.1,<2.0)", "mypy (>=1.3.0,<1.4.0)", "parameterized (>=0.7,<1.0)", "pytest (>=6.2,<8)", "pytest-cov (>=2.10,<5)", "pytest-env (>=0.6,<1)", "pytest-rerunfailures (>=9.1,<12)", "pytest-xdist (>=2.5,<4)", "pyyaml (>=6.0,<7.0)", "requests (>=2.28,<3.0)", "ruamel.yaml (==0.17.21)", "ruff (>=0.4.5,<0.5.0)", "tenacity (>=8.0,<9.0)", "types-PyYAML (>=6.0,<7.0)", "types-jsonschema (>=3.2,<4.0)"]
+dev = ["black (==24.3.0)", "boto3 (>=1.34.0,<2.0.0)", "boto3-stubs[appconfig,serverlessrepo] (>=1.34.0,<2.0.0)", "cloudformation-cli (>=0.2.39,<0.3.0)", "coverage (>=5.3,<8)", "dateparser (>=1.1,<2.0)", "mypy (>=1.3.0,<1.4.0)", "parameterized (>=0.7,<1.0)", "pytest (>=6.2,<8)", "pytest-cov (>=2.10,<5)", "pytest-env (>=0.6,<1)", "pytest-rerunfailures (>=9.1,<12)", "pytest-xdist (>=2.5,<4)", "pyyaml (>=6.0,<7.0)", "requests (>=2.28,<3.0)", "ruamel.yaml (==0.17.21)", "ruff (>=0.4.5,<0.5.0)", "tenacity (>=9.0,<10.0)", "types-PyYAML (>=6.0,<7.0)", "types-jsonschema (>=3.2,<4.0)"]
[[package]]
name = "aws-xray-sdk"
@@ -488,7 +488,7 @@ files = [
{file = "boto3-1.38.27-py3-none-any.whl", hash = "sha256:95f5fe688795303a8a15e8b7e7f255cadab35eae459d00cc281a4fd77252ea80"},
{file = "boto3-1.38.27.tar.gz", hash = "sha256:94bd7fdd92d5701b362d4df100d21e28f8307a67ff56b6a8b0398119cf22f859"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.38.27,<1.39.0"
@@ -509,7 +509,7 @@ files = [
{file = "botocore-1.38.27-py3-none-any.whl", hash = "sha256:a785d5e9a5eda88ad6ab9ed8b87d1f2ac409d0226bba6ff801c55359e94d91a8"},
{file = "botocore-1.38.27.tar.gz", hash = "sha256:9788f7efe974328a38cbade64cc0b1e67d27944b899f88cb786ae362973133b6"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -551,26 +551,26 @@ virtualenv = ["virtualenv (>=20.0.35)"]
[[package]]
name = "cachetools"
-version = "6.1.0"
+version = "5.5.2"
description = "Extensible memoizing collections and decorators"
optional = false
-python-versions = ">=3.9"
+python-versions = ">=3.7"
groups = ["main"]
files = [
- {file = "cachetools-6.1.0-py3-none-any.whl", hash = "sha256:1c7bb3cf9193deaf3508b7c5f2a79986c13ea38965c5adcff1f84519cf39163e"},
- {file = "cachetools-6.1.0.tar.gz", hash = "sha256:b4c4f404392848db3ce7aac34950d17be4d864da4b8b66911008e430bc544587"},
+ {file = "cachetools-5.5.2-py3-none-any.whl", hash = "sha256:d26a22bcc62eb95c3beabd9f1ee5e820d3d2704fe2967cbe350e20c8ffcd3f0a"},
+ {file = "cachetools-5.5.2.tar.gz", hash = "sha256:1a661caa9175d26759571b2e19580f9d6393969e5dfca11fdb1f947a23e640d4"},
]
[[package]]
name = "certifi"
-version = "2025.4.26"
+version = "2025.6.15"
description = "Python package for providing Mozilla's CA Bundle."
optional = false
-python-versions = ">=3.6"
+python-versions = ">=3.7"
groups = ["main", "dev", "docs"]
files = [
- {file = "certifi-2025.4.26-py3-none-any.whl", hash = "sha256:30350364dfe371162649852c63336a15c70c6510c2ad5015b21c2345311805f3"},
- {file = "certifi-2025.4.26.tar.gz", hash = "sha256:0a816057ea3cdefcef70270d2c515e4506bbc954f417fa5ade2021213bb8f0c6"},
+ {file = "certifi-2025.6.15-py3-none-any.whl", hash = "sha256:2e0c7ce7cb5d8f8634ca55d2ba7e6ec2689a2fd6537d8dec1296a477a4910057"},
+ {file = "certifi-2025.6.15.tar.gz", hash = "sha256:d747aa5a8b9bbbb1bb8c22bb13e22bd1f18e9796defa16bab421f7f7a317323b"},
]
[[package]]
@@ -668,14 +668,14 @@ files = [
[[package]]
name = "cfn-lint"
-version = "1.35.4"
+version = "1.36.1"
description = "Checks CloudFormation templates for practices and behaviour that could potentially be improved"
optional = false
python-versions = ">=3.9"
groups = ["dev"]
files = [
- {file = "cfn_lint-1.35.4-py3-none-any.whl", hash = "sha256:4649797b4a6975a8ca5ebbf51e568a52383fa5b7d591f92266b8803735e5a52f"},
- {file = "cfn_lint-1.35.4.tar.gz", hash = "sha256:da38218367217b909884ec2efe361b3992868f140b1d5f37dc64a9e328d9ddb9"},
+ {file = "cfn_lint-1.36.1-py3-none-any.whl", hash = "sha256:1f5eae322cabc9b73a717b83bf3a73709e2193fbf7fd399339e016a0838de7a3"},
+ {file = "cfn_lint-1.36.1.tar.gz", hash = "sha256:ccb85398fb1d8d8d87edafe207ab6a133c4ab9aefd3ff93591b6ad8c3ccab291"},
]
[package.dependencies]
@@ -802,6 +802,7 @@ description = "Composable command line interface toolkit"
optional = false
python-versions = ">=3.7"
groups = ["main", "dev", "docs"]
+markers = "python_version < \"3.10\""
files = [
{file = "click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2"},
{file = "click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a"},
@@ -810,6 +811,22 @@ files = [
[package.dependencies]
colorama = {version = "*", markers = "platform_system == \"Windows\""}
+[[package]]
+name = "click"
+version = "8.2.1"
+description = "Composable command line interface toolkit"
+optional = false
+python-versions = ">=3.10"
+groups = ["main", "dev", "docs"]
+markers = "python_version >= \"3.10\""
+files = [
+ {file = "click-8.2.1-py3-none-any.whl", hash = "sha256:61a3265b914e850b85317d0b3109c7f8cd35a670f963866005d6ef1d5175a12b"},
+ {file = "click-8.2.1.tar.gz", hash = "sha256:27c491cc05d968d271d5a1db13e3b5a184636d9d930f148c50b038f0d0646202"},
+]
+
+[package.dependencies]
+colorama = {version = "*", markers = "platform_system == \"Windows\""}
+
[[package]]
name = "colorama"
version = "0.4.6"
@@ -1418,7 +1435,7 @@ files = [
{file = "filelock-3.18.0-py3-none-any.whl", hash = "sha256:c401f4f8377c4464e6db25fff06205fd89bdd83b65eb0488ed1b160f780e21de"},
{file = "filelock-3.18.0.tar.gz", hash = "sha256:adbc88eabb99d2fec8c9c1b229b171f18afa655400173ddc653d5d01501fb9f2"},
]
-markers = {main = "extra == \"ray\" or extra == \"hf\""}
+markers = {main = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"hf\") or extra == \"hf\" or extra == \"ray\""}
[package.extras]
docs = ["furo (>=2024.8.6)", "sphinx (>=8.1.3)", "sphinx-autodoc-typehints (>=3)"]
@@ -1452,14 +1469,14 @@ dotenv = ["python-dotenv"]
[[package]]
name = "flask-cors"
-version = "6.0.0"
+version = "6.0.1"
description = "A Flask extension simplifying CORS support"
optional = false
python-versions = "<4.0,>=3.9"
groups = ["dev"]
files = [
- {file = "flask_cors-6.0.0-py3-none-any.whl", hash = "sha256:6332073356452343a8ccddbfec7befdc3fdd040141fe776ec9b94c262f058657"},
- {file = "flask_cors-6.0.0.tar.gz", hash = "sha256:4592c1570246bf7beee96b74bc0adbbfcb1b0318f6ba05c412e8909eceec3393"},
+ {file = "flask_cors-6.0.1-py3-none-any.whl", hash = "sha256:c7b2cbfb1a31aa0d2e5341eea03a6805349f7a61647daee1a15c46bbe981494c"},
+ {file = "flask_cors-6.0.1.tar.gz", hash = "sha256:d81bcb31f07b0985be7f48406247e9243aced229b7747219160a0559edd678db"},
]
[package.dependencies]
@@ -1473,7 +1490,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "python_version == \"3.9\" and extra == \"ray\" or (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version < \"3.10\" or python_version >= \"3.10\" and (extra == \"ray\" or extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\")"
files = [
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cc4df77d638aa2ed703b878dd093725b72a824c3c546c076e8fdf276f78ee84a"},
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:716a9973a2cc963160394f701964fe25012600f3d311f60c790400b00e568b61"},
@@ -1705,65 +1722,76 @@ dev = ["flake8", "markdown", "twine", "wheel"]
[[package]]
name = "google-api-core"
-version = "1.16.0"
+version = "2.25.1"
description = "Google API client core library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-api-core-1.16.0.tar.gz", hash = "sha256:92e962a087f1c4b8d1c5c88ade1c1dfd550047dcffb320c57ef6a534a20403e2"},
- {file = "google_api_core-1.16.0-py2.py3-none-any.whl", hash = "sha256:859f7392676761f2b160c6ee030c3422135ada4458f0948c5690a6a7c8d86294"},
+ {file = "google_api_core-2.25.1-py3-none-any.whl", hash = "sha256:8a2a56c1fef82987a524371f99f3bd0143702fecc670c72e600c1cda6bf8dbb7"},
+ {file = "google_api_core-2.25.1.tar.gz", hash = "sha256:d2aaa0b13c78c61cb3f4282c464c046e45fbd75755683c9c525e6e8f7ed0a5e8"},
]
[package.dependencies]
-google-auth = ">=0.4.0,<2.0dev"
-googleapis-common-protos = ">=1.6.0,<2.0dev"
-protobuf = ">=3.4.0"
-pytz = "*"
-requests = ">=2.18.0,<3.0.0dev"
-setuptools = ">=34.0.0"
-six = ">=1.10.0"
+google-auth = ">=2.14.1,<3.0.0"
+googleapis-common-protos = ">=1.56.2,<2.0.0"
+proto-plus = [
+ {version = ">=1.22.3,<2.0.0"},
+ {version = ">=1.25.0,<2.0.0", markers = "python_version >= \"3.13\""},
+]
+protobuf = ">=3.19.5,<3.20.0 || >3.20.0,<3.20.1 || >3.20.1,<4.21.0 || >4.21.0,<4.21.1 || >4.21.1,<4.21.2 || >4.21.2,<4.21.3 || >4.21.3,<4.21.4 || >4.21.4,<4.21.5 || >4.21.5,<7.0.0"
+requests = ">=2.18.0,<3.0.0"
[package.extras]
-grpc = ["grpcio (>=1.8.2,<2.0dev)"]
-grpcgcp = ["grpcio-gcp (>=0.2.2)"]
-grpcio-gcp = ["grpcio-gcp (>=0.2.2)"]
+async-rest = ["google-auth[aiohttp] (>=2.35.0,<3.0.0)"]
+grpc = ["grpcio (>=1.33.2,<2.0.0)", "grpcio (>=1.49.1,<2.0.0) ; python_version >= \"3.11\"", "grpcio-status (>=1.33.2,<2.0.0)", "grpcio-status (>=1.49.1,<2.0.0) ; python_version >= \"3.11\""]
+grpcgcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
+grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
[[package]]
name = "google-auth"
-version = "1.6.3"
+version = "2.40.3"
description = "Google Authentication Library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-auth-1.6.3.tar.gz", hash = "sha256:0f7c6a64927d34c1a474da92cfc59e552a5d3b940d3266606c6a28b72888b9e4"},
- {file = "google_auth-1.6.3-py2.py3-none-any.whl", hash = "sha256:20705f6803fd2c4d1cc2dcb0df09d4dfcb9a7d51fd59e94a3a28231fd93119ed"},
+ {file = "google_auth-2.40.3-py2.py3-none-any.whl", hash = "sha256:1370d4593e86213563547f97a92752fc658456fe4514c809544f330fed45a7ca"},
+ {file = "google_auth-2.40.3.tar.gz", hash = "sha256:500c3a29adedeb36ea9cf24b8d10858e152f2412e3ca37829b3fa18e33d63b77"},
]
[package.dependencies]
-cachetools = ">=2.0.0"
+cachetools = ">=2.0.0,<6.0"
pyasn1-modules = ">=0.2.1"
-rsa = ">=3.1.4"
-six = ">=1.9.0"
+rsa = ">=3.1.4,<5"
+
+[package.extras]
+aiohttp = ["aiohttp (>=3.6.2,<4.0.0)", "requests (>=2.20.0,<3.0.0)"]
+enterprise-cert = ["cryptography", "pyopenssl"]
+pyjwt = ["cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "pyjwt (>=2.0)"]
+pyopenssl = ["cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "pyopenssl (>=20.0.0)"]
+reauth = ["pyu2f (>=0.1.5)"]
+requests = ["requests (>=2.20.0,<3.0.0)"]
+testing = ["aiohttp (<3.10.0)", "aiohttp (>=3.6.2,<4.0.0)", "aioresponses", "cryptography (<39.0.0) ; python_version < \"3.8\"", "cryptography (>=38.0.3)", "flask", "freezegun", "grpcio", "mock", "oauth2client", "packaging", "pyjwt (>=2.0)", "pyopenssl (<24.3.0)", "pyopenssl (>=20.0.0)", "pytest", "pytest-asyncio", "pytest-cov", "pytest-localserver", "pyu2f (>=0.1.5)", "requests (>=2.20.0,<3.0.0)", "responses", "urllib3"]
+urllib3 = ["packaging", "urllib3"]
[[package]]
name = "google-auth-oauthlib"
-version = "0.5.3"
+version = "1.2.2"
description = "Google Authentication Library"
optional = true
python-versions = ">=3.6"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-auth-oauthlib-0.5.3.tar.gz", hash = "sha256:307d21918d61a0741882ad1fd001c67e68ad81206451d05fc4d26f79de56fc90"},
- {file = "google_auth_oauthlib-0.5.3-py2.py3-none-any.whl", hash = "sha256:9e8ff4ed2b21c174a2d6cc2172c698dbf0b1f686509774c663a83c495091fe09"},
+ {file = "google_auth_oauthlib-1.2.2-py3-none-any.whl", hash = "sha256:fd619506f4b3908b5df17b65f39ca8d66ea56986e5472eb5978fd8f3786f00a2"},
+ {file = "google_auth_oauthlib-1.2.2.tar.gz", hash = "sha256:11046fb8d3348b296302dd939ace8af0a724042e8029c1b872d87fabc9f41684"},
]
[package.dependencies]
-google-auth = ">=1.0.0"
+google-auth = ">=2.15.0"
requests-oauthlib = ">=0.7.0"
[package.extras]
@@ -1771,58 +1799,115 @@ tool = ["click (>=6.0.0)"]
[[package]]
name = "google-cloud-core"
-version = "1.4.0"
+version = "2.4.3"
description = "Google Cloud API client core library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-cloud-core-1.4.0.tar.gz", hash = "sha256:07a024a26c4eb14ee3df7e6e5021c04f8f7e9f0e83d3d47863229f3635f871ce"},
- {file = "google_cloud_core-1.4.0-py2.py3-none-any.whl", hash = "sha256:e84d4a05c58fc905b9340118e257cd043e9c02a6d31168182204afe4afd521eb"},
+ {file = "google_cloud_core-2.4.3-py2.py3-none-any.whl", hash = "sha256:5130f9f4c14b4fafdff75c79448f9495cfade0d8775facf1b09c3bf67e027f6e"},
+ {file = "google_cloud_core-2.4.3.tar.gz", hash = "sha256:1fab62d7102844b278fe6dead3af32408b1df3eb06f5c7e8634cbd40edc4da53"},
]
[package.dependencies]
-google-api-core = ">=1.16.0,<2.0.0dev"
+google-api-core = ">=1.31.6,<2.0.dev0 || >2.3.0,<3.0.0dev"
+google-auth = ">=1.25.0,<3.0dev"
[package.extras]
-grpc = ["grpcio (>=1.8.2,<2.0dev)"]
+grpc = ["grpcio (>=1.38.0,<2.0dev)", "grpcio-status (>=1.38.0,<2.0.dev0)"]
[[package]]
name = "google-cloud-storage"
-version = "1.23.0"
+version = "3.1.1"
description = "Google Cloud Storage API client library"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-cloud-storage-1.23.0.tar.gz", hash = "sha256:c66e876ae9547884fa42566a2ebfec51d280f488d7a058af9611ba90c78bed78"},
- {file = "google_cloud_storage-1.23.0-py2.py3-none-any.whl", hash = "sha256:9f59c100d3940e38567c48d54cf1a2e7591a2f38e9693dfc11a242d5e54a1626"},
+ {file = "google_cloud_storage-3.1.1-py3-none-any.whl", hash = "sha256:ba7e6ae2be5a7a08742f001e23ec6a0c17d78c620f63bf8e0e7c2cbdddb407de"},
+ {file = "google_cloud_storage-3.1.1.tar.gz", hash = "sha256:f9c8f965cafd1d38509f8e2b070339e0e9e5bf050774653bf36213d4ea6104c0"},
]
[package.dependencies]
-google-auth = ">=1.2.0"
-google-cloud-core = ">=1.0.3,<2.0dev"
-google-resumable-media = ">=0.5.0,<0.6dev"
+google-api-core = ">=2.15.0,<3.0.0"
+google-auth = ">=2.26.1,<3.0.0"
+google-cloud-core = ">=2.4.2,<3.0.0"
+google-crc32c = ">=1.1.3,<2.0.0"
+google-resumable-media = ">=2.7.2,<3.0.0"
+requests = ">=2.22.0,<3.0.0"
+
+[package.extras]
+protobuf = ["protobuf (>=3.20.2,<7.0.0)"]
+tracing = ["opentelemetry-api (>=1.1.0,<2.0.0)"]
+
+[[package]]
+name = "google-crc32c"
+version = "1.7.1"
+description = "A python wrapper of the C library 'Google CRC32C'"
+optional = true
+python-versions = ">=3.9"
+groups = ["main"]
+markers = "extra == \"gcsfs\""
+files = [
+ {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:b07d48faf8292b4db7c3d64ab86f950c2e94e93a11fd47271c28ba458e4a0d76"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:7cc81b3a2fbd932a4313eb53cc7d9dde424088ca3a0337160f35d91826880c1d"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:1c67ca0a1f5b56162951a9dae987988679a7db682d6f97ce0f6381ebf0fbea4c"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc5319db92daa516b653600794d5b9f9439a9a121f3e162f94b0e1891c7933cb"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dcdf5a64adb747610140572ed18d011896e3b9ae5195f2514b7ff678c80f1603"},
+ {file = "google_crc32c-1.7.1-cp310-cp310-win_amd64.whl", hash = "sha256:754561c6c66e89d55754106739e22fdaa93fafa8da7221b29c8b8e8270c6ec8a"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:6fbab4b935989e2c3610371963ba1b86afb09537fd0c633049be82afe153ac06"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:ed66cbe1ed9cbaaad9392b5259b3eba4a9e565420d734e6238813c428c3336c9"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ee6547b657621b6cbed3562ea7826c3e11cab01cd33b74e1f677690652883e77"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d68e17bad8f7dd9a49181a1f5a8f4b251c6dbc8cc96fb79f1d321dfd57d66f53"},
+ {file = "google_crc32c-1.7.1-cp311-cp311-win_amd64.whl", hash = "sha256:6335de12921f06e1f774d0dd1fbea6bf610abe0887a1638f64d694013138be5d"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:2d73a68a653c57281401871dd4aeebbb6af3191dcac751a76ce430df4d403194"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:22beacf83baaf59f9d3ab2bbb4db0fb018da8e5aebdce07ef9f09fce8220285e"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:19eafa0e4af11b0a4eb3974483d55d2d77ad1911e6cf6f832e1574f6781fd337"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b6d86616faaea68101195c6bdc40c494e4d76f41e07a37ffdef270879c15fb65"},
+ {file = "google_crc32c-1.7.1-cp312-cp312-win_amd64.whl", hash = "sha256:b7491bdc0c7564fcf48c0179d2048ab2f7c7ba36b84ccd3a3e1c3f7a72d3bba6"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_arm64.whl", hash = "sha256:df8b38bdaf1629d62d51be8bdd04888f37c451564c2042d36e5812da9eff3c35"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-macosx_12_0_x86_64.whl", hash = "sha256:e42e20a83a29aa2709a0cf271c7f8aefaa23b7ab52e53b322585297bb94d4638"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:905a385140bf492ac300026717af339790921f411c0dfd9aa5a9e69a08ed32eb"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6b211ddaf20f7ebeec5c333448582c224a7c90a9d98826fbab82c0ddc11348e6"},
+ {file = "google_crc32c-1.7.1-cp313-cp313-win_amd64.whl", hash = "sha256:0f99eaa09a9a7e642a61e06742856eec8b19fc0037832e03f941fe7cf0c8e4db"},
+ {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:32d1da0d74ec5634a05f53ef7df18fc646666a25efaaca9fc7dcfd4caf1d98c3"},
+ {file = "google_crc32c-1.7.1-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e10554d4abc5238823112c2ad7e4560f96c7bf3820b202660373d769d9e6e4c9"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_arm64.whl", hash = "sha256:9fc196f0b8d8bd2789352c6a522db03f89e83a0ed6b64315923c396d7a932315"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-macosx_12_0_x86_64.whl", hash = "sha256:bb5e35dcd8552f76eed9461a23de1030920a3c953c1982f324be8f97946e7127"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:f2226b6a8da04f1d9e61d3e357f2460b9551c5e6950071437e122c958a18ae14"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1f2b3522222746fff0e04a9bd0a23ea003ba3cccc8cf21385c564deb1f223242"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3bda0fcb632d390e3ea8b6b07bf6b4f4a66c9d02dcd6fbf7ba00a197c143f582"},
+ {file = "google_crc32c-1.7.1-cp39-cp39-win_amd64.whl", hash = "sha256:713121af19f1a617054c41f952294764e0c5443d5a5d9034b2cd60f5dd7e0349"},
+ {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a8e9afc74168b0b2232fb32dd202c93e46b7d5e4bf03e66ba5dc273bb3559589"},
+ {file = "google_crc32c-1.7.1-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa8136cc14dd27f34a3221c0f16fd42d8a40e4778273e61a3c19aedaa44daf6b"},
+ {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:85fef7fae11494e747c9fd1359a527e5970fc9603c90764843caabd3a16a0a48"},
+ {file = "google_crc32c-1.7.1-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6efb97eb4369d52593ad6f75e7e10d053cf00c48983f7a973105bc70b0ac4d82"},
+ {file = "google_crc32c-1.7.1.tar.gz", hash = "sha256:2bff2305f98846f3e825dbeec9ee406f89da7962accdb29356e4eadc251bd472"},
+]
+
+[package.extras]
+testing = ["pytest"]
[[package]]
name = "google-resumable-media"
-version = "0.5.1"
+version = "2.7.2"
description = "Utilities for Google Media Downloads and Resumable Uploads"
optional = true
-python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*"
+python-versions = ">=3.7"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "google-resumable-media-0.5.1.tar.gz", hash = "sha256:97155236971970382b738921f978a6f86a7b5a0b0311703d991e065d3cb55773"},
- {file = "google_resumable_media-0.5.1-py2.py3-none-any.whl", hash = "sha256:cdc64378dc9a7a7bf963a8d0c944c99b549dc0c195a9acbf1fcd465f380b9002"},
+ {file = "google_resumable_media-2.7.2-py2.py3-none-any.whl", hash = "sha256:3ce7551e9fe6d99e9a126101d2536612bb73486721951e9562fee0f90c6ababa"},
+ {file = "google_resumable_media-2.7.2.tar.gz", hash = "sha256:5280aed4629f2b60b847b0d42f9857fd4935c11af266744df33d8074cae92fe0"},
]
[package.dependencies]
-six = "*"
+google-crc32c = ">=1.0,<2.0dev"
[package.extras]
+aiohttp = ["aiohttp (>=3.6.2,<4.0.0dev)", "google-auth (>=1.22.0,<2.0dev)"]
requests = ["requests (>=2.18.0,<3.0.0dev)"]
[[package]]
@@ -1945,21 +2030,21 @@ colorama = ">=0.4"
[[package]]
name = "hf-xet"
-version = "1.1.3"
+version = "1.1.4"
description = "Fast transfer of large files with the Hugging Face Hub."
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"hf\" and (platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"arm64\" or platform_machine == \"aarch64\")"
files = [
- {file = "hf_xet-1.1.3-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:c3b508b5f583a75641aebf732853deb058953370ce8184f5dabc49f803b0819b"},
- {file = "hf_xet-1.1.3-cp37-abi3-macosx_11_0_arm64.whl", hash = "sha256:b788a61977fbe6b5186e66239e2a329a3f0b7e7ff50dad38984c0c74f44aeca1"},
- {file = "hf_xet-1.1.3-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fd2da210856444a34aad8ada2fc12f70dabed7cc20f37e90754d1d9b43bc0534"},
- {file = "hf_xet-1.1.3-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:8203f52827e3df65981984936654a5b390566336956f65765a8aa58c362bb841"},
- {file = "hf_xet-1.1.3-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:30c575a5306f8e6fda37edb866762140a435037365eba7a17ce7bd0bc0216a8b"},
- {file = "hf_xet-1.1.3-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:7c1a6aa6abed1f696f8099aa9796ca04c9ee778a58728a115607de9cc4638ff1"},
- {file = "hf_xet-1.1.3-cp37-abi3-win_amd64.whl", hash = "sha256:b578ae5ac9c056296bb0df9d018e597c8dc6390c5266f35b5c44696003cde9f3"},
- {file = "hf_xet-1.1.3.tar.gz", hash = "sha256:a5f09b1dd24e6ff6bcedb4b0ddab2d81824098bb002cf8b4ffa780545fa348c3"},
+ {file = "hf_xet-1.1.4-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:6591ab9f61ea82d261107ed90237e2ece972f6a7577d96f5f071208bbf255d1c"},
+ {file = "hf_xet-1.1.4-cp37-abi3-macosx_11_0_arm64.whl", hash = "sha256:071b0b4d4698990f746edd666c7cc42555833d22035d88db0df936677fb57d29"},
+ {file = "hf_xet-1.1.4-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b5b610831e92e41182d4c028653978b844d332d492cdcba1b920d3aca4a0207e"},
+ {file = "hf_xet-1.1.4-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:f6578bcd71393abfd60395279cc160ca808b61f5f9d535b922fcdcd3f77a708d"},
+ {file = "hf_xet-1.1.4-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:fb2bbfa2aae0e4f0baca988e7ba8d8c1a39a25adf5317461eb7069ad00505b3e"},
+ {file = "hf_xet-1.1.4-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:73346ba3e2e15ea8909a26b0862b458f15b003e6277935e3fba5bf273508d698"},
+ {file = "hf_xet-1.1.4-cp37-abi3-win_amd64.whl", hash = "sha256:52e8f8bc2029d8b911493f43cea131ac3fa1f0dc6a13c50b593c4516f02c6fc3"},
+ {file = "hf_xet-1.1.4.tar.gz", hash = "sha256:875158df90cb13547752532ed73cad9dfaad3b29e203143838f67178418d08a4"},
]
[package.extras]
@@ -2179,7 +2264,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -2223,8 +2308,6 @@ python-versions = "*"
groups = ["dev"]
files = [
{file = "jsonpath-ng-1.7.0.tar.gz", hash = "sha256:f6f5f7fd4e5ff79c785f1573b394043b39849fb2bb47bcead935d12b00beab3c"},
- {file = "jsonpath_ng-1.7.0-py2-none-any.whl", hash = "sha256:898c93fc173f0c336784a3fa63d7434297544b7198124a68f9a3ef9597b0ae6e"},
- {file = "jsonpath_ng-1.7.0-py3-none-any.whl", hash = "sha256:f3d7f9e848cba1b6da28c55b1c26ff915dc9e0b1ba7e752a53d6da8d5cbd00b6"},
]
[package.dependencies]
@@ -2340,14 +2423,14 @@ files = [
[[package]]
name = "markdown"
-version = "3.8"
+version = "3.8.2"
description = "Python implementation of John Gruber's Markdown."
optional = false
python-versions = ">=3.9"
groups = ["docs"]
files = [
- {file = "markdown-3.8-py3-none-any.whl", hash = "sha256:794a929b79c5af141ef5ab0f2f642d0f7b1872981250230e72682346f7cc90dc"},
- {file = "markdown-3.8.tar.gz", hash = "sha256:7df81e63f0df5c4b24b7d156eb81e4690595239b7d70937d0409f1b0de319c6f"},
+ {file = "markdown-3.8.2-py3-none-any.whl", hash = "sha256:5c83764dbd4e00bdd94d85a19b8d55ccca20fe35b2e678a1422b380324dd5f24"},
+ {file = "markdown-3.8.2.tar.gz", hash = "sha256:247b9a70dd12e27f67431ce62523e675b866d254f900c4fe75ce3dda62237c45"},
]
[package.dependencies]
@@ -2894,192 +2977,193 @@ portalocker = ["portalocker (>=1.4,<4)"]
[[package]]
name = "msgpack"
-version = "1.1.0"
+version = "1.1.1"
description = "MessagePack serializer"
optional = true
python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"ray\""
files = [
- {file = "msgpack-1.1.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7ad442d527a7e358a469faf43fda45aaf4ac3249c8310a82f0ccff9164e5dccd"},
- {file = "msgpack-1.1.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:74bed8f63f8f14d75eec75cf3d04ad581da6b914001b474a5d3cd3372c8cc27d"},
- {file = "msgpack-1.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:914571a2a5b4e7606997e169f64ce53a8b1e06f2cf2c3a7273aa106236d43dd5"},
- {file = "msgpack-1.1.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c921af52214dcbb75e6bdf6a661b23c3e6417f00c603dd2070bccb5c3ef499f5"},
- {file = "msgpack-1.1.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d8ce0b22b890be5d252de90d0e0d119f363012027cf256185fc3d474c44b1b9e"},
- {file = "msgpack-1.1.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:73322a6cc57fcee3c0c57c4463d828e9428275fb85a27aa2aa1a92fdc42afd7b"},
- {file = "msgpack-1.1.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:e1f3c3d21f7cf67bcf2da8e494d30a75e4cf60041d98b3f79875afb5b96f3a3f"},
- {file = "msgpack-1.1.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:64fc9068d701233effd61b19efb1485587560b66fe57b3e50d29c5d78e7fef68"},
- {file = "msgpack-1.1.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:42f754515e0f683f9c79210a5d1cad631ec3d06cea5172214d2176a42e67e19b"},
- {file = "msgpack-1.1.0-cp310-cp310-win32.whl", hash = "sha256:3df7e6b05571b3814361e8464f9304c42d2196808e0119f55d0d3e62cd5ea044"},
- {file = "msgpack-1.1.0-cp310-cp310-win_amd64.whl", hash = "sha256:685ec345eefc757a7c8af44a3032734a739f8c45d1b0ac45efc5d8977aa4720f"},
- {file = "msgpack-1.1.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:3d364a55082fb2a7416f6c63ae383fbd903adb5a6cf78c5b96cc6316dc1cedc7"},
- {file = "msgpack-1.1.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:79ec007767b9b56860e0372085f8504db5d06bd6a327a335449508bbee9648fa"},
- {file = "msgpack-1.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:6ad622bf7756d5a497d5b6836e7fc3752e2dd6f4c648e24b1803f6048596f701"},
- {file = "msgpack-1.1.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8e59bca908d9ca0de3dc8684f21ebf9a690fe47b6be93236eb40b99af28b6ea6"},
- {file = "msgpack-1.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5e1da8f11a3dd397f0a32c76165cf0c4eb95b31013a94f6ecc0b280c05c91b59"},
- {file = "msgpack-1.1.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:452aff037287acb1d70a804ffd022b21fa2bb7c46bee884dbc864cc9024128a0"},
- {file = "msgpack-1.1.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8da4bf6d54ceed70e8861f833f83ce0814a2b72102e890cbdfe4b34764cdd66e"},
- {file = "msgpack-1.1.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:41c991beebf175faf352fb940bf2af9ad1fb77fd25f38d9142053914947cdbf6"},
- {file = "msgpack-1.1.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:a52a1f3a5af7ba1c9ace055b659189f6c669cf3657095b50f9602af3a3ba0fe5"},
- {file = "msgpack-1.1.0-cp311-cp311-win32.whl", hash = "sha256:58638690ebd0a06427c5fe1a227bb6b8b9fdc2bd07701bec13c2335c82131a88"},
- {file = "msgpack-1.1.0-cp311-cp311-win_amd64.whl", hash = "sha256:fd2906780f25c8ed5d7b323379f6138524ba793428db5d0e9d226d3fa6aa1788"},
- {file = "msgpack-1.1.0-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:d46cf9e3705ea9485687aa4001a76e44748b609d260af21c4ceea7f2212a501d"},
- {file = "msgpack-1.1.0-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:5dbad74103df937e1325cc4bfeaf57713be0b4f15e1c2da43ccdd836393e2ea2"},
- {file = "msgpack-1.1.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:58dfc47f8b102da61e8949708b3eafc3504509a5728f8b4ddef84bd9e16ad420"},
- {file = "msgpack-1.1.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4676e5be1b472909b2ee6356ff425ebedf5142427842aa06b4dfd5117d1ca8a2"},
- {file = "msgpack-1.1.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:17fb65dd0bec285907f68b15734a993ad3fc94332b5bb21b0435846228de1f39"},
- {file = "msgpack-1.1.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a51abd48c6d8ac89e0cfd4fe177c61481aca2d5e7ba42044fd218cfd8ea9899f"},
- {file = "msgpack-1.1.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:2137773500afa5494a61b1208619e3871f75f27b03bcfca7b3a7023284140247"},
- {file = "msgpack-1.1.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:398b713459fea610861c8a7b62a6fec1882759f308ae0795b5413ff6a160cf3c"},
- {file = "msgpack-1.1.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:06f5fd2f6bb2a7914922d935d3b8bb4a7fff3a9a91cfce6d06c13bc42bec975b"},
- {file = "msgpack-1.1.0-cp312-cp312-win32.whl", hash = "sha256:ad33e8400e4ec17ba782f7b9cf868977d867ed784a1f5f2ab46e7ba53b6e1e1b"},
- {file = "msgpack-1.1.0-cp312-cp312-win_amd64.whl", hash = "sha256:115a7af8ee9e8cddc10f87636767857e7e3717b7a2e97379dc2054712693e90f"},
- {file = "msgpack-1.1.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:071603e2f0771c45ad9bc65719291c568d4edf120b44eb36324dcb02a13bfddf"},
- {file = "msgpack-1.1.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:0f92a83b84e7c0749e3f12821949d79485971f087604178026085f60ce109330"},
- {file = "msgpack-1.1.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:4a1964df7b81285d00a84da4e70cb1383f2e665e0f1f2a7027e683956d04b734"},
- {file = "msgpack-1.1.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:59caf6a4ed0d164055ccff8fe31eddc0ebc07cf7326a2aaa0dbf7a4001cd823e"},
- {file = "msgpack-1.1.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0907e1a7119b337971a689153665764adc34e89175f9a34793307d9def08e6ca"},
- {file = "msgpack-1.1.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:65553c9b6da8166e819a6aa90ad15288599b340f91d18f60b2061f402b9a4915"},
- {file = "msgpack-1.1.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:7a946a8992941fea80ed4beae6bff74ffd7ee129a90b4dd5cf9c476a30e9708d"},
- {file = "msgpack-1.1.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:4b51405e36e075193bc051315dbf29168d6141ae2500ba8cd80a522964e31434"},
- {file = "msgpack-1.1.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:b4c01941fd2ff87c2a934ee6055bda4ed353a7846b8d4f341c428109e9fcde8c"},
- {file = "msgpack-1.1.0-cp313-cp313-win32.whl", hash = "sha256:7c9a35ce2c2573bada929e0b7b3576de647b0defbd25f5139dcdaba0ae35a4cc"},
- {file = "msgpack-1.1.0-cp313-cp313-win_amd64.whl", hash = "sha256:bce7d9e614a04d0883af0b3d4d501171fbfca038f12c77fa838d9f198147a23f"},
- {file = "msgpack-1.1.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c40ffa9a15d74e05ba1fe2681ea33b9caffd886675412612d93ab17b58ea2fec"},
- {file = "msgpack-1.1.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f1ba6136e650898082d9d5a5217d5906d1e138024f836ff48691784bbe1adf96"},
- {file = "msgpack-1.1.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e0856a2b7e8dcb874be44fea031d22e5b3a19121be92a1e098f46068a11b0870"},
- {file = "msgpack-1.1.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:471e27a5787a2e3f974ba023f9e265a8c7cfd373632247deb225617e3100a3c7"},
- {file = "msgpack-1.1.0-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:646afc8102935a388ffc3914b336d22d1c2d6209c773f3eb5dd4d6d3b6f8c1cb"},
- {file = "msgpack-1.1.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:13599f8829cfbe0158f6456374e9eea9f44eee08076291771d8ae93eda56607f"},
- {file = "msgpack-1.1.0-cp38-cp38-win32.whl", hash = "sha256:8a84efb768fb968381e525eeeb3d92857e4985aacc39f3c47ffd00eb4509315b"},
- {file = "msgpack-1.1.0-cp38-cp38-win_amd64.whl", hash = "sha256:879a7b7b0ad82481c52d3c7eb99bf6f0645dbdec5134a4bddbd16f3506947feb"},
- {file = "msgpack-1.1.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:53258eeb7a80fc46f62fd59c876957a2d0e15e6449a9e71842b6d24419d88ca1"},
- {file = "msgpack-1.1.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:7e7b853bbc44fb03fbdba34feb4bd414322180135e2cb5164f20ce1c9795ee48"},
- {file = "msgpack-1.1.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:f3e9b4936df53b970513eac1758f3882c88658a220b58dcc1e39606dccaaf01c"},
- {file = "msgpack-1.1.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:46c34e99110762a76e3911fc923222472c9d681f1094096ac4102c18319e6468"},
- {file = "msgpack-1.1.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8a706d1e74dd3dea05cb54580d9bd8b2880e9264856ce5068027eed09680aa74"},
- {file = "msgpack-1.1.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:534480ee5690ab3cbed89d4c8971a5c631b69a8c0883ecfea96c19118510c846"},
- {file = "msgpack-1.1.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:8cf9e8c3a2153934a23ac160cc4cba0ec035f6867c8013cc6077a79823370346"},
- {file = "msgpack-1.1.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:3180065ec2abbe13a4ad37688b61b99d7f9e012a535b930e0e683ad6bc30155b"},
- {file = "msgpack-1.1.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:c5a91481a3cc573ac8c0d9aace09345d989dc4a0202b7fcb312c88c26d4e71a8"},
- {file = "msgpack-1.1.0-cp39-cp39-win32.whl", hash = "sha256:f80bc7d47f76089633763f952e67f8214cb7b3ee6bfa489b3cb6a84cfac114cd"},
- {file = "msgpack-1.1.0-cp39-cp39-win_amd64.whl", hash = "sha256:4d1b7ff2d6146e16e8bd665ac726a89c74163ef8cd39fa8c1087d4e52d3a2325"},
- {file = "msgpack-1.1.0.tar.gz", hash = "sha256:dd432ccc2c72b914e4cb77afce64aab761c1137cc698be3984eee260bcb2896e"},
+ {file = "msgpack-1.1.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:353b6fc0c36fde68b661a12949d7d49f8f51ff5fa019c1e47c87c4ff34b080ed"},
+ {file = "msgpack-1.1.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:79c408fcf76a958491b4e3b103d1c417044544b68e96d06432a189b43d1215c8"},
+ {file = "msgpack-1.1.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:78426096939c2c7482bf31ef15ca219a9e24460289c00dd0b94411040bb73ad2"},
+ {file = "msgpack-1.1.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8b17ba27727a36cb73aabacaa44b13090feb88a01d012c0f4be70c00f75048b4"},
+ {file = "msgpack-1.1.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7a17ac1ea6ec3c7687d70201cfda3b1e8061466f28f686c24f627cae4ea8efd0"},
+ {file = "msgpack-1.1.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:88d1e966c9235c1d4e2afac21ca83933ba59537e2e2727a999bf3f515ca2af26"},
+ {file = "msgpack-1.1.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:f6d58656842e1b2ddbe07f43f56b10a60f2ba5826164910968f5933e5178af75"},
+ {file = "msgpack-1.1.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:96decdfc4adcbc087f5ea7ebdcfd3dee9a13358cae6e81d54be962efc38f6338"},
+ {file = "msgpack-1.1.1-cp310-cp310-win32.whl", hash = "sha256:6640fd979ca9a212e4bcdf6eb74051ade2c690b862b679bfcb60ae46e6dc4bfd"},
+ {file = "msgpack-1.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:8b65b53204fe1bd037c40c4148d00ef918eb2108d24c9aaa20bc31f9810ce0a8"},
+ {file = "msgpack-1.1.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:71ef05c1726884e44f8b1d1773604ab5d4d17729d8491403a705e649116c9558"},
+ {file = "msgpack-1.1.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:36043272c6aede309d29d56851f8841ba907a1a3d04435e43e8a19928e243c1d"},
+ {file = "msgpack-1.1.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a32747b1b39c3ac27d0670122b57e6e57f28eefb725e0b625618d1b59bf9d1e0"},
+ {file = "msgpack-1.1.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8a8b10fdb84a43e50d38057b06901ec9da52baac6983d3f709d8507f3889d43f"},
+ {file = "msgpack-1.1.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ba0c325c3f485dc54ec298d8b024e134acf07c10d494ffa24373bea729acf704"},
+ {file = "msgpack-1.1.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:88daaf7d146e48ec71212ce21109b66e06a98e5e44dca47d853cbfe171d6c8d2"},
+ {file = "msgpack-1.1.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:d8b55ea20dc59b181d3f47103f113e6f28a5e1c89fd5b67b9140edb442ab67f2"},
+ {file = "msgpack-1.1.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:4a28e8072ae9779f20427af07f53bbb8b4aa81151054e882aee333b158da8752"},
+ {file = "msgpack-1.1.1-cp311-cp311-win32.whl", hash = "sha256:7da8831f9a0fdb526621ba09a281fadc58ea12701bc709e7b8cbc362feabc295"},
+ {file = "msgpack-1.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:5fd1b58e1431008a57247d6e7cc4faa41c3607e8e7d4aaf81f7c29ea013cb458"},
+ {file = "msgpack-1.1.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:ae497b11f4c21558d95de9f64fff7053544f4d1a17731c866143ed6bb4591238"},
+ {file = "msgpack-1.1.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:33be9ab121df9b6b461ff91baac6f2731f83d9b27ed948c5b9d1978ae28bf157"},
+ {file = "msgpack-1.1.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6f64ae8fe7ffba251fecb8408540c34ee9df1c26674c50c4544d72dbf792e5ce"},
+ {file = "msgpack-1.1.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a494554874691720ba5891c9b0b39474ba43ffb1aaf32a5dac874effb1619e1a"},
+ {file = "msgpack-1.1.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cb643284ab0ed26f6957d969fe0dd8bb17beb567beb8998140b5e38a90974f6c"},
+ {file = "msgpack-1.1.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:d275a9e3c81b1093c060c3837e580c37f47c51eca031f7b5fb76f7b8470f5f9b"},
+ {file = "msgpack-1.1.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:4fd6b577e4541676e0cc9ddc1709d25014d3ad9a66caa19962c4f5de30fc09ef"},
+ {file = "msgpack-1.1.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:bb29aaa613c0a1c40d1af111abf025f1732cab333f96f285d6a93b934738a68a"},
+ {file = "msgpack-1.1.1-cp312-cp312-win32.whl", hash = "sha256:870b9a626280c86cff9c576ec0d9cbcc54a1e5ebda9cd26dab12baf41fee218c"},
+ {file = "msgpack-1.1.1-cp312-cp312-win_amd64.whl", hash = "sha256:5692095123007180dca3e788bb4c399cc26626da51629a31d40207cb262e67f4"},
+ {file = "msgpack-1.1.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:3765afa6bd4832fc11c3749be4ba4b69a0e8d7b728f78e68120a157a4c5d41f0"},
+ {file = "msgpack-1.1.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:8ddb2bcfd1a8b9e431c8d6f4f7db0773084e107730ecf3472f1dfe9ad583f3d9"},
+ {file = "msgpack-1.1.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:196a736f0526a03653d829d7d4c5500a97eea3648aebfd4b6743875f28aa2af8"},
+ {file = "msgpack-1.1.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9d592d06e3cc2f537ceeeb23d38799c6ad83255289bb84c2e5792e5a8dea268a"},
+ {file = "msgpack-1.1.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4df2311b0ce24f06ba253fda361f938dfecd7b961576f9be3f3fbd60e87130ac"},
+ {file = "msgpack-1.1.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:e4141c5a32b5e37905b5940aacbc59739f036930367d7acce7a64e4dec1f5e0b"},
+ {file = "msgpack-1.1.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:b1ce7f41670c5a69e1389420436f41385b1aa2504c3b0c30620764b15dded2e7"},
+ {file = "msgpack-1.1.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:4147151acabb9caed4e474c3344181e91ff7a388b888f1e19ea04f7e73dc7ad5"},
+ {file = "msgpack-1.1.1-cp313-cp313-win32.whl", hash = "sha256:500e85823a27d6d9bba1d057c871b4210c1dd6fb01fbb764e37e4e8847376323"},
+ {file = "msgpack-1.1.1-cp313-cp313-win_amd64.whl", hash = "sha256:6d489fba546295983abd142812bda76b57e33d0b9f5d5b71c09a583285506f69"},
+ {file = "msgpack-1.1.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bba1be28247e68994355e028dcd668316db30c1f758d3241a7b903ac78dcd285"},
+ {file = "msgpack-1.1.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b8f93dcddb243159c9e4109c9750ba5b335ab8d48d9522c5308cd05d7e3ce600"},
+ {file = "msgpack-1.1.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2fbbc0b906a24038c9958a1ba7ae0918ad35b06cb449d398b76a7d08470b0ed9"},
+ {file = "msgpack-1.1.1-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:61e35a55a546a1690d9d09effaa436c25ae6130573b6ee9829c37ef0f18d5e78"},
+ {file = "msgpack-1.1.1-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:1abfc6e949b352dadf4bce0eb78023212ec5ac42f6abfd469ce91d783c149c2a"},
+ {file = "msgpack-1.1.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:996f2609ddf0142daba4cefd767d6db26958aac8439ee41db9cc0db9f4c4c3a6"},
+ {file = "msgpack-1.1.1-cp38-cp38-win32.whl", hash = "sha256:4d3237b224b930d58e9d83c81c0dba7aacc20fcc2f89c1e5423aa0529a4cd142"},
+ {file = "msgpack-1.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:da8f41e602574ece93dbbda1fab24650d6bf2a24089f9e9dbb4f5730ec1e58ad"},
+ {file = "msgpack-1.1.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:f5be6b6bc52fad84d010cb45433720327ce886009d862f46b26d4d154001994b"},
+ {file = "msgpack-1.1.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:3a89cd8c087ea67e64844287ea52888239cbd2940884eafd2dcd25754fb72232"},
+ {file = "msgpack-1.1.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d75f3807a9900a7d575d8d6674a3a47e9f227e8716256f35bc6f03fc597ffbf"},
+ {file = "msgpack-1.1.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d182dac0221eb8faef2e6f44701812b467c02674a322c739355c39e94730cdbf"},
+ {file = "msgpack-1.1.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1b13fe0fb4aac1aa5320cd693b297fe6fdef0e7bea5518cbc2dd5299f873ae90"},
+ {file = "msgpack-1.1.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:435807eeb1bc791ceb3247d13c79868deb22184e1fc4224808750f0d7d1affc1"},
+ {file = "msgpack-1.1.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:4835d17af722609a45e16037bb1d4d78b7bdf19d6c0128116d178956618c4e88"},
+ {file = "msgpack-1.1.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:a8ef6e342c137888ebbfb233e02b8fbd689bb5b5fcc59b34711ac47ebd504478"},
+ {file = "msgpack-1.1.1-cp39-cp39-win32.whl", hash = "sha256:61abccf9de335d9efd149e2fff97ed5974f2481b3353772e8e2dd3402ba2bd57"},
+ {file = "msgpack-1.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:40eae974c873b2992fd36424a5d9407f93e97656d999f43fca9d29f820899084"},
+ {file = "msgpack-1.1.1.tar.gz", hash = "sha256:77b79ce34a2bdab2594f490c8e80dd62a02d650b91a75159a63ec413b8d104cd"},
]
[[package]]
name = "multidict"
-version = "6.4.4"
+version = "6.5.0"
description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
-files = [
- {file = "multidict-6.4.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:8adee3ac041145ffe4488ea73fa0a622b464cc25340d98be76924d0cda8545ff"},
- {file = "multidict-6.4.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b61e98c3e2a861035aaccd207da585bdcacef65fe01d7a0d07478efac005e028"},
- {file = "multidict-6.4.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:75493f28dbadecdbb59130e74fe935288813301a8554dc32f0c631b6bdcdf8b0"},
- {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4ffc3c6a37e048b5395ee235e4a2a0d639c2349dffa32d9367a42fc20d399772"},
- {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:87cb72263946b301570b0f63855569a24ee8758aaae2cd182aae7d95fbc92ca7"},
- {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9bbf7bd39822fd07e3609b6b4467af4c404dd2b88ee314837ad1830a7f4a8299"},
- {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d1f7cbd4f1f44ddf5fd86a8675b7679176eae770f2fc88115d6dddb6cefb59bc"},
- {file = "multidict-6.4.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bb5ac9e5bfce0e6282e7f59ff7b7b9a74aa8e5c60d38186a4637f5aa764046ad"},
- {file = "multidict-6.4.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4efc31dfef8c4eeb95b6b17d799eedad88c4902daba39ce637e23a17ea078915"},
- {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:9fcad2945b1b91c29ef2b4050f590bfcb68d8ac8e0995a74e659aa57e8d78e01"},
- {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:d877447e7368c7320832acb7159557e49b21ea10ffeb135c1077dbbc0816b598"},
- {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:33a12ebac9f380714c298cbfd3e5b9c0c4e89c75fe612ae496512ee51028915f"},
- {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:0f14ea68d29b43a9bf37953881b1e3eb75b2739e896ba4a6aa4ad4c5b9ffa145"},
- {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:0327ad2c747a6600e4797d115d3c38a220fdb28e54983abe8964fd17e95ae83c"},
- {file = "multidict-6.4.4-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:d1a20707492db9719a05fc62ee215fd2c29b22b47c1b1ba347f9abc831e26683"},
- {file = "multidict-6.4.4-cp310-cp310-win32.whl", hash = "sha256:d83f18315b9fca5db2452d1881ef20f79593c4aa824095b62cb280019ef7aa3d"},
- {file = "multidict-6.4.4-cp310-cp310-win_amd64.whl", hash = "sha256:9c17341ee04545fd962ae07330cb5a39977294c883485c8d74634669b1f7fe04"},
- {file = "multidict-6.4.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:4f5f29794ac0e73d2a06ac03fd18870adc0135a9d384f4a306a951188ed02f95"},
- {file = "multidict-6.4.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c04157266344158ebd57b7120d9b0b35812285d26d0e78193e17ef57bfe2979a"},
- {file = "multidict-6.4.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:bb61ffd3ab8310d93427e460f565322c44ef12769f51f77277b4abad7b6f7223"},
- {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5e0ba18a9afd495f17c351d08ebbc4284e9c9f7971d715f196b79636a4d0de44"},
- {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:9faf1b1dcaadf9f900d23a0e6d6c8eadd6a95795a0e57fcca73acce0eb912065"},
- {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a4d1cb1327c6082c4fce4e2a438483390964c02213bc6b8d782cf782c9b1471f"},
- {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:941f1bec2f5dbd51feeb40aea654c2747f811ab01bdd3422a48a4e4576b7d76a"},
- {file = "multidict-6.4.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e5f8a146184da7ea12910a4cec51ef85e44f6268467fb489c3caf0cd512f29c2"},
- {file = "multidict-6.4.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:232b7237e57ec3c09be97206bfb83a0aa1c5d7d377faa019c68a210fa35831f1"},
- {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:55ae0721c1513e5e3210bca4fc98456b980b0c2c016679d3d723119b6b202c42"},
- {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:51d662c072579f63137919d7bb8fc250655ce79f00c82ecf11cab678f335062e"},
- {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:0e05c39962baa0bb19a6b210e9b1422c35c093b651d64246b6c2e1a7e242d9fd"},
- {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:d5b1cc3ab8c31d9ebf0faa6e3540fb91257590da330ffe6d2393d4208e638925"},
- {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:93ec84488a384cd7b8a29c2c7f467137d8a73f6fe38bb810ecf29d1ade011a7c"},
- {file = "multidict-6.4.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:b308402608493638763abc95f9dc0030bbd6ac6aff784512e8ac3da73a88af08"},
- {file = "multidict-6.4.4-cp311-cp311-win32.whl", hash = "sha256:343892a27d1a04d6ae455ecece12904d242d299ada01633d94c4f431d68a8c49"},
- {file = "multidict-6.4.4-cp311-cp311-win_amd64.whl", hash = "sha256:73484a94f55359780c0f458bbd3c39cb9cf9c182552177d2136e828269dee529"},
- {file = "multidict-6.4.4-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:dc388f75a1c00000824bf28b7633e40854f4127ede80512b44c3cfeeea1839a2"},
- {file = "multidict-6.4.4-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:98af87593a666f739d9dba5d0ae86e01b0e1a9cfcd2e30d2d361fbbbd1a9162d"},
- {file = "multidict-6.4.4-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:aff4cafea2d120327d55eadd6b7f1136a8e5a0ecf6fb3b6863e8aca32cd8e50a"},
- {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:169c4ba7858176b797fe551d6e99040c531c775d2d57b31bcf4de6d7a669847f"},
- {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:b9eb4c59c54421a32b3273d4239865cb14ead53a606db066d7130ac80cc8ec93"},
- {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7cf3bd54c56aa16fdb40028d545eaa8d051402b61533c21e84046e05513d5780"},
- {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f682c42003c7264134bfe886376299db4cc0c6cd06a3295b41b347044bcb5482"},
- {file = "multidict-6.4.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a920f9cf2abdf6e493c519492d892c362007f113c94da4c239ae88429835bad1"},
- {file = "multidict-6.4.4-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:530d86827a2df6504526106b4c104ba19044594f8722d3e87714e847c74a0275"},
- {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:ecde56ea2439b96ed8a8d826b50c57364612ddac0438c39e473fafad7ae1c23b"},
- {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:dc8c9736d8574b560634775ac0def6bdc1661fc63fa27ffdfc7264c565bcb4f2"},
- {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:7f3d3b3c34867579ea47cbd6c1f2ce23fbfd20a273b6f9e3177e256584f1eacc"},
- {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:87a728af265e08f96b6318ebe3c0f68b9335131f461efab2fc64cc84a44aa6ed"},
- {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:9f193eeda1857f8e8d3079a4abd258f42ef4a4bc87388452ed1e1c4d2b0c8740"},
- {file = "multidict-6.4.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:be06e73c06415199200e9a2324a11252a3d62030319919cde5e6950ffeccf72e"},
- {file = "multidict-6.4.4-cp312-cp312-win32.whl", hash = "sha256:622f26ea6a7e19b7c48dd9228071f571b2fbbd57a8cd71c061e848f281550e6b"},
- {file = "multidict-6.4.4-cp312-cp312-win_amd64.whl", hash = "sha256:5e2bcda30d5009996ff439e02a9f2b5c3d64a20151d34898c000a6281faa3781"},
- {file = "multidict-6.4.4-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:82ffabefc8d84c2742ad19c37f02cde5ec2a1ee172d19944d380f920a340e4b9"},
- {file = "multidict-6.4.4-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:6a2f58a66fe2c22615ad26156354005391e26a2f3721c3621504cd87c1ea87bf"},
- {file = "multidict-6.4.4-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:5883d6ee0fd9d8a48e9174df47540b7545909841ac82354c7ae4cbe9952603bd"},
- {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9abcf56a9511653fa1d052bfc55fbe53dbee8f34e68bd6a5a038731b0ca42d15"},
- {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:6ed5ae5605d4ad5a049fad2a28bb7193400700ce2f4ae484ab702d1e3749c3f9"},
- {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:bbfcb60396f9bcfa63e017a180c3105b8c123a63e9d1428a36544e7d37ca9e20"},
- {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b0f1987787f5f1e2076b59692352ab29a955b09ccc433c1f6b8e8e18666f608b"},
- {file = "multidict-6.4.4-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1d0121ccce8c812047d8d43d691a1ad7641f72c4f730474878a5aeae1b8ead8c"},
- {file = "multidict-6.4.4-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:83ec4967114295b8afd120a8eec579920c882831a3e4c3331d591a8e5bfbbc0f"},
- {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:995f985e2e268deaf17867801b859a282e0448633f1310e3704b30616d269d69"},
- {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:d832c608f94b9f92a0ec8b7e949be7792a642b6e535fcf32f3e28fab69eeb046"},
- {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:d21c1212171cf7da703c5b0b7a0e85be23b720818aef502ad187d627316d5645"},
- {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:cbebaa076aaecad3d4bb4c008ecc73b09274c952cf6a1b78ccfd689e51f5a5b0"},
- {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:c93a6fb06cc8e5d3628b2b5fda215a5db01e8f08fc15fadd65662d9b857acbe4"},
- {file = "multidict-6.4.4-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8cd8f81f1310182362fb0c7898145ea9c9b08a71081c5963b40ee3e3cac589b1"},
- {file = "multidict-6.4.4-cp313-cp313-win32.whl", hash = "sha256:3e9f1cd61a0ab857154205fb0b1f3d3ace88d27ebd1409ab7af5096e409614cd"},
- {file = "multidict-6.4.4-cp313-cp313-win_amd64.whl", hash = "sha256:8ffb40b74400e4455785c2fa37eba434269149ec525fc8329858c862e4b35373"},
- {file = "multidict-6.4.4-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:6a602151dbf177be2450ef38966f4be3467d41a86c6a845070d12e17c858a156"},
- {file = "multidict-6.4.4-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:0d2b9712211b860d123815a80b859075d86a4d54787e247d7fbee9db6832cf1c"},
- {file = "multidict-6.4.4-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:d2fa86af59f8fc1972e121ade052145f6da22758f6996a197d69bb52f8204e7e"},
- {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:50855d03e9e4d66eab6947ba688ffb714616f985838077bc4b490e769e48da51"},
- {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:5bce06b83be23225be1905dcdb6b789064fae92499fbc458f59a8c0e68718601"},
- {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:66ed0731f8e5dfd8369a883b6e564aca085fb9289aacabd9decd70568b9a30de"},
- {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:329ae97fc2f56f44d91bc47fe0972b1f52d21c4b7a2ac97040da02577e2daca2"},
- {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c27e5dcf520923d6474d98b96749e6805f7677e93aaaf62656005b8643f907ab"},
- {file = "multidict-6.4.4-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:058cc59b9e9b143cc56715e59e22941a5d868c322242278d28123a5d09cdf6b0"},
- {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:69133376bc9a03f8c47343d33f91f74a99c339e8b58cea90433d8e24bb298031"},
- {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:d6b15c55721b1b115c5ba178c77104123745b1417527ad9641a4c5e2047450f0"},
- {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:a887b77f51d3d41e6e1a63cf3bc7ddf24de5939d9ff69441387dfefa58ac2e26"},
- {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:632a3bf8f1787f7ef7d3c2f68a7bde5be2f702906f8b5842ad6da9d974d0aab3"},
- {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:a145c550900deb7540973c5cdb183b0d24bed6b80bf7bddf33ed8f569082535e"},
- {file = "multidict-6.4.4-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:cc5d83c6619ca5c9672cb78b39ed8542f1975a803dee2cda114ff73cbb076edd"},
- {file = "multidict-6.4.4-cp313-cp313t-win32.whl", hash = "sha256:3312f63261b9df49be9d57aaa6abf53a6ad96d93b24f9cc16cf979956355ce6e"},
- {file = "multidict-6.4.4-cp313-cp313t-win_amd64.whl", hash = "sha256:ba852168d814b2c73333073e1c7116d9395bea69575a01b0b3c89d2d5a87c8fb"},
- {file = "multidict-6.4.4-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:603f39bd1cf85705c6c1ba59644b480dfe495e6ee2b877908de93322705ad7cf"},
- {file = "multidict-6.4.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:fc60f91c02e11dfbe3ff4e1219c085695c339af72d1641800fe6075b91850c8f"},
- {file = "multidict-6.4.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:496bcf01c76a70a31c3d746fd39383aad8d685ce6331e4c709e9af4ced5fa221"},
- {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4219390fb5bf8e548e77b428bb36a21d9382960db5321b74d9d9987148074d6b"},
- {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:3ef4e9096ff86dfdcbd4a78253090ba13b1d183daa11b973e842465d94ae1772"},
- {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:49a29d7133b1fc214e818bbe025a77cc6025ed9a4f407d2850373ddde07fd04a"},
- {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e32053d6d3a8b0dfe49fde05b496731a0e6099a4df92154641c00aa76786aef5"},
- {file = "multidict-6.4.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8cc403092a49509e8ef2d2fd636a8ecefc4698cc57bbe894606b14579bc2a955"},
- {file = "multidict-6.4.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5363f9b2a7f3910e5c87d8b1855c478c05a2dc559ac57308117424dfaad6805c"},
- {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:2e543a40e4946cf70a88a3be87837a3ae0aebd9058ba49e91cacb0b2cd631e2b"},
- {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:60d849912350da557fe7de20aa8cf394aada6980d0052cc829eeda4a0db1c1db"},
- {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:19d08b4f22eae45bb018b9f06e2838c1e4b853c67628ef8ae126d99de0da6395"},
- {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:d693307856d1ef08041e8b6ff01d5b4618715007d288490ce2c7e29013c12b9a"},
- {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:fad6daaed41021934917f4fb03ca2db8d8a4d79bf89b17ebe77228eb6710c003"},
- {file = "multidict-6.4.4-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:c10d17371bff801af0daf8b073c30b6cf14215784dc08cd5c43ab5b7b8029bbc"},
- {file = "multidict-6.4.4-cp39-cp39-win32.whl", hash = "sha256:7e23f2f841fcb3ebd4724a40032d32e0892fbba4143e43d2a9e7695c5e50e6bd"},
- {file = "multidict-6.4.4-cp39-cp39-win_amd64.whl", hash = "sha256:4d7b50b673ffb4ff4366e7ab43cf1f0aef4bd3608735c5fbdf0bdb6f690da411"},
- {file = "multidict-6.4.4-py3-none-any.whl", hash = "sha256:bd4557071b561a8b3b6075c3ce93cf9bfb6182cb241805c3d66ced3b75eff4ac"},
- {file = "multidict-6.4.4.tar.gz", hash = "sha256:69ee9e6ba214b5245031b76233dd95408a0fd57fdb019ddcc1ead4790932a8e8"},
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+files = [
+ {file = "multidict-6.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:2e118a202904623b1d2606d1c8614e14c9444b59d64454b0c355044058066469"},
+ {file = "multidict-6.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:a42995bdcaff4e22cb1280ae7752c3ed3fbb398090c6991a2797a4a0e5ed16a9"},
+ {file = "multidict-6.5.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2261b538145723ca776e55208640fffd7ee78184d223f37c2b40b9edfe0e818a"},
+ {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0e5b19f8cd67235fab3e195ca389490415d9fef5a315b1fa6f332925dc924262"},
+ {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:177b081e4dec67c3320b16b3aa0babc178bbf758553085669382c7ec711e1ec8"},
+ {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4d30a2cc106a7d116b52ee046207614db42380b62e6b1dd2a50eba47c5ca5eb1"},
+ {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a72933bc308d7a64de37f0d51795dbeaceebdfb75454f89035cdfc6a74cfd129"},
+ {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:96d109e663d032280ef8ef62b50924b2e887d5ddf19e301844a6cb7e91a172a6"},
+ {file = "multidict-6.5.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b555329c9894332401f03b9a87016f0b707b6fccd4706793ec43b4a639e75869"},
+ {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:6994bad9d471ef2156f2b6850b51e20ee409c6b9deebc0e57be096be9faffdce"},
+ {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:b15f817276c96cde9060569023808eec966bd8da56a97e6aa8116f34ddab6534"},
+ {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b4bf507c991db535a935b2127cf057a58dbc688c9f309c72080795c63e796f58"},
+ {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:60c3f8f13d443426c55f88cf3172547bbc600a86d57fd565458b9259239a6737"},
+ {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:a10227168a24420c158747fc201d4279aa9af1671f287371597e2b4f2ff21879"},
+ {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:e3b1425fe54ccfde66b8cfb25d02be34d5dfd2261a71561ffd887ef4088b4b69"},
+ {file = "multidict-6.5.0-cp310-cp310-win32.whl", hash = "sha256:b4e47ef51237841d1087e1e1548071a6ef22e27ed0400c272174fa585277c4b4"},
+ {file = "multidict-6.5.0-cp310-cp310-win_amd64.whl", hash = "sha256:63b3b24fadc7067282c88fae5b2f366d5b3a7c15c021c2838de8c65a50eeefb4"},
+ {file = "multidict-6.5.0-cp310-cp310-win_arm64.whl", hash = "sha256:8b2d61afbafc679b7eaf08e9de4fa5d38bd5dc7a9c0a577c9f9588fb49f02dbb"},
+ {file = "multidict-6.5.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:8b4bf6bb15a05796a07a248084e3e46e032860c899c7a9b981030e61368dba95"},
+ {file = "multidict-6.5.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:46bb05d50219655c42a4b8fcda9c7ee658a09adbb719c48e65a20284e36328ea"},
+ {file = "multidict-6.5.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:54f524d73f4d54e87e03c98f6af601af4777e4668a52b1bd2ae0a4d6fc7b392b"},
+ {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:529b03600466480ecc502000d62e54f185a884ed4570dee90d9a273ee80e37b5"},
+ {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:69ad681ad7c93a41ee7005cc83a144b5b34a3838bcf7261e2b5356057b0f78de"},
+ {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3fe9fada8bc0839466b09fa3f6894f003137942984843ec0c3848846329a36ae"},
+ {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f94c6ea6405fcf81baef1e459b209a78cda5442e61b5b7a57ede39d99b5204a0"},
+ {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84ca75ad8a39ed75f079a8931435a5b51ee4c45d9b32e1740f99969a5d1cc2ee"},
+ {file = "multidict-6.5.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:be4c08f3a2a6cc42b414496017928d95898964fed84b1b2dace0c9ee763061f9"},
+ {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:046a7540cfbb4d5dc846a1fd9843f3ba980c6523f2e0c5b8622b4a5c94138ae6"},
+ {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:64306121171d988af77d74be0d8c73ee1a69cf6f96aea7fa6030c88f32a152dd"},
+ {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:b4ac1dd5eb0ecf6f7351d5a9137f30a83f7182209c5d37f61614dfdce5714853"},
+ {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:bab4a8337235365f4111a7011a1f028826ca683834ebd12de4b85e2844359c36"},
+ {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:a05b5604c5a75df14a63eeeca598d11b2c3745b9008539b70826ea044063a572"},
+ {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:67c4a640952371c9ca65b6a710598be246ef3be5ca83ed38c16a7660d3980877"},
+ {file = "multidict-6.5.0-cp311-cp311-win32.whl", hash = "sha256:fdeae096ca36c12d8aca2640b8407a9d94e961372c68435bef14e31cce726138"},
+ {file = "multidict-6.5.0-cp311-cp311-win_amd64.whl", hash = "sha256:e2977ef8b7ce27723ee8c610d1bd1765da4f3fbe5a64f9bf1fd3b4770e31fbc0"},
+ {file = "multidict-6.5.0-cp311-cp311-win_arm64.whl", hash = "sha256:82d0cf0ea49bae43d9e8c3851e21954eff716259ff42da401b668744d1760bcb"},
+ {file = "multidict-6.5.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:1bb986c8ea9d49947bc325c51eced1ada6d8d9b4c5b15fd3fcdc3c93edef5a74"},
+ {file = "multidict-6.5.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:03c0923da300120830fc467e23805d63bbb4e98b94032bd863bc7797ea5fa653"},
+ {file = "multidict-6.5.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:4c78d5ec00fdd35c91680ab5cf58368faad4bd1a8721f87127326270248de9bc"},
+ {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aadc3cb78be90a887f8f6b73945b840da44b4a483d1c9750459ae69687940c97"},
+ {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:5b02e1ca495d71e07e652e4cef91adae3bf7ae4493507a263f56e617de65dafc"},
+ {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7fe92a62326eef351668eec4e2dfc494927764a0840a1895cff16707fceffcd3"},
+ {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7673ee4f63879ecd526488deb1989041abcb101b2d30a9165e1e90c489f3f7fb"},
+ {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa097ae2a29f573de7e2d86620cbdda5676d27772d4ed2669cfa9961a0d73955"},
+ {file = "multidict-6.5.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:300da0fa4f8457d9c4bd579695496116563409e676ac79b5e4dca18e49d1c308"},
+ {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9a19bd108c35877b57393243d392d024cfbfdefe759fd137abb98f6fc910b64c"},
+ {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:0f32a1777465a35c35ddbbd7fc1293077938a69402fcc59e40b2846d04a120dd"},
+ {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:9cc1e10c14ce8112d1e6d8971fe3cdbe13e314f68bea0e727429249d4a6ce164"},
+ {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:e95c5e07a06594bdc288117ca90e89156aee8cb2d7c330b920d9c3dd19c05414"},
+ {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:40ff26f58323795f5cd2855e2718a1720a1123fb90df4553426f0efd76135462"},
+ {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:76803a29fd71869a8b59c2118c9dcfb3b8f9c8723e2cce6baeb20705459505cf"},
+ {file = "multidict-6.5.0-cp312-cp312-win32.whl", hash = "sha256:df7ecbc65a53a2ce1b3a0c82e6ad1a43dcfe7c6137733f9176a92516b9f5b851"},
+ {file = "multidict-6.5.0-cp312-cp312-win_amd64.whl", hash = "sha256:0ec1c3fbbb0b655a6540bce408f48b9a7474fd94ed657dcd2e890671fefa7743"},
+ {file = "multidict-6.5.0-cp312-cp312-win_arm64.whl", hash = "sha256:2d24a00d34808b22c1f15902899b9d82d0faeca9f56281641c791d8605eacd35"},
+ {file = "multidict-6.5.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:53d92df1752df67a928fa7f884aa51edae6f1cf00eeb38cbcf318cf841c17456"},
+ {file = "multidict-6.5.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:680210de2c38eef17ce46b8df8bf2c1ece489261a14a6e43c997d49843a27c99"},
+ {file = "multidict-6.5.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:e279259bcb936732bfa1a8eec82b5d2352b3df69d2fa90d25808cfc403cee90a"},
+ {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d1c185fc1069781e3fc8b622c4331fb3b433979850392daa5efbb97f7f9959bb"},
+ {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:6bb5f65ff91daf19ce97f48f63585e51595539a8a523258b34f7cef2ec7e0617"},
+ {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d8646b4259450c59b9286db280dd57745897897284f6308edbdf437166d93855"},
+ {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d245973d4ecc04eea0a8e5ebec7882cf515480036e1b48e65dffcfbdf86d00be"},
+ {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a133e7ddc9bc7fb053733d0ff697ce78c7bf39b5aec4ac12857b6116324c8d75"},
+ {file = "multidict-6.5.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:80d696fa38d738fcebfd53eec4d2e3aeb86a67679fd5e53c325756682f152826"},
+ {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:20d30c9410ac3908abbaa52ee5967a754c62142043cf2ba091e39681bd51d21a"},
+ {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:6c65068cc026f217e815fa519d8e959a7188e94ec163ffa029c94ca3ef9d4a73"},
+ {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:e355ac668a8c3e49c2ca8daa4c92f0ad5b705d26da3d5af6f7d971e46c096da7"},
+ {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:08db204213d0375a91a381cae0677ab95dd8c67a465eb370549daf6dbbf8ba10"},
+ {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:ffa58e3e215af8f6536dc837a990e456129857bb6fd546b3991be470abd9597a"},
+ {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:3e86eb90015c6f21658dbd257bb8e6aa18bdb365b92dd1fba27ec04e58cdc31b"},
+ {file = "multidict-6.5.0-cp313-cp313-win32.whl", hash = "sha256:f34a90fbd9959d0f857323bd3c52b3e6011ed48f78d7d7b9e04980b8a41da3af"},
+ {file = "multidict-6.5.0-cp313-cp313-win_amd64.whl", hash = "sha256:fcb2aa79ac6aef8d5b709bbfc2fdb1d75210ba43038d70fbb595b35af470ce06"},
+ {file = "multidict-6.5.0-cp313-cp313-win_arm64.whl", hash = "sha256:6dcee5e7e92060b4bb9bb6f01efcbb78c13d0e17d9bc6eec71660dd71dc7b0c2"},
+ {file = "multidict-6.5.0-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:cbbc88abea2388fde41dd574159dec2cda005cb61aa84950828610cb5010f21a"},
+ {file = "multidict-6.5.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:70b599f70ae6536e5976364d3c3cf36f40334708bd6cebdd1e2438395d5e7676"},
+ {file = "multidict-6.5.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:828bab777aa8d29d59700018178061854e3a47727e0611cb9bec579d3882de3b"},
+ {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a9695fc1462f17b131c111cf0856a22ff154b0480f86f539d24b2778571ff94d"},
+ {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:0b5ac6ebaf5d9814b15f399337ebc6d3a7f4ce9331edd404e76c49a01620b68d"},
+ {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:84a51e3baa77ded07be4766a9e41d977987b97e49884d4c94f6d30ab6acaee14"},
+ {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8de67f79314d24179e9b1869ed15e88d6ba5452a73fc9891ac142e0ee018b5d6"},
+ {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:17f78a52c214481d30550ec18208e287dfc4736f0c0148208334b105fd9e0887"},
+ {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2966d0099cb2e2039f9b0e73e7fd5eb9c85805681aa2a7f867f9d95b35356921"},
+ {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:86fb42ed5ed1971c642cc52acc82491af97567534a8e381a8d50c02169c4e684"},
+ {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:4e990cbcb6382f9eae4ec720bcac6a1351509e6fc4a5bb70e4984b27973934e6"},
+ {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:d99a59d64bb1f7f2117bec837d9e534c5aeb5dcedf4c2b16b9753ed28fdc20a3"},
+ {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:e8ef15cc97c9890212e1caf90f0d63f6560e1e101cf83aeaf63a57556689fb34"},
+ {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:b8a09aec921b34bd8b9f842f0bcfd76c6a8c033dc5773511e15f2d517e7e1068"},
+ {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:ff07b504c23b67f2044533244c230808a1258b3493aaf3ea2a0785f70b7be461"},
+ {file = "multidict-6.5.0-cp313-cp313t-win32.whl", hash = "sha256:9232a117341e7e979d210e41c04e18f1dc3a1d251268df6c818f5334301274e1"},
+ {file = "multidict-6.5.0-cp313-cp313t-win_amd64.whl", hash = "sha256:44cb5c53fb2d4cbcee70a768d796052b75d89b827643788a75ea68189f0980a1"},
+ {file = "multidict-6.5.0-cp313-cp313t-win_arm64.whl", hash = "sha256:51d33fafa82640c0217391d4ce895d32b7e84a832b8aee0dcc1b04d8981ec7f4"},
+ {file = "multidict-6.5.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:c0078358470da8dc90c37456f4a9cde9f86200949a048d53682b9cd21e5bbf2b"},
+ {file = "multidict-6.5.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:5cc7968b7d1bf8b973c307d38aa3a2f2c783f149bcac855944804252f1df5105"},
+ {file = "multidict-6.5.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0ad73a60e11aa92f1f2c9330efdeaac4531b719fc568eb8d312fd4112f34cc18"},
+ {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3233f21abdcd180b2624eb6988a1e1287210e99bca986d8320afca5005d85844"},
+ {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:bee5c0b79fca78fd2ab644ca4dc831ecf793eb6830b9f542ee5ed2c91bc35a0e"},
+ {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e053a4d690f4352ce46583080fefade9a903ce0fa9d820db1be80bdb9304fa2f"},
+ {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:42bdee30424c1f4dcda96e07ac60e2a4ede8a89f8ae2f48b5e4ccc060f294c52"},
+ {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:58b2ded1a7982cf7b8322b0645713a0086b2b3cf5bb9f7c01edfc1a9f98d20dc"},
+ {file = "multidict-6.5.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3f805b8b951d1fadc5bc18c3c93e509608ac5a883045ee33bc22e28806847c20"},
+ {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:2540395b63723da748f850568357a39cd8d8d4403ca9439f9fcdad6dd423c780"},
+ {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:c96aedff25f4e47b6697ba048b2c278f7caa6df82c7c3f02e077bcc8d47b4b76"},
+ {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:e80de5ad995de210fd02a65c2350649b8321d09bd2e44717eaefb0f5814503e8"},
+ {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:6cb9bcedd9391b313e5ec2fb3aa07c03e050550e7b9e4646c076d5c24ba01532"},
+ {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:a7d130ed7a112e25ab47309962ecafae07d073316f9d158bc7b3936b52b80121"},
+ {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:95750a9a9741cd1855d1b6cb4c6031ae01c01ad38d280217b64bfae986d39d56"},
+ {file = "multidict-6.5.0-cp39-cp39-win32.whl", hash = "sha256:7f78caf409914f108f4212b53a9033abfdc2cbab0647e9ac3a25bb0f21ab43d2"},
+ {file = "multidict-6.5.0-cp39-cp39-win_amd64.whl", hash = "sha256:220c74009507e847a3a6fc5375875f2a2e05bd9ce28cf607be0e8c94600f4472"},
+ {file = "multidict-6.5.0-cp39-cp39-win_arm64.whl", hash = "sha256:d98f4ac9c1ede7e9d04076e2e6d967e15df0079a6381b297270f6bcab661195e"},
+ {file = "multidict-6.5.0-py3-none-any.whl", hash = "sha256:5634b35f225977605385f56153bd95a7133faffc0ffe12ad26e10517537e8dfc"},
+ {file = "multidict-6.5.0.tar.gz", hash = "sha256:942bd8002492ba819426a8d7aefde3189c1b87099cdf18aaaefefcf7f3f7b6d2"},
]
[package.dependencies]
@@ -3269,7 +3353,7 @@ description = "Fundamental package for array computing in Python"
optional = true
python-versions = ">=3.10"
groups = ["main"]
-markers = "(extra == \"pandas\" or extra == \"ray\") and python_version == \"3.10\""
+markers = "python_version == \"3.10\" and (extra == \"pandas\" or extra == \"ray\")"
files = [
{file = "numpy-2.2.6-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b412caa66f72040e6d268491a59f2c43bf03eb6c96dd8f0307829feb7fa2b6fb"},
{file = "numpy-2.2.6-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8e41fd67c52b86603a91c1a505ebaef50b3314de0213461c7a6e99c9a3beff90"},
@@ -3335,7 +3419,7 @@ description = "Fundamental package for array computing in Python"
optional = true
python-versions = ">=3.11"
groups = ["main"]
-markers = "python_version >= \"3.11\" and (extra == \"pandas\" or extra == \"ray\")"
+markers = "(extra == \"pandas\" or extra == \"ray\") and python_version >= \"3.11\""
files = [
{file = "numpy-2.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c3c9fdde0fa18afa1099d6257eb82890ea4f3102847e692193b54e00312a9ae9"},
{file = "numpy-2.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:46d16f72c2192da7b83984aa5455baee640e33a9f1e61e656f29adf55e406c2b"},
@@ -3392,15 +3476,15 @@ files = [
[[package]]
name = "oauthlib"
-version = "3.2.2"
+version = "3.3.1"
description = "A generic, spec-compliant, thorough implementation of the OAuth request-signing logic"
optional = true
-python-versions = ">=3.6"
+python-versions = ">=3.8"
groups = ["main"]
markers = "extra == \"gcsfs\""
files = [
- {file = "oauthlib-3.2.2-py3-none-any.whl", hash = "sha256:8139f29aac13e25d502680e9e19963e83f16838d48a0d71c287fe40e7067fbca"},
- {file = "oauthlib-3.2.2.tar.gz", hash = "sha256:9859c40929662bec5d64f34d01c99e093149682a3f38915dc0655d5a633dd918"},
+ {file = "oauthlib-3.3.1-py3-none-any.whl", hash = "sha256:88119c938d2b8fb88561af5f6ee0eec8cc8d552b7bb1f712743136eb7523b7a1"},
+ {file = "oauthlib-3.3.1.tar.gz", hash = "sha256:0f0f8aa759826a193cf66c12ea1af1637f87b9b4622d46e866952bb022e538c9"},
]
[package.extras]
@@ -3454,7 +3538,7 @@ files = [
{file = "packaging-25.0-py3-none-any.whl", hash = "sha256:29572ef2b1f17581046b3a2227d5c611fb25ec70ca1ba8554b24b0e69331a484"},
{file = "packaging-25.0.tar.gz", hash = "sha256:d443872c98d677bf60f6a1f2f8c1cb748e8fe762d2bf9d3148b5599295b0fc4f"},
]
-markers = {main = "extra == \"ray\" or extra == \"hf\""}
+markers = {main = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"hf\") or extra == \"hf\" or extra == \"ray\""}
[[package]]
name = "paginate"
@@ -3485,51 +3569,35 @@ files = [
{file = "pandas-2.3.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a6872d695c896f00df46b71648eea332279ef4077a409e2fe94220208b6bb675"},
{file = "pandas-2.3.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f4dd97c19bd06bc557ad787a15b6489d2614ddaab5d104a0310eb314c724b2d2"},
{file = "pandas-2.3.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:034abd6f3db8b9880aaee98f4f5d4dbec7c4829938463ec046517220b2f8574e"},
- {file = "pandas-2.3.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:23c2b2dc5213810208ca0b80b8666670eb4660bbfd9d45f58592cc4ddcfd62e1"},
{file = "pandas-2.3.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:39ff73ec07be5e90330cc6ff5705c651ace83374189dcdcb46e6ff54b4a72cd6"},
{file = "pandas-2.3.0-cp310-cp310-win_amd64.whl", hash = "sha256:40cecc4ea5abd2921682b57532baea5588cc5f80f0231c624056b146887274d2"},
{file = "pandas-2.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:8adff9f138fc614347ff33812046787f7d43b3cef7c0f0171b3340cae333f6ca"},
- {file = "pandas-2.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e5f08eb9a445d07720776df6e641975665c9ea12c9d8a331e0f6890f2dcd76ef"},
{file = "pandas-2.3.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fa35c266c8cd1a67d75971a1912b185b492d257092bdd2709bbdebe574ed228d"},
{file = "pandas-2.3.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:14a0cc77b0f089d2d2ffe3007db58f170dae9b9f54e569b299db871a3ab5bf46"},
- {file = "pandas-2.3.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:c06f6f144ad0a1bf84699aeea7eff6068ca5c63ceb404798198af7eb86082e33"},
{file = "pandas-2.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ed16339bc354a73e0a609df36d256672c7d296f3f767ac07257801aa064ff73c"},
{file = "pandas-2.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:fa07e138b3f6c04addfeaf56cc7fdb96c3b68a3fe5e5401251f231fce40a0d7a"},
{file = "pandas-2.3.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:2eb4728a18dcd2908c7fccf74a982e241b467d178724545a48d0caf534b38ebf"},
- {file = "pandas-2.3.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b9d8c3187be7479ea5c3d30c32a5d73d62a621166675063b2edd21bc47614027"},
- {file = "pandas-2.3.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9ff730713d4c4f2f1c860e36c005c7cefc1c7c80c21c0688fd605aa43c9fcf09"},
{file = "pandas-2.3.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba24af48643b12ffe49b27065d3babd52702d95ab70f50e1b34f71ca703e2c0d"},
- {file = "pandas-2.3.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:404d681c698e3c8a40a61d0cd9412cc7364ab9a9cc6e144ae2992e11a2e77a20"},
{file = "pandas-2.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:6021910b086b3ca756755e86ddc64e0ddafd5e58e076c72cb1585162e5ad259b"},
{file = "pandas-2.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:094e271a15b579650ebf4c5155c05dcd2a14fd4fdd72cf4854b2f7ad31ea30be"},
{file = "pandas-2.3.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:2c7e2fc25f89a49a11599ec1e76821322439d90820108309bf42130d2f36c983"},
- {file = "pandas-2.3.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:c6da97aeb6a6d233fb6b17986234cc723b396b50a3c6804776351994f2a658fd"},
{file = "pandas-2.3.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb32dc743b52467d488e7a7c8039b821da2826a9ba4f85b89ea95274f863280f"},
{file = "pandas-2.3.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:213cd63c43263dbb522c1f8a7c9d072e25900f6975596f883f4bebd77295d4f3"},
- {file = "pandas-2.3.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:1d2b33e68d0ce64e26a4acc2e72d747292084f4e8db4c847c6f5f6cbe56ed6d8"},
{file = "pandas-2.3.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:430a63bae10b5086995db1b02694996336e5a8ac9a96b4200572b413dfdfccb9"},
{file = "pandas-2.3.0-cp313-cp313-win_amd64.whl", hash = "sha256:4930255e28ff5545e2ca404637bcc56f031893142773b3468dc021c6c32a1390"},
{file = "pandas-2.3.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:f925f1ef673b4bd0271b1809b72b3270384f2b7d9d14a189b12b7fc02574d575"},
{file = "pandas-2.3.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e78ad363ddb873a631e92a3c063ade1ecfb34cae71e9a2be6ad100f875ac1042"},
{file = "pandas-2.3.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:951805d146922aed8357e4cc5671b8b0b9be1027f0619cea132a9f3f65f2f09c"},
{file = "pandas-2.3.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a881bc1309f3fce34696d07b00f13335c41f5f5a8770a33b09ebe23261cfc67"},
- {file = "pandas-2.3.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:e1991bbb96f4050b09b5f811253c4f3cf05ee89a589379aa36cd623f21a31d6f"},
{file = "pandas-2.3.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:bb3be958022198531eb7ec2008cfc78c5b1eed51af8600c6c5d9160d89d8d249"},
- {file = "pandas-2.3.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9efc0acbbffb5236fbdf0409c04edce96bec4bdaa649d49985427bd1ec73e085"},
- {file = "pandas-2.3.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:75651c14fde635e680496148a8526b328e09fe0572d9ae9b638648c46a544ba3"},
- {file = "pandas-2.3.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bf5be867a0541a9fb47a4be0c5790a4bccd5b77b92f0a59eeec9375fafc2aa14"},
- {file = "pandas-2.3.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84141f722d45d0c2a89544dd29d35b3abfc13d2250ed7e68394eda7564bd6324"},
- {file = "pandas-2.3.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:f95a2aef32614ed86216d3c450ab12a4e82084e8102e355707a1d96e33d51c34"},
- {file = "pandas-2.3.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:e0f51973ba93a9f97185049326d75b942b9aeb472bec616a129806facb129ebb"},
- {file = "pandas-2.3.0-cp39-cp39-win_amd64.whl", hash = "sha256:b198687ca9c8529662213538a9bb1e60fa0bf0f6af89292eb68fea28743fcd5a"},
{file = "pandas-2.3.0.tar.gz", hash = "sha256:34600ab34ebf1131a7613a260a61dbe8b62c188ec0ea4c296da7c9a06b004133"},
]
[package.dependencies]
numpy = [
{version = ">=1.22.4", markers = "python_version < \"3.11\""},
- {version = ">=1.23.2", markers = "python_version == \"3.11\""},
{version = ">=1.26.0", markers = "python_version >= \"3.12\""},
+ {version = ">=1.23.2", markers = "python_version == \"3.11\""},
]
python-dateutil = ">=2.8.2"
pytz = ">=2020.1"
@@ -3631,20 +3699,20 @@ files = [
[[package]]
name = "polars"
-version = "1.30.0"
+version = "1.31.0"
description = "Blazingly fast DataFrame library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
markers = "extra == \"polars\""
files = [
- {file = "polars-1.30.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:4c33bc97c29b7112f0e689a2f8a33143973a3ff466c70b25c7fd1880225de6dd"},
- {file = "polars-1.30.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:e3d05914c364b8e39a5b10dcf97e84d76e516b3b1693880bf189a93aab3ca00d"},
- {file = "polars-1.30.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a52af3862082b868c1febeae650af8ae8a2105d2cb28f0449179a7b44f54ccf"},
- {file = "polars-1.30.0-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:ffb3ef133454275d4254442257c5f71dd6e393ce365c97997dadeb6fa9d6d4b5"},
- {file = "polars-1.30.0-cp39-abi3-win_amd64.whl", hash = "sha256:c26b633a9bd530c5fc09d317fca3bb3e16c772bd7df7549a9d8ec1934773cc5d"},
- {file = "polars-1.30.0-cp39-abi3-win_arm64.whl", hash = "sha256:476f1bde65bc7b4d9f80af370645c2981b5798d67c151055e58534e89e96f2a8"},
- {file = "polars-1.30.0.tar.gz", hash = "sha256:dfe94ae84a5efd9ba74e616e3e125b24ca155494a931890a8f17480737c4db45"},
+ {file = "polars-1.31.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:ccc68cd6877deecd46b13cbd2663ca89ab2a2cb1fe49d5cfc66a9cef166566d9"},
+ {file = "polars-1.31.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:a94c5550df397ad3c2d6adc212e59fd93d9b044ec974dd3653e121e6487a7d21"},
+ {file = "polars-1.31.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ada7940ed92bea65d5500ae7ac1f599798149df8faa5a6db150327c9ddbee4f1"},
+ {file = "polars-1.31.0-cp39-abi3-manylinux_2_24_aarch64.whl", hash = "sha256:b324e6e3e8c6cc6593f9d72fe625f06af65e8d9d47c8686583585533a5e731e1"},
+ {file = "polars-1.31.0-cp39-abi3-win_amd64.whl", hash = "sha256:3fd874d3432fc932863e8cceff2cff8a12a51976b053f2eb6326a0672134a632"},
+ {file = "polars-1.31.0-cp39-abi3-win_arm64.whl", hash = "sha256:62ef23bb9d10dca4c2b945979f9a50812ac4ace4ed9e158a6b5d32a7322e6f75"},
+ {file = "polars-1.31.0.tar.gz", hash = "sha256:59a88054a5fc0135386268ceefdbb6a6cc012d21b5b44fed4f1d3faabbdcbf32"},
]
[package.extras]
@@ -3655,7 +3723,7 @@ calamine = ["fastexcel (>=0.9)"]
cloudpickle = ["cloudpickle"]
connectorx = ["connectorx (>=0.3.2)"]
database = ["polars[adbc,connectorx,sqlalchemy]"]
-deltalake = ["deltalake (>=0.19.0)"]
+deltalake = ["deltalake (>=1.0.0)"]
excel = ["polars[calamine,openpyxl,xlsx2csv,xlsxwriter]"]
fsspec = ["fsspec"]
gpu = ["cudf-polars-cu12"]
@@ -3700,7 +3768,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:22d9962a358aedbb7a2e36187ff273adeaab9743373a272976d2e348d08c7770"},
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0d0fda578d1dc3f77b6b5a5dce3b9ad69a8250a891760a548df850a5e8da87f3"},
@@ -3802,6 +3870,25 @@ files = [
{file = "propcache-0.3.2.tar.gz", hash = "sha256:20d7d62e4e7ef05f221e0db2856b979540686342e7dd9973b815599c7057e168"},
]
+[[package]]
+name = "proto-plus"
+version = "1.26.1"
+description = "Beautiful, Pythonic protocol buffers"
+optional = true
+python-versions = ">=3.7"
+groups = ["main"]
+markers = "extra == \"gcsfs\""
+files = [
+ {file = "proto_plus-1.26.1-py3-none-any.whl", hash = "sha256:13285478c2dcf2abb829db158e1047e2f1e8d63a077d94263c2b88b043c75a66"},
+ {file = "proto_plus-1.26.1.tar.gz", hash = "sha256:21a515a4c4c0088a773899e23c7bbade3d18f9c66c73edd4c7ee3816bc96a012"},
+]
+
+[package.dependencies]
+protobuf = ">=3.19.0,<7.0.0"
+
+[package.extras]
+testing = ["google-api-core (>=1.31.5)"]
+
[[package]]
name = "protobuf"
version = "6.31.1"
@@ -3809,7 +3896,7 @@ description = ""
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"ray\" or extra == \"gcsfs\""
+markers = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"gcsfs\") or extra == \"gcsfs\" or extra == \"ray\""
files = [
{file = "protobuf-6.31.1-cp310-abi3-win32.whl", hash = "sha256:7fa17d5a29c2e04b7d90e5e32388b8bfd0e7107cd8e616feef7ed3fa6bdab5c9"},
{file = "protobuf-6.31.1-cp310-abi3-win_amd64.whl", hash = "sha256:426f59d2964864a1a366254fa703b8632dcec0790d8862d30034d8245e1cd447"},
@@ -3878,7 +3965,6 @@ files = [
{file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:bb89f0a835bcfc1d42ccd5f41f04870c1b936d8507c6df12b7737febc40f0909"},
{file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:f0c2d907a1e102526dd2986df638343388b94c33860ff3bbe1384130828714b1"},
{file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:f8157bed2f51db683f31306aa497311b560f2265998122abe1dce6428bd86567"},
- {file = "psycopg2_binary-2.9.10-cp313-cp313-win_amd64.whl", hash = "sha256:27422aa5f11fbcd9b18da48373eb67081243662f9b46e6fd07c3eb46e4535142"},
{file = "psycopg2_binary-2.9.10-cp38-cp38-macosx_12_0_x86_64.whl", hash = "sha256:eb09aa7f9cecb45027683bb55aebaaf45a0df8bf6de68801a6afdc7947bb09d4"},
{file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b73d6d7f0ccdad7bc43e6d34273f70d587ef62f824d7261c4ae9b8b1b6af90e8"},
{file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ce5ab4bf46a211a8e924d307c1b1fcda82368586a19d0a24f8ae166f5c784864"},
@@ -4008,7 +4094,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
+markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -4518,7 +4604,7 @@ description = "World timezone definitions, modern and historical"
optional = true
python-versions = "*"
groups = ["main"]
-markers = "extra == \"gcsfs\" or extra == \"pandas\" or extra == \"ray\""
+markers = "extra == \"pandas\" or extra == \"ray\""
files = [
{file = "pytz-2025.2-py2.py3-none-any.whl", hash = "sha256:5ddf76296dd8c44c26eb8f4b6f35488f3ccbf6fbbd7adee0b7262d43f0ec2f00"},
{file = "pytz-2025.2.tar.gz", hash = "sha256:360b9e3dbb49a209c21ad61809c7fb453643e048b38924c765813546746e81c3"},
@@ -4613,7 +4699,7 @@ files = [
{file = "PyYAML-6.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:39693e1f8320ae4f43943590b49779ffb98acb81f788220ea932a6b6c51004d8"},
{file = "pyyaml-6.0.2.tar.gz", hash = "sha256:d584d9ec91ad65861cc08d42e834324ef890a082e591037abe114850ff7bbc3e"},
]
-markers = {main = "extra == \"ray\" or extra == \"hf\""}
+markers = {main = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"hf\") or extra == \"hf\" or extra == \"ray\""}
[[package]]
name = "pyyaml-env-tag"
@@ -5137,7 +5223,7 @@ files = [
{file = "s3transfer-0.13.0-py3-none-any.whl", hash = "sha256:0148ef34d6dd964d0d8cf4311b2b21c474693e57c2e069ec708ce043d2b527be"},
{file = "s3transfer-0.13.0.tar.gz", hash = "sha256:f5e6db74eb7776a37208001113ea7aa97695368242b364d73e91c981ac522177"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.4,<2.0a.0"
@@ -5151,12 +5237,11 @@ version = "80.9.0"
description = "Easily download, build, install, upgrade, and uninstall Python packages"
optional = false
python-versions = ">=3.9"
-groups = ["main", "dev"]
+groups = ["dev"]
files = [
{file = "setuptools-80.9.0-py3-none-any.whl", hash = "sha256:062d34222ad13e0cc312a4c02d73f059e86a4acbfbdea8f8f76b28c99f306922"},
{file = "setuptools-80.9.0.tar.gz", hash = "sha256:f36b47402ecde768dbfafc46e8e4207b4360c654f1f3bb84475f0a28628fb19c"},
]
-markers = {main = "extra == \"gcsfs\""}
[package.extras]
check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1) ; sys_platform != \"cygwin\"", "ruff (>=0.8.0) ; sys_platform != \"cygwin\""]
@@ -5646,7 +5731,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"daft\" or extra == \"hf\""
+markers = "extra == \"hf\" or extra == \"daft\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -5723,15 +5808,15 @@ socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"]
[[package]]
name = "urllib3"
-version = "2.4.0"
+version = "2.5.0"
description = "HTTP library with thread-safe connection pooling, file post, and more."
optional = false
python-versions = ">=3.9"
groups = ["main", "dev", "docs"]
markers = "python_version >= \"3.10\""
files = [
- {file = "urllib3-2.4.0-py3-none-any.whl", hash = "sha256:4e16665048960a0900c702d4a66415956a584919c03361cac9f1df5c5dd7e813"},
- {file = "urllib3-2.4.0.tar.gz", hash = "sha256:414bc6535b787febd7567804cc015fee39daab8ad86268f1310a9250697de466"},
+ {file = "urllib3-2.5.0-py3-none-any.whl", hash = "sha256:e6b01673c0fa6a13e374b50871808eb3bf7046c4b125b216f6bf1cc604cff0dc"},
+ {file = "urllib3-2.5.0.tar.gz", hash = "sha256:3fc47733c7e419d4bc3f6b3dc2b4f890bb743906a30d56ba4a5bfa4bbff92760"},
]
[package.extras]
@@ -5931,7 +6016,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6032e6da6abd41e4acda34d75a816012717000fa6839f37124a47fcefc49bec4"},
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2c7b34d804b8cf9b214f05015c4fee2ebe7ed05cf581e7192c06555c71f4446a"},
From bb45772da340541d8c3d80485be3002fddec795b Mon Sep 17 00:00:00 2001
From: NikitaMatskevich
Date: Fri, 20 Jun 2025 17:11:08 +0200
Subject: [PATCH 079/112] Support ADLS with Pyarrow file IO (#2111)
# Rationale for this change
Starting from version 20, PyArrow supports ADLS filesystem. This PR adds
Pyarrow Azure support to Pyiceberg.
PyArrow is the [default
IO](https://github.com/apache/iceberg-python/blob/main/pyiceberg/io/__init__.py#L366-L369)
for Pyiceberg catalogs. In Azure environment it handles wider spectrum
of auth strategies then Fsspec, including, for instance, [Managed
Identities](https://learn.microsoft.com/en-us/entra/identity/managed-identities-azure-resources/overview).
Also, prior to this PR
https://github.com/apache/iceberg-python/pull/1663 (that is not merged
yet) there was no support for wasb(s) with Fsspec.
See the corresponding issue for more details:
https://github.com/apache/iceberg-python/issues/2112
# Are these changes tested?
Tests are added under tests/io/test_pyarrow.py.
# Are there any user-facing changes?
There are no API breaking changes. Direct impact of the PR: Pyarrow
FileIO in Pyiceberg supports Azure cloud environment. Examples of impact
for final users:
- Pyiceberg is usable in services with Managed Identities auth strategy.
- Pyiceberg is usable with wasb(s) schemes in Azure.
---------
Co-authored-by: Kevin Liu
Co-authored-by: Kevin Liu
---
mkdocs/docs/configuration.md | 24 +++--
pyiceberg/io/__init__.py | 4 +
pyiceberg/io/pyarrow.py | 47 ++++++++
tests/conftest.py | 79 ++++++++++----
tests/io/test_pyarrow.py | 201 ++++++++++++++++++++++++++++++++++-
5 files changed, 321 insertions(+), 34 deletions(-)
diff --git a/mkdocs/docs/configuration.md b/mkdocs/docs/configuration.md
index 3037e0f5a2..539fa48266 100644
--- a/mkdocs/docs/configuration.md
+++ b/mkdocs/docs/configuration.md
@@ -146,16 +146,20 @@ For the FileIO there are several configuration options available:
-| Key | Example | Description |
-| ---------------------- | ----------------------------------------------------------------------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |
-| adls.connection-string | AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqF...;BlobEndpoint= | A [connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string). This could be used to use FileIO with any adls-compatible object storage service that has a different endpoint (like [azurite](https://github.com/azure/azurite)). |
-| adls.account-name | devstoreaccount1 | The account that you want to connect to |
-| adls.account-key | Eby8vdM02xNOcqF... | The key to authentication against the account. |
-| adls.sas-token | NuHOuuzdQN7VRM%2FOpOeqBlawRCA845IY05h9eu1Yte4%3D | The shared access signature |
-| adls.tenant-id | ad667be4-b811-11ed-afa1-0242ac120002 | The tenant-id |
-| adls.client-id | ad667be4-b811-11ed-afa1-0242ac120002 | The client-id |
-| adls.client-secret | oCA3R6P\*ka#oa1Sms2J74z... | The client-secret |
-| adls.account-host | accountname1.blob.core.windows.net | The storage account host. See [AzureBlobFileSystem](https://github.com/fsspec/adlfs/blob/adb9c53b74a0d420625b86dd00fbe615b43201d2/adlfs/spec.py#L125) for reference |
+| Key | Example | Description |
+|------------------------------|---------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| adls.connection-string | AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqF...;BlobEndpoint= | A [connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string). This could be used to use FileIO with any adls-compatible object storage service that has a different endpoint (like [azurite](https://github.com/azure/azurite)). |
+| adls.account-name | devstoreaccount1 | The account that you want to connect to |
+| adls.account-key | Eby8vdM02xNOcqF... | The key to authentication against the account. |
+| adls.sas-token | NuHOuuzdQN7VRM%2FOpOeqBlawRCA845IY05h9eu1Yte4%3D | The shared access signature |
+| adls.tenant-id | ad667be4-b811-11ed-afa1-0242ac120002 | The tenant-id |
+| adls.client-id | ad667be4-b811-11ed-afa1-0242ac120002 | The client-id |
+| adls.client-secret | oCA3R6P\*ka#oa1Sms2J74z... | The client-secret |
+| adls.account-host | accountname1.blob.core.windows.net | The storage account host. See [AzureBlobFileSystem](https://github.com/fsspec/adlfs/blob/adb9c53b74a0d420625b86dd00fbe615b43201d2/adlfs/spec.py#L125) for reference |
+| adls.blob-storage-authority | .blob.core.windows.net | The hostname[:port] of the Blob Service. Defaults to `.blob.core.windows.net`. Useful for connecting to a local emulator, like [azurite](https://github.com/azure/azurite). See [AzureFileSystem](https://arrow.apache.org/docs/python/filesystems.html#azure-storage-file-system) for reference |
+| adls.dfs-storage-authority | .dfs.core.windows.net | The hostname[:port] of the Data Lake Gen 2 Service. Defaults to `.dfs.core.windows.net`. Useful for connecting to a local emulator, like [azurite](https://github.com/azure/azurite). See [AzureFileSystem](https://arrow.apache.org/docs/python/filesystems.html#azure-storage-file-system) for reference |
+| adls.blob-storage-scheme | https | Either `http` or `https`. Defaults to `https`. Useful for connecting to a local emulator, like [azurite](https://github.com/azure/azurite). See [AzureFileSystem](https://arrow.apache.org/docs/python/filesystems.html#azure-storage-file-system) for reference |
+| adls.dfs-storage-scheme | https | Either `http` or `https`. Defaults to `https`. Useful for connecting to a local emulator, like [azurite](https://github.com/azure/azurite). See [AzureFileSystem](https://arrow.apache.org/docs/python/filesystems.html#azure-storage-file-system) for reference |
diff --git a/pyiceberg/io/__init__.py b/pyiceberg/io/__init__.py
index ba9738070f..2a73aba7f2 100644
--- a/pyiceberg/io/__init__.py
+++ b/pyiceberg/io/__init__.py
@@ -80,6 +80,10 @@
ADLS_CLIENT_ID = "adls.client-id"
ADLS_CLIENT_SECRET = "adls.client-secret"
ADLS_ACCOUNT_HOST = "adls.account-host"
+ADLS_BLOB_STORAGE_AUTHORITY = "adls.blob-storage-authority"
+ADLS_DFS_STORAGE_AUTHORITY = "adls.dfs-storage-authority"
+ADLS_BLOB_STORAGE_SCHEME = "adls.blob-storage-scheme"
+ADLS_DFS_STORAGE_SCHEME = "adls.dfs-storage-scheme"
GCS_TOKEN = "gcs.oauth2.token"
GCS_TOKEN_EXPIRES_AT_MS = "gcs.oauth2.token-expires-at"
GCS_PROJECT_ID = "gcs.project-id"
diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py
index eeacaccfe2..c23be93ead 100644
--- a/pyiceberg/io/pyarrow.py
+++ b/pyiceberg/io/pyarrow.py
@@ -84,6 +84,13 @@
)
from pyiceberg.expressions.visitors import visit as boolean_expression_visit
from pyiceberg.io import (
+ ADLS_ACCOUNT_KEY,
+ ADLS_ACCOUNT_NAME,
+ ADLS_BLOB_STORAGE_AUTHORITY,
+ ADLS_BLOB_STORAGE_SCHEME,
+ ADLS_DFS_STORAGE_AUTHORITY,
+ ADLS_DFS_STORAGE_SCHEME,
+ ADLS_SAS_TOKEN,
AWS_ACCESS_KEY_ID,
AWS_REGION,
AWS_ROLE_ARN,
@@ -390,6 +397,9 @@ def _initialize_fs(self, scheme: str, netloc: Optional[str] = None) -> FileSyste
elif scheme in {"gs", "gcs"}:
return self._initialize_gcs_fs()
+ elif scheme in {"abfs", "abfss", "wasb", "wasbs"}:
+ return self._initialize_azure_fs()
+
elif scheme in {"file"}:
return self._initialize_local_fs()
@@ -471,6 +481,43 @@ def _initialize_s3_fs(self, netloc: Optional[str]) -> FileSystem:
return S3FileSystem(**client_kwargs)
+ def _initialize_azure_fs(self) -> FileSystem:
+ from packaging import version
+
+ MIN_PYARROW_VERSION_SUPPORTING_AZURE_FS = "20.0.0"
+ if version.parse(pyarrow.__version__) < version.parse(MIN_PYARROW_VERSION_SUPPORTING_AZURE_FS):
+ raise ImportError(
+ f"pyarrow version >= {MIN_PYARROW_VERSION_SUPPORTING_AZURE_FS} required for AzureFileSystem support, "
+ f"but found version {pyarrow.__version__}."
+ )
+
+ from pyarrow.fs import AzureFileSystem
+
+ client_kwargs: Dict[str, str] = {}
+
+ if account_name := self.properties.get(ADLS_ACCOUNT_NAME):
+ client_kwargs["account_name"] = account_name
+
+ if account_key := self.properties.get(ADLS_ACCOUNT_KEY):
+ client_kwargs["account_key"] = account_key
+
+ if blob_storage_authority := self.properties.get(ADLS_BLOB_STORAGE_AUTHORITY):
+ client_kwargs["blob_storage_authority"] = blob_storage_authority
+
+ if dfs_storage_authority := self.properties.get(ADLS_DFS_STORAGE_AUTHORITY):
+ client_kwargs["dfs_storage_authority"] = dfs_storage_authority
+
+ if blob_storage_scheme := self.properties.get(ADLS_BLOB_STORAGE_SCHEME):
+ client_kwargs["blob_storage_scheme"] = blob_storage_scheme
+
+ if dfs_storage_scheme := self.properties.get(ADLS_DFS_STORAGE_SCHEME):
+ client_kwargs["dfs_storage_scheme"] = dfs_storage_scheme
+
+ if sas_token := self.properties.get(ADLS_SAS_TOKEN):
+ client_kwargs["sas_token"] = sas_token
+
+ return AzureFileSystem(**client_kwargs)
+
def _initialize_hdfs_fs(self, scheme: str, netloc: Optional[str]) -> FileSystem:
from pyarrow.fs import HadoopFileSystem
diff --git a/tests/conftest.py b/tests/conftest.py
index 729e29cb0c..a584f98c10 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -52,6 +52,12 @@
from pyiceberg.catalog.noop import NoopCatalog
from pyiceberg.expressions import BoundReference
from pyiceberg.io import (
+ ADLS_ACCOUNT_KEY,
+ ADLS_ACCOUNT_NAME,
+ ADLS_BLOB_STORAGE_AUTHORITY,
+ ADLS_BLOB_STORAGE_SCHEME,
+ ADLS_DFS_STORAGE_AUTHORITY,
+ ADLS_DFS_STORAGE_SCHEME,
GCS_PROJECT_ID,
GCS_SERVICE_HOST,
GCS_TOKEN,
@@ -348,6 +354,11 @@ def table_schema_with_all_types() -> Schema:
)
+@pytest.fixture(params=["abfs", "abfss", "wasb", "wasbs"])
+def adls_scheme(request: pytest.FixtureRequest) -> str:
+ return request.param
+
+
@pytest.fixture(scope="session")
def pyarrow_schema_simple_without_ids() -> "pa.Schema":
import pyarrow as pa
@@ -2088,6 +2099,26 @@ def fsspec_fileio_gcs(request: pytest.FixtureRequest) -> FsspecFileIO:
return fsspec.FsspecFileIO(properties=properties)
+@pytest.fixture
+def adls_fsspec_fileio(request: pytest.FixtureRequest) -> Generator[FsspecFileIO, None, None]:
+ from azure.storage.blob import BlobServiceClient
+
+ azurite_url = request.config.getoption("--adls.endpoint")
+ azurite_account_name = request.config.getoption("--adls.account-name")
+ azurite_account_key = request.config.getoption("--adls.account-key")
+ azurite_connection_string = f"DefaultEndpointsProtocol=http;AccountName={azurite_account_name};AccountKey={azurite_account_key};BlobEndpoint={azurite_url}/{azurite_account_name};"
+ properties = {
+ "adls.connection-string": azurite_connection_string,
+ "adls.account-name": azurite_account_name,
+ }
+
+ bbs = BlobServiceClient.from_connection_string(conn_str=azurite_connection_string)
+ bbs.create_container("tests")
+ yield fsspec.FsspecFileIO(properties=properties)
+ bbs.delete_container("tests")
+ bbs.close()
+
+
@pytest.fixture
def pyarrow_fileio_gcs(request: pytest.FixtureRequest) -> "PyArrowFileIO":
from pyiceberg.io.pyarrow import PyArrowFileIO
@@ -2101,6 +2132,34 @@ def pyarrow_fileio_gcs(request: pytest.FixtureRequest) -> "PyArrowFileIO":
return PyArrowFileIO(properties=properties)
+@pytest.fixture
+def pyarrow_fileio_adls(request: pytest.FixtureRequest) -> Generator[Any, None, None]:
+ from azure.storage.blob import BlobServiceClient
+
+ from pyiceberg.io.pyarrow import PyArrowFileIO
+
+ azurite_url = request.config.getoption("--adls.endpoint")
+ azurite_scheme, azurite_authority = azurite_url.split("://", 1)
+
+ azurite_account_name = request.config.getoption("--adls.account-name")
+ azurite_account_key = request.config.getoption("--adls.account-key")
+ azurite_connection_string = f"DefaultEndpointsProtocol=http;AccountName={azurite_account_name};AccountKey={azurite_account_key};BlobEndpoint={azurite_url}/{azurite_account_name};"
+ properties = {
+ ADLS_ACCOUNT_NAME: azurite_account_name,
+ ADLS_ACCOUNT_KEY: azurite_account_key,
+ ADLS_BLOB_STORAGE_AUTHORITY: azurite_authority,
+ ADLS_DFS_STORAGE_AUTHORITY: azurite_authority,
+ ADLS_BLOB_STORAGE_SCHEME: azurite_scheme,
+ ADLS_DFS_STORAGE_SCHEME: azurite_scheme,
+ }
+
+ bbs = BlobServiceClient.from_connection_string(conn_str=azurite_connection_string)
+ bbs.create_container("warehouse")
+ yield PyArrowFileIO(properties=properties)
+ bbs.delete_container("warehouse")
+ bbs.close()
+
+
def aws_credentials() -> None:
os.environ["AWS_ACCESS_KEY_ID"] = "testing"
os.environ["AWS_SECRET_ACCESS_KEY"] = "testing"
@@ -2162,26 +2221,6 @@ def fixture_dynamodb(_aws_credentials: None) -> Generator[boto3.client, None, No
yield boto3.client("dynamodb", region_name="us-east-1")
-@pytest.fixture
-def adls_fsspec_fileio(request: pytest.FixtureRequest) -> Generator[FsspecFileIO, None, None]:
- from azure.storage.blob import BlobServiceClient
-
- azurite_url = request.config.getoption("--adls.endpoint")
- azurite_account_name = request.config.getoption("--adls.account-name")
- azurite_account_key = request.config.getoption("--adls.account-key")
- azurite_connection_string = f"DefaultEndpointsProtocol=http;AccountName={azurite_account_name};AccountKey={azurite_account_key};BlobEndpoint={azurite_url}/{azurite_account_name};"
- properties = {
- "adls.connection-string": azurite_connection_string,
- "adls.account-name": azurite_account_name,
- }
-
- bbs = BlobServiceClient.from_connection_string(conn_str=azurite_connection_string)
- bbs.create_container("tests")
- yield fsspec.FsspecFileIO(properties=properties)
- bbs.delete_container("tests")
- bbs.close()
-
-
@pytest.fixture(scope="session")
def empty_home_dir_path(tmp_path_factory: pytest.TempPathFactory) -> str:
home_path = str(tmp_path_factory.mktemp("home"))
diff --git a/tests/io/test_pyarrow.py b/tests/io/test_pyarrow.py
index 9daefacdca..850b1292a8 100644
--- a/tests/io/test_pyarrow.py
+++ b/tests/io/test_pyarrow.py
@@ -24,9 +24,11 @@
from unittest.mock import MagicMock, patch
from uuid import uuid4
+import pyarrow
import pyarrow as pa
import pyarrow.parquet as pq
import pytest
+from packaging import version
from pyarrow.fs import FileType, LocalFileSystem, S3FileSystem
from pyiceberg.exceptions import ResolveError
@@ -106,6 +108,11 @@
from tests.catalog.test_base import InMemoryCatalog
from tests.conftest import UNIFIED_AWS_SESSION_PROPERTIES
+skip_if_pyarrow_too_old = pytest.mark.skipif(
+ version.parse(pyarrow.__version__) < version.parse("20.0.0"),
+ reason="Requires pyarrow version >= 20.0.0",
+)
+
def test_pyarrow_infer_local_fs_from_path() -> None:
"""Test path with `file` scheme and no scheme both use LocalFileSystem"""
@@ -1672,7 +1679,7 @@ def test_new_output_file_gcs(pyarrow_fileio_gcs: PyArrowFileIO) -> None:
@pytest.mark.gcs
@pytest.mark.skip(reason="Open issue on Arrow: https://github.com/apache/arrow/issues/36993")
def test_write_and_read_file_gcs(pyarrow_fileio_gcs: PyArrowFileIO) -> None:
- """Test writing and reading a file using FsspecInputFile and FsspecOutputFile"""
+ """Test writing and reading a file using PyArrowFile"""
location = f"gs://warehouse/{uuid4()}.txt"
output_file = pyarrow_fileio_gcs.new_output(location=location)
with output_file.create() as f:
@@ -1689,7 +1696,7 @@ def test_write_and_read_file_gcs(pyarrow_fileio_gcs: PyArrowFileIO) -> None:
@pytest.mark.gcs
def test_getting_length_of_file_gcs(pyarrow_fileio_gcs: PyArrowFileIO) -> None:
- """Test getting the length of an FsspecInputFile and FsspecOutputFile"""
+ """Test getting the length of PyArrowFile"""
filename = str(uuid4())
output_file = pyarrow_fileio_gcs.new_output(location=f"gs://warehouse/{filename}")
@@ -1753,7 +1760,7 @@ def test_read_specified_bytes_for_file_gcs(pyarrow_fileio_gcs: PyArrowFileIO) ->
@pytest.mark.gcs
@pytest.mark.skip(reason="Open issue on Arrow: https://github.com/apache/arrow/issues/36993")
def test_raise_on_opening_file_not_found_gcs(pyarrow_fileio_gcs: PyArrowFileIO) -> None:
- """Test that an fsspec input file raises appropriately when the gcs file is not found"""
+ """Test that PyArrowFile raises appropriately when the gcs file is not found"""
filename = str(uuid4())
input_file = pyarrow_fileio_gcs.new_input(location=f"gs://warehouse/{filename}")
@@ -1815,7 +1822,7 @@ def test_converting_an_outputfile_to_an_inputfile_gcs(pyarrow_fileio_gcs: PyArro
@pytest.mark.gcs
@pytest.mark.skip(reason="Open issue on Arrow: https://github.com/apache/arrow/issues/36993")
def test_writing_avro_file_gcs(generated_manifest_entry_file: str, pyarrow_fileio_gcs: PyArrowFileIO) -> None:
- """Test that bytes match when reading a local avro file, writing it using fsspec file-io, and then reading it again"""
+ """Test that bytes match when reading a local avro file, writing it using pyarrow file-io, and then reading it again"""
filename = str(uuid4())
with PyArrowFileIO().new_input(location=generated_manifest_entry_file).open() as f:
b1 = f.read()
@@ -1828,6 +1835,192 @@ def test_writing_avro_file_gcs(generated_manifest_entry_file: str, pyarrow_filei
pyarrow_fileio_gcs.delete(f"gs://warehouse/{filename}")
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_new_input_file_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test creating a new input file from pyarrow file-io"""
+ filename = str(uuid4())
+
+ input_file = pyarrow_fileio_adls.new_input(f"{adls_scheme}://warehouse/{filename}")
+
+ assert isinstance(input_file, PyArrowFile)
+ assert input_file.location == f"{adls_scheme}://warehouse/{filename}"
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_new_output_file_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test creating a new output file from pyarrow file-io"""
+ filename = str(uuid4())
+
+ output_file = pyarrow_fileio_adls.new_output(f"{adls_scheme}://warehouse/{filename}")
+
+ assert isinstance(output_file, PyArrowFile)
+ assert output_file.location == f"{adls_scheme}://warehouse/{filename}"
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_write_and_read_file_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test writing and reading a file using PyArrowFile"""
+ location = f"{adls_scheme}://warehouse/{uuid4()}.txt"
+ output_file = pyarrow_fileio_adls.new_output(location=location)
+ with output_file.create() as f:
+ assert f.write(b"foo") == 3
+
+ assert output_file.exists()
+
+ input_file = pyarrow_fileio_adls.new_input(location=location)
+ with input_file.open() as f:
+ assert f.read() == b"foo"
+
+ pyarrow_fileio_adls.delete(input_file)
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_getting_length_of_file_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test getting the length of PyArrowFile"""
+ filename = str(uuid4())
+
+ output_file = pyarrow_fileio_adls.new_output(location=f"{adls_scheme}://warehouse/{filename}")
+ with output_file.create() as f:
+ f.write(b"foobar")
+
+ assert len(output_file) == 6
+
+ input_file = pyarrow_fileio_adls.new_input(location=f"{adls_scheme}://warehouse/{filename}")
+ assert len(input_file) == 6
+
+ pyarrow_fileio_adls.delete(output_file)
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_file_tell_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ location = f"{adls_scheme}://warehouse/{uuid4()}"
+
+ output_file = pyarrow_fileio_adls.new_output(location=location)
+ with output_file.create() as write_file:
+ write_file.write(b"foobar")
+
+ input_file = pyarrow_fileio_adls.new_input(location=location)
+ with input_file.open() as f:
+ f.seek(0)
+ assert f.tell() == 0
+ f.seek(1)
+ assert f.tell() == 1
+ f.seek(3)
+ assert f.tell() == 3
+ f.seek(0)
+ assert f.tell() == 0
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_read_specified_bytes_for_file_adls(pyarrow_fileio_adls: PyArrowFileIO) -> None:
+ location = f"abfss://warehouse/{uuid4()}"
+
+ output_file = pyarrow_fileio_adls.new_output(location=location)
+ with output_file.create() as write_file:
+ write_file.write(b"foo")
+
+ input_file = pyarrow_fileio_adls.new_input(location=location)
+ with input_file.open() as f:
+ f.seek(0)
+ assert b"f" == f.read(1)
+ f.seek(0)
+ assert b"fo" == f.read(2)
+ f.seek(1)
+ assert b"o" == f.read(1)
+ f.seek(1)
+ assert b"oo" == f.read(2)
+ f.seek(0)
+ assert b"foo" == f.read(999) # test reading amount larger than entire content length
+
+ pyarrow_fileio_adls.delete(input_file)
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_raise_on_opening_file_not_found_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test that PyArrowFile raises appropriately when the adls file is not found"""
+
+ filename = str(uuid4())
+ input_file = pyarrow_fileio_adls.new_input(location=f"{adls_scheme}://warehouse/{filename}")
+ with pytest.raises(FileNotFoundError) as exc_info:
+ input_file.open().read()
+
+ assert filename in str(exc_info.value)
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_checking_if_a_file_exists_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test checking if a file exists"""
+ non_existent_file = pyarrow_fileio_adls.new_input(location=f"{adls_scheme}://warehouse/does-not-exist.txt")
+ assert not non_existent_file.exists()
+
+ location = f"{adls_scheme}://warehouse/{uuid4()}"
+ output_file = pyarrow_fileio_adls.new_output(location=location)
+ assert not output_file.exists()
+ with output_file.create() as f:
+ f.write(b"foo")
+
+ existing_input_file = pyarrow_fileio_adls.new_input(location=location)
+ assert existing_input_file.exists()
+
+ existing_output_file = pyarrow_fileio_adls.new_output(location=location)
+ assert existing_output_file.exists()
+
+ pyarrow_fileio_adls.delete(existing_output_file)
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_closing_a_file_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test closing an output file and input file"""
+ filename = str(uuid4())
+ output_file = pyarrow_fileio_adls.new_output(location=f"{adls_scheme}://warehouse/{filename}")
+ with output_file.create() as write_file:
+ write_file.write(b"foo")
+ assert not write_file.closed # type: ignore
+ assert write_file.closed # type: ignore
+
+ input_file = pyarrow_fileio_adls.new_input(location=f"{adls_scheme}://warehouse/{filename}")
+ with input_file.open() as f:
+ assert not f.closed # type: ignore
+ assert f.closed # type: ignore
+
+ pyarrow_fileio_adls.delete(f"{adls_scheme}://warehouse/{filename}")
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_converting_an_outputfile_to_an_inputfile_adls(pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test converting an output file to an input file"""
+ filename = str(uuid4())
+ output_file = pyarrow_fileio_adls.new_output(location=f"{adls_scheme}://warehouse/{filename}")
+ input_file = output_file.to_input_file()
+ assert input_file.location == output_file.location
+
+
+@pytest.mark.adls
+@skip_if_pyarrow_too_old
+def test_writing_avro_file_adls(generated_manifest_entry_file: str, pyarrow_fileio_adls: PyArrowFileIO, adls_scheme: str) -> None:
+ """Test that bytes match when reading a local avro file, writing it using pyarrow file-io, and then reading it again"""
+ filename = str(uuid4())
+ with PyArrowFileIO().new_input(location=generated_manifest_entry_file).open() as f:
+ b1 = f.read()
+ with pyarrow_fileio_adls.new_output(location=f"{adls_scheme}://warehouse/{filename}").create() as out_f:
+ out_f.write(b1)
+ with pyarrow_fileio_adls.new_input(location=f"{adls_scheme}://warehouse/{filename}").open() as in_f:
+ b2 = in_f.read()
+ assert b1 == b2 # Check that bytes of read from local avro file match bytes written to s3
+
+ pyarrow_fileio_adls.delete(f"{adls_scheme}://warehouse/{filename}")
+
+
def test_parse_location() -> None:
def check_results(location: str, expected_schema: str, expected_netloc: str, expected_uri: str) -> None:
schema, netloc, uri = PyArrowFileIO.parse_location(location)
From 64d6241d192d438c20a72034b6aa7fa5dcc8a0b0 Mon Sep 17 00:00:00 2001
From: koenvo
Date: Fri, 20 Jun 2025 20:35:29 +0200
Subject: [PATCH 080/112] Improve upsert memory pressure (#1995)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
### Summary
This PR updates the upsert logic to use batch processing. The main goal
is to prevent out-of-memory (OOM) issues when updating large tables by
avoiding loading all data at once.
**Note:** This has only been tested against the unit tests—no real-world
datasets have been evaluated yet.
This PR partially depends on functionality introduced in
[#1817](https://github.com/apache/iceberg/pull/1817).
---
### Notes
- Duplicate detection across multiple batches is **not** possible with
this approach.
- ~All data is read sequentially, which may be slower than the parallel
read used by `to_arrow`.~ fixed using `concurrent_tasks` parameter
---
### Performance Comparison
In setups with many small files, network and metadata overhead become
the dominant factor. This impacts batch reading performance, as each
file contributes relatively more overhead than payload. In the test
setup used here, metadata access was the largest cost.
#### Using `to_arrow_batch_reader` (sequential):
- **Scan:** 9993.50 ms
- **To list:** 19811.09 ms
#### Using `to_arrow` (parallel):
- **Scan:** 10607.88 ms
---------
Co-authored-by: Fokko Driesprong
---
pyiceberg/io/pyarrow.py | 84 +++++++++++++++++--------------------
pyiceberg/table/__init__.py | 58 +++++++++++++++----------
2 files changed, 75 insertions(+), 67 deletions(-)
diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py
index c23be93ead..cc81e37119 100644
--- a/pyiceberg/io/pyarrow.py
+++ b/pyiceberg/io/pyarrow.py
@@ -25,7 +25,6 @@
from __future__ import annotations
-import concurrent.futures
import fnmatch
import functools
import itertools
@@ -36,7 +35,6 @@
import uuid
import warnings
from abc import ABC, abstractmethod
-from concurrent.futures import Future
from copy import copy
from dataclasses import dataclass
from enum import Enum
@@ -70,7 +68,6 @@
FileSystem,
FileType,
)
-from sortedcontainers import SortedList
from pyiceberg.conversions import to_bytes
from pyiceberg.exceptions import ResolveError
@@ -1586,47 +1583,20 @@ def to_table(self, tasks: Iterable[FileScanTask]) -> pa.Table:
ResolveError: When a required field cannot be found in the file
ValueError: When a field type in the file cannot be projected to the schema type
"""
- deletes_per_file = _read_all_delete_files(self._io, tasks)
- executor = ExecutorFactory.get_or_create()
-
- def _table_from_scan_task(task: FileScanTask) -> pa.Table:
- batches = list(self._record_batches_from_scan_tasks_and_deletes([task], deletes_per_file))
- if len(batches) > 0:
- return pa.Table.from_batches(batches)
- else:
- return None
-
- futures = [
- executor.submit(
- _table_from_scan_task,
- task,
- )
- for task in tasks
- ]
- total_row_count = 0
- # for consistent ordering, we need to maintain future order
- futures_index = {f: i for i, f in enumerate(futures)}
- completed_futures: SortedList[Future[pa.Table]] = SortedList(iterable=[], key=lambda f: futures_index[f])
- for future in concurrent.futures.as_completed(futures):
- completed_futures.add(future)
- if table_result := future.result():
- total_row_count += len(table_result)
- # stop early if limit is satisfied
- if self._limit is not None and total_row_count >= self._limit:
- break
-
- # by now, we've either completed all tasks or satisfied the limit
- if self._limit is not None:
- _ = [f.cancel() for f in futures if not f.done()]
-
- tables = [f.result() for f in completed_futures if f.result()]
-
arrow_schema = schema_to_pyarrow(self._projected_schema, include_field_ids=False)
- if len(tables) < 1:
- return pa.Table.from_batches([], schema=arrow_schema)
-
- result = pa.concat_tables(tables, promote_options="permissive")
+ batches = self.to_record_batches(tasks)
+ try:
+ first_batch = next(batches)
+ except StopIteration:
+ # Empty
+ return arrow_schema.empty_table()
+
+ # Note: cannot use pa.Table.from_batches(itertools.chain([first_batch], batches)))
+ # as different batches can use different schema's (due to large_ types)
+ result = pa.concat_tables(
+ (pa.Table.from_batches([batch]) for batch in itertools.chain([first_batch], batches)), promote_options="permissive"
+ )
if property_as_bool(self._io.properties, PYARROW_USE_LARGE_TYPES_ON_READ, False):
deprecation_message(
@@ -1636,9 +1606,6 @@ def _table_from_scan_task(task: FileScanTask) -> pa.Table:
)
result = result.cast(arrow_schema)
- if self._limit is not None:
- return result.slice(0, self._limit)
-
return result
def to_record_batches(self, tasks: Iterable[FileScanTask]) -> Iterator[pa.RecordBatch]:
@@ -1660,7 +1627,32 @@ def to_record_batches(self, tasks: Iterable[FileScanTask]) -> Iterator[pa.Record
ValueError: When a field type in the file cannot be projected to the schema type
"""
deletes_per_file = _read_all_delete_files(self._io, tasks)
- return self._record_batches_from_scan_tasks_and_deletes(tasks, deletes_per_file)
+
+ total_row_count = 0
+ executor = ExecutorFactory.get_or_create()
+
+ def batches_for_task(task: FileScanTask) -> List[pa.RecordBatch]:
+ # Materialize the iterator here to ensure execution happens within the executor.
+ # Otherwise, the iterator would be lazily consumed later (in the main thread),
+ # defeating the purpose of using executor.map.
+ return list(self._record_batches_from_scan_tasks_and_deletes([task], deletes_per_file))
+
+ limit_reached = False
+ for batches in executor.map(batches_for_task, tasks):
+ for batch in batches:
+ current_batch_size = len(batch)
+ if self._limit is not None and total_row_count + current_batch_size >= self._limit:
+ yield batch.slice(0, self._limit - total_row_count)
+
+ limit_reached = True
+ break
+ else:
+ yield batch
+ total_row_count += current_batch_size
+
+ if limit_reached:
+ # This break will also cancel all running tasks in the executor
+ break
def _record_batches_from_scan_tasks_and_deletes(
self, tasks: Iterable[FileScanTask], deletes_per_file: Dict[str, List[ChunkedArray]]
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 294a6bac34..f6d28a5ecf 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -781,39 +781,55 @@ def upsert(
matched_predicate = upsert_util.create_match_filter(df, join_cols)
# We must use Transaction.table_metadata for the scan. This includes all uncommitted - but relevant - changes.
- matched_iceberg_table = DataScan(
+ matched_iceberg_record_batches = DataScan(
table_metadata=self.table_metadata,
io=self._table.io,
row_filter=matched_predicate,
case_sensitive=case_sensitive,
- ).to_arrow()
+ ).to_arrow_batch_reader()
- update_row_cnt = 0
- insert_row_cnt = 0
+ batches_to_overwrite = []
+ overwrite_predicates = []
+ rows_to_insert = df
- if when_matched_update_all:
- # function get_rows_to_update is doing a check on non-key columns to see if any of the values have actually changed
- # we don't want to do just a blanket overwrite for matched rows if the actual non-key column data hasn't changed
- # this extra step avoids unnecessary IO and writes
- rows_to_update = upsert_util.get_rows_to_update(df, matched_iceberg_table, join_cols)
+ for batch in matched_iceberg_record_batches:
+ rows = pa.Table.from_batches([batch])
- update_row_cnt = len(rows_to_update)
+ if when_matched_update_all:
+ # function get_rows_to_update is doing a check on non-key columns to see if any of the values have actually changed
+ # we don't want to do just a blanket overwrite for matched rows if the actual non-key column data hasn't changed
+ # this extra step avoids unnecessary IO and writes
+ rows_to_update = upsert_util.get_rows_to_update(df, rows, join_cols)
- if len(rows_to_update) > 0:
- # build the match predicate filter
- overwrite_mask_predicate = upsert_util.create_match_filter(rows_to_update, join_cols)
+ if len(rows_to_update) > 0:
+ # build the match predicate filter
+ overwrite_mask_predicate = upsert_util.create_match_filter(rows_to_update, join_cols)
- self.overwrite(rows_to_update, overwrite_filter=overwrite_mask_predicate)
+ batches_to_overwrite.append(rows_to_update)
+ overwrite_predicates.append(overwrite_mask_predicate)
- if when_not_matched_insert_all:
- expr_match = upsert_util.create_match_filter(matched_iceberg_table, join_cols)
- expr_match_bound = bind(self.table_metadata.schema(), expr_match, case_sensitive=case_sensitive)
- expr_match_arrow = expression_to_pyarrow(expr_match_bound)
- rows_to_insert = df.filter(~expr_match_arrow)
+ if when_not_matched_insert_all:
+ expr_match = upsert_util.create_match_filter(rows, join_cols)
+ expr_match_bound = bind(self.table_metadata.schema(), expr_match, case_sensitive=case_sensitive)
+ expr_match_arrow = expression_to_pyarrow(expr_match_bound)
- insert_row_cnt = len(rows_to_insert)
+ # Filter rows per batch.
+ rows_to_insert = rows_to_insert.filter(~expr_match_arrow)
- if insert_row_cnt > 0:
+ update_row_cnt = 0
+ insert_row_cnt = 0
+
+ if batches_to_overwrite:
+ rows_to_update = pa.concat_tables(batches_to_overwrite)
+ update_row_cnt = len(rows_to_update)
+ self.overwrite(
+ rows_to_update,
+ overwrite_filter=Or(*overwrite_predicates) if len(overwrite_predicates) > 1 else overwrite_predicates[0],
+ )
+
+ if when_not_matched_insert_all:
+ insert_row_cnt = len(rows_to_insert)
+ if rows_to_insert:
self.append(rows_to_insert)
return UpsertResult(rows_updated=update_row_cnt, rows_inserted=insert_row_cnt)
From 555898a0d33f5d76012957bf81f42e2fda78e074 Mon Sep 17 00:00:00 2001
From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com>
Date: Sat, 21 Jun 2025 02:53:10 +0800
Subject: [PATCH 081/112] Type mismatch Avro zstd decompression (#2128)
# Rationale for this change
The return type of the decompress method in `ZStandardCodec` should be
`bytes`, but it currently returns a `bytearray`, which causes an
exception when reading Avro files compressed with zstd.
```text
def new_decoder(b: bytes) -> BinaryDecoder:
try:
from pyiceberg.avro.decoder_fast import CythonBinaryDecoder
> return CythonBinaryDecoder(b)
E TypeError: Argument 'input_contents' has incorrect type (expected bytes, got bytearray)
```
# Are these changes tested?
Yes, `test_write_manifest`
# Are there any user-facing changes?
No.
---
pyiceberg/avro/codecs/zstandard_codec.py | 2 +-
tests/utils/test_manifest.py | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/pyiceberg/avro/codecs/zstandard_codec.py b/pyiceberg/avro/codecs/zstandard_codec.py
index a048f68490..4cc815214f 100644
--- a/pyiceberg/avro/codecs/zstandard_codec.py
+++ b/pyiceberg/avro/codecs/zstandard_codec.py
@@ -39,7 +39,7 @@ def decompress(data: bytes) -> bytes:
if not chunk:
break
uncompressed.extend(chunk)
- return uncompressed
+ return bytes(uncompressed)
except ImportError:
diff --git a/tests/utils/test_manifest.py b/tests/utils/test_manifest.py
index c838c02ac6..825431c776 100644
--- a/tests/utils/test_manifest.py
+++ b/tests/utils/test_manifest.py
@@ -358,7 +358,7 @@ def test_write_empty_manifest() -> None:
@pytest.mark.parametrize("format_version", [1, 2])
-@pytest.mark.parametrize("compression", ["null", "deflate"])
+@pytest.mark.parametrize("compression", ["null", "deflate", "zstd"])
def test_write_manifest(
generated_manifest_file_file_v1: str,
generated_manifest_file_file_v2: str,
From 907876a10929272e60e36a791eae7a17bf2431d5 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Fri, 20 Jun 2025 22:38:11 +0200
Subject: [PATCH 082/112] Expose S3 retry strategy (#2110)
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
mkdocs/docs/configuration.md | 33 +++++++++++++++++----------------
pyiceberg/io/__init__.py | 1 +
pyiceberg/io/pyarrow.py | 22 ++++++++++++++++++++++
tests/io/test_pyarrow.py | 23 +++++++++++++++++++++--
4 files changed, 61 insertions(+), 18 deletions(-)
diff --git a/mkdocs/docs/configuration.md b/mkdocs/docs/configuration.md
index 539fa48266..8503a846d6 100644
--- a/mkdocs/docs/configuration.md
+++ b/mkdocs/docs/configuration.md
@@ -109,23 +109,24 @@ For the FileIO there are several configuration options available:
-| Key | Example | Description |
-|-----------------------------|----------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| s3.endpoint | | Configure an alternative endpoint of the S3 service for the FileIO to access. This could be used to use S3FileIO with any s3-compatible object storage service that has a different endpoint, or access a private S3 endpoint in a virtual private cloud. |
-| s3.access-key-id | admin | Configure the static access key id used to access the FileIO. |
-| s3.secret-access-key | password | Configure the static secret access key used to access the FileIO. |
-| s3.session-token | AQoDYXdzEJr... | Configure the static session token used to access the FileIO. |
-| s3.role-session-name | session | An optional identifier for the assumed role session. |
-| s3.role-arn | arn:aws:... | AWS Role ARN. If provided instead of access_key and secret_key, temporary credentials will be fetched by assuming this role. |
-| s3.signer | bearer | Configure the signature version of the FileIO. |
-| s3.signer.uri | | Configure the remote signing uri if it differs from the catalog uri. Remote signing is only implemented for `FsspecFileIO`. The final request is sent to `/`. |
-| s3.signer.endpoint | v1/main/s3-sign | Configure the remote signing endpoint. Remote signing is only implemented for `FsspecFileIO`. The final request is sent to `/`. (default : v1/aws/s3/sign). |
-| s3.region | us-west-2 | Configure the default region used to initialize an `S3FileSystem`. `PyArrowFileIO` attempts to automatically tries to resolve the region if this isn't set (only supported for AWS S3 Buckets). |
-| s3.resolve-region | False | Only supported for `PyArrowFileIO`, when enabled, it will always try to resolve the location of the bucket (only supported for AWS S3 Buckets). |
-| s3.proxy-uri | | Configure the proxy server to be used by the FileIO. |
-| s3.connect-timeout | 60.0 | Configure socket connection timeout, in seconds. |
-| s3.request-timeout | 60.0 | Configure socket read timeouts on Windows and macOS, in seconds. |
+| Key | Example | Description |
+|-----------------------------|----------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| s3.endpoint | | Configure an alternative endpoint of the S3 service for the FileIO to access. This could be used to use S3FileIO with any s3-compatible object storage service that has a different endpoint, or access a private S3 endpoint in a virtual private cloud. |
+| s3.access-key-id | admin | Configure the static access key id used to access the FileIO. |
+| s3.secret-access-key | password | Configure the static secret access key used to access the FileIO. |
+| s3.session-token | AQoDYXdzEJr... | Configure the static session token used to access the FileIO. |
+| s3.role-session-name | session | An optional identifier for the assumed role session. |
+| s3.role-arn | arn:aws:... | AWS Role ARN. If provided instead of access_key and secret_key, temporary credentials will be fetched by assuming this role. |
+| s3.signer | bearer | Configure the signature version of the FileIO. |
+| s3.signer.uri | | Configure the remote signing uri if it differs from the catalog uri. Remote signing is only implemented for `FsspecFileIO`. The final request is sent to `/`. |
+| s3.signer.endpoint | v1/main/s3-sign | Configure the remote signing endpoint. Remote signing is only implemented for `FsspecFileIO`. The final request is sent to `/`. (default : v1/aws/s3/sign). |
+| s3.region | us-west-2 | Configure the default region used to initialize an `S3FileSystem`. `PyArrowFileIO` attempts to automatically tries to resolve the region if this isn't set (only supported for AWS S3 Buckets). |
+| s3.resolve-region | False | Only supported for `PyArrowFileIO`, when enabled, it will always try to resolve the location of the bucket (only supported for AWS S3 Buckets). |
+| s3.proxy-uri | | Configure the proxy server to be used by the FileIO. |
+| s3.connect-timeout | 60.0 | Configure socket connection timeout, in seconds. |
+| s3.request-timeout | 60.0 | Configure socket read timeouts on Windows and macOS, in seconds. |
| s3.force-virtual-addressing | False | Whether to use virtual addressing of buckets. If true, then virtual addressing is always enabled. If false, then virtual addressing is only enabled if endpoint_override is empty. This can be used for non-AWS backends that only support virtual hosted-style access. |
+| s3.retry-strategy-impl | None | Ability to set a custom S3 retry strategy. A full path to a class needs to be given that extends the [S3RetryStrategy](https://github.com/apache/arrow/blob/639201bfa412db26ce45e73851432018af6c945e/python/pyarrow/_s3fs.pyx#L110) base class. |
diff --git a/pyiceberg/io/__init__.py b/pyiceberg/io/__init__.py
index 2a73aba7f2..0b03a7dd4f 100644
--- a/pyiceberg/io/__init__.py
+++ b/pyiceberg/io/__init__.py
@@ -68,6 +68,7 @@
S3_ROLE_ARN = "s3.role-arn"
S3_ROLE_SESSION_NAME = "s3.role-session-name"
S3_FORCE_VIRTUAL_ADDRESSING = "s3.force-virtual-addressing"
+S3_RETRY_STRATEGY_IMPL = "s3.retry-strategy-impl"
HDFS_HOST = "hdfs.host"
HDFS_PORT = "hdfs.port"
HDFS_USER = "hdfs.user"
diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py
index cc81e37119..3e49885e58 100644
--- a/pyiceberg/io/pyarrow.py
+++ b/pyiceberg/io/pyarrow.py
@@ -27,6 +27,7 @@
import fnmatch
import functools
+import importlib
import itertools
import logging
import operator
@@ -63,6 +64,7 @@
import pyarrow.lib
import pyarrow.parquet as pq
from pyarrow import ChunkedArray
+from pyarrow._s3fs import S3RetryStrategy
from pyarrow.fs import (
FileInfo,
FileSystem,
@@ -111,6 +113,7 @@
S3_REGION,
S3_REQUEST_TIMEOUT,
S3_RESOLVE_REGION,
+ S3_RETRY_STRATEGY_IMPL,
S3_ROLE_ARN,
S3_ROLE_SESSION_NAME,
S3_SECRET_ACCESS_KEY,
@@ -214,6 +217,20 @@ def _cached_resolve_s3_region(bucket: str) -> Optional[str]:
return None
+def _import_retry_strategy(impl: str) -> Optional[S3RetryStrategy]:
+ try:
+ path_parts = impl.split(".")
+ if len(path_parts) < 2:
+ raise ValueError(f"retry-strategy-impl should be full path (module.CustomS3RetryStrategy), got: {impl}")
+ module_name, class_name = ".".join(path_parts[:-1]), path_parts[-1]
+ module = importlib.import_module(module_name)
+ class_ = getattr(module, class_name)
+ return class_()
+ except (ModuleNotFoundError, AttributeError):
+ warnings.warn(f"Could not initialize S3 retry strategy: {impl}")
+ return None
+
+
class UnsupportedPyArrowTypeException(Exception):
"""Cannot convert PyArrow type to corresponding Iceberg type."""
@@ -476,6 +493,11 @@ def _initialize_s3_fs(self, netloc: Optional[str]) -> FileSystem:
if self.properties.get(S3_FORCE_VIRTUAL_ADDRESSING) is not None:
client_kwargs["force_virtual_addressing"] = property_as_bool(self.properties, S3_FORCE_VIRTUAL_ADDRESSING, False)
+ if (retry_strategy_impl := self.properties.get(S3_RETRY_STRATEGY_IMPL)) and (
+ retry_instance := _import_retry_strategy(retry_strategy_impl)
+ ):
+ client_kwargs["retry_strategy"] = retry_instance
+
return S3FileSystem(**client_kwargs)
def _initialize_azure_fs(self) -> FileSystem:
diff --git a/tests/io/test_pyarrow.py b/tests/io/test_pyarrow.py
index 850b1292a8..92494455af 100644
--- a/tests/io/test_pyarrow.py
+++ b/tests/io/test_pyarrow.py
@@ -19,6 +19,7 @@
import os
import tempfile
import uuid
+import warnings
from datetime import date
from typing import Any, List, Optional
from unittest.mock import MagicMock, patch
@@ -29,7 +30,7 @@
import pyarrow.parquet as pq
import pytest
from packaging import version
-from pyarrow.fs import FileType, LocalFileSystem, S3FileSystem
+from pyarrow.fs import AwsDefaultS3RetryStrategy, FileType, LocalFileSystem, S3FileSystem
from pyiceberg.exceptions import ResolveError
from pyiceberg.expressions import (
@@ -57,7 +58,7 @@
Or,
)
from pyiceberg.expressions.literals import literal
-from pyiceberg.io import InputStream, OutputStream, load_file_io
+from pyiceberg.io import S3_RETRY_STRATEGY_IMPL, InputStream, OutputStream, load_file_io
from pyiceberg.io.pyarrow import (
ICEBERG_SCHEMA,
ArrowScan,
@@ -2512,3 +2513,21 @@ def test_pyarrow_io_multi_fs() -> None:
# Same PyArrowFileIO instance resolves local file input to LocalFileSystem
assert isinstance(pyarrow_file_io.new_input("file:///path/to/file")._filesystem, LocalFileSystem)
+
+
+class SomeRetryStrategy(AwsDefaultS3RetryStrategy):
+ def __init__(self) -> None:
+ super().__init__()
+ warnings.warn("Initialized SomeRetryStrategy 👍")
+
+
+def test_retry_strategy() -> None:
+ io = PyArrowFileIO(properties={S3_RETRY_STRATEGY_IMPL: "tests.io.test_pyarrow.SomeRetryStrategy"})
+ with pytest.warns(UserWarning, match="Initialized SomeRetryStrategy.*"):
+ io.new_input("s3://bucket/path/to/file")
+
+
+def test_retry_strategy_not_found() -> None:
+ io = PyArrowFileIO(properties={S3_RETRY_STRATEGY_IMPL: "pyiceberg.DoesNotExist"})
+ with pytest.warns(UserWarning, match="Could not initialize S3 retry strategy: pyiceberg.DoesNotExist"):
+ io.new_input("s3://bucket/path/to/file")
From c55b550acdaddaa9f9751ada7820d6efc48a337e Mon Sep 17 00:00:00 2001
From: Kevin Liu
Date: Fri, 20 Jun 2025 17:11:00 -0400
Subject: [PATCH 083/112] Refactor `Makefile` and run s3/adls/gcs tests (#2125)
Closes #2124
# Rationale for this change
Refactor `Makefile`
* Grouped commands and added comments
* Added `COVERAGE` param to run test with coverage
* Added `COVERAGE_FAIL_UNDER` param to specify coverage threshold to
pass
* Change test coverage threshold to `85` for unit tests, we are
currently at `87`
* Change test coverage threshold to `75` for integration tests, we are
currently at `77`
CI
* Add s3/adls/gcs integration tests to run in CI
* Run tests with coverage report
Note the `gcsfs` issue was resolved by #2127
# Are these changes tested?
Yes
# Are there any user-facing changes?
No
---
.github/workflows/python-ci.yml | 34 ++++++--
Makefile | 141 ++++++++++++++++++++------------
2 files changed, 118 insertions(+), 57 deletions(-)
diff --git a/.github/workflows/python-ci.yml b/.github/workflows/python-ci.yml
index 0e0b0c0f62..c8cdbcc58a 100644
--- a/.github/workflows/python-ci.yml
+++ b/.github/workflows/python-ci.yml
@@ -62,10 +62,12 @@ jobs:
run: sudo apt-get update && sudo apt-get install -y libkrb5-dev # for kerberos
- name: Install
run: make install-dependencies
- - name: Linters
+ - name: Run linters
run: make lint
- - name: Tests
- run: make test-coverage-unit
+ - name: Run unit tests with coverage
+ run: COVERAGE=1 make test
+ - name: Generate coverage report (85%) # Coverage threshold should only increase over time — never decrease it!
+ run: COVERAGE_FAIL_UNDER=85 make coverage-report
integration-test:
runs-on: ubuntu-22.04
@@ -79,8 +81,30 @@ jobs:
run: sudo apt-get update && sudo apt-get install -y libkrb5-dev # for kerberos
- name: Install
run: make install
- - name: Run integration tests
- run: make test-coverage-integration
+
+ - name: Run integration tests with coverage
+ run: COVERAGE=1 make test-integration
+ - name: Show debug logs
+ if: ${{ failure() }}
+ run: docker compose -f dev/docker-compose.yml logs
+
+ - name: Run s3 integration tests with coverage
+ run: COVERAGE=1 make test-s3
- name: Show debug logs
if: ${{ failure() }}
run: docker compose -f dev/docker-compose.yml logs
+
+ - name: Run adls integration tests with coverage
+ run: COVERAGE=1 make test-adls
+ - name: Show debug logs
+ if: ${{ failure() }}
+ run: docker compose -f dev/docker-compose-azurite.yml logs
+
+ - name: Run gcs integration tests with coverage
+ run: COVERAGE=1 make test-gcs
+ - name: Show debug logs
+ if: ${{ failure() }}
+ run: docker compose -f dev/docker-compose-gcs-server.yml logs
+
+ - name: Generate coverage report (75%) # Coverage threshold should only increase over time — never decrease it!
+ run: COVERAGE_FAIL_UNDER=75 make coverage-report
diff --git a/Makefile b/Makefile
index cd6056b027..859d8dfa23 100644
--- a/Makefile
+++ b/Makefile
@@ -14,47 +14,80 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
+# ========================
+# Configuration Variables
+# ========================
+PYTEST_ARGS ?= -v # Override with e.g. PYTEST_ARGS="-vv --tb=short"
+COVERAGE ?= 0 # Set COVERAGE=1 to enable coverage: make test COVERAGE=1
+COVERAGE_FAIL_UNDER ?= 85 # Minimum coverage % to pass: make coverage-report COVERAGE_FAIL_UNDER=70
-help: ## Display this help
- @awk 'BEGIN {FS = ":.*##"; printf "\nUsage:\n make \033[36m\033[0m\n"} /^[a-zA-Z_-]+:.*?##/ { printf " \033[36m%-20s\033[0m %s\n", $$1, $$2 } /^##@/ { printf "\n\033[1m%s\033[0m\n", substr($$0, 5) } ' $(MAKEFILE_LIST)
+ifeq ($(COVERAGE),1)
+ TEST_RUNNER = poetry run coverage run --parallel-mode --source=pyiceberg -m
+else
+ TEST_RUNNER = poetry run
+endif
POETRY_VERSION = 2.1.1
-install-poetry: ## Ensure Poetry is installed and the correct version is being used.
+
+# ============
+# Help Section
+# ============
+
+##@ General
+
+help: ## Display this help message
+ @awk 'BEGIN {FS = ":.*##"; printf "\nUsage:\n make \033[36m\033[0m\n"} /^[a-zA-Z_-]+:.*?##/ { printf " \033[36m%-25s\033[0m %s\n", $$1, $$2 } /^##@/ { printf "\n\033[1m%s\033[0m\n", substr($$0, 5) } ' $(MAKEFILE_LIST)
+
+# ==================
+# Installation Tasks
+# ==================
+
+##@ Setup
+
+install-poetry: ## Ensure Poetry is installed at the specified version
@if ! command -v poetry &> /dev/null; then \
- echo "Poetry could not be found. Installing..."; \
+ echo "Poetry not found. Installing..."; \
pip install --user poetry==$(POETRY_VERSION); \
else \
INSTALLED_VERSION=$$(pip show poetry | grep Version | awk '{print $$2}'); \
if [ "$$INSTALLED_VERSION" != "$(POETRY_VERSION)" ]; then \
- echo "Poetry version $$INSTALLED_VERSION does not match required version $(POETRY_VERSION). Updating..."; \
+ echo "Updating Poetry to version $(POETRY_VERSION)..."; \
pip install --user --upgrade poetry==$(POETRY_VERSION); \
else \
- echo "Poetry version $$INSTALLED_VERSION is already installed."; \
- fi \
+ echo "Poetry version $(POETRY_VERSION) already installed."; \
+ fi; \
fi
-install-dependencies: ## Install dependencies including dev, docs, and all extras
+install-dependencies: ## Install all dependencies including extras
poetry install --all-extras
-install: | install-poetry install-dependencies
+install: install-poetry install-dependencies ## Install Poetry and dependencies
+
+# ===============
+# Code Validation
+# ===============
+
+##@ Quality
check-license: ## Check license headers
./dev/check-license
-lint: ## lint
+lint: ## Run code linters via pre-commit
poetry run pre-commit run --all-files
-test: ## Run all unit tests, can add arguments with PYTEST_ARGS="-vv"
- poetry run pytest tests/ -m "(unmarked or parametrize) and not integration" ${PYTEST_ARGS}
+# ===============
+# Testing Section
+# ===============
-test-s3: # Run tests marked with s3, can add arguments with PYTEST_ARGS="-vv"
- sh ./dev/run-minio.sh
- poetry run pytest tests/ -m s3 ${PYTEST_ARGS}
+##@ Testing
-test-integration: | test-integration-setup test-integration-exec ## Run all integration tests, can add arguments with PYTEST_ARGS="-vv"
+test: ## Run all unit tests (excluding integration)
+ $(TEST_RUNNER) pytest tests/ -m "(unmarked or parametrize) and not integration" $(PYTEST_ARGS)
-test-integration-setup: # Prepare the environment for integration
+test-integration: test-integration-setup test-integration-exec ## Run integration tests
+
+test-integration-setup: ## Start Docker services for integration tests
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 up -d
@@ -62,58 +95,62 @@ test-integration-setup: # Prepare the environment for integration
docker compose -f dev/docker-compose-integration.yml cp ./dev/provision.py spark-iceberg:/opt/spark/provision.py
docker compose -f dev/docker-compose-integration.yml exec -T spark-iceberg ipython ./provision.py
-test-integration-exec: # Execute integration tests, can add arguments with PYTEST_ARGS="-vv"
- poetry run pytest tests/ -v -m integration ${PYTEST_ARGS}
+test-integration-exec: ## Run integration tests (excluding provision)
+ $(TEST_RUNNER) pytest tests/ -m integration $(PYTEST_ARGS)
-test-integration-rebuild:
+test-integration-rebuild: ## Rebuild integration Docker services from scratch
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-adls: ## Run tests marked with adls, can add arguments with PYTEST_ARGS="-vv"
+test-s3: ## Run tests marked with @pytest.mark.s3
+ sh ./dev/run-minio.sh
+ $(TEST_RUNNER) pytest tests/ -m s3 $(PYTEST_ARGS)
+
+test-adls: ## Run tests marked with @pytest.mark.adls
sh ./dev/run-azurite.sh
- poetry run pytest tests/ -m adls ${PYTEST_ARGS}
+ $(TEST_RUNNER) pytest tests/ -m adls $(PYTEST_ARGS)
-test-gcs: ## Run tests marked with gcs, can add arguments with PYTEST_ARGS="-vv"
+test-gcs: ## Run tests marked with @pytest.mark.gcs
sh ./dev/run-gcs-server.sh
- poetry run pytest tests/ -m gcs ${PYTEST_ARGS}
+ $(TEST_RUNNER) pytest tests/ -m gcs $(PYTEST_ARGS)
-test-coverage-unit: # Run test with coverage for unit tests, can add arguments with PYTEST_ARGS="-vv"
- poetry run coverage run --source=pyiceberg/ --data-file=.coverage.unit -m pytest tests/ -v -m "(unmarked or parametrize) and not integration" ${PYTEST_ARGS}
-
-test-coverage-integration: # Run test with coverage for integration tests, can add arguments with PYTEST_ARGS="-vv"
- 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 up -d
- sh ./dev/run-azurite.sh
- sh ./dev/run-gcs-server.sh
- sleep 10
- docker compose -f dev/docker-compose-integration.yml cp ./dev/provision.py spark-iceberg:/opt/spark/provision.py
- docker compose -f dev/docker-compose-integration.yml exec -T spark-iceberg ipython ./provision.py
- poetry run coverage run --source=pyiceberg/ --data-file=.coverage.integration -m pytest tests/ -v -m integration ${PYTEST_ARGS}
+test-coverage: COVERAGE=1
+test-coverage: test test-integration test-s3 test-adls test-gcs coverage-report ## Run all tests with coverage and report
-test-coverage: | test-coverage-unit test-coverage-integration ## Run all tests with coverage including unit and integration tests
- poetry run coverage combine .coverage.unit .coverage.integration
- poetry run coverage report -m --fail-under=90
+coverage-report: ## Combine and report coverage
+ poetry run coverage combine
+ poetry run coverage report -m --fail-under=$(COVERAGE_FAIL_UNDER)
poetry run coverage html
poetry run coverage xml
+# ================
+# Documentation
+# ================
-clean: ## Clean up the project Python working environment
- @echo "Cleaning up Cython and Python cached files"
- @rm -rf build dist *.egg-info
- @find . -name "*.so" -exec echo Deleting {} \; -delete
- @find . -name "*.pyc" -exec echo Deleting {} \; -delete
- @find . -name "__pycache__" -exec echo Deleting {} \; -exec rm -rf {} +
- @find . -name "*.pyd" -exec echo Deleting {} \; -delete
- @find . -name "*.pyo" -exec echo Deleting {} \; -delete
- @echo "Cleanup complete"
+##@ Documentation
-docs-install:
+docs-install: ## Install docs dependencies
poetry install --with docs
-docs-serve:
+docs-serve: ## Serve local docs preview (hot reload)
poetry run mkdocs serve -f mkdocs/mkdocs.yml
-docs-build:
+docs-build: ## Build the static documentation site
poetry run mkdocs build -f mkdocs/mkdocs.yml --strict
+
+# ===================
+# Project Maintenance
+# ===================
+
+##@ Maintenance
+
+clean: ## Remove build artifacts and caches
+ @echo "Cleaning up Cython and Python cached files..."
+ @rm -rf build dist *.egg-info
+ @find . -name "*.so" -exec echo Deleting {} \; -delete
+ @find . -name "*.pyc" -exec echo Deleting {} \; -delete
+ @find . -name "__pycache__" -exec echo Deleting {} \; -exec rm -rf {} +
+ @find . -name "*.pyd" -exec echo Deleting {} \; -delete
+ @find . -name "*.pyo" -exec echo Deleting {} \; -delete
+ @echo "Cleanup complete."
From 4357b53d92d850e44a45fab77f59618891974d69 Mon Sep 17 00:00:00 2001
From: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
Date: Sun, 22 Jun 2025 14:07:04 -0400
Subject: [PATCH 084/112] perf: optimize `table.add_files` and `inspect.files`
(#2133)
Should help with #2130 and #2132
Modifies `Table.add_files` to explicitly use `inspect.data_files` and
also parallelize `inspect._files`
I didn't see anywhere else where looping over manifest entries was
parallelized, so seems better to parallelize across manifests than
within.
No changes here but should be faster.
---------
Co-authored-by: Kevin Liu
---
pyiceberg/table/__init__.py | 2 +-
pyiceberg/table/inspect.py | 11 +++++++----
2 files changed, 8 insertions(+), 5 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index f6d28a5ecf..1246c6c19e 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -855,7 +855,7 @@ def add_files(
import pyarrow.compute as pc
expr = pc.field("file_path").isin(file_paths)
- referenced_files = [file["file_path"] for file in self._table.inspect.files().filter(expr).to_pylist()]
+ referenced_files = [file["file_path"] for file in self._table.inspect.data_files().filter(expr).to_pylist()]
if referenced_files:
raise ValueError(f"Cannot add files that are already referenced by table, files: {', '.join(referenced_files)}")
diff --git a/pyiceberg/table/inspect.py b/pyiceberg/table/inspect.py
index cce5250ad5..3bb0268a05 100644
--- a/pyiceberg/table/inspect.py
+++ b/pyiceberg/table/inspect.py
@@ -650,11 +650,14 @@ def _files(self, snapshot_id: Optional[int] = None, data_file_filter: Optional[S
snapshot = self._get_snapshot(snapshot_id)
io = self.tbl.io
- files_table: list[pa.Table] = []
- for manifest_list in snapshot.manifests(io):
- files_table.append(self._get_files_from_manifest(manifest_list, data_file_filter))
- return pa.concat_tables(files_table)
+ executor = ExecutorFactory.get_or_create()
+ results = list(
+ executor.map(
+ lambda manifest_list: self._get_files_from_manifest(manifest_list, data_file_filter), snapshot.manifests(io)
+ )
+ )
+ return pa.concat_tables(results)
def files(self, snapshot_id: Optional[int] = None) -> "pa.Table":
return self._files(snapshot_id)
From 292591736f9ce7a9e294ed6fc7f8503c9c7508e6 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Mon, 23 Jun 2025 22:48:17 +0200
Subject: [PATCH 085/112] Run `test_create_table_transaction` for Rest V1
tables (#2137)
# Rationale for this change
Moar test coverage :)
# Are these changes tested?
# Are there any user-facing changes?
---
tests/integration/test_writes/test_writes.py | 11 -----------
1 file changed, 11 deletions(-)
diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py
index 493b163b95..033a9f7c0d 100644
--- a/tests/integration/test_writes/test_writes.py
+++ b/tests/integration/test_writes/test_writes.py
@@ -40,7 +40,6 @@
from pyiceberg.catalog import Catalog, load_catalog
from pyiceberg.catalog.hive import HiveCatalog
-from pyiceberg.catalog.rest import RestCatalog
from pyiceberg.catalog.sql import SqlCatalog
from pyiceberg.exceptions import NoSuchTableError
from pyiceberg.expressions import And, EqualTo, GreaterThanOrEqual, In, LessThan, Not
@@ -885,11 +884,6 @@ def test_write_and_evolve(session_catalog: Catalog, format_version: int) -> None
@pytest.mark.parametrize("format_version", [1, 2])
@pytest.mark.parametrize("catalog", [pytest.lazy_fixture("session_catalog_hive"), pytest.lazy_fixture("session_catalog")])
def test_create_table_transaction(catalog: Catalog, format_version: int) -> None:
- if format_version == 1 and isinstance(catalog, RestCatalog):
- pytest.skip(
- "There is a bug in the REST catalog image (https://github.com/apache/iceberg/issues/8756) that prevents create and commit a staged version 1 table"
- )
-
identifier = f"default.arrow_create_table_transaction_{catalog.name}_{format_version}"
try:
@@ -942,11 +936,6 @@ def test_create_table_transaction(catalog: Catalog, format_version: int) -> None
@pytest.mark.parametrize("format_version", [1, 2])
@pytest.mark.parametrize("catalog", [pytest.lazy_fixture("session_catalog_hive"), pytest.lazy_fixture("session_catalog")])
def test_create_table_with_non_default_values(catalog: Catalog, table_schema_with_all_types: Schema, format_version: int) -> None:
- if format_version == 1 and isinstance(catalog, RestCatalog):
- pytest.skip(
- "There is a bug in the REST catalog image (https://github.com/apache/iceberg/issues/8756) that prevents create and commit a staged version 1 table"
- )
-
identifier = f"default.arrow_create_table_transaction_with_non_default_values_{catalog.name}_{format_version}"
identifier_ref = f"default.arrow_create_table_transaction_with_non_default_values_ref_{catalog.name}_{format_version}"
From a742534d6fa6cd66b248457378d7040d37779691 Mon Sep 17 00:00:00 2001
From: Antoine <89391685+elca-anh@users.noreply.github.com>
Date: Tue, 24 Jun 2025 00:17:22 +0200
Subject: [PATCH 086/112] Add warehouse parameter to the REST Catalog doc
(#2066)
# Rationale for this change
Missing parameter in REST Catalog documentation
# Are these changes tested?
Doc only
# Are there any user-facing changes?
Doc only
---------
Co-authored-by: Fokko Driesprong
Co-authored-by: Kevin Liu
---
mkdocs/docs/configuration.md | 1 +
1 file changed, 1 insertion(+)
diff --git a/mkdocs/docs/configuration.md b/mkdocs/docs/configuration.md
index 8503a846d6..922e60e1f8 100644
--- a/mkdocs/docs/configuration.md
+++ b/mkdocs/docs/configuration.md
@@ -351,6 +351,7 @@ catalog:
| rest.signing-name | execute-api | The service signing name to use when SigV4 signing a request |
| oauth2-server-uri | | Authentication URL to use for client credentials authentication (default: uri + 'v1/oauth/tokens') |
| snapshot-loading-mode | refs | The snapshots to return in the body of the metadata. Setting the value to `all` would return the full set of snapshots currently valid for the table. Setting the value to `refs` would load all snapshots referenced by branches or tags. |
+| warehouse | myWarehouse | Warehouse location or identifier to request from the catalog service. May be used to determine server-side overrides, such as the warehouse location. |
From 7cb8b1b7e71afbdabc738d0ce25f7ed1100ac67e Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 23 Jun 2025 22:23:04 -0400
Subject: [PATCH 087/112] Build: Bump mypy-boto3-glue from 1.38.22 to 1.38.42
(#2144)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [mypy-boto3-glue](https://github.com/youtype/mypy_boto3_builder)
from 1.38.22 to 1.38.42.
Release notes
Sourced from mypy-boto3-glue's
releases.
8.8.0 - Python 3.8 runtime is back
Changed
[services] install_requires section is
calculated based on dependencies in use, so
typing-extensions version is set properly
[all] Replaced typing imports with
collections.abc with a fallback to typing for
Python <3.9
[all] Added aliases for builtins.list,
builtins.set, builtins.dict, and
builtins.type, so Python 3.8 runtime should work as
expected again (reported by @YHallouard in #340
and @Omri-Ben-Yair
in #336)
[all] Unions use the same type annotations as the rest
of the structures due to proper fallbacks
Fixed
[services] Universal input/output shapes were not
replaced properly in service subresources
[docs] Simplified doc links rendering for services
[services] Cleaned up unnecessary imports in
client.pyi
[builder] Import records with fallback are always
rendered
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 88 +++++++++++++++++++++++++++--------------------------
1 file changed, 45 insertions(+), 43 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 887d029eff..0783784472 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -59,7 +59,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -72,7 +72,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5421af8f22a98f640261ee48aae3a37f0c41371e99412d55eaf2f8a46d5dad29"},
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0fcda86f6cb318ba36ed8f1396a6a4a3fd8f856f84d426584392083d10da4de0"},
@@ -202,7 +202,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "python_version == \"3.9\" and extra == \"ray\" or (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version < \"3.10\" or python_version >= \"3.10\" and (extra == \"ray\" or extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -268,7 +268,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -285,7 +285,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "python_version == \"3.9\" and extra == \"ray\" or (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version < \"3.10\" or python_version >= \"3.10\" and (extra == \"ray\" or extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\")"}
+markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -488,7 +488,7 @@ files = [
{file = "boto3-1.38.27-py3-none-any.whl", hash = "sha256:95f5fe688795303a8a15e8b7e7f255cadab35eae459d00cc281a4fd77252ea80"},
{file = "boto3-1.38.27.tar.gz", hash = "sha256:94bd7fdd92d5701b362d4df100d21e28f8307a67ff56b6a8b0398119cf22f859"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.38.27,<1.39.0"
@@ -509,7 +509,7 @@ files = [
{file = "botocore-1.38.27-py3-none-any.whl", hash = "sha256:a785d5e9a5eda88ad6ab9ed8b87d1f2ac409d0226bba6ff801c55359e94d91a8"},
{file = "botocore-1.38.27.tar.gz", hash = "sha256:9788f7efe974328a38cbade64cc0b1e67d27944b899f88cb786ae362973133b6"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -802,7 +802,6 @@ description = "Composable command line interface toolkit"
optional = false
python-versions = ">=3.7"
groups = ["main", "dev", "docs"]
-markers = "python_version < \"3.10\""
files = [
{file = "click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2"},
{file = "click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a"},
@@ -811,22 +810,6 @@ files = [
[package.dependencies]
colorama = {version = "*", markers = "platform_system == \"Windows\""}
-[[package]]
-name = "click"
-version = "8.2.1"
-description = "Composable command line interface toolkit"
-optional = false
-python-versions = ">=3.10"
-groups = ["main", "dev", "docs"]
-markers = "python_version >= \"3.10\""
-files = [
- {file = "click-8.2.1-py3-none-any.whl", hash = "sha256:61a3265b914e850b85317d0b3109c7f8cd35a670f963866005d6ef1d5175a12b"},
- {file = "click-8.2.1.tar.gz", hash = "sha256:27c491cc05d968d271d5a1db13e3b5a184636d9d930f148c50b038f0d0646202"},
-]
-
-[package.dependencies]
-colorama = {version = "*", markers = "platform_system == \"Windows\""}
-
[[package]]
name = "colorama"
version = "0.4.6"
@@ -1435,7 +1418,7 @@ files = [
{file = "filelock-3.18.0-py3-none-any.whl", hash = "sha256:c401f4f8377c4464e6db25fff06205fd89bdd83b65eb0488ed1b160f780e21de"},
{file = "filelock-3.18.0.tar.gz", hash = "sha256:adbc88eabb99d2fec8c9c1b229b171f18afa655400173ddc653d5d01501fb9f2"},
]
-markers = {main = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"hf\") or extra == \"hf\" or extra == \"ray\""}
+markers = {main = "extra == \"ray\" or extra == \"hf\""}
[package.extras]
docs = ["furo (>=2024.8.6)", "sphinx (>=8.1.3)", "sphinx-autodoc-typehints (>=3)"]
@@ -1490,7 +1473,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "python_version == \"3.9\" and extra == \"ray\" or (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version < \"3.10\" or python_version >= \"3.10\" and (extra == \"ray\" or extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cc4df77d638aa2ed703b878dd093725b72a824c3c546c076e8fdf276f78ee84a"},
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:716a9973a2cc963160394f701964fe25012600f3d311f60c790400b00e568b61"},
@@ -2264,7 +2247,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -2308,6 +2291,8 @@ python-versions = "*"
groups = ["dev"]
files = [
{file = "jsonpath-ng-1.7.0.tar.gz", hash = "sha256:f6f5f7fd4e5ff79c785f1573b394043b39849fb2bb47bcead935d12b00beab3c"},
+ {file = "jsonpath_ng-1.7.0-py2-none-any.whl", hash = "sha256:898c93fc173f0c336784a3fa63d7434297544b7198124a68f9a3ef9597b0ae6e"},
+ {file = "jsonpath_ng-1.7.0-py3-none-any.whl", hash = "sha256:f3d7f9e848cba1b6da28c55b1c26ff915dc9e0b1ba7e752a53d6da8d5cbd00b6"},
]
[package.dependencies]
@@ -3052,7 +3037,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "multidict-6.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:2e118a202904623b1d2606d1c8614e14c9444b59d64454b0c355044058066469"},
{file = "multidict-6.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:a42995bdcaff4e22cb1280ae7752c3ed3fbb398090c6991a2797a4a0e5ed16a9"},
@@ -3186,14 +3171,14 @@ typing-extensions = {version = "*", markers = "python_version < \"3.12\""}
[[package]]
name = "mypy-boto3-glue"
-version = "1.38.22"
-description = "Type annotations for boto3 Glue 1.38.22 service generated with mypy-boto3-builder 8.11.0"
+version = "1.38.42"
+description = "Type annotations for boto3 Glue 1.38.42 service generated with mypy-boto3-builder 8.11.0"
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "mypy_boto3_glue-1.38.22-py3-none-any.whl", hash = "sha256:4fe34c858cbee41e8ad30382305c01b0dd9c1da4c84f894860b9249ddabb4a58"},
- {file = "mypy_boto3_glue-1.38.22.tar.gz", hash = "sha256:a9c529fafaaa9845d39c3204b3fb6cbbb633fa747faf6a084a2b2a381ef12a2b"},
+ {file = "mypy_boto3_glue-1.38.42-py3-none-any.whl", hash = "sha256:f49401710e8795a5d12ae0ad14bb6a96fd8ac23477853569421253a51a8e790c"},
+ {file = "mypy_boto3_glue-1.38.42.tar.gz", hash = "sha256:c34967cd54ddd5d8d8afc1464eaa847b6430c32b7424559199c5af3d9006b4f1"},
]
[package.dependencies]
@@ -3353,7 +3338,7 @@ description = "Fundamental package for array computing in Python"
optional = true
python-versions = ">=3.10"
groups = ["main"]
-markers = "python_version == \"3.10\" and (extra == \"pandas\" or extra == \"ray\")"
+markers = "(extra == \"pandas\" or extra == \"ray\") and python_version == \"3.10\""
files = [
{file = "numpy-2.2.6-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b412caa66f72040e6d268491a59f2c43bf03eb6c96dd8f0307829feb7fa2b6fb"},
{file = "numpy-2.2.6-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8e41fd67c52b86603a91c1a505ebaef50b3314de0213461c7a6e99c9a3beff90"},
@@ -3419,7 +3404,7 @@ description = "Fundamental package for array computing in Python"
optional = true
python-versions = ">=3.11"
groups = ["main"]
-markers = "(extra == \"pandas\" or extra == \"ray\") and python_version >= \"3.11\""
+markers = "python_version >= \"3.11\" and (extra == \"pandas\" or extra == \"ray\")"
files = [
{file = "numpy-2.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c3c9fdde0fa18afa1099d6257eb82890ea4f3102847e692193b54e00312a9ae9"},
{file = "numpy-2.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:46d16f72c2192da7b83984aa5455baee640e33a9f1e61e656f29adf55e406c2b"},
@@ -3538,7 +3523,7 @@ files = [
{file = "packaging-25.0-py3-none-any.whl", hash = "sha256:29572ef2b1f17581046b3a2227d5c611fb25ec70ca1ba8554b24b0e69331a484"},
{file = "packaging-25.0.tar.gz", hash = "sha256:d443872c98d677bf60f6a1f2f8c1cb748e8fe762d2bf9d3148b5599295b0fc4f"},
]
-markers = {main = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"hf\") or extra == \"hf\" or extra == \"ray\""}
+markers = {main = "extra == \"ray\" or extra == \"hf\""}
[[package]]
name = "paginate"
@@ -3569,35 +3554,51 @@ files = [
{file = "pandas-2.3.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a6872d695c896f00df46b71648eea332279ef4077a409e2fe94220208b6bb675"},
{file = "pandas-2.3.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f4dd97c19bd06bc557ad787a15b6489d2614ddaab5d104a0310eb314c724b2d2"},
{file = "pandas-2.3.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:034abd6f3db8b9880aaee98f4f5d4dbec7c4829938463ec046517220b2f8574e"},
+ {file = "pandas-2.3.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:23c2b2dc5213810208ca0b80b8666670eb4660bbfd9d45f58592cc4ddcfd62e1"},
{file = "pandas-2.3.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:39ff73ec07be5e90330cc6ff5705c651ace83374189dcdcb46e6ff54b4a72cd6"},
{file = "pandas-2.3.0-cp310-cp310-win_amd64.whl", hash = "sha256:40cecc4ea5abd2921682b57532baea5588cc5f80f0231c624056b146887274d2"},
{file = "pandas-2.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:8adff9f138fc614347ff33812046787f7d43b3cef7c0f0171b3340cae333f6ca"},
+ {file = "pandas-2.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e5f08eb9a445d07720776df6e641975665c9ea12c9d8a331e0f6890f2dcd76ef"},
{file = "pandas-2.3.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fa35c266c8cd1a67d75971a1912b185b492d257092bdd2709bbdebe574ed228d"},
{file = "pandas-2.3.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:14a0cc77b0f089d2d2ffe3007db58f170dae9b9f54e569b299db871a3ab5bf46"},
+ {file = "pandas-2.3.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:c06f6f144ad0a1bf84699aeea7eff6068ca5c63ceb404798198af7eb86082e33"},
{file = "pandas-2.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ed16339bc354a73e0a609df36d256672c7d296f3f767ac07257801aa064ff73c"},
{file = "pandas-2.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:fa07e138b3f6c04addfeaf56cc7fdb96c3b68a3fe5e5401251f231fce40a0d7a"},
{file = "pandas-2.3.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:2eb4728a18dcd2908c7fccf74a982e241b467d178724545a48d0caf534b38ebf"},
+ {file = "pandas-2.3.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b9d8c3187be7479ea5c3d30c32a5d73d62a621166675063b2edd21bc47614027"},
+ {file = "pandas-2.3.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9ff730713d4c4f2f1c860e36c005c7cefc1c7c80c21c0688fd605aa43c9fcf09"},
{file = "pandas-2.3.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba24af48643b12ffe49b27065d3babd52702d95ab70f50e1b34f71ca703e2c0d"},
+ {file = "pandas-2.3.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:404d681c698e3c8a40a61d0cd9412cc7364ab9a9cc6e144ae2992e11a2e77a20"},
{file = "pandas-2.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:6021910b086b3ca756755e86ddc64e0ddafd5e58e076c72cb1585162e5ad259b"},
{file = "pandas-2.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:094e271a15b579650ebf4c5155c05dcd2a14fd4fdd72cf4854b2f7ad31ea30be"},
{file = "pandas-2.3.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:2c7e2fc25f89a49a11599ec1e76821322439d90820108309bf42130d2f36c983"},
+ {file = "pandas-2.3.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:c6da97aeb6a6d233fb6b17986234cc723b396b50a3c6804776351994f2a658fd"},
{file = "pandas-2.3.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bb32dc743b52467d488e7a7c8039b821da2826a9ba4f85b89ea95274f863280f"},
{file = "pandas-2.3.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:213cd63c43263dbb522c1f8a7c9d072e25900f6975596f883f4bebd77295d4f3"},
+ {file = "pandas-2.3.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:1d2b33e68d0ce64e26a4acc2e72d747292084f4e8db4c847c6f5f6cbe56ed6d8"},
{file = "pandas-2.3.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:430a63bae10b5086995db1b02694996336e5a8ac9a96b4200572b413dfdfccb9"},
{file = "pandas-2.3.0-cp313-cp313-win_amd64.whl", hash = "sha256:4930255e28ff5545e2ca404637bcc56f031893142773b3468dc021c6c32a1390"},
{file = "pandas-2.3.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:f925f1ef673b4bd0271b1809b72b3270384f2b7d9d14a189b12b7fc02574d575"},
{file = "pandas-2.3.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:e78ad363ddb873a631e92a3c063ade1ecfb34cae71e9a2be6ad100f875ac1042"},
{file = "pandas-2.3.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:951805d146922aed8357e4cc5671b8b0b9be1027f0619cea132a9f3f65f2f09c"},
{file = "pandas-2.3.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1a881bc1309f3fce34696d07b00f13335c41f5f5a8770a33b09ebe23261cfc67"},
+ {file = "pandas-2.3.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:e1991bbb96f4050b09b5f811253c4f3cf05ee89a589379aa36cd623f21a31d6f"},
{file = "pandas-2.3.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:bb3be958022198531eb7ec2008cfc78c5b1eed51af8600c6c5d9160d89d8d249"},
+ {file = "pandas-2.3.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:9efc0acbbffb5236fbdf0409c04edce96bec4bdaa649d49985427bd1ec73e085"},
+ {file = "pandas-2.3.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:75651c14fde635e680496148a8526b328e09fe0572d9ae9b638648c46a544ba3"},
+ {file = "pandas-2.3.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bf5be867a0541a9fb47a4be0c5790a4bccd5b77b92f0a59eeec9375fafc2aa14"},
+ {file = "pandas-2.3.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84141f722d45d0c2a89544dd29d35b3abfc13d2250ed7e68394eda7564bd6324"},
+ {file = "pandas-2.3.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:f95a2aef32614ed86216d3c450ab12a4e82084e8102e355707a1d96e33d51c34"},
+ {file = "pandas-2.3.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:e0f51973ba93a9f97185049326d75b942b9aeb472bec616a129806facb129ebb"},
+ {file = "pandas-2.3.0-cp39-cp39-win_amd64.whl", hash = "sha256:b198687ca9c8529662213538a9bb1e60fa0bf0f6af89292eb68fea28743fcd5a"},
{file = "pandas-2.3.0.tar.gz", hash = "sha256:34600ab34ebf1131a7613a260a61dbe8b62c188ec0ea4c296da7c9a06b004133"},
]
[package.dependencies]
numpy = [
{version = ">=1.22.4", markers = "python_version < \"3.11\""},
- {version = ">=1.26.0", markers = "python_version >= \"3.12\""},
{version = ">=1.23.2", markers = "python_version == \"3.11\""},
+ {version = ">=1.26.0", markers = "python_version >= \"3.12\""},
]
python-dateutil = ">=2.8.2"
pytz = ">=2020.1"
@@ -3768,7 +3769,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:22d9962a358aedbb7a2e36187ff273adeaab9743373a272976d2e348d08c7770"},
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0d0fda578d1dc3f77b6b5a5dce3b9ad69a8250a891760a548df850a5e8da87f3"},
@@ -3896,7 +3897,7 @@ description = ""
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"gcsfs\") or extra == \"gcsfs\" or extra == \"ray\""
+markers = "extra == \"ray\" or extra == \"gcsfs\""
files = [
{file = "protobuf-6.31.1-cp310-abi3-win32.whl", hash = "sha256:7fa17d5a29c2e04b7d90e5e32388b8bfd0e7107cd8e616feef7ed3fa6bdab5c9"},
{file = "protobuf-6.31.1-cp310-abi3-win_amd64.whl", hash = "sha256:426f59d2964864a1a366254fa703b8632dcec0790d8862d30034d8245e1cd447"},
@@ -3965,6 +3966,7 @@ files = [
{file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:bb89f0a835bcfc1d42ccd5f41f04870c1b936d8507c6df12b7737febc40f0909"},
{file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:f0c2d907a1e102526dd2986df638343388b94c33860ff3bbe1384130828714b1"},
{file = "psycopg2_binary-2.9.10-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:f8157bed2f51db683f31306aa497311b560f2265998122abe1dce6428bd86567"},
+ {file = "psycopg2_binary-2.9.10-cp313-cp313-win_amd64.whl", hash = "sha256:27422aa5f11fbcd9b18da48373eb67081243662f9b46e6fd07c3eb46e4535142"},
{file = "psycopg2_binary-2.9.10-cp38-cp38-macosx_12_0_x86_64.whl", hash = "sha256:eb09aa7f9cecb45027683bb55aebaaf45a0df8bf6de68801a6afdc7947bb09d4"},
{file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b73d6d7f0ccdad7bc43e6d34273f70d587ef62f824d7261c4ae9b8b1b6af90e8"},
{file = "psycopg2_binary-2.9.10-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ce5ab4bf46a211a8e924d307c1b1fcda82368586a19d0a24f8ae166f5c784864"},
@@ -4094,7 +4096,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
+markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -4699,7 +4701,7 @@ files = [
{file = "PyYAML-6.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:39693e1f8320ae4f43943590b49779ffb98acb81f788220ea932a6b6c51004d8"},
{file = "pyyaml-6.0.2.tar.gz", hash = "sha256:d584d9ec91ad65861cc08d42e834324ef890a082e591037abe114850ff7bbc3e"},
]
-markers = {main = "python_version == \"3.9\" and (extra == \"ray\" or extra == \"hf\") or extra == \"hf\" or extra == \"ray\""}
+markers = {main = "extra == \"ray\" or extra == \"hf\""}
[[package]]
name = "pyyaml-env-tag"
@@ -5223,7 +5225,7 @@ files = [
{file = "s3transfer-0.13.0-py3-none-any.whl", hash = "sha256:0148ef34d6dd964d0d8cf4311b2b21c474693e57c2e069ec708ce043d2b527be"},
{file = "s3transfer-0.13.0.tar.gz", hash = "sha256:f5e6db74eb7776a37208001113ea7aa97695368242b364d73e91c981ac522177"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.4,<2.0a.0"
@@ -5731,7 +5733,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"hf\" or extra == \"daft\""
+markers = "extra == \"daft\" or extra == \"hf\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -6016,7 +6018,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6032e6da6abd41e4acda34d75a816012717000fa6839f37124a47fcefc49bec4"},
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2c7b34d804b8cf9b214f05015c4fee2ebe7ed05cf581e7192c06555c71f4446a"},
From f1d2af20b1728edd946ea48896c74edf6408b457 Mon Sep 17 00:00:00 2001
From: Kevin Liu
Date: Tue, 24 Jun 2025 03:45:57 -0400
Subject: [PATCH 088/112] test: use `create or replace` (#2140)
# Rationale for this change
Follow up to #2137
Remove all mentions of `CREATE OR REPLACE` issue,
https://github.com/apache/iceberg/issues/8756
# Are these changes tested?
# Are there any user-facing changes?
---
dev/provision.py | 6 +-----
1 file changed, 1 insertion(+), 5 deletions(-)
diff --git a/dev/provision.py b/dev/provision.py
index 837189204e..231f5123ce 100644
--- a/dev/provision.py
+++ b/dev/provision.py
@@ -273,13 +273,9 @@
"""
)
- # There is an issue with CREATE OR REPLACE
- # https://github.com/apache/iceberg/issues/8756
- spark.sql(f"DROP TABLE IF EXISTS {catalog_name}.default.test_table_version")
-
spark.sql(
f"""
- CREATE TABLE {catalog_name}.default.test_table_version (
+ CREATE OR REPLACE TABLE {catalog_name}.default.test_table_version (
dt date,
number integer,
letter string
From 75eaea300f8c437d53361b31368205bcc4497fc6 Mon Sep 17 00:00:00 2001
From: Kevin Liu
Date: Tue, 24 Jun 2025 10:54:22 -0400
Subject: [PATCH 089/112] Update assertion for testing locally (#2113)
# Rationale for this change
`make test` fails locally since `Config().get_known_catalogs()` also
reads my local `~/.pyiceberg.yaml`
Follow up to #2088
# Are these changes tested?
# Are there any user-facing changes?
---
tests/utils/test_config.py | 7 +++----
1 file changed, 3 insertions(+), 4 deletions(-)
diff --git a/tests/utils/test_config.py b/tests/utils/test_config.py
index 24a867f812..d70b0345f8 100644
--- a/tests/utils/test_config.py
+++ b/tests/utils/test_config.py
@@ -59,10 +59,9 @@ def test_fix_nested_objects_from_environment_variables() -> None:
@mock.patch.dict(os.environ, EXAMPLE_ENV)
@mock.patch.dict(os.environ, {"PYICEBERG_CATALOG__DEVELOPMENT__URI": "https://dev.service.io/api"})
def test_list_all_known_catalogs() -> None:
- assert Config().get_known_catalogs() == [
- "production",
- "development",
- ]
+ catalogs = Config().get_known_catalogs()
+ assert "production" in catalogs
+ assert "development" in catalogs
def test_from_configuration_files(tmp_path_factory: pytest.TempPathFactory) -> None:
From 8c47fb333ca0dc9170829df0a62901139b4c2f92 Mon Sep 17 00:00:00 2001
From: Kaushik Srinivasan
Date: Tue, 24 Jun 2025 15:45:52 -0400
Subject: [PATCH 090/112] Validate added data files for snapshot compatibility
(#2050)
Closes #1929
# Rationale for this change
- Since we want to support snapshot write compatibility (#1772) and is
part of the following parent issue #819
# Are these changes tested?
Yes
# Are there any user-facing changes?
No
---------
Co-authored-by: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
Co-authored-by: Fokko Driesprong
---
pyiceberg/table/update/validate.py | 127 ++++++++++++++++++++++-----
tests/table/test_validate.py | 135 ++++++++++++++++++++++++++++-
2 files changed, 240 insertions(+), 22 deletions(-)
diff --git a/pyiceberg/table/update/validate.py b/pyiceberg/table/update/validate.py
index 32aabac28c..b49c4abe07 100644
--- a/pyiceberg/table/update/validate.py
+++ b/pyiceberg/table/update/validate.py
@@ -14,17 +14,19 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
-from typing import Iterator, Optional
+from typing import Iterator, Optional, Set
from pyiceberg.exceptions import ValidationException
from pyiceberg.expressions import BooleanExpression
from pyiceberg.expressions.visitors import ROWS_CANNOT_MATCH, _InclusiveMetricsEvaluator
from pyiceberg.manifest import ManifestContent, ManifestEntry, ManifestEntryStatus, ManifestFile
+from pyiceberg.schema import Schema
from pyiceberg.table import Table
from pyiceberg.table.snapshots import Operation, Snapshot, ancestors_between
from pyiceberg.typedef import Record
-VALIDATE_DATA_FILES_EXIST_OPERATIONS = {Operation.OVERWRITE, Operation.REPLACE, Operation.DELETE}
+VALIDATE_DATA_FILES_EXIST_OPERATIONS: Set[Operation] = {Operation.OVERWRITE, Operation.REPLACE, Operation.DELETE}
+VALIDATE_ADDED_DATA_FILES_OPERATIONS: Set[Operation] = {Operation.APPEND, Operation.OVERWRITE}
def _validation_history(
@@ -77,6 +79,47 @@ def _validation_history(
return manifests_files, snapshots
+def _filter_manifest_entries(
+ entry: ManifestEntry,
+ snapshot_ids: set[int],
+ data_filter: Optional[BooleanExpression],
+ partition_set: Optional[dict[int, set[Record]]],
+ entry_status: Optional[ManifestEntryStatus],
+ schema: Schema,
+) -> bool:
+ """Filter manifest entries based on data filter and partition set.
+
+ Args:
+ entry: Manifest entry to filter
+ snapshot_ids: set of snapshot ids to match data files
+ data_filter: Optional filter to match data files
+ partition_set: Optional set of partitions to match data files
+ entry_status: Optional status to match data files
+ schema: schema for filtering
+
+ Returns:
+ True if the entry should be included, False otherwise
+ """
+ if entry.snapshot_id not in snapshot_ids:
+ return False
+
+ if entry_status is not None and entry.status != entry_status:
+ return False
+
+ if data_filter is not None:
+ evaluator = _InclusiveMetricsEvaluator(schema, data_filter)
+ if evaluator.eval(entry.data_file) is ROWS_CANNOT_MATCH:
+ return False
+
+ if partition_set is not None:
+ partition = entry.data_file.partition
+ spec_id = entry.data_file.spec_id
+ if spec_id not in partition_set or partition not in partition_set[spec_id]:
+ return False
+
+ return True
+
+
def _deleted_data_files(
table: Table,
starting_snapshot: Snapshot,
@@ -108,27 +151,12 @@ def _deleted_data_files(
ManifestContent.DATA,
)
- if data_filter is not None:
- evaluator = _InclusiveMetricsEvaluator(table.schema(), data_filter).eval
-
for manifest in manifests:
for entry in manifest.fetch_manifest_entry(table.io, discard_deleted=False):
- if entry.snapshot_id not in snapshot_ids:
- continue
-
- if entry.status != ManifestEntryStatus.DELETED:
- continue
-
- if data_filter is not None and evaluator(entry.data_file) is ROWS_CANNOT_MATCH:
- continue
-
- if partition_set is not None:
- spec_id = entry.data_file.spec_id
- partition = entry.data_file.partition
- if spec_id not in partition_set or partition not in partition_set[spec_id]:
- continue
-
- yield entry
+ if _filter_manifest_entries(
+ entry, snapshot_ids, data_filter, partition_set, ManifestEntryStatus.DELETED, table.schema()
+ ):
+ yield entry
def _validate_deleted_data_files(
@@ -150,3 +178,60 @@ def _validate_deleted_data_files(
if any(conflicting_entries):
conflicting_snapshots = {entry.snapshot_id for entry in conflicting_entries}
raise ValidationException(f"Deleted data files were found matching the filter for snapshots {conflicting_snapshots}!")
+
+
+def _added_data_files(
+ table: Table,
+ starting_snapshot: Snapshot,
+ data_filter: Optional[BooleanExpression],
+ partition_set: Optional[dict[int, set[Record]]],
+ parent_snapshot: Optional[Snapshot],
+) -> Iterator[ManifestEntry]:
+ """Return manifest entries for data files added between the starting snapshot and parent snapshot.
+
+ Args:
+ table: Table to get the history from
+ starting_snapshot: Starting snapshot to get the history from
+ data_filter: Optional filter to match data files
+ partition_set: Optional set of partitions to match data files
+ parent_snapshot: Parent snapshot to get the history from
+
+ Returns:
+ Iterator of manifest entries for added data files matching the conditions
+ """
+ if parent_snapshot is None:
+ return
+
+ manifests, snapshot_ids = _validation_history(
+ table,
+ parent_snapshot,
+ starting_snapshot,
+ VALIDATE_ADDED_DATA_FILES_OPERATIONS,
+ ManifestContent.DATA,
+ )
+
+ for manifest in manifests:
+ for entry in manifest.fetch_manifest_entry(table.io):
+ if _filter_manifest_entries(entry, snapshot_ids, data_filter, partition_set, None, table.schema()):
+ yield entry
+
+
+def _validate_added_data_files(
+ table: Table,
+ starting_snapshot: Snapshot,
+ data_filter: Optional[BooleanExpression],
+ parent_snapshot: Optional[Snapshot],
+) -> None:
+ """Validate that no files matching a filter have been added to the table since a starting snapshot.
+
+ Args:
+ table: Table to validate
+ starting_snapshot: Snapshot current at the start of the operation
+ data_filter: Expression used to find added data files
+ parent_snapshot: Ending snapshot on the branch being validated
+
+ """
+ conflicting_entries = _added_data_files(table, starting_snapshot, data_filter, None, parent_snapshot)
+ if any(conflicting_entries):
+ conflicting_snapshots = {entry.snapshot_id for entry in conflicting_entries if entry.snapshot_id is not None}
+ raise ValidationException(f"Added data files were found matching the filter for snapshots {conflicting_snapshots}!")
diff --git a/tests/table/test_validate.py b/tests/table/test_validate.py
index 1da8718805..570f680860 100644
--- a/tests/table/test_validate.py
+++ b/tests/table/test_validate.py
@@ -25,7 +25,13 @@
from pyiceberg.manifest import ManifestContent, ManifestEntry, ManifestEntryStatus, ManifestFile
from pyiceberg.table import Table
from pyiceberg.table.snapshots import Operation, Snapshot, Summary
-from pyiceberg.table.update.validate import _deleted_data_files, _validate_deleted_data_files, _validation_history
+from pyiceberg.table.update.validate import (
+ _added_data_files,
+ _deleted_data_files,
+ _validate_added_data_files,
+ _validate_deleted_data_files,
+ _validation_history,
+)
@pytest.fixture
@@ -217,3 +223,130 @@ class DummyEntry:
data_filter=None,
parent_snapshot=oldest_snapshot,
)
+
+
+@pytest.mark.parametrize("operation", [Operation.APPEND, Operation.OVERWRITE])
+def test_validate_added_data_files_conflicting_count(
+ table_v2_with_extensive_snapshots_and_manifests: tuple[Table, dict[int, list[ManifestFile]]],
+ operation: Operation,
+) -> None:
+ table, mock_manifests = table_v2_with_extensive_snapshots_and_manifests
+
+ snapshot_history = 100
+ snapshots = table.snapshots()
+ for i in range(1, snapshot_history + 1):
+ altered_snapshot = snapshots[-i]
+ altered_snapshot = altered_snapshot.model_copy(update={"summary": Summary(operation=operation)})
+ snapshots[-i] = altered_snapshot
+
+ table.metadata = table.metadata.model_copy(
+ update={"snapshots": snapshots},
+ )
+
+ oldest_snapshot = table.snapshots()[-snapshot_history]
+ newest_snapshot = cast(Snapshot, table.current_snapshot())
+
+ def mock_read_manifest_side_effect(self: Snapshot, io: FileIO) -> list[ManifestFile]:
+ """Mock the manifests method to use the snapshot_id for lookup."""
+ snapshot_id = self.snapshot_id
+ if snapshot_id in mock_manifests:
+ return mock_manifests[snapshot_id]
+ return []
+
+ def mock_fetch_manifest_entry(self: ManifestFile, io: FileIO, discard_deleted: bool = True) -> list[ManifestEntry]:
+ return [
+ ManifestEntry.from_args(
+ status=ManifestEntryStatus.ADDED,
+ snapshot_id=self.added_snapshot_id,
+ )
+ ]
+
+ with (
+ patch("pyiceberg.table.snapshots.Snapshot.manifests", new=mock_read_manifest_side_effect),
+ patch("pyiceberg.manifest.ManifestFile.fetch_manifest_entry", new=mock_fetch_manifest_entry),
+ ):
+ result = list(
+ _added_data_files(
+ table=table,
+ starting_snapshot=newest_snapshot,
+ data_filter=None,
+ parent_snapshot=oldest_snapshot,
+ partition_set=None,
+ )
+ )
+
+ # since we only look at the ManifestContent.Data files
+ assert len(result) == snapshot_history / 2
+
+
+@pytest.mark.parametrize("operation", [Operation.DELETE, Operation.REPLACE])
+def test_validate_added_data_files_non_conflicting_count(
+ table_v2_with_extensive_snapshots_and_manifests: tuple[Table, dict[int, list[ManifestFile]]],
+ operation: Operation,
+) -> None:
+ table, mock_manifests = table_v2_with_extensive_snapshots_and_manifests
+
+ snapshot_history = 100
+ snapshots = table.snapshots()
+ for i in range(1, snapshot_history + 1):
+ altered_snapshot = snapshots[-i]
+ altered_snapshot = altered_snapshot.model_copy(update={"summary": Summary(operation=operation)})
+ snapshots[-i] = altered_snapshot
+
+ table.metadata = table.metadata.model_copy(
+ update={"snapshots": snapshots},
+ )
+
+ oldest_snapshot = table.snapshots()[-snapshot_history]
+ newest_snapshot = cast(Snapshot, table.current_snapshot())
+
+ def mock_read_manifest_side_effect(self: Snapshot, io: FileIO) -> list[ManifestFile]:
+ """Mock the manifests method to use the snapshot_id for lookup."""
+ snapshot_id = self.snapshot_id
+ if snapshot_id in mock_manifests:
+ return mock_manifests[snapshot_id]
+ return []
+
+ def mock_fetch_manifest_entry(self: ManifestFile, io: FileIO, discard_deleted: bool = True) -> list[ManifestEntry]:
+ return [
+ ManifestEntry.from_args(
+ status=ManifestEntryStatus.ADDED,
+ snapshot_id=self.added_snapshot_id,
+ )
+ ]
+
+ with (
+ patch("pyiceberg.table.snapshots.Snapshot.manifests", new=mock_read_manifest_side_effect),
+ patch("pyiceberg.manifest.ManifestFile.fetch_manifest_entry", new=mock_fetch_manifest_entry),
+ ):
+ result = list(
+ _added_data_files(
+ table=table,
+ starting_snapshot=newest_snapshot,
+ data_filter=None,
+ parent_snapshot=oldest_snapshot,
+ partition_set=None,
+ )
+ )
+
+ assert len(result) == 0
+
+
+def test_validate_added_data_files_raises_on_conflict(
+ table_v2_with_extensive_snapshots_and_manifests: tuple[Table, dict[int, list[ManifestFile]]],
+) -> None:
+ table, _ = table_v2_with_extensive_snapshots_and_manifests
+ oldest_snapshot = table.snapshots()[0]
+ newest_snapshot = cast(Snapshot, table.current_snapshot())
+
+ class DummyEntry:
+ snapshot_id = 123
+
+ with patch("pyiceberg.table.update.validate._added_data_files", return_value=[DummyEntry()]):
+ with pytest.raises(ValidationException):
+ _validate_added_data_files(
+ table=table,
+ starting_snapshot=newest_snapshot,
+ data_filter=None,
+ parent_snapshot=oldest_snapshot,
+ )
From f027a86dd1f514468b625d7b1026333b60979de2 Mon Sep 17 00:00:00 2001
From: gardenia
Date: Tue, 24 Jun 2025 21:01:33 +0100
Subject: [PATCH 091/112] Add `hive.kerberos-service-name` configuration option
(#2032) (#2141)
# Rationale for this change
Added new configuration parameter hive.kerberos-service-name (#2032)
hive.kerberos-service-name
Defaults to "hive"
# Are these changes tested?
added unit test.
# Are there any user-facing changes?
this change adds an optional configuration parameter for the hive
catalog (hive.kerberos-service-name) which defaults to "hive". the
change includes doc updates.
Co-authored-by: Colm Dougan
---
mkdocs/docs/configuration.md | 9 +++++----
pyiceberg/catalog/hive.py | 14 ++++++++++++--
tests/catalog/test_hive.py | 18 ++++++++++++++++--
3 files changed, 33 insertions(+), 8 deletions(-)
diff --git a/mkdocs/docs/configuration.md b/mkdocs/docs/configuration.md
index 922e60e1f8..bc514e39af 100644
--- a/mkdocs/docs/configuration.md
+++ b/mkdocs/docs/configuration.md
@@ -439,10 +439,11 @@ catalog:
s3.secret-access-key: password
```
-| Key | Example | Description |
-|------------------------------| ------- | --------------------------------- |
-| hive.hive2-compatible | true | Using Hive 2.x compatibility mode |
-| hive.kerberos-authentication | true | Using authentication via Kerberos |
+| Key | Example | Description |
+|------------------------------| ------- | ------------------------------------ |
+| hive.hive2-compatible | true | Using Hive 2.x compatibility mode |
+| hive.kerberos-authentication | true | Using authentication via Kerberos |
+| hive.kerberos-service-name | hive | Kerberos service name (default hive) |
When using Hive 2.x, make sure to set the compatibility flag:
diff --git a/pyiceberg/catalog/hive.py b/pyiceberg/catalog/hive.py
index 05dd408fc2..09437dd1b6 100644
--- a/pyiceberg/catalog/hive.py
+++ b/pyiceberg/catalog/hive.py
@@ -130,6 +130,8 @@
HIVE_KERBEROS_AUTH = "hive.kerberos-authentication"
HIVE_KERBEROS_AUTH_DEFAULT = False
+HIVE_KERBEROS_SERVICE_NAME = "hive.kerberos-service-name"
+HIVE_KERBEROS_SERVICE_NAME_DEFAULT = "hive"
LOCK_CHECK_MIN_WAIT_TIME = "lock-check-min-wait-time"
LOCK_CHECK_MAX_WAIT_TIME = "lock-check-max-wait-time"
@@ -149,9 +151,16 @@ class _HiveClient:
_transport: TTransport
_ugi: Optional[List[str]]
- def __init__(self, uri: str, ugi: Optional[str] = None, kerberos_auth: Optional[bool] = HIVE_KERBEROS_AUTH_DEFAULT):
+ def __init__(
+ self,
+ uri: str,
+ ugi: Optional[str] = None,
+ kerberos_auth: Optional[bool] = HIVE_KERBEROS_AUTH_DEFAULT,
+ kerberos_service_name: Optional[str] = HIVE_KERBEROS_SERVICE_NAME,
+ ):
self._uri = uri
self._kerberos_auth = kerberos_auth
+ self._kerberos_service_name = kerberos_service_name
self._ugi = ugi.split(":") if ugi else None
self._transport = self._init_thrift_transport()
@@ -161,7 +170,7 @@ def _init_thrift_transport(self) -> TTransport:
if not self._kerberos_auth:
return TTransport.TBufferedTransport(socket)
else:
- return TTransport.TSaslClientTransport(socket, host=url_parts.hostname, service="hive")
+ return TTransport.TSaslClientTransport(socket, host=url_parts.hostname, service=self._kerberos_service_name)
def _client(self) -> Client:
protocol = TBinaryProtocol.TBinaryProtocol(self._transport)
@@ -314,6 +323,7 @@ def _create_hive_client(properties: Dict[str, str]) -> _HiveClient:
uri,
properties.get("ugi"),
property_as_bool(properties, HIVE_KERBEROS_AUTH, HIVE_KERBEROS_AUTH_DEFAULT),
+ properties.get(HIVE_KERBEROS_SERVICE_NAME, HIVE_KERBEROS_SERVICE_NAME_DEFAULT),
)
except BaseException as e:
last_exception = e
diff --git a/tests/catalog/test_hive.py b/tests/catalog/test_hive.py
index fef0d6acc6..a36425ebea 100644
--- a/tests/catalog/test_hive.py
+++ b/tests/catalog/test_hive.py
@@ -48,6 +48,7 @@
DO_NOT_UPDATE_STATS,
DO_NOT_UPDATE_STATS_DEFAULT,
HIVE_KERBEROS_AUTH,
+ HIVE_KERBEROS_SERVICE_NAME,
LOCK_CHECK_MAX_WAIT_TIME,
LOCK_CHECK_MIN_WAIT_TIME,
LOCK_CHECK_RETRIES,
@@ -1300,7 +1301,20 @@ def test_create_hive_client_success() -> None:
with patch("pyiceberg.catalog.hive._HiveClient", return_value=MagicMock()) as mock_hive_client:
client = HiveCatalog._create_hive_client(properties)
- mock_hive_client.assert_called_once_with("thrift://localhost:10000", "user", False)
+ mock_hive_client.assert_called_once_with("thrift://localhost:10000", "user", False, "hive")
+ assert client is not None
+
+
+def test_create_hive_client_with_kerberos_success() -> None:
+ properties = {
+ "uri": "thrift://localhost:10000",
+ "ugi": "user",
+ HIVE_KERBEROS_AUTH: "true",
+ HIVE_KERBEROS_SERVICE_NAME: "hiveuser",
+ }
+ with patch("pyiceberg.catalog.hive._HiveClient", return_value=MagicMock()) as mock_hive_client:
+ client = HiveCatalog._create_hive_client(properties)
+ mock_hive_client.assert_called_once_with("thrift://localhost:10000", "user", True, "hiveuser")
assert client is not None
@@ -1313,7 +1327,7 @@ def test_create_hive_client_multiple_uris() -> None:
client = HiveCatalog._create_hive_client(properties)
assert mock_hive_client.call_count == 2
mock_hive_client.assert_has_calls(
- [call("thrift://localhost:10000", "user", False), call("thrift://localhost:10001", "user", False)]
+ [call("thrift://localhost:10000", "user", False, "hive"), call("thrift://localhost:10001", "user", False, "hive")]
)
assert client is not None
From cdd6e818acfde769b1325de7049151f6db550130 Mon Sep 17 00:00:00 2001
From: Carlo Cancellieri
Date: Thu, 26 Jun 2025 17:17:00 +0200
Subject: [PATCH 092/112] Relax REST client `/v1/config` response constraints
(#2148)
fixing #2122
# Rationale for this change
# Are these changes tested?
Yes tested locally
# Are there any user-facing changes?
Nope, just use them
---------
Co-authored-by: Kevin Liu
Co-authored-by: Fokko Driesprong
---
pyiceberg/catalog/rest/__init__.py | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/pyiceberg/catalog/rest/__init__.py b/pyiceberg/catalog/rest/__init__.py
index 3f59a196ea..6215d17a4f 100644
--- a/pyiceberg/catalog/rest/__init__.py
+++ b/pyiceberg/catalog/rest/__init__.py
@@ -178,8 +178,8 @@ class RegisterTableRequest(IcebergBaseModel):
class ConfigResponse(IcebergBaseModel):
- defaults: Properties = Field()
- overrides: Properties = Field()
+ defaults: Optional[Properties] = Field(default_factory=dict)
+ overrides: Optional[Properties] = Field(default_factory=dict)
class ListNamespaceResponse(IcebergBaseModel):
From aa9db672ae9c3bde8d37dcd44ff1f7e0d415df08 Mon Sep 17 00:00:00 2001
From: Kevin Liu
Date: Sun, 29 Jun 2025 01:15:08 -0400
Subject: [PATCH 093/112] dep: update `multidict`, previous version yanked
(#2157)
---
poetry.lock | 254 ++++++++++++++++++++++++++--------------------------
1 file changed, 127 insertions(+), 127 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 0783784472..22adbe43d5 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -59,7 +59,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -72,7 +72,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5421af8f22a98f640261ee48aae3a37f0c41371e99412d55eaf2f8a46d5dad29"},
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0fcda86f6cb318ba36ed8f1396a6a4a3fd8f856f84d426584392083d10da4de0"},
@@ -202,7 +202,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -268,7 +268,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -285,7 +285,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -488,7 +488,7 @@ files = [
{file = "boto3-1.38.27-py3-none-any.whl", hash = "sha256:95f5fe688795303a8a15e8b7e7f255cadab35eae459d00cc281a4fd77252ea80"},
{file = "boto3-1.38.27.tar.gz", hash = "sha256:94bd7fdd92d5701b362d4df100d21e28f8307a67ff56b6a8b0398119cf22f859"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.38.27,<1.39.0"
@@ -509,7 +509,7 @@ files = [
{file = "botocore-1.38.27-py3-none-any.whl", hash = "sha256:a785d5e9a5eda88ad6ab9ed8b87d1f2ac409d0226bba6ff801c55359e94d91a8"},
{file = "botocore-1.38.27.tar.gz", hash = "sha256:9788f7efe974328a38cbade64cc0b1e67d27944b899f88cb786ae362973133b6"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -1473,7 +1473,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cc4df77d638aa2ed703b878dd093725b72a824c3c546c076e8fdf276f78ee84a"},
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:716a9973a2cc963160394f701964fe25012600f3d311f60c790400b00e568b61"},
@@ -2247,7 +2247,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -3032,123 +3032,123 @@ files = [
[[package]]
name = "multidict"
-version = "6.5.0"
+version = "6.6.2"
description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
-files = [
- {file = "multidict-6.5.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:2e118a202904623b1d2606d1c8614e14c9444b59d64454b0c355044058066469"},
- {file = "multidict-6.5.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:a42995bdcaff4e22cb1280ae7752c3ed3fbb398090c6991a2797a4a0e5ed16a9"},
- {file = "multidict-6.5.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2261b538145723ca776e55208640fffd7ee78184d223f37c2b40b9edfe0e818a"},
- {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0e5b19f8cd67235fab3e195ca389490415d9fef5a315b1fa6f332925dc924262"},
- {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:177b081e4dec67c3320b16b3aa0babc178bbf758553085669382c7ec711e1ec8"},
- {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4d30a2cc106a7d116b52ee046207614db42380b62e6b1dd2a50eba47c5ca5eb1"},
- {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a72933bc308d7a64de37f0d51795dbeaceebdfb75454f89035cdfc6a74cfd129"},
- {file = "multidict-6.5.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:96d109e663d032280ef8ef62b50924b2e887d5ddf19e301844a6cb7e91a172a6"},
- {file = "multidict-6.5.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b555329c9894332401f03b9a87016f0b707b6fccd4706793ec43b4a639e75869"},
- {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:6994bad9d471ef2156f2b6850b51e20ee409c6b9deebc0e57be096be9faffdce"},
- {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:b15f817276c96cde9060569023808eec966bd8da56a97e6aa8116f34ddab6534"},
- {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b4bf507c991db535a935b2127cf057a58dbc688c9f309c72080795c63e796f58"},
- {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:60c3f8f13d443426c55f88cf3172547bbc600a86d57fd565458b9259239a6737"},
- {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:a10227168a24420c158747fc201d4279aa9af1671f287371597e2b4f2ff21879"},
- {file = "multidict-6.5.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:e3b1425fe54ccfde66b8cfb25d02be34d5dfd2261a71561ffd887ef4088b4b69"},
- {file = "multidict-6.5.0-cp310-cp310-win32.whl", hash = "sha256:b4e47ef51237841d1087e1e1548071a6ef22e27ed0400c272174fa585277c4b4"},
- {file = "multidict-6.5.0-cp310-cp310-win_amd64.whl", hash = "sha256:63b3b24fadc7067282c88fae5b2f366d5b3a7c15c021c2838de8c65a50eeefb4"},
- {file = "multidict-6.5.0-cp310-cp310-win_arm64.whl", hash = "sha256:8b2d61afbafc679b7eaf08e9de4fa5d38bd5dc7a9c0a577c9f9588fb49f02dbb"},
- {file = "multidict-6.5.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:8b4bf6bb15a05796a07a248084e3e46e032860c899c7a9b981030e61368dba95"},
- {file = "multidict-6.5.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:46bb05d50219655c42a4b8fcda9c7ee658a09adbb719c48e65a20284e36328ea"},
- {file = "multidict-6.5.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:54f524d73f4d54e87e03c98f6af601af4777e4668a52b1bd2ae0a4d6fc7b392b"},
- {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:529b03600466480ecc502000d62e54f185a884ed4570dee90d9a273ee80e37b5"},
- {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:69ad681ad7c93a41ee7005cc83a144b5b34a3838bcf7261e2b5356057b0f78de"},
- {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3fe9fada8bc0839466b09fa3f6894f003137942984843ec0c3848846329a36ae"},
- {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f94c6ea6405fcf81baef1e459b209a78cda5442e61b5b7a57ede39d99b5204a0"},
- {file = "multidict-6.5.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84ca75ad8a39ed75f079a8931435a5b51ee4c45d9b32e1740f99969a5d1cc2ee"},
- {file = "multidict-6.5.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:be4c08f3a2a6cc42b414496017928d95898964fed84b1b2dace0c9ee763061f9"},
- {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:046a7540cfbb4d5dc846a1fd9843f3ba980c6523f2e0c5b8622b4a5c94138ae6"},
- {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:64306121171d988af77d74be0d8c73ee1a69cf6f96aea7fa6030c88f32a152dd"},
- {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:b4ac1dd5eb0ecf6f7351d5a9137f30a83f7182209c5d37f61614dfdce5714853"},
- {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:bab4a8337235365f4111a7011a1f028826ca683834ebd12de4b85e2844359c36"},
- {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:a05b5604c5a75df14a63eeeca598d11b2c3745b9008539b70826ea044063a572"},
- {file = "multidict-6.5.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:67c4a640952371c9ca65b6a710598be246ef3be5ca83ed38c16a7660d3980877"},
- {file = "multidict-6.5.0-cp311-cp311-win32.whl", hash = "sha256:fdeae096ca36c12d8aca2640b8407a9d94e961372c68435bef14e31cce726138"},
- {file = "multidict-6.5.0-cp311-cp311-win_amd64.whl", hash = "sha256:e2977ef8b7ce27723ee8c610d1bd1765da4f3fbe5a64f9bf1fd3b4770e31fbc0"},
- {file = "multidict-6.5.0-cp311-cp311-win_arm64.whl", hash = "sha256:82d0cf0ea49bae43d9e8c3851e21954eff716259ff42da401b668744d1760bcb"},
- {file = "multidict-6.5.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:1bb986c8ea9d49947bc325c51eced1ada6d8d9b4c5b15fd3fcdc3c93edef5a74"},
- {file = "multidict-6.5.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:03c0923da300120830fc467e23805d63bbb4e98b94032bd863bc7797ea5fa653"},
- {file = "multidict-6.5.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:4c78d5ec00fdd35c91680ab5cf58368faad4bd1a8721f87127326270248de9bc"},
- {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aadc3cb78be90a887f8f6b73945b840da44b4a483d1c9750459ae69687940c97"},
- {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:5b02e1ca495d71e07e652e4cef91adae3bf7ae4493507a263f56e617de65dafc"},
- {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7fe92a62326eef351668eec4e2dfc494927764a0840a1895cff16707fceffcd3"},
- {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7673ee4f63879ecd526488deb1989041abcb101b2d30a9165e1e90c489f3f7fb"},
- {file = "multidict-6.5.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa097ae2a29f573de7e2d86620cbdda5676d27772d4ed2669cfa9961a0d73955"},
- {file = "multidict-6.5.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:300da0fa4f8457d9c4bd579695496116563409e676ac79b5e4dca18e49d1c308"},
- {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:9a19bd108c35877b57393243d392d024cfbfdefe759fd137abb98f6fc910b64c"},
- {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:0f32a1777465a35c35ddbbd7fc1293077938a69402fcc59e40b2846d04a120dd"},
- {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:9cc1e10c14ce8112d1e6d8971fe3cdbe13e314f68bea0e727429249d4a6ce164"},
- {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:e95c5e07a06594bdc288117ca90e89156aee8cb2d7c330b920d9c3dd19c05414"},
- {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:40ff26f58323795f5cd2855e2718a1720a1123fb90df4553426f0efd76135462"},
- {file = "multidict-6.5.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:76803a29fd71869a8b59c2118c9dcfb3b8f9c8723e2cce6baeb20705459505cf"},
- {file = "multidict-6.5.0-cp312-cp312-win32.whl", hash = "sha256:df7ecbc65a53a2ce1b3a0c82e6ad1a43dcfe7c6137733f9176a92516b9f5b851"},
- {file = "multidict-6.5.0-cp312-cp312-win_amd64.whl", hash = "sha256:0ec1c3fbbb0b655a6540bce408f48b9a7474fd94ed657dcd2e890671fefa7743"},
- {file = "multidict-6.5.0-cp312-cp312-win_arm64.whl", hash = "sha256:2d24a00d34808b22c1f15902899b9d82d0faeca9f56281641c791d8605eacd35"},
- {file = "multidict-6.5.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:53d92df1752df67a928fa7f884aa51edae6f1cf00eeb38cbcf318cf841c17456"},
- {file = "multidict-6.5.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:680210de2c38eef17ce46b8df8bf2c1ece489261a14a6e43c997d49843a27c99"},
- {file = "multidict-6.5.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:e279259bcb936732bfa1a8eec82b5d2352b3df69d2fa90d25808cfc403cee90a"},
- {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d1c185fc1069781e3fc8b622c4331fb3b433979850392daa5efbb97f7f9959bb"},
- {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:6bb5f65ff91daf19ce97f48f63585e51595539a8a523258b34f7cef2ec7e0617"},
- {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d8646b4259450c59b9286db280dd57745897897284f6308edbdf437166d93855"},
- {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d245973d4ecc04eea0a8e5ebec7882cf515480036e1b48e65dffcfbdf86d00be"},
- {file = "multidict-6.5.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a133e7ddc9bc7fb053733d0ff697ce78c7bf39b5aec4ac12857b6116324c8d75"},
- {file = "multidict-6.5.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:80d696fa38d738fcebfd53eec4d2e3aeb86a67679fd5e53c325756682f152826"},
- {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:20d30c9410ac3908abbaa52ee5967a754c62142043cf2ba091e39681bd51d21a"},
- {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:6c65068cc026f217e815fa519d8e959a7188e94ec163ffa029c94ca3ef9d4a73"},
- {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:e355ac668a8c3e49c2ca8daa4c92f0ad5b705d26da3d5af6f7d971e46c096da7"},
- {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:08db204213d0375a91a381cae0677ab95dd8c67a465eb370549daf6dbbf8ba10"},
- {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:ffa58e3e215af8f6536dc837a990e456129857bb6fd546b3991be470abd9597a"},
- {file = "multidict-6.5.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:3e86eb90015c6f21658dbd257bb8e6aa18bdb365b92dd1fba27ec04e58cdc31b"},
- {file = "multidict-6.5.0-cp313-cp313-win32.whl", hash = "sha256:f34a90fbd9959d0f857323bd3c52b3e6011ed48f78d7d7b9e04980b8a41da3af"},
- {file = "multidict-6.5.0-cp313-cp313-win_amd64.whl", hash = "sha256:fcb2aa79ac6aef8d5b709bbfc2fdb1d75210ba43038d70fbb595b35af470ce06"},
- {file = "multidict-6.5.0-cp313-cp313-win_arm64.whl", hash = "sha256:6dcee5e7e92060b4bb9bb6f01efcbb78c13d0e17d9bc6eec71660dd71dc7b0c2"},
- {file = "multidict-6.5.0-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:cbbc88abea2388fde41dd574159dec2cda005cb61aa84950828610cb5010f21a"},
- {file = "multidict-6.5.0-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:70b599f70ae6536e5976364d3c3cf36f40334708bd6cebdd1e2438395d5e7676"},
- {file = "multidict-6.5.0-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:828bab777aa8d29d59700018178061854e3a47727e0611cb9bec579d3882de3b"},
- {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a9695fc1462f17b131c111cf0856a22ff154b0480f86f539d24b2778571ff94d"},
- {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:0b5ac6ebaf5d9814b15f399337ebc6d3a7f4ce9331edd404e76c49a01620b68d"},
- {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:84a51e3baa77ded07be4766a9e41d977987b97e49884d4c94f6d30ab6acaee14"},
- {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8de67f79314d24179e9b1869ed15e88d6ba5452a73fc9891ac142e0ee018b5d6"},
- {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:17f78a52c214481d30550ec18208e287dfc4736f0c0148208334b105fd9e0887"},
- {file = "multidict-6.5.0-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2966d0099cb2e2039f9b0e73e7fd5eb9c85805681aa2a7f867f9d95b35356921"},
- {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:86fb42ed5ed1971c642cc52acc82491af97567534a8e381a8d50c02169c4e684"},
- {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:4e990cbcb6382f9eae4ec720bcac6a1351509e6fc4a5bb70e4984b27973934e6"},
- {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:d99a59d64bb1f7f2117bec837d9e534c5aeb5dcedf4c2b16b9753ed28fdc20a3"},
- {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:e8ef15cc97c9890212e1caf90f0d63f6560e1e101cf83aeaf63a57556689fb34"},
- {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:b8a09aec921b34bd8b9f842f0bcfd76c6a8c033dc5773511e15f2d517e7e1068"},
- {file = "multidict-6.5.0-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:ff07b504c23b67f2044533244c230808a1258b3493aaf3ea2a0785f70b7be461"},
- {file = "multidict-6.5.0-cp313-cp313t-win32.whl", hash = "sha256:9232a117341e7e979d210e41c04e18f1dc3a1d251268df6c818f5334301274e1"},
- {file = "multidict-6.5.0-cp313-cp313t-win_amd64.whl", hash = "sha256:44cb5c53fb2d4cbcee70a768d796052b75d89b827643788a75ea68189f0980a1"},
- {file = "multidict-6.5.0-cp313-cp313t-win_arm64.whl", hash = "sha256:51d33fafa82640c0217391d4ce895d32b7e84a832b8aee0dcc1b04d8981ec7f4"},
- {file = "multidict-6.5.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:c0078358470da8dc90c37456f4a9cde9f86200949a048d53682b9cd21e5bbf2b"},
- {file = "multidict-6.5.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:5cc7968b7d1bf8b973c307d38aa3a2f2c783f149bcac855944804252f1df5105"},
- {file = "multidict-6.5.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:0ad73a60e11aa92f1f2c9330efdeaac4531b719fc568eb8d312fd4112f34cc18"},
- {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3233f21abdcd180b2624eb6988a1e1287210e99bca986d8320afca5005d85844"},
- {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:bee5c0b79fca78fd2ab644ca4dc831ecf793eb6830b9f542ee5ed2c91bc35a0e"},
- {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e053a4d690f4352ce46583080fefade9a903ce0fa9d820db1be80bdb9304fa2f"},
- {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:42bdee30424c1f4dcda96e07ac60e2a4ede8a89f8ae2f48b5e4ccc060f294c52"},
- {file = "multidict-6.5.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:58b2ded1a7982cf7b8322b0645713a0086b2b3cf5bb9f7c01edfc1a9f98d20dc"},
- {file = "multidict-6.5.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3f805b8b951d1fadc5bc18c3c93e509608ac5a883045ee33bc22e28806847c20"},
- {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:2540395b63723da748f850568357a39cd8d8d4403ca9439f9fcdad6dd423c780"},
- {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:c96aedff25f4e47b6697ba048b2c278f7caa6df82c7c3f02e077bcc8d47b4b76"},
- {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:e80de5ad995de210fd02a65c2350649b8321d09bd2e44717eaefb0f5814503e8"},
- {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:6cb9bcedd9391b313e5ec2fb3aa07c03e050550e7b9e4646c076d5c24ba01532"},
- {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:a7d130ed7a112e25ab47309962ecafae07d073316f9d158bc7b3936b52b80121"},
- {file = "multidict-6.5.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:95750a9a9741cd1855d1b6cb4c6031ae01c01ad38d280217b64bfae986d39d56"},
- {file = "multidict-6.5.0-cp39-cp39-win32.whl", hash = "sha256:7f78caf409914f108f4212b53a9033abfdc2cbab0647e9ac3a25bb0f21ab43d2"},
- {file = "multidict-6.5.0-cp39-cp39-win_amd64.whl", hash = "sha256:220c74009507e847a3a6fc5375875f2a2e05bd9ce28cf607be0e8c94600f4472"},
- {file = "multidict-6.5.0-cp39-cp39-win_arm64.whl", hash = "sha256:d98f4ac9c1ede7e9d04076e2e6d967e15df0079a6381b297270f6bcab661195e"},
- {file = "multidict-6.5.0-py3-none-any.whl", hash = "sha256:5634b35f225977605385f56153bd95a7133faffc0ffe12ad26e10517537e8dfc"},
- {file = "multidict-6.5.0.tar.gz", hash = "sha256:942bd8002492ba819426a8d7aefde3189c1b87099cdf18aaaefefcf7f3f7b6d2"},
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+files = [
+ {file = "multidict-6.6.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cfd9c74d337e710d7ee26e72a7dbedbd60e0c58d3df7c5ccbb748857e977783c"},
+ {file = "multidict-6.6.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9d2c5867a1bd182041a950e9ec3dd3622926260434655bd5d94a62d889100787"},
+ {file = "multidict-6.6.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:bc8551dd0000ce3f1d909906415ec18970fedb78e685dcac3a0b331a3422d810"},
+ {file = "multidict-6.6.2-cp310-cp310-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:9a23d9360f656c316518c8534685ca7c9f18877f782c11bcfce97ff1012ba256"},
+ {file = "multidict-6.6.2-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:37fe64cfc6f73fce34f2ef9e099efb8333650b85b50929ba37789311283f476f"},
+ {file = "multidict-6.6.2-cp310-cp310-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:2429b25566ff8c12cdf472ee82084ea96ea085675822d6d85aee85efd1d36cc0"},
+ {file = "multidict-6.6.2-cp310-cp310-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:66c596bd9bc833bad98445539ad53165b214c2c87bf386dbb819fabd1acdb462"},
+ {file = "multidict-6.6.2-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:eb8c006b565a0e53b298e9d48ef5aafe343f77de65c4fa7adb3d3b752a22d10b"},
+ {file = "multidict-6.6.2-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:7d09a7ade505d4556aa00c18f5635c9e7fe5973b98fee4e034162b02e48da7bc"},
+ {file = "multidict-6.6.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:6c95573274630213748ecee465410d4e5e44532d97ba9b09481968efd3c1fd2c"},
+ {file = "multidict-6.6.2-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:e45ebeeee1ce0f9a68151cee1afe02eef56f3b6977a580873c179175e5108275"},
+ {file = "multidict-6.6.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:329ecbdd77402648ebcb077b342ad6e67396dcf377c67418a733e88476ff3a11"},
+ {file = "multidict-6.6.2-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:f449699e273579a7eda79e36a8b7a6aae06a601d115c54e1aeebf08e07ea3ea1"},
+ {file = "multidict-6.6.2-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:ed4bb96a59976e4da7e1fbe3a7c37bcb4a16f3b20c5bba8af9a0ce459e14039a"},
+ {file = "multidict-6.6.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:4d05496c2779af4698ba8a841b226247a9a515210eff3a029f48d5345255b1d3"},
+ {file = "multidict-6.6.2-cp310-cp310-win32.whl", hash = "sha256:f96af5fbf6bab448d6dab34e8126f32f86de65034539d4a7077193f7b64a08f6"},
+ {file = "multidict-6.6.2-cp310-cp310-win_amd64.whl", hash = "sha256:44468089034383be86735f64f5d7daa6a1297e338b739403871a63750b95866d"},
+ {file = "multidict-6.6.2-cp310-cp310-win_arm64.whl", hash = "sha256:4e36b00dfb630a81f8efd4eb8a67b5b45f0918da3f2c8c4c14d16fc12b682d33"},
+ {file = "multidict-6.6.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:f89ce85a8f949e4dde3a4fb374500630cf425a32d8c53a2892b8fbfc556720dc"},
+ {file = "multidict-6.6.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:73d6c5bed784d70e790b6f014c1a9456e76401ab32bdacbf8bf4a6c2f1654b12"},
+ {file = "multidict-6.6.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a94cff417557f5a665e81c642e88d0062d19203a6b470b84fd62007d9c4c23c6"},
+ {file = "multidict-6.6.2-cp311-cp311-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:ebe5fc33f384c0277d3e9ecfc2f1fb3a5d2e6ac03deda39b84cf97611857241a"},
+ {file = "multidict-6.6.2-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:9723d0ba4d4eb50e7abcc235d4db5ce562c6cc20caa902967d8d915ccd73cd75"},
+ {file = "multidict-6.6.2-cp311-cp311-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:fcb85c7bedeaace74fbb631186789a23706bf22a44224b36340576370b3d9fd5"},
+ {file = "multidict-6.6.2-cp311-cp311-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:7d037a8777504c29cb65b90b6bff6c5bf4e5129487a3f7e4517ae97c3dcf675e"},
+ {file = "multidict-6.6.2-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:d082bf71da51352247b94622c9c8fd905b22bd4a7e94e252d4336492f595abe5"},
+ {file = "multidict-6.6.2-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:ce75c8986b17bc35f22240b7a4d9561530551e1f715b6cc23e136e7c06e6922a"},
+ {file = "multidict-6.6.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:f38e7b738de36f7caa707add650555d9bc01af9276000ba0427fda08153861f7"},
+ {file = "multidict-6.6.2-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:3e88f5cf86bbe5303d056cec9ccc854408571a3f6d41ce90ca9fdc2df47e3810"},
+ {file = "multidict-6.6.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:2b62ab7680362cfaf03059f7cb680d3bacd7d8adf176b0a86045e882486bbffc"},
+ {file = "multidict-6.6.2-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:5b39f3c6c22f4f227fe8994eb916ea12e4d46f65fdbc2d977d20750a85be581b"},
+ {file = "multidict-6.6.2-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:b256569fc1f8b52d0c3280503f2b5031e90cdb4469f67bfef32f53ef2b07922d"},
+ {file = "multidict-6.6.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:32791619e6b41cce97b6510fb68ee215c75012aa3c53778057a538ae473d3abd"},
+ {file = "multidict-6.6.2-cp311-cp311-win32.whl", hash = "sha256:7eb520abcf9f0e983fa4c666586161a2137d78096546ea19744b50ceae1bc401"},
+ {file = "multidict-6.6.2-cp311-cp311-win_amd64.whl", hash = "sha256:d2f765d6793815a82e517a1095c02ccbeab47f0a98fc244fe551625c519ade4d"},
+ {file = "multidict-6.6.2-cp311-cp311-win_arm64.whl", hash = "sha256:1d465e6cbbcadcbb8b3b08827fe9af889cc035500764b313aafb82c7e8e4b0a9"},
+ {file = "multidict-6.6.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:6e5e426aff6b5b32167b1185909ea390e51e59c7a6adfe65de16746e5739d8c1"},
+ {file = "multidict-6.6.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:0c4617af352d3e03b5febd040100d1bba67ac844e0f7780c8a124358883119dd"},
+ {file = "multidict-6.6.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:65854da6c2f065f7e52c4385727494d72b25eaf4e901b15fb3f61e21bb0b52eb"},
+ {file = "multidict-6.6.2-cp312-cp312-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:c477d3bc9a1aa0214f5639a8c1b4a6b3cd9faea5a861b4001a6df62294dcc952"},
+ {file = "multidict-6.6.2-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:d19a4bc7c5c1a25424812a26e8dccb18fff65a5f24515d2f3b85302ca3f3914f"},
+ {file = "multidict-6.6.2-cp312-cp312-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:54ffaf44483b082602c1e1831472e3369c005f092271dbbcad2f7b12d1e84019"},
+ {file = "multidict-6.6.2-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:a526df4fe21d2dc68265c046685042fc89187dc40754489f32f7efc05e264b0f"},
+ {file = "multidict-6.6.2-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:73e8763061f0a38cec6d084b696964ee7b7e50c10c89a64b20be7044dca36a74"},
+ {file = "multidict-6.6.2-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:81a13031b6831e4de5a89fe391d7f8b60a929f2d22dad403c69d60853e5ba1ca"},
+ {file = "multidict-6.6.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6421f4d5acd632af56ae90906755b242e518d59f5313a7b41cd55fb59adfcd74"},
+ {file = "multidict-6.6.2-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:3fed2cfff9d8e3316fc4c5aca40f33d7cd29b5a9a4cbf4aa17dfcae592ccb17c"},
+ {file = "multidict-6.6.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:eb97a4eed516fb3d63028fc0a8a8661e1acdf7925eace9c85153ff967926331c"},
+ {file = "multidict-6.6.2-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:9bbef50bfefe84213b791c9a326d3221fa31181ba39576049a55c1eef9768109"},
+ {file = "multidict-6.6.2-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:1d7d15b9285d439c3ca80b97b0ed6cc98a2df22c481de1848b77117563ddba14"},
+ {file = "multidict-6.6.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:5244c5c12889d84b9b7bf22f875e52c5ba4daa89c8ab92719863a14cd76dd04d"},
+ {file = "multidict-6.6.2-cp312-cp312-win32.whl", hash = "sha256:a2ec0e52d7b298d53983cc4987fe76a25e033305f58d597fbcc1ff139b5e417e"},
+ {file = "multidict-6.6.2-cp312-cp312-win_amd64.whl", hash = "sha256:96d2d55c01ce4ec000a1b6eadbaa22971c91ec512819abee8b5b13f4af3fd566"},
+ {file = "multidict-6.6.2-cp312-cp312-win_arm64.whl", hash = "sha256:a0af3b15eab84e0d4f62a365927070d7f200db7efb8bb1e17de7c14fab5183bb"},
+ {file = "multidict-6.6.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:c7f285ab85059a75b053027231626aeeabb4432191420d5c83cd91e2e462d25e"},
+ {file = "multidict-6.6.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:6f6fae153d1c78c37c2203b46c3062e942297eede21ebabea15fbfcaa7fa51be"},
+ {file = "multidict-6.6.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:2c051b7191d185fc7add9cdc52326acf93791884e51062605da0ff4371f679a1"},
+ {file = "multidict-6.6.2-cp313-cp313-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:dc609fb7fadeb0e2d3a752dcfbb62fc23a2d5cc88316fe199366f73aa74a3215"},
+ {file = "multidict-6.6.2-cp313-cp313-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:5245f0cba904051470c1b9c281533301457bc952992ac0d71a8668c2d10a4134"},
+ {file = "multidict-6.6.2-cp313-cp313-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:4409bbc4595a182908b539b156f882a5d72688a91343e0d3b0782c5cf1e85322"},
+ {file = "multidict-6.6.2-cp313-cp313-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:8ebd518dcc942616a539310e5deb0c29fbf4d0efa80de60186b53a2905192997"},
+ {file = "multidict-6.6.2-cp313-cp313-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:a50596d0b2f950015f80d277b42922cf30e0e1fb2348af3da5f4a07808f2c385"},
+ {file = "multidict-6.6.2-cp313-cp313-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:c6c2d7686d2f9ecb18192455aa04345a05646f45a286d67b31b438eaf749a91e"},
+ {file = "multidict-6.6.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:140fe62aaac9c401918a6cc5919afb99c1c676c6b449f58a6702db8ed333f394"},
+ {file = "multidict-6.6.2-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:2a7ea17e5d3cab04047b8880da1224a124c1ee7a8703dddce2cb66e6931c70f3"},
+ {file = "multidict-6.6.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:a88f0fab41095ff860987a4f4f2c12bf96193c0bce8b59f574c20c46542a4e5a"},
+ {file = "multidict-6.6.2-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:35c3c207c03c3d81d25afaa99df9f04158011d069a0716bbfc1c37e1006bab7c"},
+ {file = "multidict-6.6.2-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:845b5d2f1a8f4a9c4115ef84ab4f6cd4f35dbc2cebd5ab0a3d84d79510b3a27c"},
+ {file = "multidict-6.6.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:ac9cf00f77d7bb3c796a08812a9a4fcad052a313373a5e294b3fb1c9efe042fd"},
+ {file = "multidict-6.6.2-cp313-cp313-win32.whl", hash = "sha256:21a6477462132909c072671b51e74786eb6b9ce267257e305a7c924df79838a6"},
+ {file = "multidict-6.6.2-cp313-cp313-win_amd64.whl", hash = "sha256:09dc7f1bfb1571bfed0c32f8f66e2065a48488ed0da5b58de7a9be58079c95e6"},
+ {file = "multidict-6.6.2-cp313-cp313-win_arm64.whl", hash = "sha256:7a7748bffbfd398bd3e82cbb1c78dcf91f1dd67d1a23388f508adfb606cd4d77"},
+ {file = "multidict-6.6.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:0d2471a261c977b71b9bf1f3bb2aab1dc8a807a43f018a9d5fb420723fa9c77e"},
+ {file = "multidict-6.6.2-cp313-cp313t-macosx_10_13_x86_64.whl", hash = "sha256:03f20dfe1e118ce85467b88a2cab5e93fd750489731b3dd7a550d1da27238d80"},
+ {file = "multidict-6.6.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:7f40cff60aa06eec355988f35670057fa843893652648b658e6fa3402a725d72"},
+ {file = "multidict-6.6.2-cp313-cp313t-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:aaecf4537cce47944b7eb142ba047e031650cb2150d5692d49133f58d7d8fcbf"},
+ {file = "multidict-6.6.2-cp313-cp313t-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:5fbadc2c5c7bc633212e65df2908485679fa791b309c6636eafbd111c353af3d"},
+ {file = "multidict-6.6.2-cp313-cp313t-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:a914d3b7d4f4f22d34588b5af75ddb496a9947f67b2a8a4ea515d23118d338b0"},
+ {file = "multidict-6.6.2-cp313-cp313t-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:57dc09924085acf2bba3923d6f043a90fd527dac5f05874e3f740f45f1ca1c3c"},
+ {file = "multidict-6.6.2-cp313-cp313t-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:2a00b5f20e601edc82759734c6b64cd68795c1a9691735b599cba51c5d406914"},
+ {file = "multidict-6.6.2-cp313-cp313t-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:dcfa8cafb01c892d3dae719083127af0cf2e3eb0ce4c73b33c9f88af60f2a313"},
+ {file = "multidict-6.6.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:bd7a2bf9148f0dcab31207bb6c92522043b1781b43f5053c08f29a755cb5c91b"},
+ {file = "multidict-6.6.2-cp313-cp313t-musllinux_1_2_armv7l.whl", hash = "sha256:db34ee9ebe45126fc7f90206531704ac0d2da268101180886380fe601bffe124"},
+ {file = "multidict-6.6.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:b7c25f8e195d4fe34270208a568c843cfc85b2906ae20600ea8bbb2248ea9774"},
+ {file = "multidict-6.6.2-cp313-cp313t-musllinux_1_2_ppc64le.whl", hash = "sha256:b8fb447ff4ebe073c2f4e250d9f253712f1b6eb8f2830d4f09942f50359d85ff"},
+ {file = "multidict-6.6.2-cp313-cp313t-musllinux_1_2_s390x.whl", hash = "sha256:f4d5732f6bb3bf875fffbc9b155ab2c3b65924405d76fde6ea6c21253eab58c7"},
+ {file = "multidict-6.6.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:b33458d93a8c06e028ffe0ca155852b0cf2078b90081e791a7aafe9380a3ee2b"},
+ {file = "multidict-6.6.2-cp313-cp313t-win32.whl", hash = "sha256:43a7ddcf8f1e7ccae2197745152d4f97bb22b1b21afec05e271751dae56a576e"},
+ {file = "multidict-6.6.2-cp313-cp313t-win_amd64.whl", hash = "sha256:6d0d1dbbe970870e23a198d2b62f81cc0b145cca3eea1ba60670125a3184561c"},
+ {file = "multidict-6.6.2-cp313-cp313t-win_arm64.whl", hash = "sha256:3ead8284906e416fd990d44e964286393096fe0c0eedd4102fbc3a935250172a"},
+ {file = "multidict-6.6.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:6f6abb40cdee1a74b4f60c6c73c5206cd6b0c7b201ee88bea5386d793c55ffa5"},
+ {file = "multidict-6.6.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:e3a33a028a5653d73b28a4cd01a089f0be3138bd8fad906eb74225f2ed965bed"},
+ {file = "multidict-6.6.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:48c320085d29f2dc8cbc99f76a7bdb86a3d7fefc1c8c1069de4bd95334246308"},
+ {file = "multidict-6.6.2-cp39-cp39-manylinux1_i686.manylinux2014_i686.manylinux_2_17_i686.manylinux_2_5_i686.whl", hash = "sha256:ef0451c94925832b781b420d23305a69e1d422ee4c31f212f5c7763dce9ae138"},
+ {file = "multidict-6.6.2-cp39-cp39-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:f01091a5e0c5a362532edaa838f2215810764be5cc16520f3ca72f33aa080993"},
+ {file = "multidict-6.6.2-cp39-cp39-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:de9ea1af48ed753f7098ccab23ffcd8ca19cdd9731ce14affb901ef746f13ae3"},
+ {file = "multidict-6.6.2-cp39-cp39-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:e791e0650f48791acdd30a5210a97653c65d13e45b69664f80a831f4ee57aaaa"},
+ {file = "multidict-6.6.2-cp39-cp39-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:3eb3c2a4afce00fb2b4338f3d22a9babb8b7117aa6ed7e7297d300ba4584ea38"},
+ {file = "multidict-6.6.2-cp39-cp39-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:5ffa6be22fbf76b1d805c614c323b7da095faed7996e189fd77028bb3fb12e16"},
+ {file = "multidict-6.6.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:135925116dfb07f3b6e1625d3af26a00eb5db3e08fe9fe1c116a89dbdb3c74ff"},
+ {file = "multidict-6.6.2-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:e4625a9c9aa1427ee98d43042cefdb83102c336b1295aa683e83b045885a3f91"},
+ {file = "multidict-6.6.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:a928b34b9cc291c6825b217b5b4846d6bc71fddadfcbdd93b4153c6c510d084d"},
+ {file = "multidict-6.6.2-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:41c799c55b790c41fc331fbbc4763fc47090bfa773d45a0b5748a71453d0b11b"},
+ {file = "multidict-6.6.2-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:fe71c7b1763b26f6b368f9e45bf9a1116388d073931bd37a1ed5951af56e43a0"},
+ {file = "multidict-6.6.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:96b745578a33f017b0cee33350c06c986614b04e50d3d899844225077fe831f9"},
+ {file = "multidict-6.6.2-cp39-cp39-win32.whl", hash = "sha256:324cfe383f150123aa70c637781f1d9fd89cb1a583d38468766cb06ff2ea04d7"},
+ {file = "multidict-6.6.2-cp39-cp39-win_amd64.whl", hash = "sha256:f729d139b74441b44d4b8559566d1e9b39e8e282105e390efd5cfcf57b72ea7f"},
+ {file = "multidict-6.6.2-cp39-cp39-win_arm64.whl", hash = "sha256:ddc9ff0c3254b6fbc0411ecdbb952995f1479a6647ba58ff5ca5816ab7ed31ef"},
+ {file = "multidict-6.6.2-py3-none-any.whl", hash = "sha256:a7d14275ff2f85a8ff3c2a32e30f94b9fc8a2125b59a4ecc32271a347fad6e78"},
+ {file = "multidict-6.6.2.tar.gz", hash = "sha256:c1e8b8b0523c0361a78ce9b99d9850c51cf25e1fa3c5686030ce75df6fdf2918"},
]
[package.dependencies]
@@ -3769,7 +3769,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:22d9962a358aedbb7a2e36187ff273adeaab9743373a272976d2e348d08c7770"},
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0d0fda578d1dc3f77b6b5a5dce3b9ad69a8250a891760a548df850a5e8da87f3"},
@@ -4096,7 +4096,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
+markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -5225,7 +5225,7 @@ files = [
{file = "s3transfer-0.13.0-py3-none-any.whl", hash = "sha256:0148ef34d6dd964d0d8cf4311b2b21c474693e57c2e069ec708ce043d2b527be"},
{file = "s3transfer-0.13.0.tar.gz", hash = "sha256:f5e6db74eb7776a37208001113ea7aa97695368242b364d73e91c981ac522177"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.4,<2.0a.0"
@@ -5733,7 +5733,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"daft\" or extra == \"hf\""
+markers = "extra == \"hf\" or extra == \"daft\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -6018,7 +6018,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6032e6da6abd41e4acda34d75a816012717000fa6839f37124a47fcefc49bec4"},
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2c7b34d804b8cf9b214f05015c4fee2ebe7ed05cf581e7192c06555c71f4446a"},
From cd10a6254e4188d3fbbc2724838d8f0be588a52c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 1 Jul 2025 10:39:50 -0400
Subject: [PATCH 094/112] Build: Bump mypy-boto3-dynamodb from 1.38.4 to 1.39.0
(#2163)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps
[mypy-boto3-dynamodb](https://github.com/youtype/mypy_boto3_builder)
from 1.38.4 to 1.39.0.
Release notes
Sourced from mypy-boto3-dynamodb's
releases.
8.8.0 - Python 3.8 runtime is back
Changed
[services] install_requires section is
calculated based on dependencies in use, so
typing-extensions version is set properly
[all] Replaced typing imports with
collections.abc with a fallback to typing for
Python <3.9
[all] Added aliases for builtins.list,
builtins.set, builtins.dict, and
builtins.type, so Python 3.8 runtime should work as
expected again (reported by @YHallouard in #340
and @Omri-Ben-Yair
in #336)
[all] Unions use the same type annotations as the rest
of the structures due to proper fallbacks
Fixed
[services] Universal input/output shapes were not
replaced properly in service subresources
[docs] Simplified doc links rendering for services
[services] Cleaned up unnecessary imports in
client.pyi
[builder] Import records with fallback are always
rendered
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 38 +++++++++++++++++++-------------------
1 file changed, 19 insertions(+), 19 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 22adbe43d5..ae3a49a468 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -59,7 +59,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -72,7 +72,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5421af8f22a98f640261ee48aae3a37f0c41371e99412d55eaf2f8a46d5dad29"},
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0fcda86f6cb318ba36ed8f1396a6a4a3fd8f856f84d426584392083d10da4de0"},
@@ -202,7 +202,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -268,7 +268,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -285,7 +285,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -488,7 +488,7 @@ files = [
{file = "boto3-1.38.27-py3-none-any.whl", hash = "sha256:95f5fe688795303a8a15e8b7e7f255cadab35eae459d00cc281a4fd77252ea80"},
{file = "boto3-1.38.27.tar.gz", hash = "sha256:94bd7fdd92d5701b362d4df100d21e28f8307a67ff56b6a8b0398119cf22f859"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.38.27,<1.39.0"
@@ -509,7 +509,7 @@ files = [
{file = "botocore-1.38.27-py3-none-any.whl", hash = "sha256:a785d5e9a5eda88ad6ab9ed8b87d1f2ac409d0226bba6ff801c55359e94d91a8"},
{file = "botocore-1.38.27.tar.gz", hash = "sha256:9788f7efe974328a38cbade64cc0b1e67d27944b899f88cb786ae362973133b6"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -1473,7 +1473,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
+markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cc4df77d638aa2ed703b878dd093725b72a824c3c546c076e8fdf276f78ee84a"},
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:716a9973a2cc963160394f701964fe25012600f3d311f60c790400b00e568b61"},
@@ -2247,7 +2247,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -3037,7 +3037,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "multidict-6.6.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cfd9c74d337e710d7ee26e72a7dbedbd60e0c58d3df7c5ccbb748857e977783c"},
{file = "multidict-6.6.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9d2c5867a1bd182041a950e9ec3dd3622926260434655bd5d94a62d889100787"},
@@ -3156,14 +3156,14 @@ typing-extensions = {version = ">=4.1.0", markers = "python_version < \"3.11\""}
[[package]]
name = "mypy-boto3-dynamodb"
-version = "1.38.4"
-description = "Type annotations for boto3 DynamoDB 1.38.4 service generated with mypy-boto3-builder 8.10.1"
+version = "1.39.0"
+description = "Type annotations for boto3 DynamoDB 1.39.0 service generated with mypy-boto3-builder 8.11.0"
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "mypy_boto3_dynamodb-1.38.4-py3-none-any.whl", hash = "sha256:6b29d89c649eeb1e894118bee002cb8b1304c78da735b1503aa08e46b0abfdec"},
- {file = "mypy_boto3_dynamodb-1.38.4.tar.gz", hash = "sha256:5cf3787631e312b3d75f89a6cbbbd4ad786a76f5d565af023febf03fbf23c0b5"},
+ {file = "mypy_boto3_dynamodb-1.39.0-py3-none-any.whl", hash = "sha256:3a136f9d764fa5e1b2ff464fa9599533fd00e65affe47bd28a40d920ece707a4"},
+ {file = "mypy_boto3_dynamodb-1.39.0.tar.gz", hash = "sha256:c3bafc7b4f8d59bac9a7436c7ccfb6fe32991bc7fc88c62264eaad06ae63f8a8"},
]
[package.dependencies]
@@ -3769,7 +3769,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:22d9962a358aedbb7a2e36187ff273adeaab9743373a272976d2e348d08c7770"},
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0d0fda578d1dc3f77b6b5a5dce3b9ad69a8250a891760a548df850a5e8da87f3"},
@@ -4096,7 +4096,7 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\""}
+markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -5225,7 +5225,7 @@ files = [
{file = "s3transfer-0.13.0-py3-none-any.whl", hash = "sha256:0148ef34d6dd964d0d8cf4311b2b21c474693e57c2e069ec708ce043d2b527be"},
{file = "s3transfer-0.13.0.tar.gz", hash = "sha256:f5e6db74eb7776a37208001113ea7aa97695368242b364d73e91c981ac522177"},
]
-markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.4,<2.0a.0"
@@ -5733,7 +5733,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"hf\" or extra == \"daft\""
+markers = "extra == \"daft\" or extra == \"hf\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -6018,7 +6018,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
+markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
files = [
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6032e6da6abd41e4acda34d75a816012717000fa6839f37124a47fcefc49bec4"},
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2c7b34d804b8cf9b214f05015c4fee2ebe7ed05cf581e7192c06555c71f4446a"},
From d158d007672cd905ff2af4960ee95fb73451835c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 1 Jul 2025 10:40:08 -0400
Subject: [PATCH 095/112] Build: Bump mypy-boto3-glue from 1.38.42 to 1.39.0
(#2164)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [mypy-boto3-glue](https://github.com/youtype/mypy_boto3_builder)
from 1.38.42 to 1.39.0.
Release notes
Sourced from mypy-boto3-glue's
releases.
8.8.0 - Python 3.8 runtime is back
Changed
[services] install_requires section is
calculated based on dependencies in use, so
typing-extensions version is set properly
[all] Replaced typing imports with
collections.abc with a fallback to typing for
Python <3.9
[all] Added aliases for builtins.list,
builtins.set, builtins.dict, and
builtins.type, so Python 3.8 runtime should work as
expected again (reported by @YHallouard in #340
and @Omri-Ben-Yair
in #336)
[all] Unions use the same type annotations as the rest
of the structures due to proper fallbacks
Fixed
[services] Universal input/output shapes were not
replaced properly in service subresources
[docs] Simplified doc links rendering for services
[services] Cleaned up unnecessary imports in
client.pyi
[builder] Import records with fallback are always
rendered
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index ae3a49a468..4093a340c1 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -3171,14 +3171,14 @@ typing-extensions = {version = "*", markers = "python_version < \"3.12\""}
[[package]]
name = "mypy-boto3-glue"
-version = "1.38.42"
-description = "Type annotations for boto3 Glue 1.38.42 service generated with mypy-boto3-builder 8.11.0"
+version = "1.39.0"
+description = "Type annotations for boto3 Glue 1.39.0 service generated with mypy-boto3-builder 8.11.0"
optional = false
python-versions = ">=3.8"
groups = ["dev"]
files = [
- {file = "mypy_boto3_glue-1.38.42-py3-none-any.whl", hash = "sha256:f49401710e8795a5d12ae0ad14bb6a96fd8ac23477853569421253a51a8e790c"},
- {file = "mypy_boto3_glue-1.38.42.tar.gz", hash = "sha256:c34967cd54ddd5d8d8afc1464eaa847b6430c32b7424559199c5af3d9006b4f1"},
+ {file = "mypy_boto3_glue-1.39.0-py3-none-any.whl", hash = "sha256:6ba8cc33d77402ced0247df81da11906dada15bc02dab098517d8d2686ef2369"},
+ {file = "mypy_boto3_glue-1.39.0.tar.gz", hash = "sha256:e3fa80713eb57c09af1d7f7057071b36f66e196bb86b47661a1973dc71fcebaf"},
]
[package.dependencies]
From 030d67678220571a25f873faf00d8885f742bdc7 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 1 Jul 2025 10:40:22 -0400
Subject: [PATCH 096/112] Build: Bump huggingface-hub from 0.33.0 to 0.33.1
(#2165)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [huggingface-hub](https://github.com/huggingface/huggingface_hub)
from 0.33.0 to 0.33.1.
Release notes
Sourced from huggingface-hub's
releases.
[v0.33.1]: Inference Providers Bug Fixes, Tiny-Agents Message
handling Improvement, and Inference Endpoints Health Check Update
Full Changelog: https://github.com/huggingface/huggingface_hub/compare/v0.33.0...v0.33.1
This release introduces bug fixes for chat completion type
compatibility and feature extraction parameters, enhanced message
handling in tiny-agents, and updated inference endpoint health
check:
- [Tiny agents] Add tool call to messages #3159
by
@NielsRogge
- fix: update payload preparation to merge parameters into the output
dictionary #3160
by
@mishig25
- fix(inference_endpoints): use GET healthRoute instead of GET / to
check status #3165
by
@mfuntowicz
- Recursive filter_none in Inference Providers #3178
by
@Wauplin
Commits
c8ffea9
Release: v0.33.1
e76fc3a
[Tiny agents] Add tool call to messages (#3159)
57eacd2
fix(inference_endpoints): use GET healthRoute instead of
GET / to check sta...
24a0c75
Recursive filter_none in Inference Providers (#3178)
4ada292
fix: update payload preparation to merge parameters into the output
dictionar...
- See full diff in compare
view
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index 4093a340c1..bd7f086ca5 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -2035,15 +2035,15 @@ tests = ["pytest"]
[[package]]
name = "huggingface-hub"
-version = "0.33.0"
+version = "0.33.1"
description = "Client library to download and publish models, datasets and other repos on the huggingface.co hub"
optional = true
python-versions = ">=3.8.0"
groups = ["main"]
markers = "extra == \"hf\""
files = [
- {file = "huggingface_hub-0.33.0-py3-none-any.whl", hash = "sha256:e8668875b40c68f9929150d99727d39e5ebb8a05a98e4191b908dc7ded9074b3"},
- {file = "huggingface_hub-0.33.0.tar.gz", hash = "sha256:aa31f70d29439d00ff7a33837c03f1f9dd83971ce4e29ad664d63ffb17d3bb97"},
+ {file = "huggingface_hub-0.33.1-py3-none-any.whl", hash = "sha256:ec8d7444628210c0ba27e968e3c4c973032d44dcea59ca0d78ef3f612196f095"},
+ {file = "huggingface_hub-0.33.1.tar.gz", hash = "sha256:589b634f979da3ea4b8bdb3d79f97f547840dc83715918daf0b64209c0844c7b"},
]
[package.dependencies]
From aa3dde6ce7b50cd4e9b407b3012f44e76eebc443 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 1 Jul 2025 12:27:46 -0400
Subject: [PATCH 097/112] Build: Bump pyroaring from 1.0.1 to 1.0.2 (#2162)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Bumps [pyroaring](https://github.com/Ezibenroc/PyRoaringBitMap) from
1.0.1 to 1.0.2.
Release notes
Sourced from pyroaring's
releases.
1.0.2
What's Changed
New Contributors
Full Changelog: https://github.com/Ezibenroc/PyRoaringBitMap/compare/1.0.1...1.0.2
Commits
[](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores)
Dependabot will resolve any conflicts with this PR as long as you don't
alter it yourself. You can also trigger a rebase manually by commenting
`@dependabot rebase`.
[//]: # (dependabot-automerge-start)
[//]: # (dependabot-automerge-end)
---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR:
- `@dependabot rebase` will rebase this PR
- `@dependabot recreate` will recreate this PR, overwriting any edits
that have been made to it
- `@dependabot merge` will merge this PR after your CI passes on it
- `@dependabot squash and merge` will squash and merge this PR after
your CI passes on it
- `@dependabot cancel merge` will cancel a previously requested merge
and block automerging
- `@dependabot reopen` will reopen this PR if it is closed
- `@dependabot close` will close this PR and stop Dependabot recreating
it. You can achieve the same result by closing it manually
- `@dependabot show ignore conditions` will show all
of the ignore conditions of the specified dependency
- `@dependabot ignore this major version` will close this PR and stop
Dependabot creating any more for this major version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this minor version` will close this PR and stop
Dependabot creating any more for this minor version (unless you reopen
the PR or upgrade to it yourself)
- `@dependabot ignore this dependency` will close this PR and stop
Dependabot creating any more for this dependency (unless you reopen the
PR or upgrade to it yourself)
Signed-off-by: dependabot[bot]
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
poetry.lock | 170 ++++++++++++++++++++++++++--------------------------
1 file changed, 85 insertions(+), 85 deletions(-)
diff --git a/poetry.lock b/poetry.lock
index bd7f086ca5..889b3abe7a 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -4379,96 +4379,96 @@ files = [
[[package]]
name = "pyroaring"
-version = "1.0.1"
+version = "1.0.2"
description = "Library for handling efficiently sorted integer sets."
optional = false
python-versions = "*"
groups = ["main"]
files = [
- {file = "pyroaring-1.0.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:91b033fab1f32d3bf0149a9669bb496e8dd6cdaf81020948406c20d845bcd3db"},
- {file = "pyroaring-1.0.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2f0635b4634f91b6a65749f4ff90d669782e079d6cf633ddd20b10b053322197"},
- {file = "pyroaring-1.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:83732c0337a54508e758547cbd869169cdbef0005042532e38a8bad23458712f"},
- {file = "pyroaring-1.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d09198f6680a71a5d4cd058b14988160acab7f771f5acf4c4022d712ac72a34b"},
- {file = "pyroaring-1.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b96cdc047e614312ac4aebf7ff314db9ef3891138268cffa910ea02476c9411b"},
- {file = "pyroaring-1.0.1-cp310-cp310-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:7121aebcbb58c2aa49e2a265ded513a04d6ae7626d0078c695c5a12306261584"},
- {file = "pyroaring-1.0.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e69599d814bef20f437d9af6f4575312d4326aef80e3678d927080d3791fe59f"},
- {file = "pyroaring-1.0.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:d652dcc46fba3dc2c2c9b379024050d26d4a2dd74794ee35aae8fb860733ff7c"},
- {file = "pyroaring-1.0.1-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:222f178f2c62ca6b867963267db2cbc91d37ba23eb36258c5f02e15c3c1cb225"},
- {file = "pyroaring-1.0.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:b4e04d885cb732bcd5fc77aa687c030cb3500eca1ff9e3bd95ddd23d335df8d3"},
- {file = "pyroaring-1.0.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:837ef791c8f49a6787f94b6b1cf5a9002d702ef31b98b31f2e72fed3bbda4b1f"},
- {file = "pyroaring-1.0.1-cp310-cp310-win32.whl", hash = "sha256:1a731b313a76669a1ff6d48552462ba2f7492fb11b8ef67941fa0309d57fad91"},
- {file = "pyroaring-1.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:990b34a6f54f04df1581e74252641b2570244fd143dceebc1861c74ae333e509"},
- {file = "pyroaring-1.0.1-cp310-cp310-win_arm64.whl", hash = "sha256:266c3fffaaa12a76490269848a8ae74550b78fc0dd95c24e685389a1ce114e1a"},
- {file = "pyroaring-1.0.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:eebdd6c3427f0a30663a5c73646ec5397146d1db5de49bd57cf05fe18da93e09"},
- {file = "pyroaring-1.0.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:5ac4ad77298ec6b04f41f26fadcc6fcbd50fc42a434c1a13e4ceff7a344b12c7"},
- {file = "pyroaring-1.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:7abae38e94d2be4b1a525515549e97b5579fe8a219edf82f6e898ef2c1b9cef1"},
- {file = "pyroaring-1.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:09f33b16857b435bfe362a91c54ec97b4fc4012ce70faf3ba26dcdf1f2a2d16e"},
- {file = "pyroaring-1.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b53d65fc57bd7c8ee592663a95832512bc9403181eb45d6798f4141bee4feb93"},
- {file = "pyroaring-1.0.1-cp311-cp311-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:3ef6c020d6d4e95cfe51cadfc0d72c79cefebff7b83c19cbc3edda4634612c6e"},
- {file = "pyroaring-1.0.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:567bd21b13140cfbb9833d6c893590fec52f5d3ba12f102d371523a206efc271"},
- {file = "pyroaring-1.0.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8a355106643b2293dd4940d3afb8abbfaffe370dfeba110efbd1c157cfbe02e7"},
- {file = "pyroaring-1.0.1-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:99aa16b6e1fcccf60dd8885cd671df0161761413cae2d383d5b4c8917788b653"},
- {file = "pyroaring-1.0.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:99914d2d759c81cfd0cb23a0f4516caa8a580f9a9ba3bc7ebc6861ea0e7ae38a"},
- {file = "pyroaring-1.0.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:0a449c20042b0146fe2bf88700eb4083f778e3a41bcfdde1c1c80993bad9b7a1"},
- {file = "pyroaring-1.0.1-cp311-cp311-win32.whl", hash = "sha256:a65741944b637cc3fcaf3c3e54812a2c78ea2cb5135bbb8b4975f4eb7cc04b44"},
- {file = "pyroaring-1.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:f085909a12ad6edc1b5dbb31bb9eb54186e8fde71cbafc457d1a39ba2f2d3c0a"},
- {file = "pyroaring-1.0.1-cp311-cp311-win_arm64.whl", hash = "sha256:08ed2b7fe9fe8dcd097811c338c487e8608425d791083af354f91d3af81a67b1"},
- {file = "pyroaring-1.0.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:b0ecba0933fd1ac9037fb7bc832fa36f515982522c4013e681baa1470fcda9d7"},
- {file = "pyroaring-1.0.1-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:5402e465194d427ee48025c01c78daa1fb40656106561ffe47b338c6277fdd99"},
- {file = "pyroaring-1.0.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:86b834db274d84e3a50938f1adf4662ad06c23c4d30d9eb0bbf246173ba3ddcc"},
- {file = "pyroaring-1.0.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b21b42abdd2fbaf905fc75000ff90084ea180759edbc04e32dc44eaf6204cd09"},
- {file = "pyroaring-1.0.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2c8d14855a59aaca23a77897f0f9cb6f2d9960ab35d1393a6a3af3c929436e34"},
- {file = "pyroaring-1.0.1-cp312-cp312-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:9af47449b4c72fbb9a5a755d46a1ad65f13f684f7b5eacfacd686e8fec471820"},
- {file = "pyroaring-1.0.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:70efe866c90c28dc21ada69baaffac0ed137fc541c2d4d87cd261969a0f59a78"},
- {file = "pyroaring-1.0.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:603cee6c744069dc600d7b2f76cbe2f26aea8b212db69cc57bdb45200f963c32"},
- {file = "pyroaring-1.0.1-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:90259cddb3c2d418578d26681c1ac60be7479763514df1f3fe7ca389226c828e"},
- {file = "pyroaring-1.0.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:26b727159fa2cce214ddc21438bb80193e66a480628650343612446ab3a291b7"},
- {file = "pyroaring-1.0.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:31f4adaa5d5232f5ee0dca696eff32391a656e75a92184767b79fcdbe6e7441f"},
- {file = "pyroaring-1.0.1-cp312-cp312-win32.whl", hash = "sha256:803a6727bb2dca6566ba4da8e3951a09d3b9faffb4b3601481674c8aecff604a"},
- {file = "pyroaring-1.0.1-cp312-cp312-win_amd64.whl", hash = "sha256:f4e4f34b140a250d0386e9ae4c4049509cc833ef7772d86e01b6aae93370bb22"},
- {file = "pyroaring-1.0.1-cp312-cp312-win_arm64.whl", hash = "sha256:7952ab4bb8ae176aba746efd872dfaa9e124125810cbe3dc479f5ffaefd8a952"},
- {file = "pyroaring-1.0.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:c53e3511534f8a9176972003b2bcb1abae60c6d22dce4bddc61c99770e426dbf"},
- {file = "pyroaring-1.0.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:7e9b73dc8b2bcb27f20a3be4947cd89841f41f53426bee8f03f8ad68c7cd0b90"},
- {file = "pyroaring-1.0.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:a25e8ccc7ab2b78a6d937e288a6b15bb89f07e76007273010f8154669c1902be"},
- {file = "pyroaring-1.0.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9af383e5b63cef57beb005de343faa5c984574f3e5f23904fc74bba53bbdbfbc"},
- {file = "pyroaring-1.0.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cf2b2ea69d218911b244abf05cdc52fb5f45ddb4ef2730921d6b56ce1ba6d57a"},
- {file = "pyroaring-1.0.1-cp313-cp313-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:f3068dd1dfc560c92093cf10de8bf7683d6d9b3550bb5ec674a9e5a47e4c9cfa"},
- {file = "pyroaring-1.0.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:40b10657164e53ed2618ec7490e166bf37b062abbd64e6096179dd47beb4af70"},
- {file = "pyroaring-1.0.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:b813c7136eecb112379406634a158efcbfbe3ce0503421fc2f8f7c4bf44526cb"},
- {file = "pyroaring-1.0.1-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:3d5c047cb9c673c5445fc9f9ba2229609434ae01601e5b0195ac354ad942ee54"},
- {file = "pyroaring-1.0.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:8b92fa384e5ac7d6dd69601d5dd828eecf810d7978c92bc85128834bf72e6572"},
- {file = "pyroaring-1.0.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:eff8546f6e96bfb57321cc796cfd32bc9e757de930896f96333fd0d14dfb5e3c"},
- {file = "pyroaring-1.0.1-cp313-cp313-win32.whl", hash = "sha256:eab133dd4836706e1639fe2c936598ba3464a082c77803b614c4a807f5ea10fe"},
- {file = "pyroaring-1.0.1-cp313-cp313-win_amd64.whl", hash = "sha256:14b01e0d8acfe2f7b786fc7c6b441c1e67d337835f2b946142e6d79d1137f782"},
- {file = "pyroaring-1.0.1-cp313-cp313-win_arm64.whl", hash = "sha256:e0537a2e3d5ebd64e5b017e5e11461489b613dae83437a683330ee26012bb828"},
- {file = "pyroaring-1.0.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:11519baeb313a32bc038012293a40e50b8e5262791d797eb96dd52e4f1b11aff"},
- {file = "pyroaring-1.0.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:6d255eac739dcaad758f1f561561d13e355156adb023f8107b5594509e093980"},
- {file = "pyroaring-1.0.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:98a08b3964665ba22b37dba1a91fb8f4501f6f5d43ae83befa7516daea0bec09"},
- {file = "pyroaring-1.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4c6b401e6f4c5f6d37cb6250435b62b2537cae779e3c09eece639a9cc6ac91e8"},
- {file = "pyroaring-1.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bf6f2083990043dc31254c14991f6a5bc02cb60a4c8481586625eaedd2306386"},
- {file = "pyroaring-1.0.1-cp38-cp38-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:c85bffdab56ef10559e5091dde68c5b05a65321a7877c72cea6cbbbfa82e61c3"},
- {file = "pyroaring-1.0.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:62918add0aeb120252abb7d2f27251025b35adeb7cdef291269ee7de829f2f43"},
- {file = "pyroaring-1.0.1-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:26544c0f99785df03b44923acaccb13358193bc29bcfb4dc74e0d268025af0c9"},
- {file = "pyroaring-1.0.1-cp38-cp38-musllinux_1_2_armv7l.whl", hash = "sha256:829319976e80c5842ed74824d5b54863d56439643304af3bc20937fc56a54b45"},
- {file = "pyroaring-1.0.1-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:733e409daa9528bc3662d46c1ca44e152cc630af500ee87a570608b8ba9d4cb5"},
- {file = "pyroaring-1.0.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:e788c1d899ed728b31255114fa755cdef8b437d059f7969da448d249e8bc4f7e"},
- {file = "pyroaring-1.0.1-cp38-cp38-win32.whl", hash = "sha256:db2de0a399092ab4d294e93de3c6af004e2f0164b9c8e6f9f6a269c29de6d64c"},
- {file = "pyroaring-1.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:71790cbf2a8c6945698933debb73c9ab8f1ae70f98a4f548c48d257acf078a6d"},
- {file = "pyroaring-1.0.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:de325ee535599e5259befa0edbc98c105c4bcc2e7ff0b9165a37f43448c41174"},
- {file = "pyroaring-1.0.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:7b30e47c8d01d78129111bb3ac5d23c64842a63cafeea94810d5c50d1721c3d4"},
- {file = "pyroaring-1.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:2f184c7672dd77fa4a1b860d933918d2ffe9a1a6339f29d9afc6840515c64fb6"},
- {file = "pyroaring-1.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:47058ca48f2e4cdae5db179653e30b9dde5d4d42a0e877501dcad7116cabc8b9"},
- {file = "pyroaring-1.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:91dbbb20646d59c5ecde5f3003bca9727b362e740ec53a770b37aa5b4a386c98"},
- {file = "pyroaring-1.0.1-cp39-cp39-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:7ca8e5454d1b9bbbe89b2bb038139142de39503d1499589a019ff7ef38dc3f0b"},
- {file = "pyroaring-1.0.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c5668c8e85c74bd1c6ef2e6f97f35af9a94f6f6254032efd5b87728a3073fd5b"},
- {file = "pyroaring-1.0.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:56915a8ed3c6cca5839c1a61c3e0d0f339be2da271c9bcc8999ff8acf2a1a687"},
- {file = "pyroaring-1.0.1-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:09a670334a5df5e0891690d4e86e1f30537cb47bfcf51cd223a7a809fd3eb8b6"},
- {file = "pyroaring-1.0.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:da67411dd64a2b7f8794df9a8e2c99d89803a94a3fe65340410fc85032e22390"},
- {file = "pyroaring-1.0.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:c3562556856096fbaf10d72d2e350edb00e8c7906c6cf9bb86a17845b631f059"},
- {file = "pyroaring-1.0.1-cp39-cp39-win32.whl", hash = "sha256:6e194e7c89d6865e90190c30cfe68dd3e641092c4fe48e13ecb67e2368f02489"},
- {file = "pyroaring-1.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:5e7d9bbdacd557ef0f0e5d4456b0cbd3e85130c8ef102dcfe484f6a4af5df444"},
- {file = "pyroaring-1.0.1-cp39-cp39-win_arm64.whl", hash = "sha256:15cd0fc12ee797bbb28332bdbac2d65a7826c0b6ab50b7337e75ee41f92c9503"},
- {file = "pyroaring-1.0.1.tar.gz", hash = "sha256:e0ab5f8a18a7ba99b8f7a152dca300ef5bd9eff0a7df56a08714114497b63f10"},
+ {file = "pyroaring-1.0.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:20967e63a7d17758e290f8121239707d086b7720de57d04844e1b0aa977786cb"},
+ {file = "pyroaring-1.0.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e6e8de37d22e7aa9a482db7153b9e2a2c966e1951ecf5aaa08cec3264ee773f9"},
+ {file = "pyroaring-1.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1387a4dd91374864775fe864c058821535a7ebfa96e208622a21db962d66af15"},
+ {file = "pyroaring-1.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:39ae367d11eb6cdf5b1afd939ebc063f5cabf0120efee794f96289eaa4962789"},
+ {file = "pyroaring-1.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a0f55f288c72b5326a3a64fb7104e92c71501490ab4bc5ec48f54379e3f7e655"},
+ {file = "pyroaring-1.0.2-cp310-cp310-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:4302399ea8b8fcfa54e5a847d8089001f3b3c2c294b5902598e1db30a749c74e"},
+ {file = "pyroaring-1.0.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:25f4f98d99eb1fdd20300ea45241f6304f90d55820d1b3e70c73a09d4ef1bffc"},
+ {file = "pyroaring-1.0.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:986c5c7bd2831bd5ffa7cb14aa7f20cbf9c762619bcf6a5a4e141d14ba4aa732"},
+ {file = "pyroaring-1.0.2-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:4c4865df32b4c980fb977140a09077d59a4f2cc86f47524c5d76b64650fec0de"},
+ {file = "pyroaring-1.0.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:d32be36debb293e8e6bfc8e51f257a1e95b9bc820e0a91196d87cfebe0575af6"},
+ {file = "pyroaring-1.0.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:6c55caa802473ce136cbbe2f51506d5046c8b71d8e265b8fb0df011f4534af44"},
+ {file = "pyroaring-1.0.2-cp310-cp310-win32.whl", hash = "sha256:703f377d0c50d7bfce0734b1256140cffaa666d9b0621fe74708b1141e8a85a7"},
+ {file = "pyroaring-1.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:e85e3bc78fce1aa6a83cfc3a67cd36bf02c9261e4f426b6040aa0bd97a89820d"},
+ {file = "pyroaring-1.0.2-cp310-cp310-win_arm64.whl", hash = "sha256:108a808a07a7bb773205ab75f7cc22f235f8dffeea85c4bd28b6417fe5480a30"},
+ {file = "pyroaring-1.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:85a79f88d94d7586921472720c9a9fcedc5db79bff310413be049c2ca3afb349"},
+ {file = "pyroaring-1.0.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:10e82996c70ed344773582be70b34e5b20078626fd2da5063ab396f12f2764ba"},
+ {file = "pyroaring-1.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:76af7c7c72a61be1ed08062c8f7aed8020879ada0e77d2bcee6fb40657c32399"},
+ {file = "pyroaring-1.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:01cca981c5a05a4c05f8e1d5908fc50e143d9d3f68dc516525ab2e1e8eefacb6"},
+ {file = "pyroaring-1.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a8de91fe9bacbae5eb3dbabddc03145da1ce3ce64f421262abe9ea5e99043274"},
+ {file = "pyroaring-1.0.2-cp311-cp311-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:32901ca5c5c8bc6a1aa90b006d9c75f918639f639c615bf12ba9b9d41b008d01"},
+ {file = "pyroaring-1.0.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c56152bee9d6e2aef7535ed3874155709c73ccd4860921072a0655488abb9db8"},
+ {file = "pyroaring-1.0.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:7dbdd3f4b3efc0c6dcac67aced444ca3a13e2346ace9172f1f5a74a78128777c"},
+ {file = "pyroaring-1.0.2-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:08a127d1a740326a9954d00c5efec56bff31695b05a798a8dcd55d17f880ca15"},
+ {file = "pyroaring-1.0.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:e87aca7ddde5eb729ff4513e44a6c3a1d844d8a50b67b21dd98420a0f47b53d3"},
+ {file = "pyroaring-1.0.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:613fe8ade494a94de4e4c4226c7b526b2be802bb4280a983d6ec68b903130f2d"},
+ {file = "pyroaring-1.0.2-cp311-cp311-win32.whl", hash = "sha256:35267f9439cccc6023a71aa929df38cda1f4146392595dc35f2ddc1263bbd56e"},
+ {file = "pyroaring-1.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:bd31fece30655ad55b2ae2e0b1f397880c78a1052013a577ca4eafbcba5c1afa"},
+ {file = "pyroaring-1.0.2-cp311-cp311-win_arm64.whl", hash = "sha256:b78bfbc2c56c78cd054b3df22215e7d51144ca906e3f115e22b8eade72c7f079"},
+ {file = "pyroaring-1.0.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:d9ec623abbfcf50771e460f202d41700bdfb7dc28c6f1b0be0843dd8b61d50ac"},
+ {file = "pyroaring-1.0.2-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:986a1245b22294ce5ba1b6f4299aea9a410d534718fce0fc6af6fe2251bb8981"},
+ {file = "pyroaring-1.0.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:617b1affe6a09025a0207f6fa1d6f2266ef540c5ab9ab29287311f35825ae63a"},
+ {file = "pyroaring-1.0.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:67c0782b46e552c71d2598f50cad326446364fab02ec51b37ec26037d78263bc"},
+ {file = "pyroaring-1.0.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:47fe576f01b806c5677e08cc60d3765f872f7aeef9204914dacb3ba72067d4b0"},
+ {file = "pyroaring-1.0.2-cp312-cp312-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:96291b1f4f77ca13ceb3716605bcb63ae91e17c9539bd07fc77509226a7e8d1f"},
+ {file = "pyroaring-1.0.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:bbf02fc3873efefe37429a91d1c384c6c259b3e5b5f4220c9436550b5c480aa3"},
+ {file = "pyroaring-1.0.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:0ce11a6d57d6d89759eeb87a727d1657cb1f03ced71d186812799e07879c568e"},
+ {file = "pyroaring-1.0.2-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:decefea17be7ec3cfe461022ebb73d369f0fe07d39ba813d168267ff82a3f748"},
+ {file = "pyroaring-1.0.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:04e6b56384aab6ca2f37e4d868b49689a0e9cf4198d8fdb2f6475a8f13d937ba"},
+ {file = "pyroaring-1.0.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:b7d74f12761ee614ebd6337641100c370bc982099fbb109cdd9b23b40ae96273"},
+ {file = "pyroaring-1.0.2-cp312-cp312-win32.whl", hash = "sha256:5a5facb5d5c529f4b6967ce75f76806bf8161694dc3d5d4c0f9de3cf614b7735"},
+ {file = "pyroaring-1.0.2-cp312-cp312-win_amd64.whl", hash = "sha256:906f07c97487fd0319aa36e36f8ed54b9fdf275297b394cf36803592527b186c"},
+ {file = "pyroaring-1.0.2-cp312-cp312-win_arm64.whl", hash = "sha256:51ed8a5677c409c4b896e931b4baaa0f95f6a4c8fe56b6dc0e7f8f6728487465"},
+ {file = "pyroaring-1.0.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:f14a300354b2c23de16cc421dc54af4987cb3da4d90df9eddf3bb081a664093d"},
+ {file = "pyroaring-1.0.2-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:987fc92e443dfcea57b31bd7f0b06e1d46b7361c7d1b3adf5c9d49614195e7ff"},
+ {file = "pyroaring-1.0.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:bc40e70fe9917f32776f741ce93acf1de8897260cadf27397252f1e4125a91da"},
+ {file = "pyroaring-1.0.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4b482f64a08bd23e3ff7a025fd431da048d4ee7b0dd6e75e83182b1ff418008b"},
+ {file = "pyroaring-1.0.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0f9b1a1d309b29de793e2958c1e9808218c449a7516f682627a18ea5782b0e2a"},
+ {file = "pyroaring-1.0.2-cp313-cp313-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:cf4822333443ff7cdedce36da8365a442ad40e8c42fb1b835a589892cb08de65"},
+ {file = "pyroaring-1.0.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3695317b9f8a95ced9ab2426fcae59f23b540c8d812178e607e090f5546f2359"},
+ {file = "pyroaring-1.0.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:83d92fdf6c236e539fe3258375fd407b822dd7207cbed56a520d364b7cf3a516"},
+ {file = "pyroaring-1.0.2-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:2bc091dab0811359f4fbabbf771485b12e2086c5bd2f27570cca8860e56e3837"},
+ {file = "pyroaring-1.0.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:f1744c415b75e4cd3b3d26c2d7a2cda6c8f3e6a713e42751189adfe1d41954e7"},
+ {file = "pyroaring-1.0.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:25b223c71a34e0bf4fe004312723918941337ae4770ec22e1ae43050fc45422f"},
+ {file = "pyroaring-1.0.2-cp313-cp313-win32.whl", hash = "sha256:41e96f770240d773c13137250dd0c51ab898c435d48d77eae37853939a9bb402"},
+ {file = "pyroaring-1.0.2-cp313-cp313-win_amd64.whl", hash = "sha256:a73ce95a6205f2b341147619790ecc75e96a9ca265f9797fc816324df6c26c38"},
+ {file = "pyroaring-1.0.2-cp313-cp313-win_arm64.whl", hash = "sha256:3912f2ea93b27b57e914b7d082a6849ff9048644c803de8bb85d7a15cdb420bd"},
+ {file = "pyroaring-1.0.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:44d28fe7cd95be08172e66ec75e87ffae1e77c3e4ddb0dbf4a45b7a593590a77"},
+ {file = "pyroaring-1.0.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1d60ac82595cf2f0dfce8513cc91838c589bcf6959f5ab7e1801e6efcc4d79f9"},
+ {file = "pyroaring-1.0.2-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:a0dda1ca2f5419c06f01a99faf4a5962bf74fe18e6b21bf563e704f00a087a2a"},
+ {file = "pyroaring-1.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5b72de42162bccb12bda41a9b9e68323fbfa7850973f83d1f85480dc9a57b9a3"},
+ {file = "pyroaring-1.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7db9b2fa9ed2f4310f6c2c46f6b56755f15029a37964664ec02fa76156895798"},
+ {file = "pyroaring-1.0.2-cp38-cp38-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:45aa198a2ae487b6c50447215525be8b226ed088f652f6593835a46dba7251e7"},
+ {file = "pyroaring-1.0.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:eddbad0004c27c47f5f088150f18824bff70d67e4868db74665acf47e1f1be96"},
+ {file = "pyroaring-1.0.2-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:73bb24c851e1860b455421dff2b4373573ebcef62369d82e49485a1d326e0a2c"},
+ {file = "pyroaring-1.0.2-cp38-cp38-musllinux_1_2_armv7l.whl", hash = "sha256:3a4f7f3819780d86335bdbc39221b5e62cd1c224cddd4e8ba224085295ddbb9a"},
+ {file = "pyroaring-1.0.2-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:35218c167bbd8546119da0eea18dd3297c3aa44b3ec26093a41b44fb853fbb75"},
+ {file = "pyroaring-1.0.2-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:1768e31604e145f045d4e0a223fd7c8a5ebf6419e67581141ed673a93980cd3b"},
+ {file = "pyroaring-1.0.2-cp38-cp38-win32.whl", hash = "sha256:2957fd752d3b39408ff197fe94c3d1d20e5c7725b9f117c97efe9be7d0dffe1e"},
+ {file = "pyroaring-1.0.2-cp38-cp38-win_amd64.whl", hash = "sha256:c2d5528e9b8162a313bac99bd920ff45b53a87faaea20edf62e903dcb5373d4f"},
+ {file = "pyroaring-1.0.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:171bd25428136eb5d379f21185788d32d86c0bbb2a789e112ecadb80067e4918"},
+ {file = "pyroaring-1.0.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:226dd9e63eb22fa2ad045461b79781bb6d4682e43853007cc54db197ad8e05f5"},
+ {file = "pyroaring-1.0.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:db99ede0f2292d3df142b69b87f11fd7b0beacab11ed104fff1e0db87ca52477"},
+ {file = "pyroaring-1.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9c057963f8b257a1f3dcf4b8bf3272bf241eaac2b7414b17ec3b7eef73b03097"},
+ {file = "pyroaring-1.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:aeb3276817f2f29ed6712c1b5c68e80f334ad435245d19c79cbade8f848a4c04"},
+ {file = "pyroaring-1.0.2-cp39-cp39-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:a88d5215576a59acad827b1cc5f575cad832dc61626a0c26507d459a3099a266"},
+ {file = "pyroaring-1.0.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5f8fa9b97c5b64f13586646ca999d67a6d7064fff79a6ad044316a835e55aedb"},
+ {file = "pyroaring-1.0.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:78ff45b76e858db13d76b13487865575cabff85ec777f3d95c3ff75395237d81"},
+ {file = "pyroaring-1.0.2-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:48b9ac5f5b557698c53030e271b4a135090846e32b17bb414077d888aa666199"},
+ {file = "pyroaring-1.0.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:36a2564957d17ad4c6caa62e4bb19e519d038daf5b6b7c5ef274c0d1283c71fc"},
+ {file = "pyroaring-1.0.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:f705b6126e906bf3ee43c64dcba06bde6b13854528dd5ca302cfdc388a52b31c"},
+ {file = "pyroaring-1.0.2-cp39-cp39-win32.whl", hash = "sha256:08e73bcd138e7db6ed6dcceaab608db635604d94f012d11639dcea27e7048f4d"},
+ {file = "pyroaring-1.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:bc905d10d7a0c269dc93f6526925deb402069c9d7d265210bb54ffd9650ab5b9"},
+ {file = "pyroaring-1.0.2-cp39-cp39-win_arm64.whl", hash = "sha256:5da91936b591db7bf2165f56e47fe0696595a2c9e30c9399df814cfeb6b92c82"},
+ {file = "pyroaring-1.0.2.tar.gz", hash = "sha256:aa4a1de882bf4a245c7c0dee8594b898e76501bc32afc0cc9ff4138784462839"},
]
[[package]]
From 3a37b525c11d3344cc126a63df4b88409005221c Mon Sep 17 00:00:00 2001
From: vinjai
Date: Thu, 3 Jul 2025 04:24:18 +0530
Subject: [PATCH 098/112] Feature: Write to branches (#941)
Fixes: https://github.com/apache/iceberg-python/issues/306
---------
Co-authored-by: Kevin Liu
---
pyiceberg/cli/console.py | 6 +-
pyiceberg/table/__init__.py | 97 +++++++---
pyiceberg/table/update/__init__.py | 8 +-
pyiceberg/table/update/snapshot.py | 178 ++++++++++++-------
pyiceberg/utils/concurrent.py | 10 +-
tests/integration/test_deletes.py | 29 +++
tests/integration/test_writes/test_writes.py | 160 ++++++++++++++++-
tests/table/test_init.py | 32 +++-
8 files changed, 409 insertions(+), 111 deletions(-)
diff --git a/pyiceberg/cli/console.py b/pyiceberg/cli/console.py
index 6be4df12cc..3fbd9c9fc9 100644
--- a/pyiceberg/cli/console.py
+++ b/pyiceberg/cli/console.py
@@ -33,7 +33,7 @@
from pyiceberg.cli.output import ConsoleOutput, JsonOutput, Output
from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchPropertyException, NoSuchTableError
from pyiceberg.table import TableProperties
-from pyiceberg.table.refs import SnapshotRef
+from pyiceberg.table.refs import SnapshotRef, SnapshotRefType
from pyiceberg.utils.properties import property_as_int
@@ -417,7 +417,7 @@ def list_refs(ctx: Context, identifier: str, type: str, verbose: bool) -> None:
refs = table.refs()
if type:
type = type.lower()
- if type not in {"branch", "tag"}:
+ if type not in {SnapshotRefType.BRANCH, SnapshotRefType.TAG}:
raise ValueError(f"Type must be either branch or tag, got: {type}")
relevant_refs = [
@@ -431,7 +431,7 @@ def list_refs(ctx: Context, identifier: str, type: str, verbose: bool) -> None:
def _retention_properties(ref: SnapshotRef, table_properties: Dict[str, str]) -> Dict[str, str]:
retention_properties = {}
- if ref.snapshot_ref_type == "branch":
+ if ref.snapshot_ref_type == SnapshotRefType.BRANCH:
default_min_snapshots_to_keep = property_as_int(
table_properties,
TableProperties.MIN_SNAPSHOTS_TO_KEEP,
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 1246c6c19e..28d19e4aaf 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -87,7 +87,7 @@
from pyiceberg.table.name_mapping import (
NameMapping,
)
-from pyiceberg.table.refs import SnapshotRef
+from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRef
from pyiceberg.table.snapshots import (
Snapshot,
SnapshotLogEntry,
@@ -397,7 +397,7 @@ def _build_partition_predicate(self, partition_records: Set[Record]) -> BooleanE
expr = Or(expr, match_partition_expression)
return expr
- def _append_snapshot_producer(self, snapshot_properties: Dict[str, str]) -> _FastAppendFiles:
+ def _append_snapshot_producer(self, snapshot_properties: Dict[str, str], branch: Optional[str]) -> _FastAppendFiles:
"""Determine the append type based on table properties.
Args:
@@ -410,7 +410,7 @@ def _append_snapshot_producer(self, snapshot_properties: Dict[str, str]) -> _Fas
TableProperties.MANIFEST_MERGE_ENABLED,
TableProperties.MANIFEST_MERGE_ENABLED_DEFAULT,
)
- update_snapshot = self.update_snapshot(snapshot_properties=snapshot_properties)
+ update_snapshot = self.update_snapshot(snapshot_properties=snapshot_properties, branch=branch)
return update_snapshot.merge_append() if manifest_merge_enabled else update_snapshot.fast_append()
def update_schema(self, allow_incompatible_changes: bool = False, case_sensitive: bool = True) -> UpdateSchema:
@@ -430,13 +430,16 @@ def update_schema(self, allow_incompatible_changes: bool = False, case_sensitive
name_mapping=self.table_metadata.name_mapping(),
)
- def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> UpdateSnapshot:
+ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT, branch: Optional[str] = None) -> UpdateSnapshot:
"""Create a new UpdateSnapshot to produce a new snapshot for the table.
Returns:
A new UpdateSnapshot
"""
- return UpdateSnapshot(self, io=self._table.io, snapshot_properties=snapshot_properties)
+ if branch is None:
+ branch = MAIN_BRANCH
+
+ return UpdateSnapshot(self, io=self._table.io, branch=branch, snapshot_properties=snapshot_properties)
def rewrite_manifests(self) -> RewriteManifestsResult:
if self._table.current_snapshot() is None:
@@ -455,13 +458,14 @@ def update_statistics(self) -> UpdateStatistics:
"""
return UpdateStatistics(transaction=self)
- def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None:
+ def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT, branch: Optional[str] = None) -> None:
"""
Shorthand API for appending a PyArrow table to a table transaction.
Args:
df: The Arrow dataframe that will be appended to overwrite the table
snapshot_properties: Custom properties to be added to the snapshot summary
+ branch: Branch Reference to run the append operation
"""
try:
import pyarrow as pa
@@ -484,7 +488,7 @@ def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT)
self.table_metadata.schema(), provided_schema=df.schema, downcast_ns_timestamp_to_us=downcast_ns_timestamp_to_us
)
- with self._append_snapshot_producer(snapshot_properties) as append_files:
+ with self._append_snapshot_producer(snapshot_properties, branch=branch) as append_files:
# skip writing data files if the dataframe is empty
if df.shape[0] > 0:
data_files = list(
@@ -495,7 +499,9 @@ def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT)
for data_file in data_files:
append_files.append_data_file(data_file)
- def dynamic_partition_overwrite(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None:
+ def dynamic_partition_overwrite(
+ self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT, branch: Optional[str] = None
+ ) -> None:
"""
Shorthand for overwriting existing partitions with a PyArrow table.
@@ -506,6 +512,7 @@ def dynamic_partition_overwrite(self, df: pa.Table, snapshot_properties: Dict[st
Args:
df: The Arrow dataframe that will be used to overwrite the table
snapshot_properties: Custom properties to be added to the snapshot summary
+ branch: Branch Reference to run the dynamic partition overwrite operation
"""
try:
import pyarrow as pa
@@ -544,9 +551,9 @@ def dynamic_partition_overwrite(self, df: pa.Table, snapshot_properties: Dict[st
partitions_to_overwrite = {data_file.partition for data_file in data_files}
delete_filter = self._build_partition_predicate(partition_records=partitions_to_overwrite)
- self.delete(delete_filter=delete_filter, snapshot_properties=snapshot_properties)
+ self.delete(delete_filter=delete_filter, snapshot_properties=snapshot_properties, branch=branch)
- with self._append_snapshot_producer(snapshot_properties) as append_files:
+ with self._append_snapshot_producer(snapshot_properties, branch=branch) as append_files:
append_files.commit_uuid = append_snapshot_commit_uuid
for data_file in data_files:
append_files.append_data_file(data_file)
@@ -557,6 +564,7 @@ def overwrite(
overwrite_filter: Union[BooleanExpression, str] = ALWAYS_TRUE,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
case_sensitive: bool = True,
+ branch: Optional[str] = None,
) -> None:
"""
Shorthand for adding a table overwrite with a PyArrow table to the transaction.
@@ -571,8 +579,9 @@ def overwrite(
df: The Arrow dataframe that will be used to overwrite the table
overwrite_filter: ALWAYS_TRUE when you overwrite all the data,
or a boolean expression in case of a partial overwrite
- case_sensitive: A bool determine if the provided `overwrite_filter` is case-sensitive
snapshot_properties: Custom properties to be added to the snapshot summary
+ case_sensitive: A bool determine if the provided `overwrite_filter` is case-sensitive
+ branch: Branch Reference to run the overwrite operation
"""
try:
import pyarrow as pa
@@ -597,9 +606,14 @@ def overwrite(
if overwrite_filter != AlwaysFalse():
# Only delete when the filter is != AlwaysFalse
- self.delete(delete_filter=overwrite_filter, case_sensitive=case_sensitive, snapshot_properties=snapshot_properties)
+ self.delete(
+ delete_filter=overwrite_filter,
+ case_sensitive=case_sensitive,
+ snapshot_properties=snapshot_properties,
+ branch=branch,
+ )
- with self._append_snapshot_producer(snapshot_properties) as append_files:
+ with self._append_snapshot_producer(snapshot_properties, branch=branch) as append_files:
# skip writing data files if the dataframe is empty
if df.shape[0] > 0:
data_files = _dataframe_to_data_files(
@@ -613,6 +627,7 @@ def delete(
delete_filter: Union[str, BooleanExpression],
snapshot_properties: Dict[str, str] = EMPTY_DICT,
case_sensitive: bool = True,
+ branch: Optional[str] = None,
) -> None:
"""
Shorthand for deleting record from a table.
@@ -626,6 +641,7 @@ def delete(
delete_filter: A boolean expression to delete rows from a table
snapshot_properties: Custom properties to be added to the snapshot summary
case_sensitive: A bool determine if the provided `delete_filter` is case-sensitive
+ branch: Branch Reference to run the delete operation
"""
from pyiceberg.io.pyarrow import (
ArrowScan,
@@ -642,7 +658,7 @@ def delete(
if isinstance(delete_filter, str):
delete_filter = _parse_row_filter(delete_filter)
- with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot:
+ with self.update_snapshot(snapshot_properties=snapshot_properties, branch=branch).delete() as delete_snapshot:
delete_snapshot.delete_by_predicate(delete_filter, case_sensitive)
# Check if there are any files that require an actual rewrite of a data file
@@ -650,7 +666,10 @@ def delete(
bound_delete_filter = bind(self.table_metadata.schema(), delete_filter, case_sensitive)
preserve_row_filter = _expression_to_complementary_pyarrow(bound_delete_filter)
- files = self._scan(row_filter=delete_filter, case_sensitive=case_sensitive).plan_files()
+ file_scan = self._scan(row_filter=delete_filter, case_sensitive=case_sensitive)
+ if branch is not None:
+ file_scan = file_scan.use_ref(branch)
+ files = file_scan.plan_files()
commit_uuid = uuid.uuid4()
counter = itertools.count(0)
@@ -692,7 +711,9 @@ def delete(
)
if len(replaced_files) > 0:
- with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as overwrite_snapshot:
+ with self.update_snapshot(
+ snapshot_properties=snapshot_properties, branch=branch
+ ).overwrite() as overwrite_snapshot:
overwrite_snapshot.commit_uuid = commit_uuid
for original_data_file, replaced_data_files in replaced_files:
overwrite_snapshot.delete_data_file(original_data_file)
@@ -709,6 +730,7 @@ def upsert(
when_matched_update_all: bool = True,
when_not_matched_insert_all: bool = True,
case_sensitive: bool = True,
+ branch: Optional[str] = None,
) -> UpsertResult:
"""Shorthand API for performing an upsert to an iceberg table.
@@ -719,6 +741,7 @@ def upsert(
when_matched_update_all: Bool indicating to update rows that are matched but require an update due to a value in a non-key column changing
when_not_matched_insert_all: Bool indicating new rows to be inserted that do not match any existing rows in the table
case_sensitive: Bool indicating if the match should be case-sensitive
+ branch: Branch Reference to run the upsert operation
To learn more about the identifier-field-ids: https://iceberg.apache.org/spec/#identifier-field-ids
@@ -781,12 +804,18 @@ def upsert(
matched_predicate = upsert_util.create_match_filter(df, join_cols)
# We must use Transaction.table_metadata for the scan. This includes all uncommitted - but relevant - changes.
- matched_iceberg_record_batches = DataScan(
+
+ matched_iceberg_record_batches_scan = DataScan(
table_metadata=self.table_metadata,
io=self._table.io,
row_filter=matched_predicate,
case_sensitive=case_sensitive,
- ).to_arrow_batch_reader()
+ )
+
+ if branch is not None:
+ matched_iceberg_record_batches_scan = matched_iceberg_record_batches_scan.use_ref(branch)
+
+ matched_iceberg_record_batches = matched_iceberg_record_batches_scan.to_arrow_batch_reader()
batches_to_overwrite = []
overwrite_predicates = []
@@ -825,12 +854,13 @@ def upsert(
self.overwrite(
rows_to_update,
overwrite_filter=Or(*overwrite_predicates) if len(overwrite_predicates) > 1 else overwrite_predicates[0],
+ branch=branch,
)
if when_not_matched_insert_all:
insert_row_cnt = len(rows_to_insert)
if rows_to_insert:
- self.append(rows_to_insert)
+ self.append(rows_to_insert, branch=branch)
return UpsertResult(rows_updated=update_row_cnt, rows_inserted=insert_row_cnt)
@@ -1271,6 +1301,7 @@ def upsert(
when_matched_update_all: bool = True,
when_not_matched_insert_all: bool = True,
case_sensitive: bool = True,
+ branch: Optional[str] = None,
) -> UpsertResult:
"""Shorthand API for performing an upsert to an iceberg table.
@@ -1281,6 +1312,7 @@ def upsert(
when_matched_update_all: Bool indicating to update rows that are matched but require an update due to a value in a non-key column changing
when_not_matched_insert_all: Bool indicating new rows to be inserted that do not match any existing rows in the table
case_sensitive: Bool indicating if the match should be case-sensitive
+ branch: Branch Reference to run the upsert operation
To learn more about the identifier-field-ids: https://iceberg.apache.org/spec/#identifier-field-ids
@@ -1313,29 +1345,34 @@ def upsert(
when_matched_update_all=when_matched_update_all,
when_not_matched_insert_all=when_not_matched_insert_all,
case_sensitive=case_sensitive,
+ branch=branch,
)
- def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None:
+ def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT, branch: Optional[str] = None) -> None:
"""
Shorthand API for appending a PyArrow table to the table.
Args:
df: The Arrow dataframe that will be appended to overwrite the table
snapshot_properties: Custom properties to be added to the snapshot summary
+ branch: Branch Reference to run the append operation
"""
with self.transaction() as tx:
- tx.append(df=df, snapshot_properties=snapshot_properties)
+ tx.append(df=df, snapshot_properties=snapshot_properties, branch=branch)
- def dynamic_partition_overwrite(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None:
+ def dynamic_partition_overwrite(
+ self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT, branch: Optional[str] = None
+ ) -> None:
"""Shorthand for dynamic overwriting the table with a PyArrow table.
Old partitions are auto detected and replaced with data files created for input arrow table.
Args:
df: The Arrow dataframe that will be used to overwrite the table
snapshot_properties: Custom properties to be added to the snapshot summary
+ branch: Branch Reference to run the dynamic partition overwrite operation
"""
with self.transaction() as tx:
- tx.dynamic_partition_overwrite(df=df, snapshot_properties=snapshot_properties)
+ tx.dynamic_partition_overwrite(df=df, snapshot_properties=snapshot_properties, branch=branch)
def overwrite(
self,
@@ -1343,6 +1380,7 @@ def overwrite(
overwrite_filter: Union[BooleanExpression, str] = ALWAYS_TRUE,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
case_sensitive: bool = True,
+ branch: Optional[str] = None,
) -> None:
"""
Shorthand for overwriting the table with a PyArrow table.
@@ -1359,10 +1397,15 @@ def overwrite(
or a boolean expression in case of a partial overwrite
snapshot_properties: Custom properties to be added to the snapshot summary
case_sensitive: A bool determine if the provided `overwrite_filter` is case-sensitive
+ branch: Branch Reference to run the overwrite operation
"""
with self.transaction() as tx:
tx.overwrite(
- df=df, overwrite_filter=overwrite_filter, case_sensitive=case_sensitive, snapshot_properties=snapshot_properties
+ df=df,
+ overwrite_filter=overwrite_filter,
+ case_sensitive=case_sensitive,
+ snapshot_properties=snapshot_properties,
+ branch=branch,
)
def delete(
@@ -1370,6 +1413,7 @@ def delete(
delete_filter: Union[BooleanExpression, str] = ALWAYS_TRUE,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
case_sensitive: bool = True,
+ branch: Optional[str] = None,
) -> None:
"""
Shorthand for deleting rows from the table.
@@ -1378,9 +1422,12 @@ def delete(
delete_filter: The predicate that used to remove rows
snapshot_properties: Custom properties to be added to the snapshot summary
case_sensitive: A bool determine if the provided `delete_filter` is case-sensitive
+ branch: Branch Reference to run the delete operation
"""
with self.transaction() as tx:
- tx.delete(delete_filter=delete_filter, case_sensitive=case_sensitive, snapshot_properties=snapshot_properties)
+ tx.delete(
+ delete_filter=delete_filter, case_sensitive=case_sensitive, snapshot_properties=snapshot_properties, branch=branch
+ )
def add_files(
self, file_paths: List[str], snapshot_properties: Dict[str, str] = EMPTY_DICT, check_duplicate_files: bool = True
diff --git a/pyiceberg/table/update/__init__.py b/pyiceberg/table/update/__init__.py
index 4905c31bfb..6653f119f0 100644
--- a/pyiceberg/table/update/__init__.py
+++ b/pyiceberg/table/update/__init__.py
@@ -29,7 +29,7 @@
from pyiceberg.partitioning import PARTITION_FIELD_ID_START, PartitionSpec
from pyiceberg.schema import Schema
from pyiceberg.table.metadata import SUPPORTED_TABLE_FORMAT_VERSION, TableMetadata, TableMetadataUtil
-from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRef
+from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
from pyiceberg.table.snapshots import (
MetadataLogEntry,
Snapshot,
@@ -139,7 +139,7 @@ class AddSnapshotUpdate(IcebergBaseModel):
class SetSnapshotRefUpdate(IcebergBaseModel):
action: Literal["set-snapshot-ref"] = Field(default="set-snapshot-ref")
ref_name: str = Field(alias="ref-name")
- type: Literal["tag", "branch"]
+ type: Literal[SnapshotRefType.TAG, SnapshotRefType.BRANCH]
snapshot_id: int = Field(alias="snapshot-id")
max_ref_age_ms: Annotated[Optional[int], Field(alias="max-ref-age-ms", default=None)]
max_snapshot_age_ms: Annotated[Optional[int], Field(alias="max-snapshot-age-ms", default=None)]
@@ -702,6 +702,10 @@ class AssertRefSnapshotId(ValidatableTableRequirement):
def validate(self, base_metadata: Optional[TableMetadata]) -> None:
if base_metadata is None:
raise CommitFailedException("Requirement failed: current table metadata is missing")
+ elif len(base_metadata.snapshots) == 0 and self.ref != MAIN_BRANCH:
+ raise CommitFailedException(
+ f"Requirement failed: Table has no snapshots and can only be written to the {MAIN_BRANCH} BRANCH."
+ )
elif snapshot_ref := base_metadata.refs.get(self.ref):
ref_type = snapshot_ref.snapshot_ref_type
if self.snapshot_id is None:
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 264c5242fc..ef1a6c2e1f 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -57,7 +57,7 @@
from pyiceberg.partitioning import (
PartitionSpec,
)
-from pyiceberg.table.refs import SnapshotRefType
+from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRefType
from pyiceberg.table.snapshots import (
Operation,
Snapshot,
@@ -109,6 +109,7 @@ class _SnapshotProducer(UpdateTableMetadata[U], Generic[U]):
_manifest_num_counter: itertools.count[int]
_deleted_data_files: Set[DataFile]
_compression: AvroCompressionCodec
+ _target_branch = MAIN_BRANCH
def __init__(
self,
@@ -117,16 +118,13 @@ def __init__(
io: FileIO,
commit_uuid: Optional[uuid.UUID] = None,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ branch: str = MAIN_BRANCH,
) -> None:
super().__init__(transaction)
self.commit_uuid = commit_uuid or uuid.uuid4()
self._io = io
self._operation = operation
self._snapshot_id = self._transaction.table_metadata.new_snapshot_id()
- # Since we only support the main branch for now
- self._parent_snapshot_id = (
- snapshot.snapshot_id if (snapshot := self._transaction.table_metadata.current_snapshot()) else None
- )
self._added_data_files = []
self._deleted_data_files = set()
self.snapshot_properties = snapshot_properties
@@ -136,6 +134,20 @@ def __init__(
self._compression = self._transaction.table_metadata.properties.get( # type: ignore
TableProperties.WRITE_AVRO_COMPRESSION, TableProperties.WRITE_AVRO_COMPRESSION_DEFAULT
)
+ self._target_branch = self._validate_target_branch(branch=branch)
+ self._parent_snapshot_id = (
+ snapshot.snapshot_id if (snapshot := self._transaction.table_metadata.snapshot_by_name(self._target_branch)) else None
+ )
+
+ def _validate_target_branch(self, branch: str) -> str:
+ # Default is already set to MAIN_BRANCH. So branch name can't be None.
+ if branch is None:
+ raise ValueError("Invalid branch name: null")
+ if branch in self._transaction.table_metadata.refs:
+ ref = self._transaction.table_metadata.refs[branch]
+ if ref.snapshot_ref_type != SnapshotRefType.BRANCH:
+ raise ValueError(f"{branch} is a tag, not a branch. Tags cannot be targets for producing snapshots")
+ return branch
def append_data_file(self, data_file: DataFile) -> _SnapshotProducer[U]:
self._added_data_files.append(data_file)
@@ -285,10 +297,20 @@ def _commit(self) -> UpdatesAndRequirements:
(
AddSnapshotUpdate(snapshot=snapshot),
SetSnapshotRefUpdate(
- snapshot_id=self._snapshot_id, parent_snapshot_id=self._parent_snapshot_id, ref_name="main", type="branch"
+ snapshot_id=self._snapshot_id,
+ parent_snapshot_id=self._parent_snapshot_id,
+ ref_name=self._target_branch,
+ type=SnapshotRefType.BRANCH,
+ ),
+ ),
+ (
+ AssertRefSnapshotId(
+ snapshot_id=self._transaction.table_metadata.refs[self._target_branch].snapshot_id
+ if self._target_branch in self._transaction.table_metadata.refs
+ else None,
+ ref=self._target_branch,
),
),
- (AssertRefSnapshotId(snapshot_id=self._transaction.table_metadata.current_snapshot_id, ref="main"),),
)
@property
@@ -336,10 +358,11 @@ def __init__(
operation: Operation,
transaction: Transaction,
io: FileIO,
+ branch: str,
commit_uuid: Optional[uuid.UUID] = None,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
):
- super().__init__(operation, transaction, io, commit_uuid, snapshot_properties)
+ super().__init__(operation, transaction, io, commit_uuid, snapshot_properties, branch)
self._predicate = AlwaysFalse()
self._case_sensitive = True
@@ -399,47 +422,53 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
total_deleted_entries = []
partial_rewrites_needed = False
self._deleted_data_files = set()
- if snapshot := self._transaction.table_metadata.current_snapshot():
- for manifest_file in snapshot.manifests(io=self._io):
- if manifest_file.content == ManifestContent.DATA:
- if not manifest_evaluators[manifest_file.partition_spec_id](manifest_file):
- # If the manifest isn't relevant, we can just keep it in the manifest-list
- existing_manifests.append(manifest_file)
- else:
- # It is relevant, let's check out the content
- deleted_entries = []
- existing_entries = []
- for entry in manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True):
- if strict_metrics_evaluator(entry.data_file) == ROWS_MUST_MATCH:
- # Based on the metadata, it can be dropped right away
- deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED))
- self._deleted_data_files.add(entry.data_file)
- else:
- # Based on the metadata, we cannot determine if it can be deleted
- existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING))
- if inclusive_metrics_evaluator(entry.data_file) != ROWS_MIGHT_NOT_MATCH:
- partial_rewrites_needed = True
-
- if len(deleted_entries) > 0:
- total_deleted_entries += deleted_entries
-
- # Rewrite the manifest
- if len(existing_entries) > 0:
- with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
- avro_compression=self._compression,
- ) as writer:
- for existing_entry in existing_entries:
- writer.add_entry(existing_entry)
- existing_manifests.append(writer.to_manifest_file())
- else:
+
+ # Determine the snapshot to read manifests from for deletion
+ # Should be the current tip of the _target_branch
+ parent_snapshot_id_for_delete_source = self._parent_snapshot_id
+ if parent_snapshot_id_for_delete_source is not None:
+ snapshot = self._transaction.table_metadata.snapshot_by_id(parent_snapshot_id_for_delete_source)
+ if snapshot: # Ensure snapshot is found
+ for manifest_file in snapshot.manifests(io=self._io):
+ if manifest_file.content == ManifestContent.DATA:
+ if not manifest_evaluators[manifest_file.partition_spec_id](manifest_file):
+ # If the manifest isn't relevant, we can just keep it in the manifest-list
existing_manifests.append(manifest_file)
- else:
- existing_manifests.append(manifest_file)
+ else:
+ # It is relevant, let's check out the content
+ deleted_entries = []
+ existing_entries = []
+ for entry in manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True):
+ if strict_metrics_evaluator(entry.data_file) == ROWS_MUST_MATCH:
+ # Based on the metadata, it can be dropped right away
+ deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED))
+ self._deleted_data_files.add(entry.data_file)
+ else:
+ # Based on the metadata, we cannot determine if it can be deleted
+ existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING))
+ if inclusive_metrics_evaluator(entry.data_file) != ROWS_MIGHT_NOT_MATCH:
+ partial_rewrites_needed = True
+
+ if len(deleted_entries) > 0:
+ total_deleted_entries += deleted_entries
+
+ # Rewrite the manifest
+ if len(existing_entries) > 0:
+ with write_manifest(
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
+ avro_compression=self._compression,
+ ) as writer:
+ for existing_entry in existing_entries:
+ writer.add_entry(existing_entry)
+ existing_manifests.append(writer.to_manifest_file())
+ else:
+ existing_manifests.append(manifest_file)
+ else:
+ existing_manifests.append(manifest_file)
return existing_manifests, total_deleted_entries, partial_rewrites_needed
@@ -505,12 +534,13 @@ def __init__(
operation: Operation,
transaction: Transaction,
io: FileIO,
+ branch: str,
commit_uuid: Optional[uuid.UUID] = None,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
from pyiceberg.table import TableProperties
- super().__init__(operation, transaction, io, commit_uuid, snapshot_properties)
+ super().__init__(operation, transaction, io, commit_uuid, snapshot_properties, branch)
self._target_size_bytes = property_as_int(
self._transaction.table_metadata.properties,
TableProperties.MANIFEST_TARGET_SIZE_BYTES,
@@ -703,7 +733,7 @@ def _existing_manifests(self) -> List[ManifestFile]:
"""Determine if there are any existing manifest files."""
existing_files = []
- if snapshot := self._transaction.table_metadata.current_snapshot():
+ if snapshot := self._transaction.table_metadata.snapshot_by_name(name=self._target_branch):
for manifest_file in snapshot.manifests(io=self._io):
entries = manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True)
found_deleted_data_files = [entry.data_file for entry in entries if entry.data_file in self._deleted_data_files]
@@ -721,19 +751,17 @@ def _existing_manifests(self) -> List[ManifestFile]:
snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
- [
- writer.add_entry(
- ManifestEntry.from_args(
- status=ManifestEntryStatus.EXISTING,
- snapshot_id=entry.snapshot_id,
- sequence_number=entry.sequence_number,
- file_sequence_number=entry.file_sequence_number,
- data_file=entry.data_file,
+ for entry in entries:
+ if entry.data_file not in found_deleted_data_files:
+ writer.add_entry(
+ ManifestEntry.from_args(
+ status=ManifestEntryStatus.EXISTING,
+ snapshot_id=entry.snapshot_id,
+ sequence_number=entry.sequence_number,
+ file_sequence_number=entry.file_sequence_number,
+ data_file=entry.data_file,
+ )
)
- )
- for entry in entries
- if entry.data_file not in found_deleted_data_files
- ]
existing_files.append(writer.to_manifest_file())
return existing_files
@@ -774,21 +802,37 @@ def _get_entries(manifest: ManifestFile) -> List[ManifestEntry]:
class UpdateSnapshot:
_transaction: Transaction
_io: FileIO
+ _branch: str
_snapshot_properties: Dict[str, str]
- def __init__(self, transaction: Transaction, io: FileIO, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None:
+ def __init__(
+ self,
+ transaction: Transaction,
+ io: FileIO,
+ branch: str,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ ) -> None:
self._transaction = transaction
self._io = io
self._snapshot_properties = snapshot_properties
+ self._branch = branch
def fast_append(self) -> _FastAppendFiles:
return _FastAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
+ operation=Operation.APPEND,
+ transaction=self._transaction,
+ io=self._io,
+ branch=self._branch,
+ snapshot_properties=self._snapshot_properties,
)
def merge_append(self) -> _MergeAppendFiles:
return _MergeAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties
+ operation=Operation.APPEND,
+ transaction=self._transaction,
+ io=self._io,
+ branch=self._branch,
+ snapshot_properties=self._snapshot_properties,
)
def rewrite(self) -> _RewriteManifests:
@@ -803,10 +847,11 @@ def overwrite(self, commit_uuid: Optional[uuid.UUID] = None) -> _OverwriteFiles:
return _OverwriteFiles(
commit_uuid=commit_uuid,
operation=Operation.OVERWRITE
- if self._transaction.table_metadata.current_snapshot() is not None
+ if self._transaction.table_metadata.snapshot_by_name(name=self._branch) is not None
else Operation.APPEND,
transaction=self._transaction,
io=self._io,
+ branch=self._branch,
snapshot_properties=self._snapshot_properties,
)
@@ -815,6 +860,7 @@ def delete(self) -> _DeleteFiles:
operation=Operation.DELETE,
transaction=self._transaction,
io=self._io,
+ branch=self._branch,
snapshot_properties=self._snapshot_properties,
)
diff --git a/pyiceberg/utils/concurrent.py b/pyiceberg/utils/concurrent.py
index 805599bf41..751cbd9bbb 100644
--- a/pyiceberg/utils/concurrent.py
+++ b/pyiceberg/utils/concurrent.py
@@ -25,6 +25,11 @@
class ExecutorFactory:
_instance: Optional[Executor] = None
+ @staticmethod
+ def max_workers() -> Optional[int]:
+ """Return the max number of workers configured."""
+ return Config().get_int("max-workers")
+
@staticmethod
def get_or_create() -> Executor:
"""Return the same executor in each call."""
@@ -33,8 +38,3 @@ def get_or_create() -> Executor:
ExecutorFactory._instance = ThreadPoolExecutor(max_workers=max_workers)
return ExecutorFactory._instance
-
- @staticmethod
- def max_workers() -> Optional[int]:
- """Return the max number of workers configured."""
- return Config().get_int("max-workers")
diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py
index 527f659640..abf8502ac7 100644
--- a/tests/integration/test_deletes.py
+++ b/tests/integration/test_deletes.py
@@ -894,3 +894,32 @@ def test_overwrite_with_filter_case_insensitive(test_table: Table) -> None:
test_table.overwrite(df=new_table, overwrite_filter=f"Idx == {record_to_overwrite['idx']}", case_sensitive=False)
assert record_to_overwrite not in test_table.scan().to_arrow().to_pylist()
assert new_record_to_insert in test_table.scan().to_arrow().to_pylist()
+
+
+@pytest.mark.integration
+@pytest.mark.parametrize("format_version", [1, 2])
+@pytest.mark.filterwarnings("ignore:Delete operation did not match any records")
+def test_delete_on_empty_table(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None:
+ identifier = f"default.test_delete_on_empty_table_{format_version}"
+
+ run_spark_commands(
+ spark,
+ [
+ f"DROP TABLE IF EXISTS {identifier}",
+ f"""
+ CREATE TABLE {identifier} (
+ volume int
+ )
+ USING iceberg
+ TBLPROPERTIES('format-version' = {format_version})
+ """,
+ ],
+ )
+
+ tbl = session_catalog.load_table(identifier)
+
+ # Perform a delete operation on the empty table
+ tbl.delete(AlwaysTrue())
+
+ # Assert that no new snapshot was created because no rows were deleted
+ assert len(tbl.snapshots()) == 0
diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py
index 033a9f7c0d..b66601f683 100644
--- a/tests/integration/test_writes/test_writes.py
+++ b/tests/integration/test_writes/test_writes.py
@@ -41,12 +41,13 @@
from pyiceberg.catalog import Catalog, load_catalog
from pyiceberg.catalog.hive import HiveCatalog
from pyiceberg.catalog.sql import SqlCatalog
-from pyiceberg.exceptions import NoSuchTableError
+from pyiceberg.exceptions import CommitFailedException, NoSuchTableError
from pyiceberg.expressions import And, EqualTo, GreaterThanOrEqual, In, LessThan, Not
from pyiceberg.io.pyarrow import _dataframe_to_data_files
from pyiceberg.partitioning import PartitionField, PartitionSpec
from pyiceberg.schema import Schema
from pyiceberg.table import TableProperties
+from pyiceberg.table.refs import MAIN_BRANCH
from pyiceberg.table.sorting import SortDirection, SortField, SortOrder
from pyiceberg.transforms import DayTransform, HourTransform, IdentityTransform
from pyiceberg.types import (
@@ -1856,3 +1857,160 @@ def test_avro_compression_codecs(session_catalog: Catalog, arrow_table_with_null
with tbl.io.new_input(current_snapshot.manifest_list).open() as f:
reader = fastavro.reader(f)
assert reader.codec == "null"
+
+
+@pytest.mark.integration
+def test_append_to_non_existing_branch(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_non_existing_branch"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [])
+ with pytest.raises(
+ CommitFailedException, match=f"Table has no snapshots and can only be written to the {MAIN_BRANCH} BRANCH."
+ ):
+ tbl.append(arrow_table_with_null, branch="non_existing_branch")
+
+
+@pytest.mark.integration
+def test_append_to_existing_branch(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_existing_branch_append"
+ branch = "existing_branch"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_null])
+
+ assert tbl.metadata.current_snapshot_id is not None
+
+ tbl.manage_snapshots().create_branch(snapshot_id=tbl.metadata.current_snapshot_id, branch_name=branch).commit()
+ tbl.append(arrow_table_with_null, branch=branch)
+
+ assert len(tbl.scan().use_ref(branch).to_arrow()) == 6
+ assert len(tbl.scan().to_arrow()) == 3
+ branch_snapshot = tbl.metadata.snapshot_by_name(branch)
+ assert branch_snapshot is not None
+ main_snapshot = tbl.metadata.snapshot_by_name("main")
+ assert main_snapshot is not None
+ assert branch_snapshot.parent_snapshot_id == main_snapshot.snapshot_id
+
+
+@pytest.mark.integration
+def test_delete_to_existing_branch(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_existing_branch_delete"
+ branch = "existing_branch"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_null])
+
+ assert tbl.metadata.current_snapshot_id is not None
+
+ tbl.manage_snapshots().create_branch(snapshot_id=tbl.metadata.current_snapshot_id, branch_name=branch).commit()
+ tbl.delete(delete_filter="int = 9", branch=branch)
+
+ assert len(tbl.scan().use_ref(branch).to_arrow()) == 2
+ assert len(tbl.scan().to_arrow()) == 3
+ branch_snapshot = tbl.metadata.snapshot_by_name(branch)
+ assert branch_snapshot is not None
+ main_snapshot = tbl.metadata.snapshot_by_name("main")
+ assert main_snapshot is not None
+ assert branch_snapshot.parent_snapshot_id == main_snapshot.snapshot_id
+
+
+@pytest.mark.integration
+def test_overwrite_to_existing_branch(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_existing_branch_overwrite"
+ branch = "existing_branch"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_null])
+
+ assert tbl.metadata.current_snapshot_id is not None
+
+ tbl.manage_snapshots().create_branch(snapshot_id=tbl.metadata.current_snapshot_id, branch_name=branch).commit()
+ tbl.overwrite(arrow_table_with_null, branch=branch)
+
+ assert len(tbl.scan().use_ref(branch).to_arrow()) == 3
+ assert len(tbl.scan().to_arrow()) == 3
+ branch_snapshot = tbl.metadata.snapshot_by_name(branch)
+ assert branch_snapshot is not None and branch_snapshot.parent_snapshot_id is not None
+ delete_snapshot = tbl.metadata.snapshot_by_id(branch_snapshot.parent_snapshot_id)
+ assert delete_snapshot is not None
+ main_snapshot = tbl.metadata.snapshot_by_name("main")
+ assert main_snapshot is not None
+ assert (
+ delete_snapshot.parent_snapshot_id == main_snapshot.snapshot_id
+ ) # Currently overwrite is a delete followed by an append operation
+
+
+@pytest.mark.integration
+def test_intertwined_branch_writes(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ identifier = "default.test_intertwined_branch_operations"
+ branch1 = "existing_branch_1"
+ branch2 = "existing_branch_2"
+
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_null])
+
+ assert tbl.metadata.current_snapshot_id is not None
+
+ tbl.manage_snapshots().create_branch(snapshot_id=tbl.metadata.current_snapshot_id, branch_name=branch1).commit()
+
+ tbl.delete("int = 9", branch=branch1)
+
+ tbl.append(arrow_table_with_null)
+
+ tbl.manage_snapshots().create_branch(snapshot_id=tbl.metadata.current_snapshot_id, branch_name=branch2).commit()
+
+ tbl.overwrite(arrow_table_with_null, branch=branch2)
+
+ assert len(tbl.scan().use_ref(branch1).to_arrow()) == 2
+ assert len(tbl.scan().use_ref(branch2).to_arrow()) == 3
+ assert len(tbl.scan().to_arrow()) == 6
+
+
+@pytest.mark.integration
+def test_branch_spark_write_py_read(session_catalog: Catalog, spark: SparkSession, arrow_table_with_null: pa.Table) -> None:
+ # Initialize table with branch
+ identifier = "default.test_branch_spark_write_py_read"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_null])
+ branch = "existing_spark_branch"
+
+ # Create branch in Spark
+ spark.sql(f"ALTER TABLE {identifier} CREATE BRANCH {branch}")
+
+ # Spark Write
+ spark.sql(
+ f"""
+ DELETE FROM {identifier}.branch_{branch}
+ WHERE int = 9
+ """
+ )
+
+ # Refresh table to get new refs
+ tbl.refresh()
+
+ # Python Read
+ assert len(tbl.scan().to_arrow()) == 3
+ assert len(tbl.scan().use_ref(branch).to_arrow()) == 2
+
+
+@pytest.mark.integration
+def test_branch_py_write_spark_read(session_catalog: Catalog, spark: SparkSession, arrow_table_with_null: pa.Table) -> None:
+ # Initialize table with branch
+ identifier = "default.test_branch_py_write_spark_read"
+ tbl = _create_table(session_catalog, identifier, {"format-version": "2"}, [arrow_table_with_null])
+ branch = "existing_py_branch"
+
+ assert tbl.metadata.current_snapshot_id is not None
+
+ # Create branch
+ tbl.manage_snapshots().create_branch(snapshot_id=tbl.metadata.current_snapshot_id, branch_name=branch).commit()
+
+ # Python Write
+ tbl.delete("int = 9", branch=branch)
+
+ # Spark Read
+ main_df = spark.sql(
+ f"""
+ SELECT *
+ FROM {identifier}
+ """
+ )
+ branch_df = spark.sql(
+ f"""
+ SELECT *
+ FROM {identifier}.branch_{branch}
+ """
+ )
+ assert main_df.count() == 3
+ assert branch_df.count() == 2
diff --git a/tests/table/test_init.py b/tests/table/test_init.py
index 6165dadec4..89524a861c 100644
--- a/tests/table/test_init.py
+++ b/tests/table/test_init.py
@@ -50,7 +50,7 @@
_match_deletes_to_data_file,
)
from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadataUtil, TableMetadataV2, _generate_snapshot_id
-from pyiceberg.table.refs import SnapshotRef
+from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
from pyiceberg.table.snapshots import (
MetadataLogEntry,
Operation,
@@ -1000,28 +1000,42 @@ def test_assert_table_uuid(table_v2: Table) -> None:
def test_assert_ref_snapshot_id(table_v2: Table) -> None:
base_metadata = table_v2.metadata
- AssertRefSnapshotId(ref="main", snapshot_id=base_metadata.current_snapshot_id).validate(base_metadata)
+ AssertRefSnapshotId(ref=MAIN_BRANCH, snapshot_id=base_metadata.current_snapshot_id).validate(base_metadata)
with pytest.raises(CommitFailedException, match="Requirement failed: current table metadata is missing"):
- AssertRefSnapshotId(ref="main", snapshot_id=1).validate(None)
+ AssertRefSnapshotId(ref=MAIN_BRANCH, snapshot_id=1).validate(None)
with pytest.raises(
CommitFailedException,
- match="Requirement failed: branch main was created concurrently",
+ match=f"Requirement failed: branch {MAIN_BRANCH} was created concurrently",
):
- AssertRefSnapshotId(ref="main", snapshot_id=None).validate(base_metadata)
+ AssertRefSnapshotId(ref=MAIN_BRANCH, snapshot_id=None).validate(base_metadata)
with pytest.raises(
CommitFailedException,
- match="Requirement failed: branch main has changed: expected id 1, found 3055729675574597004",
+ match=f"Requirement failed: branch {MAIN_BRANCH} has changed: expected id 1, found 3055729675574597004",
):
- AssertRefSnapshotId(ref="main", snapshot_id=1).validate(base_metadata)
+ AssertRefSnapshotId(ref=MAIN_BRANCH, snapshot_id=1).validate(base_metadata)
+
+ non_existing_ref = "not_exist_branch_or_tag"
+ assert table_v2.refs().get("not_exist_branch_or_tag") is None
+
+ with pytest.raises(
+ CommitFailedException,
+ match=f"Requirement failed: branch or tag {non_existing_ref} is missing, expected 1",
+ ):
+ AssertRefSnapshotId(ref=non_existing_ref, snapshot_id=1).validate(base_metadata)
+
+ # existing Tag in metadata: test
+ ref_tag = table_v2.refs().get("test")
+ assert ref_tag is not None
+ assert ref_tag.snapshot_ref_type == SnapshotRefType.TAG, "TAG test should be present in table to be tested"
with pytest.raises(
CommitFailedException,
- match="Requirement failed: branch or tag not_exist is missing, expected 1",
+ match="Requirement failed: tag test has changed: expected id 3055729675574597004, found 3051729675574597004",
):
- AssertRefSnapshotId(ref="not_exist", snapshot_id=1).validate(base_metadata)
+ AssertRefSnapshotId(ref="test", snapshot_id=3055729675574597004).validate(base_metadata)
def test_assert_last_assigned_field_id(table_v2: Table) -> None:
From 0b0640011a094c303431df3ccefec430ea2c62e0 Mon Sep 17 00:00:00 2001
From: frankliee
Date: Thu, 3 Jul 2025 17:04:46 +0800
Subject: [PATCH 099/112] Hive: update hive storage descriptor after commit
schema change (#2036)
# Rationale for this change
Like iceberg jar, we should also update hive storage descriptor after
commit metadata
see:
https://github.com/apache/iceberg/blob/b504f9c51c6c0e0a5c0c5ff53f295e69b67d8e59/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java#L170
# Are these changes tested?
new UTs
# Are there any user-facing changes?
No
---
dev/hive/core-site.xml | 5 ++++
pyiceberg/catalog/hive.py | 6 +++++
tests/integration/test_writes/test_writes.py | 24 ++++++++++++++++++++
3 files changed, 35 insertions(+)
diff --git a/dev/hive/core-site.xml b/dev/hive/core-site.xml
index b77332b83b..f5a9473b51 100644
--- a/dev/hive/core-site.xml
+++ b/dev/hive/core-site.xml
@@ -50,4 +50,9 @@
fs.s3a.path.style.access
true
+
+ hive.metastore.disallow.incompatible.col.type.changes
+ false
+
+
diff --git a/pyiceberg/catalog/hive.py b/pyiceberg/catalog/hive.py
index 09437dd1b6..cc9cd028c4 100644
--- a/pyiceberg/catalog/hive.py
+++ b/pyiceberg/catalog/hive.py
@@ -561,6 +561,12 @@ def commit_table(
previous_metadata_location=current_table.metadata_location,
metadata_properties=updated_staged_table.properties,
)
+ # Update hive's schema and properties
+ hive_table.sd = _construct_hive_storage_descriptor(
+ updated_staged_table.schema(),
+ updated_staged_table.location(),
+ property_as_bool(updated_staged_table.properties, HIVE2_COMPATIBLE, HIVE2_COMPATIBLE_DEFAULT),
+ )
open_client.alter_table_with_environment_context(
dbname=database_name,
tbl_name=table_name,
diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py
index b66601f683..30a0968680 100644
--- a/tests/integration/test_writes/test_writes.py
+++ b/tests/integration/test_writes/test_writes.py
@@ -1148,6 +1148,30 @@ def test_hive_catalog_storage_descriptor(
assert spark.sql("SELECT * FROM hive.default.test_storage_descriptor").count() == 3
+@pytest.mark.integration
+@pytest.mark.parametrize("format_version", [1, 2])
+def test_hive_catalog_storage_descriptor_has_changed(
+ session_catalog_hive: HiveCatalog,
+ pa_schema: pa.Schema,
+ arrow_table_with_null: pa.Table,
+ spark: SparkSession,
+ format_version: int,
+) -> None:
+ tbl = _create_table(
+ session_catalog_hive, "default.test_storage_descriptor", {"format-version": format_version}, [arrow_table_with_null]
+ )
+
+ with tbl.transaction() as tx:
+ with tx.update_schema() as schema:
+ schema.update_column("string_long", doc="this is string_long")
+ schema.update_column("binary", doc="this is binary")
+
+ with session_catalog_hive._client as open_client:
+ hive_table = session_catalog_hive._get_hive_table(open_client, "default", "test_storage_descriptor")
+ assert "this is string_long" in str(hive_table.sd)
+ assert "this is binary" in str(hive_table.sd)
+
+
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [pytest.lazy_fixture("session_catalog_hive"), pytest.lazy_fixture("session_catalog")])
def test_sanitize_character_partitioned(catalog: Catalog) -> None:
From fb8b82ddf65d28b99cd7003bb269e8db7d32ef5d Mon Sep 17 00:00:00 2001
From: summermousa-vendia
Date: Thu, 3 Jul 2025 04:15:06 -0500
Subject: [PATCH 100/112] fix: Propagate Table properties to Glue (#2097)
# Rationale for this change
See https://github.com/apache/iceberg-python/pull/2013
Closes https://github.com/apache/iceberg-python/issues/2064
Continuing the trend, but with glue.
# Are these changes tested?
See test below
# Are there any user-facing changes?
When a user specifies property update on commit table, those parameters
will be passed to the glue client.
---
pyiceberg/catalog/glue.py | 12 ++++++++++--
tests/catalog/test_glue.py | 2 ++
2 files changed, 12 insertions(+), 2 deletions(-)
diff --git a/pyiceberg/catalog/glue.py b/pyiceberg/catalog/glue.py
index 88ad8aa433..4eb4164e57 100644
--- a/pyiceberg/catalog/glue.py
+++ b/pyiceberg/catalog/glue.py
@@ -140,12 +140,20 @@
def _construct_parameters(
- metadata_location: str, glue_table: Optional["TableTypeDef"] = None, prev_metadata_location: Optional[str] = None
+ metadata_location: str,
+ glue_table: Optional["TableTypeDef"] = None,
+ prev_metadata_location: Optional[str] = None,
+ metadata_properties: Optional[Properties] = None,
) -> Properties:
new_parameters = glue_table.get("Parameters", {}) if glue_table else {}
new_parameters.update({TABLE_TYPE: ICEBERG.upper(), METADATA_LOCATION: metadata_location})
if prev_metadata_location:
new_parameters[PREVIOUS_METADATA_LOCATION] = prev_metadata_location
+
+ if metadata_properties:
+ for key, value in metadata_properties.items():
+ new_parameters[key] = str(value)
+
return new_parameters
@@ -236,7 +244,7 @@ def _construct_table_input(
table_input: "TableInputTypeDef" = {
"Name": table_name,
"TableType": EXTERNAL_TABLE,
- "Parameters": _construct_parameters(metadata_location, glue_table, prev_metadata_location),
+ "Parameters": _construct_parameters(metadata_location, glue_table, prev_metadata_location, properties),
"StorageDescriptor": {
"Columns": _to_columns(metadata),
"Location": metadata.location,
diff --git a/tests/catalog/test_glue.py b/tests/catalog/test_glue.py
index 399c9e8bb1..0ff43cd52b 100644
--- a/tests/catalog/test_glue.py
+++ b/tests/catalog/test_glue.py
@@ -791,6 +791,8 @@ def test_commit_table_properties(
Name=table_name,
)
assert table_info["Table"]["Description"] == "test_description"
+ assert table_info["Table"]["Parameters"]["test_a"] == "test_aa"
+ assert table_info["Table"]["Parameters"]["test_c"] == "test_c"
@mock_aws
From c29a36066000df029685665c52486be07ebbe594 Mon Sep 17 00:00:00 2001
From: Ethan Knox
Date: Thu, 3 Jul 2025 16:45:42 -0400
Subject: [PATCH 101/112] Documented `row_filter` expressions (#1862)
# Rationale for this change
scan's `row_filter` param is not super intuitive. I got tired of reading
over the expression and parser code as I'm trying to build out
statements, so I had some docs made up.
# Are these changes tested?
They are docs only, so not really?
# Are there any user-facing changes?
Yes there are docs for the expression and string syntaxes of
`row_filter` now.
---------
Co-authored-by: Fokko Driesprong
---
mkdocs/docs/SUMMARY.md | 2 +
mkdocs/docs/expression-dsl.md | 259 +++++++++++++++++++++++++++++++
mkdocs/docs/row-filter-syntax.md | 172 ++++++++++++++++++++
3 files changed, 433 insertions(+)
create mode 100644 mkdocs/docs/expression-dsl.md
create mode 100644 mkdocs/docs/row-filter-syntax.md
diff --git a/mkdocs/docs/SUMMARY.md b/mkdocs/docs/SUMMARY.md
index c344b2fdd2..d268bcc4b0 100644
--- a/mkdocs/docs/SUMMARY.md
+++ b/mkdocs/docs/SUMMARY.md
@@ -24,6 +24,8 @@
- [Configuration](configuration.md)
- [CLI](cli.md)
- [API](api.md)
+ - [Row Filter Syntax](row-filter-syntax.md)
+ - [Expression DSL](expression-dsl.md)
- [Contributing](contributing.md)
- [Community](community.md)
- Releases
diff --git a/mkdocs/docs/expression-dsl.md b/mkdocs/docs/expression-dsl.md
new file mode 100644
index 0000000000..e8d551d0e6
--- /dev/null
+++ b/mkdocs/docs/expression-dsl.md
@@ -0,0 +1,259 @@
+
+
+# Expression DSL
+
+The PyIceberg library provides a powerful expression DSL (Domain Specific Language) for building complex row filter expressions. This guide will help you understand how to use the expression DSL effectively. This DSL allows you to build type-safe expressions for use in the `row_filter` scan argument.
+
+They are composed of terms, predicates, and logical operators.
+
+## Basic Concepts
+
+### Terms
+
+Terms are the basic building blocks of expressions. They represent references to fields in your data:
+
+```python
+from pyiceberg.expressions import Reference
+
+# Create a reference to a field named "age"
+age_field = Reference("age")
+```
+
+### Predicates
+
+Predicates are expressions that evaluate to a boolean value. They can be combined using logical operators.
+
+#### Literal Predicates
+
+```python
+from pyiceberg.expressions import EqualTo, NotEqualTo, LessThan, LessThanOrEqual, GreaterThan, GreaterThanOrEqual
+
+# age equals 18
+age_equals_18 = EqualTo("age", 18)
+
+# age is not equal to 18
+age_not_equals_18 = NotEqualTo("age", 18)
+
+# age is less than 18
+age_less_than_18 = LessThan("age", 18)
+
+# Less than or equal to
+age_less_than_or_equal_18 = LessThanOrEqual("age", 18)
+
+# Greater than
+age_greater_than_18 = GreaterThan("age", 18)
+
+# Greater than or equal to
+age_greater_than_or_equal_18 = GreaterThanOrEqual("age", 18)
+```
+
+#### Set Predicates
+
+```python
+from pyiceberg.expressions import In, NotIn
+
+# age is one of 18, 19, 20
+age_in_set = In("age", [18, 19, 20])
+
+# age is not 18, 19, oer 20
+age_not_in_set = NotIn("age", [18, 19, 20])
+```
+
+#### Unary Predicates
+
+```python
+from pyiceberg.expressions import IsNull, NotNull
+
+# Is null
+name_is_null = IsNull("name")
+
+# Is not null
+name_is_not_null = NotNull("name")
+```
+
+#### String Predicates
+
+```python
+from pyiceberg.expressions import StartsWith, NotStartsWith
+
+# TRUE for 'Johnathan', FALSE for 'Johan'
+name_starts_with = StartsWith("name", "John")
+
+# FALSE for 'Johnathan', TRUE for 'Johan'
+name_not_starts_with = NotStartsWith("name", "John")
+```
+
+### Logical Operators
+
+You can combine predicates using logical operators:
+
+```python
+from pyiceberg.expressions import And, Or, Not
+
+# TRUE for 25, FALSE for 67 and 15
+age_between = And(
+ GreaterThanOrEqual("age", 18),
+ LessThanOrEqual("age", 65)
+)
+
+# FALSE for 25, TRUE for 67 and 15
+age_outside = Or(
+ LessThan("age", 18),
+ GreaterThan("age", 65)
+)
+
+# NOT operator
+not_adult = Not(GreaterThanOrEqual("age", 18))
+```
+
+## Advanced Usage
+
+### Complex Expressions
+
+You can build complex expressions by combining multiple predicates and operators:
+
+```python
+from pyiceberg.expressions import And, Or, Not, EqualTo, GreaterThan, LessThan, In
+
+# (age >= 18 AND age <= 65) AND (status = 'active' OR status = 'pending')
+complex_filter = And(
+ And(
+ GreaterThanOrEqual("age", 18),
+ LessThanOrEqual("age", 65)
+ ),
+ Or(
+ EqualTo("status", "active"),
+ EqualTo("status", "pending")
+ )
+)
+
+# NOT (age < 18 OR age > 65)
+age_in_range = Not(
+ Or(
+ LessThan("age", 18),
+ GreaterThan("age", 65)
+ )
+)
+```
+
+### Type Safety
+
+The expression DSL provides type safety through Python's type system. When you create expressions, the types are checked at runtime:
+
+```python
+from pyiceberg.expressions import EqualTo
+
+# This will work
+age_equals_18 = EqualTo("age", 18)
+
+# This will raise a TypeError if the field type doesn't match
+age_equals_18 = EqualTo("age", "18") # Will fail if age is an integer field
+```
+
+## Best Practices
+
+1. **Use Type Hints**: Always use type hints when working with expressions to catch type-related errors early.
+
+2. **Break Down Complex Expressions**: For complex expressions, break them down into smaller, more manageable parts:
+
+```python
+# Instead of this:
+complex_filter = And(
+ And(
+ GreaterThanOrEqual("age", 18),
+ LessThanOrEqual("age", 65)
+ ),
+ Or(
+ EqualTo("status", "active"),
+ EqualTo("status", "pending")
+ )
+)
+
+# Do this:
+age_range = And(
+ GreaterThanOrEqual("age", 18),
+ LessThanOrEqual("age", 65)
+)
+
+status_filter = Or(
+ EqualTo("status", "active"),
+ EqualTo("status", "pending")
+)
+
+complex_filter = And(age_range, status_filter)
+```
+
+## Common Pitfalls
+
+1. **Null Handling**: Be careful when using `IsNull` and `NotNull` predicates with required fields. The expression DSL will automatically optimize these cases:
+ - `IsNull` (and `IsNaN` for doubles/floats) on a required field will always return `False`
+ - `NotNull` (and `NotNaN` for doubles/floats) on a required field will always return `True`
+
+2. **String Comparisons**: When using string predicates like `StartsWith`, ensure that the field type is actually a string type.
+
+## Examples
+
+Here are some practical examples of using the expression DSL:
+
+### Basic Filtering
+
+```python
+
+from datetime import datetime
+from pyiceberg.expressions import (
+ And,
+ EqualTo,
+ GreaterThanOrEqual,
+ LessThanOrEqual,
+ GreaterThan,
+ In
+)
+
+active_adult_users_filter = And(
+ EqualTo("status", "active"),
+ GreaterThanOrEqual("age", 18)
+)
+
+
+high_value_customers = And(
+ GreaterThan("total_spent", 1000),
+ In("membership_level", ["gold", "platinum"])
+)
+
+date_range_filter = And(
+ GreaterThanOrEqual("created_at", datetime(2024, 1, 1)),
+ LessThanOrEqual("created_at", datetime(2024, 12, 31))
+)
+```
+
+### Multi-Condition Filter
+
+```python
+from pyiceberg.expressions import And, Or, Not, EqualTo, GreaterThan
+
+complex_filter = And(
+ Not(EqualTo("status", "deleted")),
+ Or(
+ And(
+ EqualTo("type", "premium"),
+ GreaterThan("subscription_months", 12)
+ ),
+ EqualTo("type", "enterprise")
+ )
+)
+```
\ No newline at end of file
diff --git a/mkdocs/docs/row-filter-syntax.md b/mkdocs/docs/row-filter-syntax.md
new file mode 100644
index 0000000000..45ce195e53
--- /dev/null
+++ b/mkdocs/docs/row-filter-syntax.md
@@ -0,0 +1,172 @@
+
+
+# Row Filter Syntax
+
+In addtion to the primary [Expression DSL](expression-dsl.md), PyIceberg provides a string-based statement interface for filtering rows in Iceberg tables. This guide explains the syntax and provides examples for supported operations.
+
+The row filter syntax is designed to be similar to SQL WHERE clauses. Here are the basic components:
+
+### Column References
+
+Columns can be referenced using either unquoted or quoted identifiers:
+
+```sql
+column_name
+"column.name"
+```
+
+### Literals
+
+The following literal types are supported:
+
+- Strings: `'hello world'`
+- Numbers: `42`, `-42`, `3.14`
+- Booleans: `true`, `false` (case insensitive)
+
+## Comparison Operations
+
+### Basic Comparisons
+
+```sql
+column = 42
+column != 42
+column > 42
+column >= 42
+column < 42
+column <= 42
+```
+
+!!! note
+ The `==` operator is an alias for `=` and `<>` is an alias for `!=`
+
+### String Comparisons
+
+```sql
+column = 'hello'
+column != 'world'
+```
+
+## NULL Checks
+
+Check for NULL values using the `IS NULL` and `IS NOT NULL` operators:
+
+```sql
+column IS NULL
+column IS NOT NULL
+```
+
+## NaN Checks
+
+For floating-point columns, you can check for NaN values:
+
+```sql
+column IS NAN
+column IS NOT NAN
+```
+
+## IN and NOT IN
+
+Check if a value is in a set of values:
+
+```sql
+column IN ('a', 'b', 'c')
+column NOT IN (1, 2, 3)
+```
+
+## LIKE Operations
+
+The LIKE operator supports pattern matching with a wildcard `%` at the end of the string:
+
+```sql
+column LIKE 'prefix%'
+column NOT LIKE 'prefix%'
+```
+
+!!! important
+ The `%` wildcard is only supported at the end of the pattern. Using it in the middle or beginning of the pattern will raise an error.
+
+## Logical Operations
+
+Combine multiple conditions using logical operators:
+
+```sql
+column1 = 42 AND column2 = 'hello'
+column1 > 0 OR column2 IS NULL
+NOT (column1 = 42)
+```
+
+!!! tip
+ Parentheses can be used to group logical operations for clarity:
+ ```sql
+ (column1 = 42 AND column2 = 'hello') OR column3 IS NULL
+ ```
+
+## Complete Examples
+
+Here are some complete examples showing how to combine different operations:
+
+```sql
+-- Complex filter with multiple conditions
+status = 'active' AND age > 18 AND NOT (country IN ('US', 'CA'))
+
+-- Filter with string pattern matching
+name LIKE 'John%' AND age >= 21
+
+-- Filter with NULL checks and numeric comparisons
+price IS NOT NULL AND price > 100 AND quantity > 0
+
+-- Filter with multiple logical operations
+(status = 'pending' OR status = 'processing') AND NOT (priority = 'low')
+```
+
+## Common Pitfalls
+
+1. **String Quoting**: Always use single quotes for string literals. Double quotes are reserved for column identifiers.
+ ```sql
+ -- Correct
+ name = 'John'
+
+ -- Incorrect
+ name = "John"
+ ```
+
+2. **Wildcard Usage**: The `%` wildcard in LIKE patterns can only appear at the end.
+ ```sql
+ -- Correct
+ name LIKE 'John%'
+
+ -- Incorrect (will raise an error)
+ name LIKE '%John%'
+ ```
+
+3. **Case Sensitivity**: Boolean literals (`true`/`false`) are case insensitive, but string comparisons are case sensitive.
+ ```sql
+ -- All valid
+ is_active = true
+ is_active = TRUE
+ is_active = True
+
+ -- Case sensitive
+ status = 'Active' -- Will not match 'active'
+ ```
+
+## Best Practices
+
+1. For complex use cases, use the primary [Expression DSL](expression-dsl.md)
+2. When using multiple conditions, consider the order of operations (NOT > AND > OR)
+3. For string comparisons, be consistent with case usage
\ No newline at end of file
From daa887d43868bf7b3dcea4e57418c90cb6c4a0d0 Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Fri, 4 Jul 2025 15:22:09 +0200
Subject: [PATCH 102/112] Fix the CI (#2171)
---
mkdocs/docs/expression-dsl.md | 2 +-
mkdocs/docs/row-filter-syntax.md | 11 +++++++----
2 files changed, 8 insertions(+), 5 deletions(-)
diff --git a/mkdocs/docs/expression-dsl.md b/mkdocs/docs/expression-dsl.md
index e8d551d0e6..7dd8b5137b 100644
--- a/mkdocs/docs/expression-dsl.md
+++ b/mkdocs/docs/expression-dsl.md
@@ -256,4 +256,4 @@ complex_filter = And(
EqualTo("type", "enterprise")
)
)
-```
\ No newline at end of file
+```
diff --git a/mkdocs/docs/row-filter-syntax.md b/mkdocs/docs/row-filter-syntax.md
index 45ce195e53..2191b9fd4a 100644
--- a/mkdocs/docs/row-filter-syntax.md
+++ b/mkdocs/docs/row-filter-syntax.md
@@ -17,11 +17,11 @@
# Row Filter Syntax
-In addtion to the primary [Expression DSL](expression-dsl.md), PyIceberg provides a string-based statement interface for filtering rows in Iceberg tables. This guide explains the syntax and provides examples for supported operations.
+In addition to the primary [Expression DSL](expression-dsl.md), PyIceberg provides a string-based statement interface for filtering rows in Iceberg tables. This guide explains the syntax and provides examples for supported operations.
The row filter syntax is designed to be similar to SQL WHERE clauses. Here are the basic components:
-### Column References
+## Column References
Columns can be referenced using either unquoted or quoted identifiers:
@@ -30,7 +30,7 @@ column_name
"column.name"
```
-### Literals
+## Literals
The following literal types are supported:
@@ -137,6 +137,7 @@ price IS NOT NULL AND price > 100 AND quantity > 0
## Common Pitfalls
1. **String Quoting**: Always use single quotes for string literals. Double quotes are reserved for column identifiers.
+
```sql
-- Correct
name = 'John'
@@ -146,6 +147,7 @@ price IS NOT NULL AND price > 100 AND quantity > 0
```
2. **Wildcard Usage**: The `%` wildcard in LIKE patterns can only appear at the end.
+
```sql
-- Correct
name LIKE 'John%'
@@ -155,6 +157,7 @@ price IS NOT NULL AND price > 100 AND quantity > 0
```
3. **Case Sensitivity**: Boolean literals (`true`/`false`) are case insensitive, but string comparisons are case sensitive.
+
```sql
-- All valid
is_active = true
@@ -169,4 +172,4 @@ price IS NOT NULL AND price > 100 AND quantity > 0
1. For complex use cases, use the primary [Expression DSL](expression-dsl.md)
2. When using multiple conditions, consider the order of operations (NOT > AND > OR)
-3. For string comparisons, be consistent with case usage
\ No newline at end of file
+3. For string comparisons, be consistent with case usage
From 5d67b6790a706cc659ce26fe6acbf2cf176864e0 Mon Sep 17 00:00:00 2001
From: Kevin Liu
Date: Sun, 6 Jul 2025 12:37:44 -0700
Subject: [PATCH 103/112] add iceberg datafusion integration (#2075)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
# Rationale for this change
- Added pyiceberg table integration so that pyiceberg `Table` can be
pass in directly to datafusion's `register_table_provider`
- Added `datafusion` as a optional dependency
- Added docs for the integration:
# Are these changes tested?
Yes
# Are there any user-facing changes?
---
mkdocs/docs/api.md | 63 +++++++++++++++++++++++++++++++++
poetry.lock | 42 +++++++++++-----------
pyiceberg/table/__init__.py | 46 ++++++++++++++++++++++++
pyproject.toml | 3 +-
tests/table/test_datafusion.py | 64 ++++++++++++++++++++++++++++++++++
5 files changed, 197 insertions(+), 21 deletions(-)
create mode 100644 tests/table/test_datafusion.py
diff --git a/mkdocs/docs/api.md b/mkdocs/docs/api.md
index d84c82ec2a..448993b4f6 100644
--- a/mkdocs/docs/api.md
+++ b/mkdocs/docs/api.md
@@ -1758,3 +1758,66 @@ shape: (11, 4)
│ 21 ┆ 566 ┆ Incorrect billing amount ┆ 2022-04-17 10:53:20 │
└───────────┴─────────────┴────────────────────────────┴─────────────────────┘
```
+
+### Apache DataFusion
+
+PyIceberg integrates with [Apache DataFusion](https://datafusion.apache.org/) through the Custom Table Provider interface ([FFI_TableProvider](https://datafusion.apache.org/python/user-guide/io/table_provider.html)) exposed through `iceberg-rust`.
+
+
+
+!!! note "Requirements"
+ This requires [`datafusion` to be installed](index.md).
+
+
+
+
+
+!!! warning "Experimental Feature"
+ The DataFusion integration is considered **experimental**.
+
+ The integration has a few caveats:
+
+ - Only works with `datafusion >= 45`
+ - Depends directly on `iceberg-rust` instead of PyIceberg's implementation
+ - Has limited features compared to the full PyIceberg API
+
+ The integration will improve as both DataFusion and `iceberg-rust` matures.
+
+
+
+PyIceberg tables can be registered directly with DataFusion's SessionContext using the table provider interface.
+
+```python
+from datafusion import SessionContext
+from pyiceberg.catalog import load_catalog
+import pyarrow as pa
+
+# Load catalog and create/load a table
+catalog = load_catalog("catalog", type="in-memory")
+catalog.create_namespace_if_not_exists("default")
+
+# Create some sample data
+data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]})
+iceberg_table = catalog.create_table("default.test", schema=data.schema)
+iceberg_table.append(data)
+
+# Register the table with DataFusion
+ctx = SessionContext()
+ctx.register_table_provider("test", iceberg_table)
+
+# Query the table using DataFusion SQL
+ctx.table("test").show()
+```
+
+This will output:
+
+```python
+DataFrame()
++---+---+
+| x | y |
++---+---+
+| 1 | 4 |
+| 2 | 5 |
+| 3 | 6 |
++---+---+
+```
diff --git a/poetry.lock b/poetry.lock
index 889b3abe7a..6aae0c58b0 100644
--- a/poetry.lock
+++ b/poetry.lock
@@ -59,7 +59,7 @@ description = "Happy Eyeballs for asyncio"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohappyeyeballs-2.6.1-py3-none-any.whl", hash = "sha256:f349ba8f4b75cb25c99c5c2d84e997e485204d2902a9597802b0371f09331fb8"},
{file = "aiohappyeyeballs-2.6.1.tar.gz", hash = "sha256:c3f9d0113123803ccadfdf3f0faa505bc78e6a72d1cc4806cbd719826e943558"},
@@ -72,7 +72,7 @@ description = "Async http client/server framework (asyncio)"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:5421af8f22a98f640261ee48aae3a37f0c41371e99412d55eaf2f8a46d5dad29"},
{file = "aiohttp-3.12.13-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0fcda86f6cb318ba36ed8f1396a6a4a3fd8f856f84d426584392083d10da4de0"},
@@ -202,7 +202,7 @@ description = "aiosignal: a list of registered asynchronous callbacks"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "aiosignal-1.3.2-py2.py3-none-any.whl", hash = "sha256:45cde58e409a301715980c2b01d0c28bdde3770d8290b5eb2173759d9acb31a5"},
{file = "aiosignal-1.3.2.tar.gz", hash = "sha256:a8c255c66fafb1e499c9351d0bf32ff2d8a0321595ebac3b93713656d2436f54"},
@@ -268,7 +268,7 @@ description = "Timeout context manager for asyncio programs"
optional = true
python-versions = ">=3.8"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and python_version <= \"3.10\""
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and python_version <= \"3.10\""
files = [
{file = "async_timeout-5.0.1-py3-none-any.whl", hash = "sha256:39e3809566ff85354557ec2398b55e096c8364bacac9405a7a1fa429e77fe76c"},
{file = "async_timeout-5.0.1.tar.gz", hash = "sha256:d9321a7a3d5a6a5e187e824d2fa0793ce379a202935782d555d6e9d2735677d3"},
@@ -285,7 +285,7 @@ files = [
{file = "attrs-25.3.0-py3-none-any.whl", hash = "sha256:427318ce031701fea540783410126f03899a97ffc6f61596ad581ac2e40e3bc3"},
{file = "attrs-25.3.0.tar.gz", hash = "sha256:75d7cefc7fb576747b2c81b4442d4d4a1ce0900973527c011d1030fd3bf4af1b"},
]
-markers = {main = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"}
+markers = {main = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"}
[package.extras]
benchmark = ["cloudpickle ; platform_python_implementation == \"CPython\"", "hypothesis", "mypy (>=1.11.1) ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pympler", "pytest (>=4.3.0)", "pytest-codspeed", "pytest-mypy-plugins ; platform_python_implementation == \"CPython\" and python_version >= \"3.10\"", "pytest-xdist[psutil]"]
@@ -488,7 +488,7 @@ files = [
{file = "boto3-1.38.27-py3-none-any.whl", hash = "sha256:95f5fe688795303a8a15e8b7e7f255cadab35eae459d00cc281a4fd77252ea80"},
{file = "boto3-1.38.27.tar.gz", hash = "sha256:94bd7fdd92d5701b362d4df100d21e28f8307a67ff56b6a8b0398119cf22f859"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.38.27,<1.39.0"
@@ -509,7 +509,7 @@ files = [
{file = "botocore-1.38.27-py3-none-any.whl", hash = "sha256:a785d5e9a5eda88ad6ab9ed8b87d1f2ac409d0226bba6ff801c55359e94d91a8"},
{file = "botocore-1.38.27.tar.gz", hash = "sha256:9788f7efe974328a38cbade64cc0b1e67d27944b899f88cb786ae362973133b6"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[package.dependencies]
jmespath = ">=0.7.1,<2.0.0"
@@ -1160,9 +1160,10 @@ files = [
name = "datafusion"
version = "47.0.0"
description = "Build and run queries against data"
-optional = false
+optional = true
python-versions = ">=3.9"
-groups = ["dev"]
+groups = ["main"]
+markers = "extra == \"datafusion\""
files = [
{file = "datafusion-47.0.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:ccd83a8e49fb39be06ddfa87023200a9ddc93d181247654ac951fa5720219d08"},
{file = "datafusion-47.0.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:43677e6284b165727031aec14d4beaa97296e991960293c61dcb66a3a9ce59b8"},
@@ -1473,7 +1474,7 @@ description = "A list-like structure which implements collections.abc.MutableSeq
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "(extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\") and (extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\" or extra == \"ray\")"
+markers = "(extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\") and (extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"ray\")"
files = [
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cc4df77d638aa2ed703b878dd093725b72a824c3c546c076e8fdf276f78ee84a"},
{file = "frozenlist-1.7.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:716a9973a2cc963160394f701964fe25012600f3d311f60c790400b00e568b61"},
@@ -2247,7 +2248,7 @@ files = [
{file = "jmespath-1.0.1-py3-none-any.whl", hash = "sha256:02e2e4cc71b5bcab88332eebf907519190dd9e6e82107fa7f83b1003a6252980"},
{file = "jmespath-1.0.1.tar.gz", hash = "sha256:90261b206d6defd58fdd5e85f478bf633a2901798906be2ad389150c5c60edbe"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\" or extra == \"s3fs\""}
[[package]]
name = "joserfc"
@@ -3037,7 +3038,7 @@ description = "multidict implementation"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "multidict-6.6.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:cfd9c74d337e710d7ee26e72a7dbedbd60e0c58d3df7c5ccbb748857e977783c"},
{file = "multidict-6.6.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9d2c5867a1bd182041a950e9ec3dd3622926260434655bd5d94a62d889100787"},
@@ -3769,7 +3770,7 @@ description = "Accelerated property cache"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:22d9962a358aedbb7a2e36187ff273adeaab9743373a272976d2e348d08c7770"},
{file = "propcache-0.3.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0d0fda578d1dc3f77b6b5a5dce3b9ad69a8250a891760a548df850a5e8da87f3"},
@@ -4036,9 +4037,10 @@ files = [
name = "pyarrow"
version = "20.0.0"
description = "Python library for Apache Arrow"
-optional = false
+optional = true
python-versions = ">=3.9"
-groups = ["main", "dev"]
+groups = ["main"]
+markers = "extra == \"pyarrow\" or extra == \"pandas\" or extra == \"duckdb\" or extra == \"ray\" or extra == \"daft\" or extra == \"datafusion\""
files = [
{file = "pyarrow-20.0.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:c7dd06fd7d7b410ca5dc839cc9d485d2bc4ae5240851bcd45d85105cc90a47d7"},
{file = "pyarrow-20.0.0-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:d5382de8dc34c943249b01c19110783d0d64b207167c728461add1ecc2db88e4"},
@@ -4096,7 +4098,6 @@ files = [
{file = "pyarrow-20.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:9965a050048ab02409fb7cbbefeedba04d3d67f2cc899eff505cc084345959ca"},
{file = "pyarrow-20.0.0.tar.gz", hash = "sha256:febc4a913592573c8d5805091a6c2b5064c8bd6e002131f01061797d91c783c1"},
]
-markers = {main = "extra == \"daft\" or extra == \"duckdb\" or extra == \"pandas\" or extra == \"pyarrow\" or extra == \"ray\""}
[package.extras]
test = ["cffi", "hypothesis", "pandas", "pytest", "pytz"]
@@ -5225,7 +5226,7 @@ files = [
{file = "s3transfer-0.13.0-py3-none-any.whl", hash = "sha256:0148ef34d6dd964d0d8cf4311b2b21c474693e57c2e069ec708ce043d2b527be"},
{file = "s3transfer-0.13.0.tar.gz", hash = "sha256:f5e6db74eb7776a37208001113ea7aa97695368242b364d73e91c981ac522177"},
]
-markers = {main = "extra == \"dynamodb\" or extra == \"glue\" or extra == \"rest-sigv4\""}
+markers = {main = "extra == \"glue\" or extra == \"dynamodb\" or extra == \"rest-sigv4\""}
[package.dependencies]
botocore = ">=1.37.4,<2.0a.0"
@@ -5733,7 +5734,7 @@ description = "Fast, Extensible Progress Meter"
optional = true
python-versions = ">=3.7"
groups = ["main"]
-markers = "extra == \"daft\" or extra == \"hf\""
+markers = "extra == \"hf\" or extra == \"daft\""
files = [
{file = "tqdm-4.67.1-py3-none-any.whl", hash = "sha256:26445eca388f82e72884e0d580d5464cd801a3ea01e63e5601bdff9ba6a48de2"},
{file = "tqdm-4.67.1.tar.gz", hash = "sha256:f8aef9c52c08c13a65f30ea34f4e5aac3fd1a34959879d7e59e63027286627f2"},
@@ -6018,7 +6019,7 @@ description = "Yet another URL library"
optional = true
python-versions = ">=3.9"
groups = ["main"]
-markers = "extra == \"adlfs\" or extra == \"gcsfs\" or extra == \"s3fs\""
+markers = "extra == \"s3fs\" or extra == \"adlfs\" or extra == \"gcsfs\""
files = [
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6032e6da6abd41e4acda34d75a816012717000fa6839f37124a47fcefc49bec4"},
{file = "yarl-1.20.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:2c7b34d804b8cf9b214f05015c4fee2ebe7ed05cf581e7192c06555c71f4446a"},
@@ -6269,6 +6270,7 @@ cffi = ["cffi (>=1.11)"]
[extras]
adlfs = ["adlfs"]
daft = ["getdaft"]
+datafusion = ["datafusion"]
duckdb = ["duckdb", "pyarrow"]
dynamodb = ["boto3"]
gcsfs = ["gcsfs"]
@@ -6291,4 +6293,4 @@ zstandard = ["zstandard"]
[metadata]
lock-version = "2.1"
python-versions = "^3.9.2, !=3.9.7"
-content-hash = "c2f45d4d591caedd7d513922884de881cf4ef30a8b431a5ceb6bb9e56711a669"
+content-hash = "c3676c4f64eeafe88af2acf9ec7428258a8ef1a92320091f2225865bffbecb6f"
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 28d19e4aaf..28cd760458 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -143,6 +143,7 @@
import pyarrow as pa
import ray
from duckdb import DuckDBPyConnection
+ from pyiceberg_core.datafusion import IcebergDataFusionTable
from pyiceberg.catalog import Catalog
@@ -1506,6 +1507,51 @@ def to_polars(self) -> pl.LazyFrame:
return pl.scan_iceberg(self)
+ def __datafusion_table_provider__(self) -> "IcebergDataFusionTable":
+ """Return the DataFusion table provider PyCapsule interface.
+
+ To support DataFusion features such as push down filtering, this function will return a PyCapsule
+ interface that conforms to the FFI Table Provider required by DataFusion. From an end user perspective
+ you should not need to call this function directly. Instead you can use ``register_table_provider`` in
+ the DataFusion SessionContext.
+
+ Returns:
+ A PyCapsule DataFusion TableProvider interface.
+
+ Example:
+ ```python
+ from datafusion import SessionContext
+ from pyiceberg.catalog import load_catalog
+ import pyarrow as pa
+ catalog = load_catalog("catalog", type="in-memory")
+ catalog.create_namespace_if_not_exists("default")
+ data = pa.table({"x": [1, 2, 3], "y": [4, 5, 6]})
+ iceberg_table = catalog.create_table("default.test", schema=data.schema)
+ iceberg_table.append(data)
+ ctx = SessionContext()
+ ctx.register_table_provider("test", iceberg_table)
+ ctx.table("test").show()
+ ```
+ Results in
+ ```
+ DataFrame()
+ +---+---+
+ | x | y |
+ +---+---+
+ | 1 | 4 |
+ | 2 | 5 |
+ | 3 | 6 |
+ +---+---+
+ ```
+ """
+ from pyiceberg_core.datafusion import IcebergDataFusionTable
+
+ return IcebergDataFusionTable(
+ identifier=self.name(),
+ metadata_location=self.metadata_location,
+ file_io_properties=self.io.properties,
+ ).__datafusion_table_provider__()
+
class StaticTable(Table):
"""Load a table directly from a metadata file (i.e., without using a catalog)."""
diff --git a/pyproject.toml b/pyproject.toml
index 4e479e9d0e..a680df1d30 100644
--- a/pyproject.toml
+++ b/pyproject.toml
@@ -84,6 +84,7 @@ pyiceberg-core = { version = "^0.5.1", optional = true }
polars = { version = "^1.21.0", optional = true }
thrift-sasl = { version = ">=0.4.3", optional = true }
kerberos = {version = "^1.3.1", optional = true}
+datafusion = { version = ">=45", optional = true }
[tool.poetry.group.dev.dependencies]
pytest = "7.4.4"
@@ -99,7 +100,6 @@ pytest-mock = "3.14.1"
pyspark = "3.5.6"
cython = "3.1.2"
deptry = ">=0.14,<0.24"
-datafusion = ">=44,<48"
docutils = "!=0.21.post1" # https://github.com/python-poetry/poetry/issues/9248#issuecomment-2026240520
mypy-boto3-glue = ">=1.28.18"
mypy-boto3-dynamodb = ">=1.28.18"
@@ -314,6 +314,7 @@ gcsfs = ["gcsfs"]
rest-sigv4 = ["boto3"]
hf = ["huggingface-hub"]
pyiceberg-core = ["pyiceberg-core"]
+datafusion = ["datafusion"]
[tool.pytest.ini_options]
markers = [
diff --git a/tests/table/test_datafusion.py b/tests/table/test_datafusion.py
new file mode 100644
index 0000000000..d9fa3e1e7b
--- /dev/null
+++ b/tests/table/test_datafusion.py
@@ -0,0 +1,64 @@
+# 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.
+
+
+from pathlib import Path
+
+import pyarrow as pa
+import pytest
+from datafusion import SessionContext
+
+from pyiceberg.catalog import Catalog, load_catalog
+
+
+@pytest.fixture(scope="session")
+def warehouse(tmp_path_factory: pytest.TempPathFactory) -> Path:
+ return tmp_path_factory.mktemp("warehouse")
+
+
+@pytest.fixture(scope="session")
+def catalog(warehouse: Path) -> Catalog:
+ catalog = load_catalog(
+ "default",
+ uri=f"sqlite:///{warehouse}/pyiceberg_catalog.db",
+ warehouse=f"file://{warehouse}",
+ )
+ return catalog
+
+
+def test_datafusion_register_pyiceberg_table(catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+ catalog.create_namespace_if_not_exists("default")
+ iceberg_table = catalog.create_table_if_not_exists(
+ "default.dataset",
+ schema=arrow_table_with_null.schema,
+ )
+ iceberg_table.append(arrow_table_with_null)
+
+ ctx = SessionContext()
+ ctx.register_table_provider("test", iceberg_table)
+
+ datafusion_table = ctx.table("test")
+ assert datafusion_table is not None
+
+ assert datafusion_table.to_arrow_table().to_pylist() == iceberg_table.scan().to_arrow().to_pylist()
+
+ from pandas.testing import assert_frame_equal
+
+ assert_frame_equal(
+ datafusion_table.to_arrow_table().to_pandas(),
+ iceberg_table.scan().to_arrow().to_pandas(),
+ )
From f11ebbd395d2c95825f20eb71765de9895e0a57e Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sun, 6 Jul 2025 22:05:53 +0200
Subject: [PATCH 104/112] Cleanup in `expression-dsl.md` (#2168)
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---
mkdocs/docs/expression-dsl.md | 19 ++-----------------
1 file changed, 2 insertions(+), 17 deletions(-)
diff --git a/mkdocs/docs/expression-dsl.md b/mkdocs/docs/expression-dsl.md
index 7dd8b5137b..cf5784dd2e 100644
--- a/mkdocs/docs/expression-dsl.md
+++ b/mkdocs/docs/expression-dsl.md
@@ -17,7 +17,7 @@
# Expression DSL
-The PyIceberg library provides a powerful expression DSL (Domain Specific Language) for building complex row filter expressions. This guide will help you understand how to use the expression DSL effectively. This DSL allows you to build type-safe expressions for use in the `row_filter` scan argument.
+The PyIceberg library provides a powerful expression Domain Specific Language (DSL) for building complex row filter expressions. This guide will help you understand how to use the expression DSL effectively. This DSL allows you to build type-safe expressions for use in the `row_filter` scan argument.
They are composed of terms, predicates, and logical operators.
@@ -151,20 +151,6 @@ age_in_range = Not(
)
```
-### Type Safety
-
-The expression DSL provides type safety through Python's type system. When you create expressions, the types are checked at runtime:
-
-```python
-from pyiceberg.expressions import EqualTo
-
-# This will work
-age_equals_18 = EqualTo("age", 18)
-
-# This will raise a TypeError if the field type doesn't match
-age_equals_18 = EqualTo("age", "18") # Will fail if age is an integer field
-```
-
## Best Practices
1. **Use Type Hints**: Always use type hints when working with expressions to catch type-related errors early.
@@ -204,7 +190,7 @@ complex_filter = And(age_range, status_filter)
- `IsNull` (and `IsNaN` for doubles/floats) on a required field will always return `False`
- `NotNull` (and `NotNaN` for doubles/floats) on a required field will always return `True`
-2. **String Comparisons**: When using string predicates like `StartsWith`, ensure that the field type is actually a string type.
+2. **String Comparisons**: When using string predicates like `StartsWith`, ensure that the field type is a string type.
## Examples
@@ -213,7 +199,6 @@ Here are some practical examples of using the expression DSL:
### Basic Filtering
```python
-
from datetime import datetime
from pyiceberg.expressions import (
And,
From 117e2bef40de05599cd2552efd7ca212a2cc410a Mon Sep 17 00:00:00 2001
From: Fokko Driesprong
Date: Sun, 6 Jul 2025 22:08:22 +0200
Subject: [PATCH 105/112] Docs: Condens Python API docs (#2139)
I noticed that the docs needed some TLC.
- Collapsed some lines to make it more compact.
- Avoid imports where possible (eg transforms).
- Update docs.
- Add an example of the `to_arrow_batch_reader` earlier in the docs.
# Rationale for this change
# Are these changes tested?
# Are there any user-facing changes?
---------
Co-authored-by: Jayce Slesar <47452474+jayceslesar@users.noreply.github.com>
---
mkdocs/docs/api.md | 141 +++++++++++++++------------------------------
1 file changed, 47 insertions(+), 94 deletions(-)
diff --git a/mkdocs/docs/api.md b/mkdocs/docs/api.md
index 448993b4f6..b8a6ea45c2 100644
--- a/mkdocs/docs/api.md
+++ b/mkdocs/docs/api.md
@@ -24,7 +24,7 @@ hide:
# Python API
-PyIceberg is based around catalogs to load tables. First step is to instantiate a catalog that loads tables. Let's use the following configuration to define a catalog called `prod`:
+(Py)Iceberg is [catalog](https://iceberg.apache.org/terms/#catalog) centric. Meaning that reading/writing data goes via a catalog. First step is to instantiate a catalog to load a table. Let's use the following configuration in `.pyiceberg.yaml` to define a REST catalog called `prod`:
```yaml
catalog:
@@ -33,7 +33,7 @@ catalog:
credential: t-1234:secret
```
-Note that multiple catalogs can be defined in the same `.pyiceberg.yaml`:
+Note that multiple catalogs can be defined in the same `.pyiceberg.yaml`, for example, in the case of a Hive and REST catalog:
```yaml
catalog:
@@ -47,13 +47,11 @@ catalog:
warehouse: my-warehouse
```
-and loaded in python by calling `load_catalog(name="hive")` and `load_catalog(name="rest")`.
+The different catalogs can be loaded in PyIceberg by their name: `load_catalog(name="hive")` and `load_catalog(name="rest")`. An overview of the configuration options can be found on the [configuration page](https://py.iceberg.apache.org/configuration/).
This information must be placed inside a file called `.pyiceberg.yaml` located either in the `$HOME` or `%USERPROFILE%` directory (depending on whether the operating system is Unix-based or Windows-based, respectively), in the current working directory, or in the `$PYICEBERG_HOME` directory (if the corresponding environment variable is set).
-For more details on possible configurations refer to the [specific page](https://py.iceberg.apache.org/configuration/).
-
-Then load the `prod` catalog:
+It is also possible to load a catalog without using a `.pyiceberg.yaml` by passing in the properties directly:
```python
from pyiceberg.catalog import load_catalog
@@ -70,13 +68,13 @@ catalog = load_catalog(
)
```
-Let's create a namespace:
+Next, create a namespace:
```python
catalog.create_namespace("docs_example")
```
-And then list them:
+Or, list existing namespaces:
```python
ns = catalog.list_namespaces()
@@ -84,12 +82,6 @@ ns = catalog.list_namespaces()
assert ns == [("docs_example",)]
```
-And then list tables in the namespace:
-
-```python
-catalog.list_tables("docs_example")
-```
-
## Create a table
To create a table from a catalog:
@@ -123,24 +115,21 @@ schema = Schema(
)
from pyiceberg.partitioning import PartitionSpec, PartitionField
-from pyiceberg.transforms import DayTransform
partition_spec = PartitionSpec(
PartitionField(
- source_id=1, field_id=1000, transform=DayTransform(), name="datetime_day"
+ source_id=1, field_id=1000, transform="day", name="datetime_day"
)
)
from pyiceberg.table.sorting import SortOrder, SortField
-from pyiceberg.transforms import IdentityTransform
# Sort on the symbol
-sort_order = SortOrder(SortField(source_id=2, transform=IdentityTransform()))
+sort_order = SortOrder(SortField(source_id=2, transform='identity'))
catalog.create_table(
identifier="docs_example.bids",
schema=schema,
- location="s3://pyiceberg",
partition_spec=partition_spec,
sort_order=sort_order,
)
@@ -153,13 +142,11 @@ To create a table using a pyarrow schema:
```python
import pyarrow as pa
-schema = pa.schema(
- [
+schema = pa.schema([
pa.field("foo", pa.string(), nullable=True),
pa.field("bar", pa.int32(), nullable=False),
pa.field("baz", pa.bool_(), nullable=True),
- ]
-)
+])
catalog.create_table(
identifier="docs_example.bids",
@@ -167,18 +154,12 @@ catalog.create_table(
)
```
-To create a table with some subsequent changes atomically in a transaction:
+Another API to create a table is using the `create_table_transaction`. This follows the same APIs when making updates to a table. This is a friendly API for both setting the partition specification and sort-order, because you don't have to deal with field-IDs.
```python
-with catalog.create_table_transaction(
- identifier="docs_example.bids",
- schema=schema,
- location="s3://pyiceberg",
- partition_spec=partition_spec,
- sort_order=sort_order,
-) as txn:
+with catalog.create_table_transaction(identifier="docs_example.bids", schema=schema) as txn:
with txn.update_schema() as update_schema:
- update_schema.add_column(path="new_column", field_type=StringType())
+ update_schema.add_column(path="new_column", field_type='string')
with txn.update_spec() as update_spec:
update_spec.add_identity("symbol")
@@ -188,6 +169,8 @@ with catalog.create_table_transaction(
## Load a table
+There are two ways of reading an Iceberg table; through a catalog, and by pointing at the Iceberg metadata directly. Reading through a catalog is preferred, and directly pointing at the metadata is read-only.
+
### Catalog table
Loading the `bids` table:
@@ -203,7 +186,7 @@ This returns a `Table` that represents an Iceberg table that can be queried and
### Static table
-To load a table directly from a metadata file (i.e., **without** using a catalog), you can use a `StaticTable` as follows:
+To load a table directly from a `metadata.json` file (i.e., **without** using a catalog), you can use a `StaticTable` as follows:
```python
from pyiceberg.table import StaticTable
@@ -213,16 +196,13 @@ static_table = StaticTable.from_metadata(
)
```
-The static-table is considered read-only.
-
-Alternatively, if your table metadata directory contains a `version-hint.text` file, you can just specify
-the table root path, and the latest metadata file will be picked automatically.
+The static-table does not allow for write operations. If your table metadata directory contains a `version-hint.text` file, you can just specify the table root path, and the latest `metadata.json` file will be resolved automatically:
```python
from pyiceberg.table import StaticTable
static_table = StaticTable.from_metadata(
- "s3://warehouse/wh/nyc.db/taxis
+ "s3://warehouse/wh/nyc.db/taxis"
)
```
@@ -236,9 +216,9 @@ catalog.table_exists("docs_example.bids")
Returns `True` if the table already exists.
-## Write support
+## Write to a table
-With PyIceberg 0.6.0 write support is added through Arrow. Let's consider an Arrow Table:
+Reading and writing is being done using [Apache Arrow](https://arrow.apache.org/). Arrow is an in-memory columnar format for fast data interchange and in-memory analytics. Let's consider the following Arrow Table:
```python
import pyarrow as pa
@@ -253,31 +233,22 @@ df = pa.Table.from_pylist(
)
```
-Next, create a table based on the schema:
+Next, create a table using the Arrow schema:
```python
from pyiceberg.catalog import load_catalog
catalog = load_catalog("default")
-from pyiceberg.schema import Schema
-from pyiceberg.types import NestedField, StringType, DoubleType
-
-schema = Schema(
- NestedField(1, "city", StringType(), required=False),
- NestedField(2, "lat", DoubleType(), required=False),
- NestedField(3, "long", DoubleType(), required=False),
-)
-
-tbl = catalog.create_table("default.cities", schema=schema)
+tbl = catalog.create_table("default.cities", schema=df.schema)
```
-Now write the data to the table:
+Next, write the data to the table. Both `append` and `overwrite` produce the same result, since the table is empty on creation:
!!! note inline end "Fast append"
- PyIceberg default to the [fast append](https://iceberg.apache.org/spec/#snapshots) to minimize the amount of data written. This enables quick writes, reducing the possibility of conflicts. The downside of the fast append is that it creates more metadata than a normal commit. [Compaction is planned](https://github.com/apache/iceberg-python/issues/270) and will automatically rewrite all the metadata when a threshold is hit, to maintain performant reads.
+ PyIceberg defaults to the [fast append](https://iceberg.apache.org/spec/#snapshots) to minimize the amount of data written. This enables fast commit operations, reducing the possibility of conflicts. The downside of the fast append is that it creates more metadata than a merge commit. [Compaction is planned](https://github.com/apache/iceberg-python/issues/270) and will automatically rewrite all the metadata when a threshold is hit, to maintain performant reads.
@@ -289,7 +260,7 @@ tbl.append(df)
tbl.overwrite(df)
```
-The data is written to the table, and when the table is read using `tbl.scan().to_arrow()`:
+Now, the data is written to the table, and the table can be read using `tbl.scan().to_arrow()`:
```python
pyarrow.Table
@@ -302,14 +273,12 @@ lat: [[52.371807,37.773972,53.11254,48.864716]]
long: [[4.896029,-122.431297,6.0989,2.349014]]
```
-You both can use `append(df)` or `overwrite(df)` since there is no data yet. If we want to add more data, we can use `.append()` again:
+If we want to add more data, we can use `.append()` again:
```python
-df = pa.Table.from_pylist(
+tbl.append(pa.Table.from_pylist(
[{"city": "Groningen", "lat": 53.21917, "long": 6.56667}],
-)
-
-tbl.append(df)
+))
```
When reading the table `tbl.scan().to_arrow()` you can see that `Groningen` is now also part of the table:
@@ -325,33 +294,30 @@ lat: [[52.371807,37.773972,53.11254,48.864716],[53.21917]]
long: [[4.896029,-122.431297,6.0989,2.349014],[6.56667]]
```
-The nested lists indicate the different Arrow buffers, where the first write results into a buffer, and the second append in a separate buffer. This is expected since it will read two parquet files.
-
-To avoid any type errors during writing, you can enforce the PyArrow table types using the Iceberg table schema:
+The nested lists indicate the different Arrow buffers. Each of the writes produce a [Parquet file](https://parquet.apache.org/) where each [row group](https://parquet.apache.org/docs/concepts/) translates into an Arrow buffer. In the case where the table is large, PyIceberg also allows the option to stream the buffers using the Arrow [RecordBatchReader](https://arrow.apache.org/docs/python/generated/pyarrow.RecordBatchReader.html), avoiding pulling everything into memory right away:
```python
-from pyiceberg.catalog import load_catalog
-import pyarrow as pa
+for buf in tbl.scan().to_arrow_batch_reader():
+ print(f"Buffer contains {len(buf)} rows")
+```
-catalog = load_catalog("default")
-table = catalog.load_table("default.cities")
-schema = table.schema().as_arrow()
+To avoid any type inconsistencies during writing, you can convert the Iceberg table schema to Arrow:
+```python
df = pa.Table.from_pylist(
- [{"city": "Groningen", "lat": 53.21917, "long": 6.56667}], schema=schema
+ [{"city": "Groningen", "lat": 53.21917, "long": 6.56667}], schema=table.schema().as_arrow()
)
-table.append(df)
+tbl.append(df)
```
-You can delete some of the data from the table by calling `tbl.delete()` with a desired `delete_filter`.
+You can delete some of the data from the table by calling `tbl.delete()` with a desired `delete_filter`. This will use the Iceberg metadata to only open up the Parquet files that contain relevant information.
```python
tbl.delete(delete_filter="city == 'Paris'")
```
-In the above example, any records where the city field value equals to `Paris` will be deleted.
-Running `tbl.scan().to_arrow()` will now yield:
+In the above example, any records where the city field value equals to `Paris` will be deleted. Running `tbl.scan().to_arrow()` will now yield:
```python
pyarrow.Table
@@ -364,30 +330,11 @@ lat: [[52.371807,37.773972,53.11254],[53.21917]]
long: [[4.896029,-122.431297,6.0989],[6.56667]]
```
-### Partial overwrites
-
-When using the `overwrite` API, you can use an `overwrite_filter` to delete data that matches the filter before appending new data into the table.
-
-For example, with an iceberg table created as:
-
-```python
-from pyiceberg.catalog import load_catalog
-
-catalog = load_catalog("default")
-
-from pyiceberg.schema import Schema
-from pyiceberg.types import NestedField, StringType, DoubleType
+In the case of `tbl.delete(delete_filter="city == 'Groningen'")`, the whole Parquet file will be dropped without checking it contents, since from the Iceberg metadata PyIceberg can derive that all the content in the file matches the predicate.
-schema = Schema(
- NestedField(1, "city", StringType(), required=False),
- NestedField(2, "lat", DoubleType(), required=False),
- NestedField(3, "long", DoubleType(), required=False),
-)
-
-tbl = catalog.create_table("default.cities", schema=schema)
-```
+### Partial overwrites
-And with initial data populating the table:
+When using the `overwrite` API, you can use an `overwrite_filter` to delete data that matches the filter before appending new data into the table. For example, consider the following Iceberg table:
```python
import pyarrow as pa
@@ -399,6 +346,12 @@ df = pa.Table.from_pylist(
{"city": "Paris", "lat": 48.864716, "long": 2.349014},
],
)
+
+from pyiceberg.catalog import load_catalog
+catalog = load_catalog("default")
+
+tbl = catalog.create_table("default.cities", schema=df.schema)
+
tbl.append(df)
```
From c7cb34c5aff3cbeeae918bbdbdbe4fedf86a7880 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?S=C3=A9bastien=20Brochet?=
Date: Mon, 7 Jul 2025 00:42:41 +0200
Subject: [PATCH 106/112] feat: add support for setting and removing table
properties on console (#2153)
# Rationale for this change
Hello!
Setting or removing table properties on console currently raise a
`Writing is WIP` error. This PR adds the code to set and remove table
properties.
# Are these changes tested?
Yes
# Are there any user-facing changes?
Yes, setting and removing table properties on console now works.
---
mkdocs/docs/cli.md | 16 ++++++++++++++++
pyiceberg/cli/console.py | 12 +++++++-----
tests/cli/test_console.py | 16 ++++++++--------
3 files changed, 31 insertions(+), 13 deletions(-)
diff --git a/mkdocs/docs/cli.md b/mkdocs/docs/cli.md
index 28e44955d7..984e0df43d 100644
--- a/mkdocs/docs/cli.md
+++ b/mkdocs/docs/cli.md
@@ -219,3 +219,19 @@ Or output in JSON for automation:
}
}
```
+
+You can also add, update or remove properties on tables or namespaces:
+
+```sh
+➜ pyiceberg properties set table nyc.taxis write.metadata.delete-after-commit.enabled true
+Set write.metadata.delete-after-commit.enabled=true on nyc.taxis
+
+➜ pyiceberg properties get table nyc.taxis
+write.metadata.delete-after-commit.enabled true
+
+➜ pyiceberg properties remove table nyc.taxis write.metadata.delete-after-commit.enabled
+Property write.metadata.delete-after-commit.enabled removed from nyc.taxis
+
+➜ pyiceberg properties get table nyc.taxis write.metadata.delete-after-commit.enabled
+Could not find property write.metadata.delete-after-commit.enabled on nyc.taxis
+```
diff --git a/pyiceberg/cli/console.py b/pyiceberg/cli/console.py
index 3fbd9c9fc9..d918f87918 100644
--- a/pyiceberg/cli/console.py
+++ b/pyiceberg/cli/console.py
@@ -361,9 +361,10 @@ def table(ctx: Context, identifier: str, property_name: str, property_value: str
catalog, output = _catalog_and_output(ctx)
identifier_tuple = Catalog.identifier_to_tuple(identifier)
- _ = catalog.load_table(identifier_tuple)
- output.text(f"Setting {property_name}={property_value} on {identifier}")
- raise NotImplementedError("Writing is WIP")
+ table = catalog.load_table(identifier_tuple)
+ with table.transaction() as tx:
+ tx.set_properties({property_name: property_value})
+ output.text(f"Set {property_name}={property_value} on {identifier}")
@properties.group()
@@ -398,8 +399,9 @@ def table(ctx: Context, identifier: str, property_name: str) -> None: # noqa: F
catalog, output = _catalog_and_output(ctx)
table = catalog.load_table(identifier)
if property_name in table.metadata.properties:
- output.exception(NotImplementedError("Writing is WIP"))
- ctx.exit(1)
+ with table.transaction() as tx:
+ tx.remove_properties(property_name)
+ output.text(f"Property {property_name} removed from {identifier}")
else:
raise NoSuchPropertyException(f"Property {property_name} does not exist on {identifier}")
diff --git a/tests/cli/test_console.py b/tests/cli/test_console.py
index fe375eb276..a0e9552236 100644
--- a/tests/cli/test_console.py
+++ b/tests/cli/test_console.py
@@ -476,8 +476,8 @@ def test_properties_set_table(catalog: InMemoryCatalog) -> None:
runner = CliRunner()
result = runner.invoke(run, ["properties", "set", "table", "default.my_table", "location", "s3://new_location"])
- assert result.exit_code == 1
- assert "Writing is WIP" in result.output
+ assert result.exit_code == 0
+ assert result.output == "Set location=s3://new_location on default.my_table\n"
def test_properties_set_table_does_not_exist(catalog: InMemoryCatalog) -> None:
@@ -518,8 +518,8 @@ def test_properties_remove_table(catalog: InMemoryCatalog) -> None:
runner = CliRunner()
result = runner.invoke(run, ["properties", "remove", "table", "default.my_table", "read.split.target.size"])
- assert result.exit_code == 1
- assert "Writing is WIP" in result.output
+ assert result.exit_code == 0
+ assert result.output == "Property read.split.target.size removed from default.my_table\n"
def test_properties_remove_table_property_does_not_exists(catalog: InMemoryCatalog) -> None:
@@ -894,8 +894,8 @@ def test_json_properties_set_table(catalog: InMemoryCatalog) -> None:
result = runner.invoke(
run, ["--output=json", "properties", "set", "table", "default.my_table", "location", "s3://new_location"]
)
- assert result.exit_code == 1
- assert "Writing is WIP" in result.output
+ assert result.exit_code == 0
+ assert result.output == """"Set location=s3://new_location on default.my_table"\n"""
def test_json_properties_set_table_does_not_exist(catalog: InMemoryCatalog) -> None:
@@ -938,8 +938,8 @@ def test_json_properties_remove_table(catalog: InMemoryCatalog) -> None:
runner = CliRunner()
result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.my_table", "read.split.target.size"])
- assert result.exit_code == 1
- assert "Writing is WIP" in result.output
+ assert result.exit_code == 0
+ assert result.output == """"Property read.split.target.size removed from default.my_table"\n"""
def test_json_properties_remove_table_property_does_not_exists(catalog: InMemoryCatalog) -> None:
From 958373253307f2a14eefd04d4c40c78029bef313 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Mon, 10 Feb 2025 19:52:51 +0200
Subject: [PATCH 107/112] first attempt at init_manifests
---
pyiceberg/manifest.py | 20 +-
pyiceberg/table/__init__.py | 54 ++++-
pyiceberg/table/update/snapshot.py | 299 +++++++++++++++++++-----
tests/integration/test_inspect_table.py | 42 ----
4 files changed, 310 insertions(+), 105 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 9e8a5ae439..b6d8273839 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -842,6 +842,24 @@ def has_added_files(self) -> bool:
def has_existing_files(self) -> bool:
return self.existing_files_count is None or self.existing_files_count > 0
+ def copy_with_snapshot_id(self, snapshot_id: int) -> ManifestFile:
+ return ManifestFile(
+ manifest_path=self.manifest_path,
+ manifest_length=self.manifest_length,
+ partition_spec_id=self.partition_spec_id,
+ content=self.content,
+ sequence_number=self.sequence_number,
+ min_sequence_number=self.min_sequence_number,
+ added_snapshot_id=snapshot_id,
+ added_files_count=self.added_files_count,
+ existing_files_count=self.existing_files_count,
+ deleted_files_count=self.deleted_files_count,
+ added_rows_count=self.added_rows_count,
+ existing_rows_count=self.existing_rows_count,
+ deleted_rows_count=self.deleted_rows_count,
+ partitions=self.partitions,
+ key_metadata=self.key_metadata,
+ )
def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List[ManifestEntry]:
"""
Read the manifest entries from the manifest file.
@@ -865,12 +883,10 @@ def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List
for entry in reader
if not discard_deleted or entry.status != ManifestEntryStatus.DELETED
]
-
def __hash__(self) -> int:
"""Return the hash of the file path."""
return hash(self.manifest_path)
-
@cached(cache=LRUCache(maxsize=128), key=lambda io, manifest_list: hashkey(manifest_list))
def _manifests(io: FileIO, manifest_list: str) -> Tuple[ManifestFile, ...]:
"""Read and cache manifests from the given manifest list, returning a tuple to prevent modification."""
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 28cd760458..f0bf55402a 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -115,7 +115,7 @@
update_table_metadata,
)
from pyiceberg.table.update.schema import UpdateSchema
-from pyiceberg.table.update.snapshot import ExpireSnapshots, ManageSnapshots, RewriteManifestsResult, UpdateSnapshot, _FastAppendFiles
+from pyiceberg.table.update.snapshot import ExpireSnapshots, ManageSnapshots, RewriteManifestsResult, UpdateSnapshot, _FastAppendFiles, _ManifestMergeManager
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
from pyiceberg.transforms import IdentityTransform
@@ -134,7 +134,7 @@
)
from pyiceberg.utils.concurrent import ExecutorFactory
from pyiceberg.utils.config import Config
-from pyiceberg.utils.properties import property_as_bool
+from pyiceberg.utils.properties import property_as_bool, property_as_int
if TYPE_CHECKING:
import daft
@@ -441,6 +441,9 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT, bran
branch = MAIN_BRANCH
return UpdateSnapshot(self, io=self._table.io, branch=branch, snapshot_properties=snapshot_properties)
+ def rewrite_manifests(self) -> None:
+ with self.update_snapshot().rewrite() as rewrite:
+ rewrite.commit()
def rewrite_manifests(self) -> RewriteManifestsResult:
if self._table.current_snapshot() is None:
@@ -1447,6 +1450,53 @@ def add_files(
file_paths=file_paths, snapshot_properties=snapshot_properties, check_duplicate_files=check_duplicate_files
)
+ def rewrite_manifests(
+ self,
+ spec_id: Optional[int] = None,
+ rewrite_all: bool = False,
+ max_manifest_size: Optional[int] = None,
+ ) -> "Table":
+
+ with self.transaction() as tx:
+ tx.rewrite_manifests()
+ ...
+ """Rewrite manifests in the table.
+
+ Args:
+ spec_id: Spec ID to be used for the rewritten manifests
+ rewrite_all: If True, rewrite all manifests. If False, only rewrite small manifests
+ max_manifest_size: Target size for manifests in bytes
+
+ Returns:
+ An updated version of the table with rewritten manifests
+ #"""
+ # return RewriteManifests(
+ # self,
+ # spec_id=spec_id,
+ # rewrite_all=rewrite_all,
+ # max_manifest_size=max_manifest_size,
+ # ).commit()
+
+ # snapshot = self.current_snapshot()
+ # manifests = []
+ # for manifest in snapshot.manifests(self.io):
+ # if manifest.content == ManifestContent.DATA:
+ # manifests.append(manifest)
+ #
+ # data_manifest_merge_manager = _ManifestMergeManager(
+ # target_size_bytes=property_as_int(
+ # self.properties,
+ # TableProperties.MANIFEST_TARGET_SIZE_BYTES,
+ # TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
+ # ),
+ # min_count_to_merge=2,
+ # merge_enabled=True,
+ # snapshot_producer=self,
+ # )
+ #
+ # data_manifest_merge_manager.merge_manifests(manifests)
+ # entries = self.inspect.entries().filter("status < 2").selectExpr("input_file_name() as manifest")
+
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index ef1a6c2e1f..52ce3feafb 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -112,12 +112,12 @@ class _SnapshotProducer(UpdateTableMetadata[U], Generic[U]):
_target_branch = MAIN_BRANCH
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
branch: str = MAIN_BRANCH,
) -> None:
super().__init__(transaction)
@@ -158,10 +158,12 @@ def delete_data_file(self, data_file: DataFile) -> _SnapshotProducer[U]:
return self
@abstractmethod
- def _deleted_entries(self) -> List[ManifestEntry]: ...
+ def _deleted_entries(self) -> List[ManifestEntry]:
+ ...
@abstractmethod
- def _existing_manifests(self) -> List[ManifestFile]: ...
+ def _existing_manifests(self) -> List[ManifestFile]:
+ ...
def _process_manifests(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
"""To perform any post-processing on the manifests before writing them to the new snapshot."""
@@ -171,11 +173,11 @@ def _manifests(self) -> List[ManifestFile]:
def _write_added_manifest() -> List[ManifestFile]:
if self._added_data_files:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.spec(),
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.spec(),
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for data_file in self._added_data_files:
@@ -202,11 +204,11 @@ def _write_delete_manifest() -> List[ManifestFile]:
partition_groups[deleted_entry.data_file.spec_id].append(deleted_entry)
for spec_id, entries in partition_groups.items():
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for entry in entries:
@@ -222,7 +224,8 @@ def _write_delete_manifest() -> List[ManifestFile]:
delete_manifests = executor.submit(_write_delete_manifest)
existing_manifests = executor.submit(self._existing_manifests)
- return self._process_manifests(added_manifests.result() + delete_manifests.result() + existing_manifests.result())
+ return self._process_manifests(
+ added_manifests.result() + delete_manifests.result() + existing_manifests.result())
def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
from pyiceberg.table import TableProperties
@@ -275,11 +278,11 @@ def _commit(self) -> UpdatesAndRequirements:
manifest_list_file_path = location_provider.new_metadata_location(file_name)
with write_manifest_list(
- format_version=self._transaction.table_metadata.format_version,
- output_file=self._io.new_output(manifest_list_file_path),
- snapshot_id=self._snapshot_id,
- parent_snapshot_id=self._parent_snapshot_id,
- sequence_number=next_sequence_number,
+ format_version=self._transaction.table_metadata.format_version,
+ output_file=self._io.new_output(manifest_list_file_path),
+ snapshot_id=self._snapshot_id,
+ parent_snapshot_id=self._parent_snapshot_id,
+ sequence_number=next_sequence_number,
avro_compression=self._compression,
) as writer:
writer.add_manifests(new_manifests)
@@ -354,13 +357,12 @@ class _DeleteFiles(_SnapshotProducer["_DeleteFiles"]):
_case_sensitive: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- branch: str,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ branch: str,commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
):
super().__init__(operation, transaction, io, commit_uuid, snapshot_properties, branch)
self._predicate = AlwaysFalse()
@@ -413,7 +415,8 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
)
manifest_evaluators: Dict[int, Callable[[ManifestFile], bool]] = KeyDefaultDict(self._build_manifest_evaluator)
- strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate, case_sensitive=self._case_sensitive).eval
+ strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate,
+ case_sensitive=self._case_sensitive).eval
inclusive_metrics_evaluator = _InclusiveMetricsEvaluator(
schema, self._predicate, case_sensitive=self._case_sensitive
).eval
@@ -489,6 +492,182 @@ def files_affected(self) -> bool:
return len(self._deleted_entries()) > 0
+class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
+ """Rewrite manifest files based on the predicate."""
+
+ KEPT_MANIFESTS_COUNT = "manifests-kept"
+ CREATED_MANIFESTS_COUNT = "manifests-created"
+ REPLACED_MANIFESTS_COUNT = "manifests-replaced"
+ PROCESSED_ENTRY_COUNT = "entries-processed"
+
+ def __init__(
+ self,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ ):
+ from pyiceberg.table import TableProperties
+
+ super().__init__(Operation.REPLACE, transaction, io, commit_uuid, snapshot_properties)
+
+ self.specs_by_id = self._transaction.table_metadata.spec().spec_id
+ self.manifest_target_size_bytes = property_as_int(
+ self._transaction.table_metadata.properties,
+ TableProperties.MANIFEST_TARGET_SIZE_BYTES,
+ TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
+ )
+ self.deleted_manifests: Set[ManifestFile] = set()
+ self.added_manifests: List[ManifestFile] = []
+ self.rewritten_added_manifests: List[ManifestFile] = []
+ self.kept_manifests: Set[ManifestFile] = set()
+ self.new_manifests: Set[ManifestFile] = set()
+ self.rewritten_manifests: Set[ManifestFile] = set()
+
+ def copy_manifest(self, manifest: ManifestFile) -> ManifestFile:
+ return ManifestFile(
+ manifest_path=manifest.manifest_path,
+ manifest_length=manifest.manifest_length,
+ partition_spec_id=manifest.partition_spec_id,
+ content=manifest.content,
+ sequence_number=manifest.sequence_number,
+ min_sequence_number=manifest.min_sequence_number,
+ added_snapshot_id=manifest.added_snapshot_id,
+ added_files_count=manifest.added_files_count,
+ existing_files_count=manifest.existing_files_count,
+ deleted_files_count=manifest.deleted_files_count,
+ added_rows_count=manifest.added_rows_count,
+ existing_rows_count=manifest.existing_rows_count,
+ deleted_rows_count=manifest.deleted_rows_count,
+ partitions=manifest.partitions,
+ key_metadata=manifest.key_metadata,
+ )
+
+ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
+ summary = {
+ self.CREATED_MANIFESTS_COUNT: str(
+ len(self.new_manifests) + len(self.added_manifests) + len(self.rewritten_added_manifests)
+ ),
+ self.KEPT_MANIFESTS_COUNT: str(len(self.kept_manifests)),
+ self.REPLACED_MANIFESTS_COUNT: str(len(self.rewritten_manifests) + len(self.deleted_manifests)),
+ self.PROCESSED_ENTRY_COUNT: str(self.entry_count),
+ }
+ return super()._summary(summary)
+
+ def delete_manifest(self, manifest):
+ self.deleted_manifests.add(manifest)
+ return self
+
+ def requires_rewrite(self, current_manifests):
+
+ if not self.rewritten_manifests:
+ # nothing yet processed so perform a full rewrite
+ return True
+
+ # if any processed manifest is not in the current manifest list, perform a full rewrite
+ return any(manifest not in current_manifests for manifest in self.rewritten_manifests)
+
+ def keep_active_manifests(self, current_manifests: List[ManifestFile]) -> None:
+ # keep any existing manifests as-is that were not processed
+ self.kept_manifests.clear()
+ for manifest in current_manifests:
+ if manifest not in self.rewritten_manifests and manifest not in self.deleted_manifests:
+ self.kept_manifests.add(manifest)
+
+ def active_files_count(self, manifests):
+ active_files_count = 0
+
+ for manifest in manifests:
+ if manifest.added_files_count is None:
+ raise ValueError("Missing file counts in {}".format(manifest.path()))
+ if manifest.existing_files_count is None:
+ raise ValueError("Missing file counts in {}".format(manifest.path()))
+ active_files_count += manifest.added_files_count
+ active_files_count += manifest.existing_files_count
+
+ return active_files_count
+
+ def validate_files_counts(self):
+ created_manifests = itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
+ created_manifests_files_count = self.active_files_count(created_manifests)
+
+ replaced_manifests = itertools.chain(self.rewritten_manifests, self.deleted_manifests)
+ replaced_manifests_files_count = self.active_files_count(replaced_manifests)
+
+ if created_manifests_files_count != replaced_manifests_files_count:
+ raise ValueError(
+ "Replaced and created manifests must have the same number of active files: {} (new), {} (old)".format(
+ created_manifests_files_count, replaced_manifests_files_count
+ )
+ )
+
+ def _existing_manifests(self) -> List[ManifestFile]:
+ """Returns the list of manifests to include in the next snapshot."""
+ return self.apply()
+
+ def _deleted_entries(self) -> List[ManifestEntry]:
+ """No entries are deleted during manifest rewriting."""
+ return []
+ def apply(self) -> List[ManifestFile]:
+ snapshot = self._transaction.table_metadata.current_snapshot()
+ current_manifests = snapshot.manifests(io=self._io)
+ current_manifest_set = set(current_manifests)
+ data_manifest_merge_manager = _ManifestMergeManager(
+ target_size_bytes=self.manifest_target_size_bytes,
+ min_count_to_merge=2,
+ merge_enabled=True,
+ snapshot_producer=self,
+ )
+ # self.validate_deleted_manifests(current_manifest_set, base.current_snapshot().snapshot_id())
+
+ if self.requires_rewrite(current_manifest_set):
+ new_manifests = data_manifest_merge_manager.merge_manifests(manifests=current_manifests)
+ # self.rewritten_manifests.add(new_manifests)
+ self.rewritten_manifests.update(new_manifests)
+ else:
+ self.keep_active_manifests(current_manifests)
+
+ self.validate_files_counts()
+
+ new_manifests_with_metadata = [
+ manifest.copy_with_snapshot_id(self.snapshot_id)
+ for manifest in itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
+ ]
+
+ # put new manifests at the beginning
+ applyi = list(new_manifests_with_metadata)
+ applyi.extend(self.kept_manifests)
+
+ return applyi
+
+ # def execute(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
+ # data_manifest_merge_manager = _ManifestMergeManager(
+ # target_size_bytes=self.manifest_target_size_bytes,
+ # min_count_to_merge=2,
+ # merge_enabled=True,
+ # snapshot_producer=self,
+ # )
+ # return data_manifest_merge_manager.merge_manifests(manifests=manifests)
+
+ def add_manifest(self, manifest):
+ if manifest.has_added_files():
+ raise ValueError("Cannot add manifest with added files")
+ if manifest.has_deleted_files():
+ raise ValueError("Cannot add manifest with deleted files")
+ if manifest.snapshot_id() is not None and manifest.snapshot_id() != -1:
+ raise ValueError("Snapshot id must be assigned during commit")
+ if manifest.sequence_number() != -1:
+ raise ValueError("Sequence must be assigned during commit")
+
+ if manifest.snapshot_id() is None:
+ self.added_manifests.append(manifest)
+ else:
+ copied_manifest = self.copy_manifest(manifest)
+ self.rewritten_added_manifests.append(copied_manifest)
+
+ return self
+
+
class _FastAppendFiles(_SnapshotProducer["_FastAppendFiles"]):
def _existing_manifests(self) -> List[ManifestFile]:
"""To determine if there are any existing manifest files.
@@ -530,13 +709,12 @@ class _MergeAppendFiles(_FastAppendFiles):
_merge_enabled: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- branch: str,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ branch: str,commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
from pyiceberg.table import TableProperties
@@ -736,7 +914,8 @@ def _existing_manifests(self) -> List[ManifestFile]:
if snapshot := self._transaction.table_metadata.snapshot_by_name(name=self._target_branch):
for manifest_file in snapshot.manifests(io=self._io):
entries = manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True)
- found_deleted_data_files = [entry.data_file for entry in entries if entry.data_file in self._deleted_data_files]
+ found_deleted_data_files = [entry.data_file for entry in entries if
+ entry.data_file in self._deleted_data_files]
if len(found_deleted_data_files) == 0:
existing_files.append(manifest_file)
@@ -744,11 +923,11 @@ def _existing_manifests(self) -> List[ManifestFile]:
# We have to rewrite the manifest file without the deleted data files
if any(entry.data_file not in found_deleted_data_files for entry in entries):
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for entry in entries:
@@ -819,10 +998,8 @@ def __init__(
def fast_append(self) -> _FastAppendFiles:
return _FastAppendFiles(
- operation=Operation.APPEND,
- transaction=self._transaction,
- io=self._io,
- branch=self._branch,
+ operation=Operation.APPEND, transaction=self._transaction, io=self._io,
+ branch=self._branch,
snapshot_properties=self._snapshot_properties,
)
@@ -872,7 +1049,8 @@ class _ManifestMergeManager(Generic[U]):
_snapshot_producer: _SnapshotProducer[U]
def __init__(
- self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool, snapshot_producer: _SnapshotProducer[U]
+ self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool,
+ snapshot_producer: _SnapshotProducer[U]
) -> None:
self._target_size_bytes = target_size_bytes
self._min_count_to_merge = min_count_to_merge
@@ -901,8 +1079,10 @@ def _create_manifest(self, spec_id: int, manifest_bin: List[ManifestFile]) -> Ma
return writer.to_manifest_file()
- def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[ManifestFile]:
- packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1, largest_bin_first=False)
+ def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[
+ ManifestFile]:
+ packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1,
+ largest_bin_first=False)
bins: List[List[ManifestFile]] = packer.pack_end(manifests, lambda m: m.manifest_length)
def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
@@ -924,7 +1104,8 @@ def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
# for consistent ordering, we need to maintain future order
futures_index = {f: i for i, f in enumerate(futures)}
- completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[], key=lambda f: futures_index[f])
+ completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[],
+ key=lambda f: futures_index[f])
for future in concurrent.futures.as_completed(futures):
completed_futures.add(future)
@@ -1025,12 +1206,12 @@ def remove_tag(self, tag_name: str) -> ManageSnapshots:
return self._remove_ref_snapshot(ref_name=tag_name)
def create_branch(
- self,
- snapshot_id: int,
- branch_name: str,
- max_ref_age_ms: Optional[int] = None,
- max_snapshot_age_ms: Optional[int] = None,
- min_snapshots_to_keep: Optional[int] = None,
+ self,
+ snapshot_id: int,
+ branch_name: str,
+ max_ref_age_ms: Optional[int] = None,
+ max_snapshot_age_ms: Optional[int] = None,
+ min_snapshots_to_keep: Optional[int] = None,
) -> ManageSnapshots:
"""
Create a new branch pointing to the given snapshot id.
diff --git a/tests/integration/test_inspect_table.py b/tests/integration/test_inspect_table.py
index 875b1f69ad..e81050a81c 100644
--- a/tests/integration/test_inspect_table.py
+++ b/tests/integration/test_inspect_table.py
@@ -1101,45 +1101,3 @@ def test_inspect_files_partitioned(spark: SparkSession, session_catalog: Catalog
.reset_index()
)
assert_frame_equal(lhs, rhs, check_dtype=False)
-
-
-def test_inspect_all_example(spark: SparkSession, session_catalog: Catalog, format_version: int) -> None:
- from pandas.testing import assert_frame_equal
-
- identifier = "default.table_metadata_all_manifests"
- try:
- session_catalog.drop_table(identifier=identifier)
- except NoSuchTableError:
- pass
-
- spark.sql(
- f"""
- CREATE TABLE {identifier} (
- id int,
- data string
- )
- PARTITIONED BY (data)
- TBLPROPERTIES ('write.update.mode'='merge-on-read',
- 'write.delete.mode'='merge-on-read')
- """
- )
- tbl = session_catalog.load_table(identifier)
- # check all_manifests when there are no snapshots
- lhs = tbl.inspect.all_manifests().to_pandas()
- rhs = spark.table(f"{identifier}.all_manifests").toPandas()
- assert_frame_equal(lhs, rhs, check_dtype=False)
-
- spark.sql(f"INSERT INTO {identifier} VALUES (1, 'a')")
-
- spark.sql(f"INSERT INTO {identifier} VALUES (2, 'b')")
-
- spark.sql(f"UPDATE {identifier} SET data = 'c' WHERE id = 1")
-
- spark.sql(f"DELETE FROM {identifier} WHERE id = 2")
-
- spark.sql(f"INSERT OVERWRITE {identifier} VALUES (1, 'a')")
-
- tbl.refresh()
-
- tbl.rewrite_manifests()
- print("efd")
From 06211b8f2d9634c48520997374ed9f138c5e0803 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Thu, 13 Feb 2025 22:19:36 +0200
Subject: [PATCH 108/112] initial implementation of rewrite_manifests
---
pyiceberg/manifest.py | 3 +
pyiceberg/table/__init__.py | 52 +--
pyiceberg/table/update/snapshot.py | 315 ++++--------------
.../test_writes/test_rewrite_manifests.py | 4 +
4 files changed, 83 insertions(+), 291 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index b6d8273839..061b2d9bd2 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -860,6 +860,7 @@ def copy_with_snapshot_id(self, snapshot_id: int) -> ManifestFile:
partitions=self.partitions,
key_metadata=self.key_metadata,
)
+
def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List[ManifestEntry]:
"""
Read the manifest entries from the manifest file.
@@ -883,10 +884,12 @@ def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List
for entry in reader
if not discard_deleted or entry.status != ManifestEntryStatus.DELETED
]
+
def __hash__(self) -> int:
"""Return the hash of the file path."""
return hash(self.manifest_path)
+
@cached(cache=LRUCache(maxsize=128), key=lambda io, manifest_list: hashkey(manifest_list))
def _manifests(io: FileIO, manifest_list: str) -> Tuple[ManifestFile, ...]:
"""Read and cache manifests from the given manifest list, returning a tuple to prevent modification."""
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index f0bf55402a..e2963179ed 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -134,7 +134,7 @@
)
from pyiceberg.utils.concurrent import ExecutorFactory
from pyiceberg.utils.config import Config
-from pyiceberg.utils.properties import property_as_bool, property_as_int
+from pyiceberg.utils.properties import property_as_bool
if TYPE_CHECKING:
import daft
@@ -443,7 +443,7 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT, bran
return UpdateSnapshot(self, io=self._table.io, branch=branch, snapshot_properties=snapshot_properties)
def rewrite_manifests(self) -> None:
with self.update_snapshot().rewrite() as rewrite:
- rewrite.commit()
+ rewrite.rewrite_manifests()
def rewrite_manifests(self) -> RewriteManifestsResult:
if self._table.current_snapshot() is None:
@@ -1453,49 +1453,17 @@ def add_files(
def rewrite_manifests(
self,
spec_id: Optional[int] = None,
- rewrite_all: bool = False,
- max_manifest_size: Optional[int] = None,
- ) -> "Table":
-
- with self.transaction() as tx:
- tx.rewrite_manifests()
- ...
- """Rewrite manifests in the table.
+ ) -> RewriteManifestsResult:
+ """
+ Shorthand API for Rewriting manifests for the table.
Args:
- spec_id: Spec ID to be used for the rewritten manifests
- rewrite_all: If True, rewrite all manifests. If False, only rewrite small manifests
- max_manifest_size: Target size for manifests in bytes
+ spec_id: Spec id of the manifests to rewrite (defaults to current spec id)
- Returns:
- An updated version of the table with rewritten manifests
- #"""
- # return RewriteManifests(
- # self,
- # spec_id=spec_id,
- # rewrite_all=rewrite_all,
- # max_manifest_size=max_manifest_size,
- # ).commit()
-
- # snapshot = self.current_snapshot()
- # manifests = []
- # for manifest in snapshot.manifests(self.io):
- # if manifest.content == ManifestContent.DATA:
- # manifests.append(manifest)
- #
- # data_manifest_merge_manager = _ManifestMergeManager(
- # target_size_bytes=property_as_int(
- # self.properties,
- # TableProperties.MANIFEST_TARGET_SIZE_BYTES,
- # TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
- # ),
- # min_count_to_merge=2,
- # merge_enabled=True,
- # snapshot_producer=self,
- # )
- #
- # data_manifest_merge_manager.merge_manifests(manifests)
- # entries = self.inspect.entries().filter("status < 2").selectExpr("input_file_name() as manifest")
+ """
+ with self.transaction() as tx:
+ rewrite_results = tx.rewrite_manifests(spec_id=spec_id)
+ return rewrite_results
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 52ce3feafb..4b9b0a5b2a 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -112,12 +112,12 @@ class _SnapshotProducer(UpdateTableMetadata[U], Generic[U]):
_target_branch = MAIN_BRANCH
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
branch: str = MAIN_BRANCH,
) -> None:
super().__init__(transaction)
@@ -158,12 +158,10 @@ def delete_data_file(self, data_file: DataFile) -> _SnapshotProducer[U]:
return self
@abstractmethod
- def _deleted_entries(self) -> List[ManifestEntry]:
- ...
+ def _deleted_entries(self) -> List[ManifestEntry]: ...
@abstractmethod
- def _existing_manifests(self) -> List[ManifestFile]:
- ...
+ def _existing_manifests(self) -> List[ManifestFile]: ...
def _process_manifests(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
"""To perform any post-processing on the manifests before writing them to the new snapshot."""
@@ -173,11 +171,11 @@ def _manifests(self) -> List[ManifestFile]:
def _write_added_manifest() -> List[ManifestFile]:
if self._added_data_files:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.spec(),
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.spec(),
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for data_file in self._added_data_files:
@@ -204,11 +202,11 @@ def _write_delete_manifest() -> List[ManifestFile]:
partition_groups[deleted_entry.data_file.spec_id].append(deleted_entry)
for spec_id, entries in partition_groups.items():
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for entry in entries:
@@ -224,8 +222,7 @@ def _write_delete_manifest() -> List[ManifestFile]:
delete_manifests = executor.submit(_write_delete_manifest)
existing_manifests = executor.submit(self._existing_manifests)
- return self._process_manifests(
- added_manifests.result() + delete_manifests.result() + existing_manifests.result())
+ return self._process_manifests(added_manifests.result() + delete_manifests.result() + existing_manifests.result())
def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
from pyiceberg.table import TableProperties
@@ -278,11 +275,11 @@ def _commit(self) -> UpdatesAndRequirements:
manifest_list_file_path = location_provider.new_metadata_location(file_name)
with write_manifest_list(
- format_version=self._transaction.table_metadata.format_version,
- output_file=self._io.new_output(manifest_list_file_path),
- snapshot_id=self._snapshot_id,
- parent_snapshot_id=self._parent_snapshot_id,
- sequence_number=next_sequence_number,
+ format_version=self._transaction.table_metadata.format_version,
+ output_file=self._io.new_output(manifest_list_file_path),
+ snapshot_id=self._snapshot_id,
+ parent_snapshot_id=self._parent_snapshot_id,
+ sequence_number=next_sequence_number,
avro_compression=self._compression,
) as writer:
writer.add_manifests(new_manifests)
@@ -302,8 +299,7 @@ def _commit(self) -> UpdatesAndRequirements:
SetSnapshotRefUpdate(
snapshot_id=self._snapshot_id,
parent_snapshot_id=self._parent_snapshot_id,
- ref_name=self._target_branch,
- type=SnapshotRefType.BRANCH,
+ ref_name=self._target_branch, type=SnapshotRefType.BRANCH,
),
),
(
@@ -357,12 +353,12 @@ class _DeleteFiles(_SnapshotProducer["_DeleteFiles"]):
_case_sensitive: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- branch: str,commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ branch: str,commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
):
super().__init__(operation, transaction, io, commit_uuid, snapshot_properties, branch)
self._predicate = AlwaysFalse()
@@ -415,8 +411,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
)
manifest_evaluators: Dict[int, Callable[[ManifestFile], bool]] = KeyDefaultDict(self._build_manifest_evaluator)
- strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate,
- case_sensitive=self._case_sensitive).eval
+ strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate, case_sensitive=self._case_sensitive).eval
inclusive_metrics_evaluator = _InclusiveMetricsEvaluator(
schema, self._predicate, case_sensitive=self._case_sensitive
).eval
@@ -458,11 +453,11 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
# Rewrite the manifest
if len(existing_entries) > 0:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for existing_entry in existing_entries:
@@ -492,182 +487,6 @@ def files_affected(self) -> bool:
return len(self._deleted_entries()) > 0
-class _RewriteManifests(_SnapshotProducer["_RewriteManifests"]):
- """Rewrite manifest files based on the predicate."""
-
- KEPT_MANIFESTS_COUNT = "manifests-kept"
- CREATED_MANIFESTS_COUNT = "manifests-created"
- REPLACED_MANIFESTS_COUNT = "manifests-replaced"
- PROCESSED_ENTRY_COUNT = "entries-processed"
-
- def __init__(
- self,
- transaction: Transaction,
- io: FileIO,
- commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
- ):
- from pyiceberg.table import TableProperties
-
- super().__init__(Operation.REPLACE, transaction, io, commit_uuid, snapshot_properties)
-
- self.specs_by_id = self._transaction.table_metadata.spec().spec_id
- self.manifest_target_size_bytes = property_as_int(
- self._transaction.table_metadata.properties,
- TableProperties.MANIFEST_TARGET_SIZE_BYTES,
- TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT,
- )
- self.deleted_manifests: Set[ManifestFile] = set()
- self.added_manifests: List[ManifestFile] = []
- self.rewritten_added_manifests: List[ManifestFile] = []
- self.kept_manifests: Set[ManifestFile] = set()
- self.new_manifests: Set[ManifestFile] = set()
- self.rewritten_manifests: Set[ManifestFile] = set()
-
- def copy_manifest(self, manifest: ManifestFile) -> ManifestFile:
- return ManifestFile(
- manifest_path=manifest.manifest_path,
- manifest_length=manifest.manifest_length,
- partition_spec_id=manifest.partition_spec_id,
- content=manifest.content,
- sequence_number=manifest.sequence_number,
- min_sequence_number=manifest.min_sequence_number,
- added_snapshot_id=manifest.added_snapshot_id,
- added_files_count=manifest.added_files_count,
- existing_files_count=manifest.existing_files_count,
- deleted_files_count=manifest.deleted_files_count,
- added_rows_count=manifest.added_rows_count,
- existing_rows_count=manifest.existing_rows_count,
- deleted_rows_count=manifest.deleted_rows_count,
- partitions=manifest.partitions,
- key_metadata=manifest.key_metadata,
- )
-
- def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
- summary = {
- self.CREATED_MANIFESTS_COUNT: str(
- len(self.new_manifests) + len(self.added_manifests) + len(self.rewritten_added_manifests)
- ),
- self.KEPT_MANIFESTS_COUNT: str(len(self.kept_manifests)),
- self.REPLACED_MANIFESTS_COUNT: str(len(self.rewritten_manifests) + len(self.deleted_manifests)),
- self.PROCESSED_ENTRY_COUNT: str(self.entry_count),
- }
- return super()._summary(summary)
-
- def delete_manifest(self, manifest):
- self.deleted_manifests.add(manifest)
- return self
-
- def requires_rewrite(self, current_manifests):
-
- if not self.rewritten_manifests:
- # nothing yet processed so perform a full rewrite
- return True
-
- # if any processed manifest is not in the current manifest list, perform a full rewrite
- return any(manifest not in current_manifests for manifest in self.rewritten_manifests)
-
- def keep_active_manifests(self, current_manifests: List[ManifestFile]) -> None:
- # keep any existing manifests as-is that were not processed
- self.kept_manifests.clear()
- for manifest in current_manifests:
- if manifest not in self.rewritten_manifests and manifest not in self.deleted_manifests:
- self.kept_manifests.add(manifest)
-
- def active_files_count(self, manifests):
- active_files_count = 0
-
- for manifest in manifests:
- if manifest.added_files_count is None:
- raise ValueError("Missing file counts in {}".format(manifest.path()))
- if manifest.existing_files_count is None:
- raise ValueError("Missing file counts in {}".format(manifest.path()))
- active_files_count += manifest.added_files_count
- active_files_count += manifest.existing_files_count
-
- return active_files_count
-
- def validate_files_counts(self):
- created_manifests = itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
- created_manifests_files_count = self.active_files_count(created_manifests)
-
- replaced_manifests = itertools.chain(self.rewritten_manifests, self.deleted_manifests)
- replaced_manifests_files_count = self.active_files_count(replaced_manifests)
-
- if created_manifests_files_count != replaced_manifests_files_count:
- raise ValueError(
- "Replaced and created manifests must have the same number of active files: {} (new), {} (old)".format(
- created_manifests_files_count, replaced_manifests_files_count
- )
- )
-
- def _existing_manifests(self) -> List[ManifestFile]:
- """Returns the list of manifests to include in the next snapshot."""
- return self.apply()
-
- def _deleted_entries(self) -> List[ManifestEntry]:
- """No entries are deleted during manifest rewriting."""
- return []
- def apply(self) -> List[ManifestFile]:
- snapshot = self._transaction.table_metadata.current_snapshot()
- current_manifests = snapshot.manifests(io=self._io)
- current_manifest_set = set(current_manifests)
- data_manifest_merge_manager = _ManifestMergeManager(
- target_size_bytes=self.manifest_target_size_bytes,
- min_count_to_merge=2,
- merge_enabled=True,
- snapshot_producer=self,
- )
- # self.validate_deleted_manifests(current_manifest_set, base.current_snapshot().snapshot_id())
-
- if self.requires_rewrite(current_manifest_set):
- new_manifests = data_manifest_merge_manager.merge_manifests(manifests=current_manifests)
- # self.rewritten_manifests.add(new_manifests)
- self.rewritten_manifests.update(new_manifests)
- else:
- self.keep_active_manifests(current_manifests)
-
- self.validate_files_counts()
-
- new_manifests_with_metadata = [
- manifest.copy_with_snapshot_id(self.snapshot_id)
- for manifest in itertools.chain(self.new_manifests, self.added_manifests, self.rewritten_added_manifests)
- ]
-
- # put new manifests at the beginning
- applyi = list(new_manifests_with_metadata)
- applyi.extend(self.kept_manifests)
-
- return applyi
-
- # def execute(self, manifests: List[ManifestFile]) -> List[ManifestFile]:
- # data_manifest_merge_manager = _ManifestMergeManager(
- # target_size_bytes=self.manifest_target_size_bytes,
- # min_count_to_merge=2,
- # merge_enabled=True,
- # snapshot_producer=self,
- # )
- # return data_manifest_merge_manager.merge_manifests(manifests=manifests)
-
- def add_manifest(self, manifest):
- if manifest.has_added_files():
- raise ValueError("Cannot add manifest with added files")
- if manifest.has_deleted_files():
- raise ValueError("Cannot add manifest with deleted files")
- if manifest.snapshot_id() is not None and manifest.snapshot_id() != -1:
- raise ValueError("Snapshot id must be assigned during commit")
- if manifest.sequence_number() != -1:
- raise ValueError("Sequence must be assigned during commit")
-
- if manifest.snapshot_id() is None:
- self.added_manifests.append(manifest)
- else:
- copied_manifest = self.copy_manifest(manifest)
- self.rewritten_added_manifests.append(copied_manifest)
-
- return self
-
-
class _FastAppendFiles(_SnapshotProducer["_FastAppendFiles"]):
def _existing_manifests(self) -> List[ManifestFile]:
"""To determine if there are any existing manifest files.
@@ -709,12 +528,12 @@ class _MergeAppendFiles(_FastAppendFiles):
_merge_enabled: bool
def __init__(
- self,
- operation: Operation,
- transaction: Transaction,
- io: FileIO,
- branch: str,commit_uuid: Optional[uuid.UUID] = None,
- snapshot_properties: Dict[str, str] = EMPTY_DICT,
+ self,
+ operation: Operation,
+ transaction: Transaction,
+ io: FileIO,
+ branch: str,commit_uuid: Optional[uuid.UUID] = None,
+ snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
from pyiceberg.table import TableProperties
@@ -914,8 +733,7 @@ def _existing_manifests(self) -> List[ManifestFile]:
if snapshot := self._transaction.table_metadata.snapshot_by_name(name=self._target_branch):
for manifest_file in snapshot.manifests(io=self._io):
entries = manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True)
- found_deleted_data_files = [entry.data_file for entry in entries if
- entry.data_file in self._deleted_data_files]
+ found_deleted_data_files = [entry.data_file for entry in entries if entry.data_file in self._deleted_data_files]
if len(found_deleted_data_files) == 0:
existing_files.append(manifest_file)
@@ -923,11 +741,11 @@ def _existing_manifests(self) -> List[ManifestFile]:
# We have to rewrite the manifest file without the deleted data files
if any(entry.data_file not in found_deleted_data_files for entry in entries):
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for entry in entries:
@@ -998,17 +816,20 @@ def __init__(
def fast_append(self) -> _FastAppendFiles:
return _FastAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io,
- branch=self._branch,
+ operation=Operation.APPEND, transaction=self._transaction, io=self._io,branch=self._branch,
snapshot_properties=self._snapshot_properties,
)
def merge_append(self) -> _MergeAppendFiles:
return _MergeAppendFiles(
- operation=Operation.APPEND,
+ operation=Operation.APPEND, transaction=self._transaction, io=self._io,branch=self._branch,
+ snapshot_properties=self._snapshot_properties,
+ )
+ def rewrite(self) -> _RewriteManifests:
+ return _RewriteManifests(
+ table=self._transaction._table,
transaction=self._transaction,
io=self._io,
- branch=self._branch,
snapshot_properties=self._snapshot_properties,
)
@@ -1049,8 +870,7 @@ class _ManifestMergeManager(Generic[U]):
_snapshot_producer: _SnapshotProducer[U]
def __init__(
- self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool,
- snapshot_producer: _SnapshotProducer[U]
+ self, target_size_bytes: int, min_count_to_merge: int, merge_enabled: bool, snapshot_producer: _SnapshotProducer[U]
) -> None:
self._target_size_bytes = target_size_bytes
self._min_count_to_merge = min_count_to_merge
@@ -1079,10 +899,8 @@ def _create_manifest(self, spec_id: int, manifest_bin: List[ManifestFile]) -> Ma
return writer.to_manifest_file()
- def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[
- ManifestFile]:
- packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1,
- largest_bin_first=False)
+ def _merge_group(self, first_manifest: ManifestFile, spec_id: int, manifests: List[ManifestFile]) -> List[ManifestFile]:
+ packer: ListPacker[ManifestFile] = ListPacker(target_weight=self._target_size_bytes, lookback=1, largest_bin_first=False)
bins: List[List[ManifestFile]] = packer.pack_end(manifests, lambda m: m.manifest_length)
def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
@@ -1104,8 +922,7 @@ def merge_bin(manifest_bin: List[ManifestFile]) -> List[ManifestFile]:
# for consistent ordering, we need to maintain future order
futures_index = {f: i for i, f in enumerate(futures)}
- completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[],
- key=lambda f: futures_index[f])
+ completed_futures: SortedList[Future[List[ManifestFile]]] = SortedList(iterable=[], key=lambda f: futures_index[f])
for future in concurrent.futures.as_completed(futures):
completed_futures.add(future)
@@ -1206,12 +1023,12 @@ def remove_tag(self, tag_name: str) -> ManageSnapshots:
return self._remove_ref_snapshot(ref_name=tag_name)
def create_branch(
- self,
- snapshot_id: int,
- branch_name: str,
- max_ref_age_ms: Optional[int] = None,
- max_snapshot_age_ms: Optional[int] = None,
- min_snapshots_to_keep: Optional[int] = None,
+ self,
+ snapshot_id: int,
+ branch_name: str,
+ max_ref_age_ms: Optional[int] = None,
+ max_snapshot_age_ms: Optional[int] = None,
+ min_snapshots_to_keep: Optional[int] = None,
) -> ManageSnapshots:
"""
Create a new branch pointing to the given snapshot id.
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index b4fd6ec37f..a46ccb243e 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -19,6 +19,7 @@
import pyarrow as pa
import pytest
+from pyspark.sql import SparkSession
from pyiceberg.catalog import Catalog
from pyiceberg.manifest import ManifestFile
@@ -100,6 +101,7 @@ def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null
tx.upgrade_table_version(format_version=2)
tbl.append(arrow_table_with_null)
+
assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
with tbl.transaction() as tx: # type: ignore[unreachable]
@@ -171,6 +173,8 @@ def test_rewrite_small_manifests_non_partitioned_table(session_catalog: Catalog,
assert len(result.added_manifests) == 1, "Action should add 1 manifest"
tbl.refresh()
+ manifests = tbl.inspect.all_manifests().to_pylist()
+ assert len(manifests) == 3, "Should have 3 manifests before rewrite"
current_snapshot = tbl.current_snapshot()
if not current_snapshot:
From 31efcbc76a4a01fb94faab7623228a27bc32b316 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Thu, 13 Feb 2025 22:21:07 +0200
Subject: [PATCH 109/112] fix lint
---
pyiceberg/table/__init__.py | 5 ++---
1 file changed, 2 insertions(+), 3 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index e2963179ed..a4746493b2 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -1453,7 +1453,7 @@ def add_files(
def rewrite_manifests(
self,
spec_id: Optional[int] = None,
- ) -> RewriteManifestsResult:
+ ) -> None:
"""
Shorthand API for Rewriting manifests for the table.
@@ -1462,8 +1462,7 @@ def rewrite_manifests(
"""
with self.transaction() as tx:
- rewrite_results = tx.rewrite_manifests(spec_id=spec_id)
- return rewrite_results
+ tx.rewrite_manifests(spec_id=spec_id)
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
From b3487ac1028467179e8e565dc855a4bfcab07745 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Sun, 23 Feb 2025 09:05:46 +0200
Subject: [PATCH 110/112] 1. support returning rewrite results 2. write tests
for rewrite manifests
---
pyiceberg/table/__init__.py | 11 +++---
pyiceberg/table/update/snapshot.py | 30 +++++++++++++++-
.../test_writes/test_rewrite_manifests.py | 35 +++++++++++++++++--
3 files changed, 68 insertions(+), 8 deletions(-)
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index a4746493b2..65582872cc 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -441,9 +441,12 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT, bran
branch = MAIN_BRANCH
return UpdateSnapshot(self, io=self._table.io, branch=branch, snapshot_properties=snapshot_properties)
- def rewrite_manifests(self) -> None:
+ def rewrite_manifests(self) -> RewriteManifestsResult:
+ if self._table.current_snapshot() is None:
+ return RewriteManifestsResult(rewritten_manifests=[], added_manifests=[])
with self.update_snapshot().rewrite() as rewrite:
- rewrite.rewrite_manifests()
+ rewritten = rewrite.rewrite_manifests()
+ return rewritten
def rewrite_manifests(self) -> RewriteManifestsResult:
if self._table.current_snapshot() is None:
@@ -1453,7 +1456,7 @@ def add_files(
def rewrite_manifests(
self,
spec_id: Optional[int] = None,
- ) -> None:
+ ) -> RewriteManifestsResult:
"""
Shorthand API for Rewriting manifests for the table.
@@ -1462,7 +1465,7 @@ def rewrite_manifests(
"""
with self.transaction() as tx:
- tx.rewrite_manifests(spec_id=spec_id)
+ return tx.rewrite_manifests(spec_id=spec_id)
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 4b9b0a5b2a..b4d7857ef4 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -650,13 +650,41 @@ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
truncate_full_table=False,
)
+ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
+ from pyiceberg.table import TableProperties
+
+ ssc = SnapshotSummaryCollector()
+ partition_summary_limit = int(
+ self._transaction.table_metadata.properties.get(
+ TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT
+ )
+ )
+ ssc.set_partition_summary_limit(partition_summary_limit)
+
+ props = {
+ "manifests-kept": str(len([])),
+ "manifests-created": str(len(self.added_manifests)),
+ "manifests-replaced": str(len(self.rewritten_manifests)),
+ "entries-processed": str(len([])),
+ }
+ previous_snapshot = (
+ self._transaction.table_metadata.snapshot_by_id(self._parent_snapshot_id)
+ if self._parent_snapshot_id is not None
+ else None
+ )
+
+ return update_snapshot_summaries(
+ summary=Summary(operation=self._operation, **ssc.build(), **props),
+ previous_summary=previous_snapshot.summary if previous_snapshot is not None else None,
+ truncate_full_table=False,
+ )
+
def rewrite_manifests(self) -> RewriteManifestsResult:
snapshot = self._table.current_snapshot()
if not snapshot:
raise ValueError("Cannot rewrite manifests without a current snapshot")
data_result = self._find_matching_manifests(snapshot, ManifestContent.DATA)
-
self.rewritten_manifests.extend(data_result.rewritten_manifests)
self.added_manifests.extend(data_result.added_manifests)
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index a46ccb243e..08114899ea 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -19,7 +19,6 @@
import pyarrow as pa
import pytest
-from pyspark.sql import SparkSession
from pyiceberg.catalog import Catalog
from pyiceberg.manifest import ManifestFile
@@ -138,6 +137,35 @@ def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null
assert expected_records_count == actual_records_count, "Record count must match"
+# @pytest.mark.integration
+# def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
+# identifier = "default.arrow_table_summaries"
+# tbl = _create_table(session_catalog, identifier, {"format-version": "2"})
+# tbl.append(arrow_table_with_null)
+# tbl.append(arrow_table_with_null)
+#
+# # tbl.rewrite_manifests()
+#
+# # records1 = [ThreeColumnRecord(1, None, "AAAA")]
+# # write_records(spark, table_location, records1)
+# before_pandas = tbl.scan().to_pandas()
+# before_count = before_pandas.shape[0]
+# tbl.refresh()
+# manifests = tbl.inspect.manifests().to_pylist()
+# assert len(manifests) == 2, "Should have 2 manifests before rewrite"
+#
+# tbl.rewrite_manifests()
+# tbl.refresh()
+#
+# after_pandas = tbl.scan().to_pandas()
+# after_count = before_pandas.shape[0]
+# manifests = tbl.inspect.manifests().to_pylist()
+# assert len(manifests) == 1, "Should have 1 manifests before rewrite"
+#
+# snaps = tbl.inspect.snapshots().to_pandas()
+# print(snaps)
+
+
@pytest.mark.integration
def test_rewrite_manifests_empty_table(session_catalog: Catalog) -> None:
# Create an unpartitioned table
@@ -173,8 +201,9 @@ def test_rewrite_small_manifests_non_partitioned_table(session_catalog: Catalog,
assert len(result.added_manifests) == 1, "Action should add 1 manifest"
tbl.refresh()
- manifests = tbl.inspect.all_manifests().to_pylist()
- assert len(manifests) == 3, "Should have 3 manifests before rewrite"
+
+ manifests = tbl.inspect.manifests()
+ assert len(manifests) == 2, "Should have 2 manifests before rewrite"
current_snapshot = tbl.current_snapshot()
if not current_snapshot:
From 0ad1eeb4e4c21952e61f00ac4dcefad34ab8b493 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Mon, 7 Jul 2025 08:39:14 +0300
Subject: [PATCH 111/112] lint and rebase
---
pyiceberg/manifest.py | 4 ---
pyiceberg/table/__init__.py | 9 +++++-
pyiceberg/table/update/snapshot.py | 32 ++++++++++---------
.../test_writes/test_rewrite_manifests.py | 1 -
4 files changed, 25 insertions(+), 21 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 061b2d9bd2..26f3ad4be5 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -885,10 +885,6 @@ def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List
if not discard_deleted or entry.status != ManifestEntryStatus.DELETED
]
- def __hash__(self) -> int:
- """Return the hash of the file path."""
- return hash(self.manifest_path)
-
@cached(cache=LRUCache(maxsize=128), key=lambda io, manifest_list: hashkey(manifest_list))
def _manifests(io: FileIO, manifest_list: str) -> Tuple[ManifestFile, ...]:
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index 65582872cc..e3e1d5af71 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -115,7 +115,13 @@
update_table_metadata,
)
from pyiceberg.table.update.schema import UpdateSchema
-from pyiceberg.table.update.snapshot import ExpireSnapshots, ManageSnapshots, RewriteManifestsResult, UpdateSnapshot, _FastAppendFiles, _ManifestMergeManager
+from pyiceberg.table.update.snapshot import (
+ ExpireSnapshots,
+ ManageSnapshots,
+ RewriteManifestsResult,
+ UpdateSnapshot,
+ _FastAppendFiles,
+)
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
from pyiceberg.transforms import IdentityTransform
@@ -441,6 +447,7 @@ def update_snapshot(self, snapshot_properties: Dict[str, str] = EMPTY_DICT, bran
branch = MAIN_BRANCH
return UpdateSnapshot(self, io=self._table.io, branch=branch, snapshot_properties=snapshot_properties)
+
def rewrite_manifests(self) -> RewriteManifestsResult:
if self._table.current_snapshot() is None:
return RewriteManifestsResult(rewritten_manifests=[], added_manifests=[])
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index b4d7857ef4..2191cab113 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -299,7 +299,8 @@ def _commit(self) -> UpdatesAndRequirements:
SetSnapshotRefUpdate(
snapshot_id=self._snapshot_id,
parent_snapshot_id=self._parent_snapshot_id,
- ref_name=self._target_branch, type=SnapshotRefType.BRANCH,
+ ref_name=self._target_branch,
+ type=SnapshotRefType.BRANCH,
),
),
(
@@ -357,7 +358,8 @@ def __init__(
operation: Operation,
transaction: Transaction,
io: FileIO,
- branch: str,commit_uuid: Optional[uuid.UUID] = None,
+ branch: str,
+ commit_uuid: Optional[uuid.UUID] = None,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
):
super().__init__(operation, transaction, io, commit_uuid, snapshot_properties, branch)
@@ -453,11 +455,11 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) ->
# Rewrite the manifest
if len(existing_entries) > 0:
with write_manifest(
- format_version=self._transaction.table_metadata.format_version,
- spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
- schema=self._transaction.table_metadata.schema(),
- output_file=self.new_manifest_output(),
- snapshot_id=self._snapshot_id,
+ format_version=self._transaction.table_metadata.format_version,
+ spec=self._transaction.table_metadata.specs()[manifest_file.partition_spec_id],
+ schema=self._transaction.table_metadata.schema(),
+ output_file=self.new_manifest_output(),
+ snapshot_id=self._snapshot_id,
avro_compression=self._compression,
) as writer:
for existing_entry in existing_entries:
@@ -532,7 +534,8 @@ def __init__(
operation: Operation,
transaction: Transaction,
io: FileIO,
- branch: str,commit_uuid: Optional[uuid.UUID] = None,
+ branch: str,
+ commit_uuid: Optional[uuid.UUID] = None,
snapshot_properties: Dict[str, str] = EMPTY_DICT,
) -> None:
from pyiceberg.table import TableProperties
@@ -844,20 +847,19 @@ def __init__(
def fast_append(self) -> _FastAppendFiles:
return _FastAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io,branch=self._branch,
+ operation=Operation.APPEND,
+ transaction=self._transaction,
+ io=self._io,
+ branch=self._branch,
snapshot_properties=self._snapshot_properties,
)
def merge_append(self) -> _MergeAppendFiles:
return _MergeAppendFiles(
- operation=Operation.APPEND, transaction=self._transaction, io=self._io,branch=self._branch,
- snapshot_properties=self._snapshot_properties,
- )
- def rewrite(self) -> _RewriteManifests:
- return _RewriteManifests(
- table=self._transaction._table,
+ operation=Operation.APPEND,
transaction=self._transaction,
io=self._io,
+ branch=self._branch,
snapshot_properties=self._snapshot_properties,
)
diff --git a/tests/integration/test_writes/test_rewrite_manifests.py b/tests/integration/test_writes/test_rewrite_manifests.py
index 08114899ea..38e85ae0a9 100644
--- a/tests/integration/test_writes/test_rewrite_manifests.py
+++ b/tests/integration/test_writes/test_rewrite_manifests.py
@@ -82,7 +82,6 @@ def table_v2_appended_with_null(session_catalog: Catalog, arrow_table_with_null:
assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
-
@pytest.mark.integration
def test_rewrite_v1_v2_manifests(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
identifier = "default.test_rewrite_v1_v2_manifests"
From 9bdbc86f95419e21c9bdd3eb18e834bde44ceab4 Mon Sep 17 00:00:00 2001
From: amitgilad
Date: Mon, 7 Jul 2025 08:57:12 +0300
Subject: [PATCH 112/112] linting errors
---
pyiceberg/manifest.py | 19 -------------------
pyiceberg/table/__init__.py | 22 ----------------------
pyiceberg/table/update/snapshot.py | 29 -----------------------------
3 files changed, 70 deletions(-)
diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py
index 26f3ad4be5..362e9085df 100644
--- a/pyiceberg/manifest.py
+++ b/pyiceberg/manifest.py
@@ -842,25 +842,6 @@ def has_added_files(self) -> bool:
def has_existing_files(self) -> bool:
return self.existing_files_count is None or self.existing_files_count > 0
- def copy_with_snapshot_id(self, snapshot_id: int) -> ManifestFile:
- return ManifestFile(
- manifest_path=self.manifest_path,
- manifest_length=self.manifest_length,
- partition_spec_id=self.partition_spec_id,
- content=self.content,
- sequence_number=self.sequence_number,
- min_sequence_number=self.min_sequence_number,
- added_snapshot_id=snapshot_id,
- added_files_count=self.added_files_count,
- existing_files_count=self.existing_files_count,
- deleted_files_count=self.deleted_files_count,
- added_rows_count=self.added_rows_count,
- existing_rows_count=self.existing_rows_count,
- deleted_rows_count=self.deleted_rows_count,
- partitions=self.partitions,
- key_metadata=self.key_metadata,
- )
-
def fetch_manifest_entry(self, io: FileIO, discard_deleted: bool = True) -> List[ManifestEntry]:
"""
Read the manifest entries from the manifest file.
diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py
index e3e1d5af71..0d8fdfba2f 100644
--- a/pyiceberg/table/__init__.py
+++ b/pyiceberg/table/__init__.py
@@ -455,14 +455,6 @@ def rewrite_manifests(self) -> RewriteManifestsResult:
rewritten = rewrite.rewrite_manifests()
return rewritten
- def rewrite_manifests(self) -> RewriteManifestsResult:
- if self._table.current_snapshot() is None:
- return RewriteManifestsResult(rewritten_manifests=[], added_manifests=[])
-
- with self.update_snapshot().rewrite() as rewrite:
- rewritten = rewrite.rewrite_manifests()
- return rewritten
-
def update_statistics(self) -> UpdateStatistics:
"""
Create a new UpdateStatistics to update the statistics of the table.
@@ -1460,20 +1452,6 @@ def add_files(
file_paths=file_paths, snapshot_properties=snapshot_properties, check_duplicate_files=check_duplicate_files
)
- def rewrite_manifests(
- self,
- spec_id: Optional[int] = None,
- ) -> RewriteManifestsResult:
- """
- Shorthand API for Rewriting manifests for the table.
-
- Args:
- spec_id: Spec id of the manifests to rewrite (defaults to current spec id)
-
- """
- with self.transaction() as tx:
- return tx.rewrite_manifests(spec_id=spec_id)
-
def update_spec(self, case_sensitive: bool = True) -> UpdateSpec:
return UpdateSpec(Transaction(self, autocommit=True), case_sensitive=case_sensitive)
diff --git a/pyiceberg/table/update/snapshot.py b/pyiceberg/table/update/snapshot.py
index 2191cab113..e182398b62 100644
--- a/pyiceberg/table/update/snapshot.py
+++ b/pyiceberg/table/update/snapshot.py
@@ -624,35 +624,6 @@ def __init__(
TableProperties.MANIFEST_MERGE_ENABLED_DEFAULT,
)
- def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
- from pyiceberg.table import TableProperties
-
- ssc = SnapshotSummaryCollector(
- int(
- self._transaction.table_metadata.properties.get(
- TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, TableProperties.WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT
- )
- )
- )
-
- props = {
- "manifests-kept": "0",
- "manifests-created": str(len(self.added_manifests)),
- "manifests-replaced": str(len(self.rewritten_manifests)),
- "entries-processed": "0",
- }
- previous_snapshot = (
- self._transaction.table_metadata.snapshot_by_id(self._parent_snapshot_id)
- if self._parent_snapshot_id is not None
- else None
- )
-
- return update_snapshot_summaries(
- summary=Summary(operation=self._operation, **ssc.build(), **props),
- previous_summary=previous_snapshot.summary if previous_snapshot is not None else None,
- truncate_full_table=False,
- )
-
def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary:
from pyiceberg.table import TableProperties