From 8d459202ed62da5708cb7ea890a831318aaa88f5 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 2 Apr 2024 16:55:39 +0200 Subject: [PATCH 01/27] Add option to delete datafiles This is done through the Iceberg metadata, resulting in efficient deletes if the data is partitioned correctly --- pyiceberg/table/__init__.py | 120 +++++++++++++++++++++++++++++- pyiceberg/table/snapshots.py | 2 +- tests/conftest.py | 7 +- tests/integration/test_deletes.py | 76 +++++++++++++++++++ 4 files changed, 200 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_deletes.py diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 2ad1f7fe81..5447fe4778 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -50,10 +50,12 @@ import pyiceberg.expressions.visitors as visitors from pyiceberg.exceptions import CommitFailedException, ResolveError, ValidationError from pyiceberg.expressions import ( + AlwaysFalse, AlwaysTrue, And, BooleanExpression, EqualTo, + Or, Reference, ) from pyiceberg.io import FileIO, load_file_io @@ -2710,6 +2712,114 @@ def _commit(self) -> UpdatesAndRequirements: ) +class DeleteFiles(_MergingSnapshotProducer): + _predicate: BooleanExpression + + def __init__( + 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() + + def _build_partition_projection(self, spec_id: int) -> BooleanExpression: + schema = self._transaction.table_metadata.schema() + spec = self._transaction.table_metadata.specs()[spec_id] + project = visitors.inclusive_projection(schema, spec) + return project(self._predicate) + + @cached_property + def partition_filters(self) -> KeyDefaultDict[int, BooleanExpression]: + return KeyDefaultDict(self._build_partition_projection) + + def _build_manifest_evaluator(self, spec_id: int) -> Callable[[ManifestFile], bool]: + schema = self._transaction.table_metadata.schema() + spec = self._transaction.table_metadata.specs()[spec_id] + return visitors.manifest_evaluator(spec, schema, self.partition_filters[spec_id], case_sensitive=True) + + def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool]: + schema = self._transaction.table_metadata.schema() + spec = self._transaction.table_metadata.specs()[spec_id] + partition_type = spec.partition_type(schema) + partition_schema = Schema(*partition_type.fields) + partition_expr = self.partition_filters[spec_id] + + return lambda data_file: visitors.expression_evaluator(partition_schema, partition_expr, case_sensitive=True)( + data_file.partition + ) + + def delete(self, predicate: BooleanExpression) -> None: + self._predicate = Or(self._predicate, predicate) + + @cached_property + def _compute_deletes(self) -> Tuple[List[ManifestFile], List[ManifestEntry]]: + schema = self._transaction.table_metadata.schema() + + def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> ManifestEntry: + return ManifestEntry( + status=status, + snapshot_id=entry.snapshot_id, + data_sequence_number=entry.data_sequence_number, + file_sequence_number=entry.file_sequence_number, + data_file=entry.data_file, + ) + + manifest_evaluators: Dict[int, Callable[[ManifestFile], bool]] = KeyDefaultDict(self._build_manifest_evaluator) + strict_metrics_evaluator = visitors._StrictMetricsEvaluator(schema, self._predicate, case_sensitive=True).eval + inclusive_metrics_evaluator = visitors._InclusiveMetricsEvaluator(schema, self._predicate, case_sensitive=True).eval + + existing_manifests = [] + total_deleted_entries = [] + if snapshot := self._transaction.table_metadata.current_snapshot(): + for num, manifest_file in enumerate(snapshot.manifests(io=self._io)): + 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): + if strict_metrics_evaluator(entry.data_file) == visitors.ROWS_MUST_MATCH: + deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED)) + elif inclusive_metrics_evaluator(entry.data_file) == visitors.ROWS_CANNOT_MATCH: + existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING)) + else: + raise ValueError("Deletes do not support rewrites of data files") + + if len(deleted_entries) > 0: + total_deleted_entries += deleted_entries + + # Rewrite the manifest + if len(existing_entries) > 0: + output_file_location = _new_manifest_path( + location=self._transaction.table_metadata.location, num=num, commit_uuid=self.commit_uuid + ) + 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._io.new_output(output_file_location), + snapshot_id=self._snapshot_id, + ) as writer: + for existing_entry in existing_entries: + writer.add_entry(existing_entry) + else: + existing_manifests.append(manifest_file) + + return existing_manifests, total_deleted_entries + + def _existing_manifests(self) -> List[ManifestFile]: + return self._compute_deletes[0] + + def _deleted_entries(self) -> List[ManifestEntry]: + return self._compute_deletes[1] + + class FastAppendFiles(_MergingSnapshotProducer): def _existing_manifests(self) -> List[ManifestFile]: """To determine if there are any existing manifest files. @@ -2787,7 +2897,7 @@ class UpdateSnapshot: _io: FileIO _snapshot_properties: Dict[str, str] - def __init__(self, transaction: Transaction, io: FileIO, snapshot_properties: Dict[str, str]) -> None: + def __init__(self, transaction: Transaction, io: FileIO, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: self._transaction = transaction self._io = io self._snapshot_properties = snapshot_properties @@ -2807,6 +2917,14 @@ def overwrite(self) -> OverwriteFiles: snapshot_properties=self._snapshot_properties, ) + def delete(self) -> DeleteFiles: + return DeleteFiles( + operation=Operation.DELETE, + transaction=self._transaction, + io=self._io, + snapshot_properties=self._snapshot_properties, + ) + class UpdateSpec(UpdateTableMetadata["UpdateSpec"]): _transaction: Transaction diff --git a/pyiceberg/table/snapshots.py b/pyiceberg/table/snapshots.py index f74ac4b7d4..24c3707ef7 100644 --- a/pyiceberg/table/snapshots.py +++ b/pyiceberg/table/snapshots.py @@ -345,7 +345,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}: + if summary.operation not in {Operation.APPEND, Operation.OVERWRITE, Operation.DELETE}: 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/tests/conftest.py b/tests/conftest.py index 48187ee685..07ae3414bb 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -46,7 +46,6 @@ import boto3 import pytest from moto import mock_aws -from pyspark.sql import SparkSession from pyiceberg import schema from pyiceberg.catalog import Catalog, load_catalog @@ -86,6 +85,7 @@ if TYPE_CHECKING: import pyarrow as pa from moto.server import ThreadedMotoServer # type: ignore + from pyspark.sql import SparkSession from pyiceberg.io.pyarrow import PyArrowFileIO @@ -1954,9 +1954,10 @@ def session_catalog() -> Catalog: @pytest.fixture(scope="session") -def spark() -> SparkSession: +def spark() -> "SparkSession": import importlib.metadata - import os + + from pyspark.sql import SparkSession spark_version = ".".join(importlib.metadata.version("pyspark").split(".")[:2]) scala_version = "2.12" diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py new file mode 100644 index 0000000000..b615065a19 --- /dev/null +++ b/tests/integration/test_deletes.py @@ -0,0 +1,76 @@ +# 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 pytest +from pyspark.sql import DataFrame, SparkSession + +from pyiceberg.catalog.rest import RestCatalog +from pyiceberg.expressions import EqualTo + + +@pytest.fixture +def test_deletes_table(spark: SparkSession) -> DataFrame: + identifier = 'default.table_partitioned_delete' + + spark.sql(f"DROP TABLE IF EXISTS {identifier}") + + spark.sql( + f""" + CREATE TABLE {identifier} ( + number_partitioned int, + number int + ) + USING iceberg + PARTITIONED BY (number_partitioned) + """ + ) + spark.sql( + f""" + INSERT INTO {identifier} VALUES (10, 20), (10, 30) + """ + ) + spark.sql( + f""" + INSERT INTO {identifier} VALUES (11, 20), (11, 30) + """ + ) + + return spark.table(identifier) + + +def test_partition_deletes(test_deletes_table: DataFrame, session_catalog: RestCatalog) -> None: + identifier = 'default.table_partitioned_delete' + + tbl = session_catalog.load_table(identifier) + + with tbl.transaction() as txn: + with txn.update_snapshot().delete() as delete: + delete.delete(EqualTo("number_partitioned", 10)) + + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} + + +def test_deletes(test_deletes_table: DataFrame, session_catalog: RestCatalog) -> None: + identifier = 'default.table_partitioned_delete' + + tbl = session_catalog.load_table(identifier) + + with tbl.transaction() as txn: + with txn.update_snapshot().delete() as delete: + delete.delete(EqualTo("number", 30)) + + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} From 87cc065f5d47d7ba011855a459baf4828cd3be39 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 2 Apr 2024 16:57:33 +0200 Subject: [PATCH 02/27] Pull in main --- pyiceberg/table/__init__.py | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index d88a92d27f..1a7fc0ad26 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -58,7 +58,10 @@ Reference, ) from pyiceberg.expressions.visitors import ( + ROWS_CANNOT_MATCH, + ROWS_MUST_MATCH, _InclusiveMetricsEvaluator, + _StrictMetricsEvaluator, expression_evaluator, inclusive_projection, manifest_evaluator, @@ -2745,7 +2748,7 @@ def __init__( def _build_partition_projection(self, spec_id: int) -> BooleanExpression: schema = self._transaction.table_metadata.schema() spec = self._transaction.table_metadata.specs()[spec_id] - project = visitors.inclusive_projection(schema, spec) + project = inclusive_projection(schema, spec) return project(self._predicate) @cached_property @@ -2755,7 +2758,7 @@ def partition_filters(self) -> KeyDefaultDict[int, BooleanExpression]: def _build_manifest_evaluator(self, spec_id: int) -> Callable[[ManifestFile], bool]: schema = self._transaction.table_metadata.schema() spec = self._transaction.table_metadata.specs()[spec_id] - return visitors.manifest_evaluator(spec, schema, self.partition_filters[spec_id], case_sensitive=True) + return manifest_evaluator(spec, schema, self.partition_filters[spec_id], case_sensitive=True) def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool]: schema = self._transaction.table_metadata.schema() @@ -2764,9 +2767,7 @@ def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool] partition_schema = Schema(*partition_type.fields) partition_expr = self.partition_filters[spec_id] - return lambda data_file: visitors.expression_evaluator(partition_schema, partition_expr, case_sensitive=True)( - data_file.partition - ) + return lambda data_file: expression_evaluator(partition_schema, partition_expr, case_sensitive=True)(data_file.partition) def delete(self, predicate: BooleanExpression) -> None: self._predicate = Or(self._predicate, predicate) @@ -2785,8 +2786,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 = visitors._StrictMetricsEvaluator(schema, self._predicate, case_sensitive=True).eval - inclusive_metrics_evaluator = visitors._InclusiveMetricsEvaluator(schema, self._predicate, case_sensitive=True).eval + strict_metrics_evaluator = _StrictMetricsEvaluator(schema, self._predicate, case_sensitive=True).eval + inclusive_metrics_evaluator = _InclusiveMetricsEvaluator(schema, self._predicate, case_sensitive=True).eval existing_manifests = [] total_deleted_entries = [] @@ -2800,9 +2801,9 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> deleted_entries = [] existing_entries = [] for entry in manifest_file.fetch_manifest_entry(io=self._io): - if strict_metrics_evaluator(entry.data_file) == visitors.ROWS_MUST_MATCH: + if strict_metrics_evaluator(entry.data_file) == ROWS_MUST_MATCH: deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED)) - elif inclusive_metrics_evaluator(entry.data_file) == visitors.ROWS_CANNOT_MATCH: + elif inclusive_metrics_evaluator(entry.data_file) == ROWS_CANNOT_MATCH: existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING)) else: raise ValueError("Deletes do not support rewrites of data files") From 234d55b4214722d62fd5264bccbf74f8c09d07c9 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 5 Apr 2024 17:16:41 +0200 Subject: [PATCH 03/27] WIP --- pyiceberg/table/__init__.py | 51 +++++++++++++++++++------------------ 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 955981efbe..a363ed9bca 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -1467,7 +1467,8 @@ def _parse_row_filter(expr: Union[str, BooleanExpression]) -> BooleanExpression: class TableScan(ABC): - table: Table + table_metadata: TableMetadata + io: FileIO row_filter: BooleanExpression selected_fields: Tuple[str, ...] case_sensitive: bool @@ -1477,7 +1478,8 @@ class TableScan(ABC): def __init__( self, - table: Table, + table_metadata: TableMetadata, + io: FileIO, row_filter: Union[str, BooleanExpression] = ALWAYS_TRUE, selected_fields: Tuple[str, ...] = ("*",), case_sensitive: bool = True, @@ -1485,7 +1487,8 @@ def __init__( options: Properties = EMPTY_DICT, limit: Optional[int] = None, ): - self.table = table + self.table_metadata = table_metadata + self.io = io self.row_filter = _parse_row_filter(row_filter) self.selected_fields = selected_fields self.case_sensitive = case_sensitive @@ -1495,16 +1498,16 @@ def __init__( def snapshot(self) -> Optional[Snapshot]: if self.snapshot_id: - return self.table.snapshot_by_id(self.snapshot_id) - return self.table.current_snapshot() + return self.table_metadata.snapshot_by_id(self.snapshot_id) + return self.table_metadata.current_snapshot() def projection(self) -> Schema: - current_schema = self.table.schema() + current_schema = self.table_metadata.schema() if self.snapshot_id is not None: - snapshot = self.table.snapshot_by_id(self.snapshot_id) + snapshot = self.table_metadata.snapshot_by_id(self.snapshot_id) if snapshot is not None: if snapshot.schema_id is not None: - snapshot_schema = self.table.schemas().get(snapshot.schema_id) + snapshot_schema = self.table_metadata.schemas().get(snapshot.schema_id) if snapshot_schema is not None: current_schema = snapshot_schema else: @@ -1625,17 +1628,6 @@ def _match_deletes_to_data_file(data_entry: ManifestEntry, positional_delete_ent class DataScan(TableScan): - def __init__( - self, - table: Table, - row_filter: Union[str, BooleanExpression] = ALWAYS_TRUE, - selected_fields: Tuple[str, ...] = ("*",), - case_sensitive: bool = True, - snapshot_id: Optional[int] = None, - options: Properties = EMPTY_DICT, - limit: Optional[int] = None, - ): - super().__init__(table, row_filter, selected_fields, case_sensitive, snapshot_id, options, limit) def _build_partition_projection(self, spec_id: int) -> BooleanExpression: project = inclusive_projection(self.table.schema(), self.table.specs()[spec_id]) @@ -2912,7 +2904,9 @@ def _commit(self) -> UpdatesAndRequirements: ) -class DeleteFiles(_MergingSnapshotProducer): +class MetadataDeleteFiles(_MergingSnapshotProducer): + """Will delete manifest entries from the current snapshot based on the predicate""" + _predicate: BooleanExpression def __init__( @@ -2954,7 +2948,7 @@ def delete(self, predicate: BooleanExpression) -> None: self._predicate = Or(self._predicate, predicate) @cached_property - def _compute_deletes(self) -> Tuple[List[ManifestFile], List[ManifestEntry]]: + def _compute_deletes(self) -> Tuple[List[ManifestFile], List[ManifestEntry], bool]: schema = self._transaction.table_metadata.schema() def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> ManifestEntry: @@ -2972,6 +2966,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> existing_manifests = [] total_deleted_entries = [] + partial_rewrites_needed = False if snapshot := self._transaction.table_metadata.current_snapshot(): for num, manifest_file in enumerate(snapshot.manifests(io=self._io)): if not manifest_evaluators[manifest_file.partition_spec_id](manifest_file): @@ -2987,7 +2982,8 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> elif inclusive_metrics_evaluator(entry.data_file) == ROWS_CANNOT_MATCH: existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING)) else: - raise ValueError("Deletes do not support rewrites of data files") + # Based on the metadata, it is unsure to say if the file can be deleted + partial_rewrites_needed = True if len(deleted_entries) > 0: total_deleted_entries += deleted_entries @@ -3006,10 +3002,11 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> ) 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) - return existing_manifests, total_deleted_entries + return existing_manifests, total_deleted_entries, partial_rewrites_needed def _existing_manifests(self) -> List[ManifestFile]: return self._compute_deletes[0] @@ -3017,6 +3014,10 @@ def _existing_manifests(self) -> List[ManifestFile]: def _deleted_entries(self) -> List[ManifestEntry]: return self._compute_deletes[1] + def rewrites_needed(self) -> bool: + """Indicates if data files need to be rewritten""" + return self._compute_deletes[2] + class FastAppendFiles(_MergingSnapshotProducer): def _existing_manifests(self) -> List[ManifestFile]: @@ -3115,8 +3116,8 @@ def overwrite(self) -> OverwriteFiles: snapshot_properties=self._snapshot_properties, ) - def delete(self) -> DeleteFiles: - return DeleteFiles( + def delete(self) -> MetadataDeleteFiles: + return MetadataDeleteFiles( operation=Operation.DELETE, transaction=self._transaction, io=self._io, From aadc89c866a29824cd4959fbc399d180e02839a7 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 5 Apr 2024 18:04:55 +0200 Subject: [PATCH 04/27] Change DataScan to accept Metadata and io For the partial deletes I want to do a scan on in memory metadata. Changing this API allows this. --- pyiceberg/io/pyarrow.py | 26 ++++++----- pyiceberg/table/__init__.py | 64 +++++++++++-------------- pyiceberg/table/metadata.py | 6 +++ tests/io/test_pyarrow.py | 93 ++++++++++++++----------------------- 4 files changed, 84 insertions(+), 105 deletions(-) diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py index 06d03e21e1..b2be460926 100644 --- a/pyiceberg/io/pyarrow.py +++ b/pyiceberg/io/pyarrow.py @@ -158,7 +158,7 @@ from pyiceberg.utils.truncate import truncate_upper_bound_binary_string, truncate_upper_bound_text_string if TYPE_CHECKING: - from pyiceberg.table import FileScanTask, Table + from pyiceberg.table import FileScanTask logger = logging.getLogger(__name__) @@ -1034,7 +1034,8 @@ def _read_all_delete_files(fs: FileSystem, tasks: Iterable[FileScanTask]) -> Dic def project_table( tasks: Iterable[FileScanTask], - table: Table, + table_metadata: TableMetadata, + io: FileIO, row_filter: BooleanExpression, projected_schema: Schema, case_sensitive: bool = True, @@ -1044,7 +1045,8 @@ def project_table( Args: tasks (Iterable[FileScanTask]): A URI or a path to a local file. - table (Table): The table that's being queried. + table_metadata (TableMetadata): The table metadata of the table that's being queried + io (FileIO): A FileIO to open streams to the object store row_filter (BooleanExpression): The expression for filtering rows. projected_schema (Schema): The output schema. case_sensitive (bool): Case sensitivity when looking up column names. @@ -1053,24 +1055,24 @@ def project_table( Raises: ResolveError: When an incompatible query is done. """ - scheme, netloc, _ = PyArrowFileIO.parse_location(table.location()) - if isinstance(table.io, PyArrowFileIO): - fs = table.io.fs_by_scheme(scheme, netloc) + scheme, netloc, _ = PyArrowFileIO.parse_location(table_metadata.location) + if isinstance(io, PyArrowFileIO): + fs = io.fs_by_scheme(scheme, netloc) else: try: from pyiceberg.io.fsspec import FsspecFileIO - if isinstance(table.io, FsspecFileIO): + if isinstance(io, FsspecFileIO): from pyarrow.fs import PyFileSystem - fs = PyFileSystem(FSSpecHandler(table.io.get_fs(scheme))) + fs = PyFileSystem(FSSpecHandler(io.get_fs(scheme))) else: - raise ValueError(f"Expected PyArrowFileIO or FsspecFileIO, got: {table.io}") + 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: {table.io}") from e + raise ValueError(f"Expected PyArrowFileIO or FsspecFileIO, got: {io}") from e - bound_row_filter = bind(table.schema(), row_filter, case_sensitive=case_sensitive) + bound_row_filter = bind(table_metadata.schema(), row_filter, case_sensitive=case_sensitive) projected_field_ids = { id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType)) @@ -1089,7 +1091,7 @@ def project_table( deletes_per_file.get(task.file.file_path), case_sensitive, limit, - table.name_mapping(), + None, ) for task in tasks ] diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 2dbc32d893..27e21c762a 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -1209,7 +1209,8 @@ def scan( limit: Optional[int] = None, ) -> DataScan: return DataScan( - table=self, + table_metadata=self.metadata, + io=self.io, row_filter=row_filter, selected_fields=selected_fields, case_sensitive=case_sensitive, @@ -1462,7 +1463,8 @@ def _parse_row_filter(expr: Union[str, BooleanExpression]) -> BooleanExpression: class TableScan(ABC): - table: Table + table_metadata: TableMetadata + io: FileIO row_filter: BooleanExpression selected_fields: Tuple[str, ...] case_sensitive: bool @@ -1472,7 +1474,8 @@ class TableScan(ABC): def __init__( self, - table: Table, + table_metadata: TableMetadata, + io: FileIO, row_filter: Union[str, BooleanExpression] = ALWAYS_TRUE, selected_fields: Tuple[str, ...] = ("*",), case_sensitive: bool = True, @@ -1480,7 +1483,8 @@ def __init__( options: Properties = EMPTY_DICT, limit: Optional[int] = None, ): - self.table = table + self.table_metadata = table_metadata + self.io = io self.row_filter = _parse_row_filter(row_filter) self.selected_fields = selected_fields self.case_sensitive = case_sensitive @@ -1490,19 +1494,20 @@ def __init__( def snapshot(self) -> Optional[Snapshot]: if self.snapshot_id: - return self.table.snapshot_by_id(self.snapshot_id) - return self.table.current_snapshot() + return self.table_metadata.snapshot_by_id(self.snapshot_id) + return self.table_metadata.current_snapshot() def projection(self) -> Schema: - current_schema = self.table.schema() + current_schema = self.table_metadata.schema() if self.snapshot_id is not None: - snapshot = self.table.snapshot_by_id(self.snapshot_id) + snapshot = self.table_metadata.snapshot_by_id(self.snapshot_id) if snapshot is not None: if snapshot.schema_id is not None: - snapshot_schema = self.table.schemas().get(snapshot.schema_id) - if snapshot_schema is not None: - current_schema = snapshot_schema - else: + try: + current_schema = next( + schema for schema in self.table_metadata.schemas if schema.schema_id == snapshot.schema_id + ) + except StopIteration: warnings.warn(f"Metadata does not contain schema with id: {snapshot.schema_id}") else: raise ValueError(f"Snapshot not found: {self.snapshot_id}") @@ -1528,7 +1533,7 @@ def update(self: S, **overrides: Any) -> S: def use_ref(self: S, name: str) -> S: if self.snapshot_id: raise ValueError(f"Cannot override ref, already set snapshot id={self.snapshot_id}") - if snapshot := self.table.snapshot_by_name(name): + if snapshot := self.table_metadata.snapshot_by_name(name): return self.update(snapshot_id=snapshot.snapshot_id) raise ValueError(f"Cannot scan unknown ref={name}") @@ -1620,20 +1625,8 @@ def _match_deletes_to_data_file(data_entry: ManifestEntry, positional_delete_ent class DataScan(TableScan): - def __init__( - self, - table: Table, - row_filter: Union[str, BooleanExpression] = ALWAYS_TRUE, - selected_fields: Tuple[str, ...] = ("*",), - case_sensitive: bool = True, - snapshot_id: Optional[int] = None, - options: Properties = EMPTY_DICT, - limit: Optional[int] = None, - ): - super().__init__(table, row_filter, selected_fields, case_sensitive, snapshot_id, options, limit) - def _build_partition_projection(self, spec_id: int) -> BooleanExpression: - project = inclusive_projection(self.table.schema(), self.table.specs()[spec_id]) + project = inclusive_projection(self.table_metadata.schema(), self.table_metadata.specs()[spec_id]) return project(self.row_filter) @cached_property @@ -1641,12 +1634,12 @@ def partition_filters(self) -> KeyDefaultDict[int, BooleanExpression]: return KeyDefaultDict(self._build_partition_projection) def _build_manifest_evaluator(self, spec_id: int) -> Callable[[ManifestFile], bool]: - spec = self.table.specs()[spec_id] - return manifest_evaluator(spec, self.table.schema(), self.partition_filters[spec_id], self.case_sensitive) + spec = self.table_metadata.specs()[spec_id] + return manifest_evaluator(spec, self.table_metadata.schema(), self.partition_filters[spec_id], self.case_sensitive) def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool]: - spec = self.table.specs()[spec_id] - partition_type = spec.partition_type(self.table.schema()) + spec = self.table_metadata.specs()[spec_id] + partition_type = spec.partition_type(self.table_metadata.schema()) partition_schema = Schema(*partition_type.fields) partition_expr = self.partition_filters[spec_id] @@ -1681,8 +1674,6 @@ def plan_files(self) -> Iterable[FileScanTask]: if not snapshot: return iter([]) - io = self.table.io - # step 1: filter manifests using partition summaries # the filter depends on the partition spec used to write the manifest file, so create a cache of filters for each spec id @@ -1690,7 +1681,7 @@ def plan_files(self) -> Iterable[FileScanTask]: manifests = [ manifest_file - for manifest_file in snapshot.manifests(io) + for manifest_file in snapshot.manifests(self.io) if manifest_evaluators[manifest_file.partition_spec_id](manifest_file) ] @@ -1699,7 +1690,7 @@ def plan_files(self) -> Iterable[FileScanTask]: partition_evaluators: Dict[int, Callable[[DataFile], bool]] = KeyDefaultDict(self._build_partition_evaluator) metrics_evaluator = _InclusiveMetricsEvaluator( - self.table.schema(), self.row_filter, self.case_sensitive, self.options.get("include_empty_files") == "true" + self.table_metadata.schema(), self.row_filter, self.case_sensitive, self.options.get("include_empty_files") == "true" ).eval min_data_sequence_number = _min_data_file_sequence_number(manifests) @@ -1713,7 +1704,7 @@ def plan_files(self) -> Iterable[FileScanTask]: lambda args: _open_manifest(*args), [ ( - io, + self.io, manifest, partition_evaluators[manifest.partition_spec_id], metrics_evaluator, @@ -1749,7 +1740,8 @@ def to_arrow(self) -> pa.Table: return project_table( self.plan_files(), - self.table, + self.table_metadata, + self.io, self.row_filter, self.projection(), case_sensitive=self.case_sensitive, diff --git a/pyiceberg/table/metadata.py b/pyiceberg/table/metadata.py index 21ed144784..65f4b7a429 100644 --- a/pyiceberg/table/metadata.py +++ b/pyiceberg/table/metadata.py @@ -278,6 +278,12 @@ def new_snapshot_id(self) -> int: return snapshot_id + def snapshot_by_name(self, name: str) -> Optional[Snapshot]: + """Return the snapshot referenced by the given name or null if no such reference exists.""" + if ref := self.refs.get(name): + return self.snapshot_by_id(ref.snapshot_id) + return None + def current_snapshot(self) -> Optional[Snapshot]: """Get the current snapshot for this table, or None if there is no current snapshot.""" if self.current_snapshot_id is not None: diff --git a/tests/io/test_pyarrow.py b/tests/io/test_pyarrow.py index b99febd6e2..46ece77880 100644 --- a/tests/io/test_pyarrow.py +++ b/tests/io/test_pyarrow.py @@ -28,7 +28,6 @@ import pytest from pyarrow.fs import FileType, LocalFileSystem -from pyiceberg.catalog.noop import NoopCatalog from pyiceberg.exceptions import ResolveError from pyiceberg.expressions import ( AlwaysFalse, @@ -72,7 +71,7 @@ from pyiceberg.manifest import DataFile, DataFileContent, FileFormat from pyiceberg.partitioning import PartitionSpec from pyiceberg.schema import Schema, make_compatible_name, visit -from pyiceberg.table import FileScanTask, Table, TableProperties +from pyiceberg.table import FileScanTask, TableProperties from pyiceberg.table.metadata import TableMetadataV2 from pyiceberg.typedef import UTF8 from pyiceberg.types import ( @@ -876,7 +875,7 @@ def project( schema: Schema, files: List[str], expr: Optional[BooleanExpression] = None, table_schema: Optional[Schema] = None ) -> pa.Table: return project_table( - [ + tasks=[ FileScanTask( DataFile( content=DataFileContent.DATA, @@ -889,21 +888,16 @@ def project( ) for file in files ], - Table( - ("namespace", "table"), - metadata=TableMetadataV2( - location="file://a/b/", - last_column_id=1, - format_version=2, - schemas=[table_schema or schema], - partition_specs=[PartitionSpec()], - ), - metadata_location="file://a/b/c.json", - io=PyArrowFileIO(), - catalog=NoopCatalog("NoopCatalog"), + table_metadata=TableMetadataV2( + location="file://a/b/", + last_column_id=1, + format_version=2, + schemas=[table_schema or schema], + partition_specs=[PartitionSpec()], ), - expr or AlwaysTrue(), - schema, + io=PyArrowFileIO(), + row_filter=expr or AlwaysTrue(), + projected_schema=schema, case_sensitive=True, ) @@ -1362,20 +1356,15 @@ def test_delete(deletes_file: str, example_task: FileScanTask, table_schema_simp with_deletes = project_table( tasks=[example_task_with_delete], - table=Table( - ("namespace", "table"), - metadata=TableMetadataV2( - location=metadata_location, - last_column_id=1, - format_version=2, - current_schema_id=1, - schemas=[table_schema_simple], - partition_specs=[PartitionSpec()], - ), - metadata_location=metadata_location, - io=load_file_io(), - catalog=NoopCatalog("noop"), + table_metadata=TableMetadataV2( + location=metadata_location, + last_column_id=1, + format_version=2, + current_schema_id=1, + schemas=[table_schema_simple], + partition_specs=[PartitionSpec()], ), + io=load_file_io(), row_filter=AlwaysTrue(), projected_schema=table_schema_simple, ) @@ -1405,20 +1394,15 @@ def test_delete_duplicates(deletes_file: str, example_task: FileScanTask, table_ with_deletes = project_table( tasks=[example_task_with_delete], - table=Table( - ("namespace", "table"), - metadata=TableMetadataV2( - location=metadata_location, - last_column_id=1, - format_version=2, - current_schema_id=1, - schemas=[table_schema_simple], - partition_specs=[PartitionSpec()], - ), - metadata_location=metadata_location, - io=load_file_io(), - catalog=NoopCatalog("noop"), + table_metadata=TableMetadataV2( + location=metadata_location, + last_column_id=1, + format_version=2, + current_schema_id=1, + schemas=[table_schema_simple], + partition_specs=[PartitionSpec()], ), + io=load_file_io(), row_filter=AlwaysTrue(), projected_schema=table_schema_simple, ) @@ -1439,21 +1423,16 @@ def test_delete_duplicates(deletes_file: str, example_task: FileScanTask, table_ def test_pyarrow_wrap_fsspec(example_task: FileScanTask, table_schema_simple: Schema) -> None: metadata_location = "file://a/b/c.json" projection = project_table( - [example_task], - Table( - ("namespace", "table"), - metadata=TableMetadataV2( - location=metadata_location, - last_column_id=1, - format_version=2, - current_schema_id=1, - schemas=[table_schema_simple], - partition_specs=[PartitionSpec()], - ), - metadata_location=metadata_location, - io=load_file_io(properties={"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}, location=metadata_location), - catalog=NoopCatalog("NoopCatalog"), + tasks=[example_task], + table_metadata=TableMetadataV2( + location=metadata_location, + last_column_id=1, + format_version=2, + current_schema_id=1, + schemas=[table_schema_simple], + partition_specs=[PartitionSpec()], ), + io=load_file_io(properties={"py-io-impl": "pyiceberg.io.fsspec.FsspecFileIO"}, location=metadata_location), case_sensitive=True, projected_schema=table_schema_simple, row_filter=AlwaysTrue(), From 7e593429bce84f76e24367140e5711b705aa8fdd Mon Sep 17 00:00:00 2001 From: HonahX Date: Sun, 7 Apr 2024 16:21:08 -0700 Subject: [PATCH 05/27] fix name-mapping issue --- pyiceberg/io/pyarrow.py | 2 +- pyiceberg/table/__init__.py | 6 +----- pyiceberg/table/metadata.py | 8 ++++++++ tests/integration/test_add_files.py | 9 +++++++++ 4 files changed, 19 insertions(+), 6 deletions(-) diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py index b2be460926..ad5be587bd 100644 --- a/pyiceberg/io/pyarrow.py +++ b/pyiceberg/io/pyarrow.py @@ -1091,7 +1091,7 @@ def project_table( deletes_per_file.get(task.file.file_path), case_sensitive, limit, - None, + table_metadata.name_mapping(), ) for task in tasks ] diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 27e21c762a..664a73e1c7 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -103,7 +103,6 @@ ) from pyiceberg.table.name_mapping import ( NameMapping, - parse_mapping_from_json, update_mapping, ) from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRef @@ -1307,10 +1306,7 @@ def update_schema(self, allow_incompatible_changes: bool = False, case_sensitive def name_mapping(self) -> Optional[NameMapping]: """Return the table's field-id NameMapping.""" - if name_mapping_json := self.properties.get(TableProperties.DEFAULT_NAME_MAPPING): - return parse_mapping_from_json(name_mapping_json) - else: - return None + return self.metadata.name_mapping() def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: """ diff --git a/pyiceberg/table/metadata.py b/pyiceberg/table/metadata.py index 65f4b7a429..ba0c885758 100644 --- a/pyiceberg/table/metadata.py +++ b/pyiceberg/table/metadata.py @@ -35,6 +35,7 @@ from pyiceberg.exceptions import ValidationError from pyiceberg.partitioning import PARTITION_FIELD_ID_START, PartitionSpec, assign_fresh_partition_spec_ids from pyiceberg.schema import Schema, assign_fresh_schema_ids +from pyiceberg.table.name_mapping import NameMapping, parse_mapping_from_json from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType from pyiceberg.table.snapshots import MetadataLogEntry, Snapshot, SnapshotLogEntry from pyiceberg.table.sorting import ( @@ -237,6 +238,13 @@ def schema(self) -> Schema: """Return the schema for this table.""" return next(schema for schema in self.schemas if schema.schema_id == self.current_schema_id) + def name_mapping(self) -> Optional[NameMapping]: + """Return the table's field-id NameMapping.""" + if name_mapping_json := self.properties.get("schema.name-mapping.default"): + return parse_mapping_from_json(name_mapping_json) + else: + return None + def spec(self) -> PartitionSpec: """Return the partition spec of this table.""" return next(spec for spec in self.partition_specs if spec.spec_id == self.default_spec_id) diff --git a/tests/integration/test_add_files.py b/tests/integration/test_add_files.py index 7c17618280..0de5d5f4ce 100644 --- a/tests/integration/test_add_files.py +++ b/tests/integration/test_add_files.py @@ -158,6 +158,9 @@ def test_add_files_to_unpartitioned_table(spark: SparkSession, session_catalog: for col in df.columns: assert df.filter(df[col].isNotNull()).count() == 5, "Expected all 5 rows to be non-null" + # check that the table can be read by pyiceberg + assert len(tbl.scan().to_arrow()) == 5, "Expected 5 rows" + @pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) @@ -255,6 +258,9 @@ def test_add_files_to_unpartitioned_table_with_schema_updates( value_count = 1 if col == "quux" else 6 assert df.filter(df[col].isNotNull()).count() == value_count, f"Expected {value_count} rows to be non-null" + # check that the table can be read by pyiceberg + assert len(tbl.scan().to_arrow()) == 6, "Expected 6 rows" + @pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) @@ -324,6 +330,9 @@ def test_add_files_to_partitioned_table(spark: SparkSession, session_catalog: Ca assert [row.file_count for row in partition_rows] == [5] assert [(row.partition.baz, row.partition.qux_month) for row in partition_rows] == [(123, 650)] + # check that the table can be read by pyiceberg + assert len(tbl.scan().to_arrow()) == 5, "Expected 5 rows" + @pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) From 5cec00a0a966c540fb8bd9bb830148bf9646cb07 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 8 Apr 2024 16:11:38 +0200 Subject: [PATCH 06/27] WIP --- pyiceberg/table/__init__.py | 39 +++++++++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 6f123876f6..d292e5242b 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -307,6 +307,19 @@ def _apply(self, updates: Tuple[TableUpdate, ...], requirements: Tuple[TableRequ return self + + def _scan( + self, + row_filter: Union[str, BooleanExpression] = ALWAYS_TRUE + ) -> DataScan: + """Minimal data scan the table with the current state of the transaction""" + return DataScan( + table_metadata=self.table_metadata, + io=self._table.io, + row_filter=row_filter, + ) + + def upgrade_table_version(self, format_version: TableVersion) -> Transaction: """Set the table to a certain version. @@ -447,6 +460,19 @@ def overwrite( for data_file in data_files: update_snapshot.append_data_file(data_file) + def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: + with self.update_snapshot(snapshot_properties=snapshot_properties) as snapshot: + with snapshot.delete() as delete: + delete.delete(delete_filter) + + # Check if there are any files that require an actual rewrite of a data file + if delete.rewrites_needed: + files = self._scan().plan_files() + + for file in files: + + + def add_files(self, file_paths: List[str]) -> None: """ Shorthand API for adding files as data files to the table transaction. @@ -1345,6 +1371,19 @@ def overwrite( with self.transaction() as tx: tx.overwrite(df=df, overwrite_filter=overwrite_filter, snapshot_properties=snapshot_properties) + def delete(self, delete_filter: BooleanExpression = ALWAYS_TRUE, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: + """ + Shorthand for deleting rows from the table + + Args: + delete_filter: The predicate that used to remove rows + snapshot_properties: Custom properties to be added to the snapshot summary + """ + with self.transaction() as tx: + tx.delete(delete_filter=delete_filter, snapshot_properties=snapshot_properties) + + + def add_files(self, file_paths: List[str]) -> None: """ Shorthand API for adding files as data files to the table. From 1723819eaa56514b3f814e95481639f096e4320b Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 9 Apr 2024 21:20:12 +0200 Subject: [PATCH 07/27] WIP --- pyiceberg/io/pyarrow.py | 66 +++++- pyiceberg/table/__init__.py | 210 +++++++++++-------- tests/catalog/integration_test_glue.py | 3 +- tests/catalog/test_sql.py | 3 +- tests/integration/test_deletes.py | 12 +- tests/integration/test_writes/test_writes.py | 3 +- tests/table/test_init.py | 6 +- tests/table/test_snapshots.py | 4 - 8 files changed, 195 insertions(+), 112 deletions(-) diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py index 74692f85b8..61299bca7f 100644 --- a/pyiceberg/io/pyarrow.py +++ b/pyiceberg/io/pyarrow.py @@ -31,6 +31,7 @@ import logging import os import re +import uuid from abc import ABC, abstractmethod from concurrent.futures import Future from copy import copy @@ -126,7 +127,6 @@ visit, visit_with_partner, ) -from pyiceberg.table import PropertyUtil, TableProperties, WriteTask from pyiceberg.table.metadata import TableMetadata from pyiceberg.table.name_mapping import NameMapping from pyiceberg.transforms import TruncateTransform @@ -159,7 +159,7 @@ from pyiceberg.utils.truncate import truncate_upper_bound_binary_string, truncate_upper_bound_text_string if TYPE_CHECKING: - from pyiceberg.table import FileScanTask + from pyiceberg.table import FileScanTask, WriteTask logger = logging.getLogger(__name__) @@ -1443,6 +1443,8 @@ class PyArrowStatisticsCollector(PreOrderSchemaVisitor[List[StatisticsCollector] _default_mode: str def __init__(self, schema: Schema, properties: Dict[str, str]): + from pyiceberg.table import TableProperties + self._schema = schema self._properties = properties self._default_mode = self._properties.get( @@ -1478,6 +1480,8 @@ def map( return k + v def primitive(self, primitive: PrimitiveType) -> List[StatisticsCollector]: + from pyiceberg.table import TableProperties + column_name = self._schema.find_column_name(self._field_id) if column_name is None: return [] @@ -1774,7 +1778,9 @@ def data_file_statistics_from_parquet_metadata( ) -def write_file(io: FileIO, table_metadata: TableMetadata, tasks: Iterator[WriteTask]) -> Iterator[DataFile]: +def write_file(io: FileIO, table_metadata: TableMetadata, tasks: Iterable["WriteTask"]) -> Iterator[DataFile]: + from pyiceberg.table import PropertyUtil, TableProperties + schema = table_metadata.schema() arrow_file_schema = schema.as_arrow() parquet_writer_kwargs = _get_parquet_writer_kwargs(table_metadata.properties) @@ -1875,6 +1881,8 @@ def parquet_files_to_data_files(io: FileIO, table_metadata: TableMetadata, file_ def _get_parquet_writer_kwargs(table_properties: Properties) -> Dict[str, Any]: + from pyiceberg.table import PropertyUtil, TableProperties + for key_pattern in [ TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, TableProperties.PARQUET_PAGE_ROW_LIMIT, @@ -1912,3 +1920,55 @@ def _get_parquet_writer_kwargs(table_properties: Properties) -> Dict[str, Any]: default=TableProperties.PARQUET_PAGE_ROW_LIMIT_DEFAULT, ), } + + +def _dataframe_to_data_files( + table_metadata: TableMetadata, + df: pa.Table, + io: FileIO, + write_uuid: Optional[uuid.UUID] = None, + counter: Optional[itertools.count[int]] = None, +) -> Iterable[DataFile]: + """Convert a PyArrow table into a DataFile. + + Returns: + An iterable that supplies datafiles that represent the table. + """ + from pyiceberg.table import PropertyUtil, TableProperties, WriteTask + + counter = counter or itertools.count(0) + write_uuid = write_uuid or uuid.uuid4() + target_file_size: int = PropertyUtil.property_as_int( # type: ignore # The property is set with non-None value. + properties=table_metadata.properties, + property_name=TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, + default=TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, + ) + + if table_metadata.spec().is_unpartitioned(): + yield from write_file( + io=io, + table_metadata=table_metadata, + tasks=iter([ + WriteTask(write_uuid=write_uuid, task_id=next(counter), record_batches=batches, schema=table_metadata.schema()) + for batches in bin_pack_arrow_table(df, target_file_size) + ]), + ) + else: + from pyiceberg.table import determine_partitions + + partitions = determine_partitions(spec=table_metadata.spec(), schema=table_metadata.schema(), arrow_table=df) + yield from write_file( + io=io, + table_metadata=table_metadata, + tasks=iter([ + WriteTask( + write_uuid=write_uuid, + task_id=next(counter), + record_batches=batches, + partition_key=partition.partition_key, + schema=table_metadata.schema(), + ) + for partition in partitions + for batches in bin_pack_arrow_table(partition.arrow_table_partition, target_file_size) + ]), + ) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index d292e5242b..3f50736f63 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -68,6 +68,7 @@ manifest_evaluator, ) from pyiceberg.io import FileIO, load_file_io +from pyiceberg.io.pyarrow import _dataframe_to_data_files, project_table from pyiceberg.manifest import ( POSITIONAL_DELETE_SCHEMA, DataFile, @@ -307,19 +308,14 @@ def _apply(self, updates: Tuple[TableUpdate, ...], requirements: Tuple[TableRequ return self - - def _scan( - self, - row_filter: Union[str, BooleanExpression] = ALWAYS_TRUE - ) -> DataScan: - """Minimal data scan the table with the current state of the transaction""" + def _scan(self, row_filter: Union[str, BooleanExpression] = ALWAYS_TRUE) -> DataScan: + """Minimal data scan the table with the current state of the transaction.""" return DataScan( table_metadata=self.table_metadata, io=self._table.io, row_filter=row_filter, ) - def upgrade_table_version(self, format_version: TableVersion) -> Transaction: """Set the table to a certain version. @@ -461,17 +457,42 @@ def overwrite( update_snapshot.append_data_file(data_file) def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: - with self.update_snapshot(snapshot_properties=snapshot_properties) as snapshot: - with snapshot.delete() as delete: - delete.delete(delete_filter) - - # Check if there are any files that require an actual rewrite of a data file - if delete.rewrites_needed: - files = self._scan().plan_files() - - for file in files: - + with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: + delete_snapshot.delete_by_predicate(delete_filter) # type: ignore + + # Check if there are any files that require an actual rewrite of a data file + if delete_snapshot.rewrites_needed: # type: ignore + with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as overwrite_snapshot: + # Potential optimization is where we check if the files actually contain relevant data. + files = self._scan(row_filter=delete_filter).plan_files() + + counter = itertools.count(0) + + # This will load the Parquet file into memory, including: + # - Filter out the rows based on the delete filter + # - Projecting it to the current schema + # - Applying the positional deletes if they are there + # When writing + # - Apply the latest partition-spec + # - And sort order when added + for original_file in files: + df = project_table( + tasks=[original_file], + table_metadata=self._table.metadata, + io=self._table.io, + row_filter=delete_filter, + projected_schema=self.table_metadata.schema(), + ) + for data_file in _dataframe_to_data_files( + io=self._table.io, + df=df, + table_metadata=self._table.metadata, + write_uuid=overwrite_snapshot.commit_uuid, + counter=counter, + ): + overwrite_snapshot.append_data_file(data_file) + overwrite_snapshot.delete_data_file(original_file.file) def add_files(self, file_paths: List[str]) -> None: """ @@ -1310,6 +1331,9 @@ def current_snapshot(self) -> Optional[Snapshot]: return self.snapshot_by_id(self.metadata.current_snapshot_id) return None + def snapshots(self) -> List[Snapshot]: + return self.metadata.snapshots + def snapshot_by_id(self, snapshot_id: int) -> Optional[Snapshot]: """Get the snapshot of this table with the given id, or None if there is no matching snapshot.""" return self.metadata.snapshot_by_id(snapshot_id) @@ -1373,7 +1397,7 @@ def overwrite( def delete(self, delete_filter: BooleanExpression = ALWAYS_TRUE, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: """ - Shorthand for deleting rows from the table + Shorthand for deleting rows from the table. Args: delete_filter: The predicate that used to remove rows @@ -1382,8 +1406,6 @@ def delete(self, delete_filter: BooleanExpression = ALWAYS_TRUE, snapshot_proper with self.transaction() as tx: tx.delete(delete_filter=delete_filter, snapshot_properties=snapshot_properties) - - def add_files(self, file_paths: List[str]) -> None: """ Shorthand API for adding files as data files to the table. @@ -1671,7 +1693,6 @@ def _match_deletes_to_data_file(data_entry: ManifestEntry, positional_delete_ent class DataScan(TableScan): - def _build_partition_projection(self, spec_id: int) -> BooleanExpression: project = inclusive_projection(self.table_metadata.schema(), self.table_metadata.specs()[spec_id]) return project(self.row_filter) @@ -1925,7 +1946,8 @@ def union_by_name(self, new_schema: Union[Schema, "pa.Schema"]) -> UpdateSchema: visit_with_partner( Catalog._convert_schema_if_needed(new_schema), -1, - UnionByNameVisitor(update_schema=self, existing_schema=self._schema, case_sensitive=self._case_sensitive), # type: ignore + UnionByNameVisitor(update_schema=self, existing_schema=self._schema, case_sensitive=self._case_sensitive), + # type: ignore PartnerIdByNameAccessor(partner_schema=self._schema, case_sensitive=self._case_sensitive), ) return self @@ -2724,52 +2746,6 @@ def _generate_manifest_list_path(location: str, snapshot_id: int, attempt: int, return f'{location}/metadata/snap-{snapshot_id}-{attempt}-{commit_uuid}.avro' -def _dataframe_to_data_files( - table_metadata: TableMetadata, df: pa.Table, io: FileIO, write_uuid: Optional[uuid.UUID] = None -) -> Iterable[DataFile]: - """Convert a PyArrow table into a DataFile. - - Returns: - An iterable that supplies datafiles that represent the table. - """ - from pyiceberg.io.pyarrow import bin_pack_arrow_table, write_file - - counter = itertools.count(0) - write_uuid = write_uuid or uuid.uuid4() - target_file_size: int = PropertyUtil.property_as_int( # type: ignore # The property is set with non-None value. - properties=table_metadata.properties, - property_name=TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, - default=TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, - ) - - if len(table_metadata.spec().fields) > 0: - partitions = _determine_partitions(spec=table_metadata.spec(), schema=table_metadata.schema(), arrow_table=df) - yield from write_file( - io=io, - table_metadata=table_metadata, - tasks=iter([ - WriteTask( - write_uuid=write_uuid, - task_id=next(counter), - record_batches=batches, - partition_key=partition.partition_key, - schema=table_metadata.schema(), - ) - for partition in partitions - for batches in bin_pack_arrow_table(partition.arrow_table_partition, target_file_size) - ]), - ) - else: - yield from write_file( - io=io, - table_metadata=table_metadata, - tasks=iter([ - WriteTask(write_uuid=write_uuid, task_id=next(counter), record_batches=batches, schema=table_metadata.schema()) - for batches in bin_pack_arrow_table(df, target_file_size) - ]), - ) - - def _parquet_files_to_data_files(table_metadata: TableMetadata, file_paths: List[str], io: FileIO) -> Iterable[DataFile]: """Convert a list files into DataFiles. @@ -2787,6 +2763,8 @@ class _MergingSnapshotProducer(UpdateTableMetadata["_MergingSnapshotProducer"]): _snapshot_id: int _parent_snapshot_id: Optional[int] _added_data_files: List[DataFile] + _deleted_data_files: Set[DataFile] + _manifest_counter: itertools.count[int] def __init__( self, @@ -2806,12 +2784,18 @@ def __init__( 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 + self._manifest_counter = itertools.count(0) def append_data_file(self, data_file: DataFile) -> _MergingSnapshotProducer: self._added_data_files.append(data_file) return self + def delete_data_file(self, data_file: DataFile) -> _MergingSnapshotProducer: + self._deleted_data_files.add(data_file) + return self + @abstractmethod def _deleted_entries(self) -> List[ManifestEntry]: ... @@ -2822,7 +2806,9 @@ def _manifests(self) -> List[ManifestFile]: def _write_added_manifest() -> List[ManifestFile]: if self._added_data_files: output_file_location = _new_manifest_path( - location=self._transaction.table_metadata.location, num=0, commit_uuid=self.commit_uuid + location=self._transaction.table_metadata.location, + num=next(self._manifest_counter), + commit_uuid=self.commit_uuid, ) with write_manifest( format_version=self._transaction.table_metadata.format_version, @@ -2850,7 +2836,9 @@ def _write_delete_manifest() -> List[ManifestFile]: deleted_entries = self._deleted_entries() if len(deleted_entries) > 0: output_file_location = _new_manifest_path( - location=self._transaction.table_metadata.location, num=1, commit_uuid=self.commit_uuid + location=self._transaction.table_metadata.location, + num=next(self._manifest_counter), + commit_uuid=self.commit_uuid, ) with write_manifest( @@ -2941,13 +2929,20 @@ def _commit(self) -> UpdatesAndRequirements: ), ( AssertTableUUID(uuid=self._transaction.table_metadata.table_uuid), - AssertRefSnapshotId(snapshot_id=self._parent_snapshot_id, ref="main"), + # AssertRefSnapshotId(snapshot_id=self._parent_snapshot_id, ref="main"), ), ) -class MetadataDeleteFiles(_MergingSnapshotProducer): - """Will delete manifest entries from the current snapshot based on the predicate""" +class DeleteFiles(_MergingSnapshotProducer): + """Will delete manifest entries from the current snapshot based on the predicate. + + This will produce a DELETE snapshot: + Data files were removed and their contents logically deleted and/or delete + files were added to delete rows. + + From the specification + """ _predicate: BooleanExpression @@ -2986,7 +2981,7 @@ def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool] return lambda data_file: expression_evaluator(partition_schema, partition_expr, case_sensitive=True)(data_file.partition) - def delete(self, predicate: BooleanExpression) -> None: + def delete_by_predicate(self, predicate: BooleanExpression) -> None: self._predicate = Or(self._predicate, predicate) @cached_property @@ -3010,7 +3005,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> total_deleted_entries = [] partial_rewrites_needed = False if snapshot := self._transaction.table_metadata.current_snapshot(): - for num, manifest_file in enumerate(snapshot.manifests(io=self._io)): + for manifest_file in snapshot.manifests(io=self._io): 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) @@ -3018,7 +3013,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> # It is relevant, let's check out the content deleted_entries = [] existing_entries = [] - for entry in manifest_file.fetch_manifest_entry(io=self._io): + for entry in manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True): if strict_metrics_evaluator(entry.data_file) == ROWS_MUST_MATCH: deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED)) elif inclusive_metrics_evaluator(entry.data_file) == ROWS_CANNOT_MATCH: @@ -3033,7 +3028,9 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> # Rewrite the manifest if len(existing_entries) > 0: output_file_location = _new_manifest_path( - location=self._transaction.table_metadata.location, num=num, commit_uuid=self.commit_uuid + location=self._transaction.table_metadata.location, + num=next(self._manifest_counter), + commit_uuid=self.commit_uuid, ) with write_manifest( format_version=self._transaction.table_metadata.format_version, @@ -3057,7 +3054,7 @@ def _deleted_entries(self) -> List[ManifestEntry]: return self._compute_deletes[1] def rewrites_needed(self) -> bool: - """Indicates if data files need to be rewritten""" + """Indicate if data files need to be rewritten.""" return self._compute_deletes[2] @@ -3091,13 +3088,50 @@ def _deleted_entries(self) -> List[ManifestEntry]: class OverwriteFiles(_MergingSnapshotProducer): + """Overwrites data from the table. This will produce an OVERWRITE snapshot. + + Data and delete files were added and removed in a logical overwrite operation. + """ + def _existing_manifests(self) -> List[ManifestFile]: - """To determine if there are any existing manifest files. + """Determine if there are any existing manifest files.""" + existing_files = [] - In the of a full overwrite, all the previous manifests are - considered deleted. - """ - return [] + 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_deletes = [_ for entry in entries if entry in self._deleted_data_files] + + if len(found_deletes) == 0: + existing_files.append(manifest_file) + else: + output_file_location = _new_manifest_path( + location=self._transaction.table_metadata.location, + num=next(self._manifest_counter), + commit_uuid=self.commit_uuid, + ) + 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._io.new_output(output_file_location), + snapshot_id=self._snapshot_id, + ) as writer: + [ + writer.add_entry( + ManifestEntry( + status=ManifestEntryStatus.EXISTING, + snapshot_id=entry.snapshot_id, + data_sequence_number=entry.data_sequence_number, + file_sequence_number=entry.file_sequence_number, + data_file=entry.data_file, + ) + ) + for entry in entries + if entry not in found_deletes + ] + existing_files.append(writer.to_manifest_file()) + return existing_files def _deleted_entries(self) -> List[ManifestEntry]: """To determine if we need to record any deleted entries. @@ -3124,7 +3158,7 @@ def _get_entries(manifest: ManifestFile) -> List[ManifestEntry]: data_file=entry.data_file, ) for entry in manifest.fetch_manifest_entry(self._io, discard_deleted=True) - if entry.data_file.content == DataFileContent.DATA + if entry.data_file.content == DataFileContent.DATA and entry.data_file in self._deleted_data_files ] list_of_entries = executor.map(_get_entries, previous_snapshot.manifests(self._io)) @@ -3158,8 +3192,8 @@ def overwrite(self) -> OverwriteFiles: snapshot_properties=self._snapshot_properties, ) - def delete(self) -> MetadataDeleteFiles: - return MetadataDeleteFiles( + def delete(self) -> DeleteFiles: + return DeleteFiles( operation=Operation.DELETE, transaction=self._transaction, io=self._io, @@ -3642,7 +3676,7 @@ def _get_table_partitions( return table_partitions -def _determine_partitions(spec: PartitionSpec, schema: Schema, arrow_table: pa.Table) -> List[TablePartition]: +def determine_partitions(spec: PartitionSpec, schema: Schema, arrow_table: pa.Table) -> List[TablePartition]: """Based on the iceberg table partition spec, slice the arrow table into partitions with their keys. Example: @@ -3651,7 +3685,7 @@ def _determine_partitions(spec: PartitionSpec, schema: Schema, arrow_table: pa.T {'year': [2020, 2022, 2022, 2021, 2022, 2022, 2022, 2019, 2021], 'n_legs': [2, 2, 2, 4, 4, 4, 4, 5, 100], 'animal': ["Flamingo", "Parrot", "Parrot", "Dog", "Horse", "Horse", "Horse","Brittle stars", "Centipede"]}. - The algrithm: + The algorithm: Firstly we group the rows into partitions by sorting with sort order [('n_legs', 'descending'), ('year', 'descending')] and null_placement of "at_end". This gives the same table as raw input. diff --git a/tests/catalog/integration_test_glue.py b/tests/catalog/integration_test_glue.py index 5cd60225c4..46884b6769 100644 --- a/tests/catalog/integration_test_glue.py +++ b/tests/catalog/integration_test_glue.py @@ -33,9 +33,8 @@ NoSuchTableError, TableAlreadyExistsError, ) -from pyiceberg.io.pyarrow import schema_to_pyarrow +from pyiceberg.io.pyarrow import _dataframe_to_data_files, schema_to_pyarrow from pyiceberg.schema import Schema -from pyiceberg.table import _dataframe_to_data_files from pyiceberg.types import IntegerType from tests.conftest import clean_up, get_bucket_name, get_s3_path diff --git a/tests/catalog/test_sql.py b/tests/catalog/test_sql.py index 99b8550602..1cea5a1e12 100644 --- a/tests/catalog/test_sql.py +++ b/tests/catalog/test_sql.py @@ -36,10 +36,9 @@ TableAlreadyExistsError, ) from pyiceberg.io import FSSPEC_FILE_IO, PY_IO_IMPL -from pyiceberg.io.pyarrow import schema_to_pyarrow +from pyiceberg.io.pyarrow import _dataframe_to_data_files, schema_to_pyarrow from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC from pyiceberg.schema import Schema -from pyiceberg.table import _dataframe_to_data_files from pyiceberg.table.snapshots import Operation from pyiceberg.table.sorting import ( NullOrder, diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index b615065a19..3face11215 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -56,10 +56,7 @@ def test_partition_deletes(test_deletes_table: DataFrame, session_catalog: RestC identifier = 'default.table_partitioned_delete' tbl = session_catalog.load_table(identifier) - - with tbl.transaction() as txn: - with txn.update_snapshot().delete() as delete: - delete.delete(EqualTo("number_partitioned", 10)) + tbl.delete(EqualTo("number_partitioned", 10)) assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} @@ -68,9 +65,6 @@ def test_deletes(test_deletes_table: DataFrame, session_catalog: RestCatalog) -> identifier = 'default.table_partitioned_delete' tbl = session_catalog.load_table(identifier) + tbl.delete(EqualTo("number", 30)) - with tbl.transaction() as txn: - with txn.update_snapshot().delete() as delete: - delete.delete(EqualTo("number", 30)) - - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 20]} diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py index e1526d2a5e..65e92521b7 100644 --- a/tests/integration/test_writes/test_writes.py +++ b/tests/integration/test_writes/test_writes.py @@ -36,7 +36,8 @@ from pyiceberg.catalog.hive import HiveCatalog from pyiceberg.catalog.sql import SqlCatalog from pyiceberg.exceptions import NoSuchTableError -from pyiceberg.table import TableProperties, _dataframe_to_data_files +from pyiceberg.io.pyarrow import _dataframe_to_data_files +from pyiceberg.table import TableProperties from tests.conftest import TEST_DATA_WITH_NULL from utils import _create_table diff --git a/tests/table/test_init.py b/tests/table/test_init.py index 2bc78f3197..feccb99d5c 100644 --- a/tests/table/test_init.py +++ b/tests/table/test_init.py @@ -64,9 +64,9 @@ UpdateSchema, _apply_table_update, _check_schema_compatible, - _determine_partitions, _match_deletes_to_data_file, _TableMetadataUpdateContext, + determine_partitions, update_table_metadata, ) from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadataUtil, TableMetadataV2, _generate_snapshot_id @@ -1166,7 +1166,7 @@ def test_partition_for_demo() -> None: PartitionField(source_id=2, field_id=1002, transform=IdentityTransform(), name="n_legs_identity"), PartitionField(source_id=1, field_id=1001, transform=IdentityTransform(), name="year_identity"), ) - result = _determine_partitions(partition_spec, test_schema, arrow_table) + result = determine_partitions(partition_spec, test_schema, arrow_table) assert {table_partition.partition_key.partition for table_partition in result} == { Record(n_legs_identity=2, year_identity=2020), Record(n_legs_identity=100, year_identity=2021), @@ -1216,7 +1216,7 @@ def test_identity_partition_on_multi_columns() -> None: } arrow_table = pa.Table.from_pydict(test_data, schema=test_pa_schema) - result = _determine_partitions(partition_spec, test_schema, arrow_table) + result = determine_partitions(partition_spec, test_schema, arrow_table) assert {table_partition.partition_key.partition for table_partition in result} == expected concatenated_arrow_table = pa.concat_tables([table_partition.arrow_table_partition for table_partition in result]) diff --git a/tests/table/test_snapshots.py b/tests/table/test_snapshots.py index e85ecce506..827de7ccc3 100644 --- a/tests/table/test_snapshots.py +++ b/tests/table/test_snapshots.py @@ -313,10 +313,6 @@ def test_invalid_operation() -> None: update_snapshot_summaries(summary=Summary(Operation.REPLACE)) assert "Operation not implemented: Operation.REPLACE" in str(e.value) - with pytest.raises(ValueError) as e: - update_snapshot_summaries(summary=Summary(Operation.DELETE)) - assert "Operation not implemented: Operation.DELETE" in str(e.value) - def test_invalid_type() -> None: with pytest.raises(ValueError) as e: From 5025b4afead58ed8e6701160b940db1307dcdb67 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 10 Apr 2024 22:32:41 +0200 Subject: [PATCH 08/27] Moar tests --- pyiceberg/table/__init__.py | 21 +++- tests/integration/test_deletes.py | 163 ++++++++++++++++++++++++------ 2 files changed, 149 insertions(+), 35 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 3f50736f63..e008791399 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -55,6 +55,7 @@ And, BooleanExpression, EqualTo, + Not, Or, Reference, ) @@ -240,6 +241,8 @@ class TableProperties: WRITE_PARTITION_SUMMARY_LIMIT = "write.summary.partition-limit" WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT = 0 + DELETE_MODE = "write.delete.mode" + DEFAULT_NAME_MAPPING = "schema.name-mapping.default" FORMAT_VERSION = "format-version" DEFAULT_FORMAT_VERSION = 2 @@ -457,11 +460,18 @@ def overwrite( update_snapshot.append_data_file(data_file) def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: + if (mode := self.table_metadata.properties.get(TableProperties.DELETE_MODE)) and mode != 'copy-on-write': + warnings.warn("PyIceberg only supports copy on write") + with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: delete_snapshot.delete_by_predicate(delete_filter) # type: ignore # Check if there are any files that require an actual rewrite of a data file - if delete_snapshot.rewrites_needed: # type: ignore + if delete_snapshot.rewrites_needed is True: # type: ignore + # When we want to filter out certain rows, we want to invert the expression + # delete id = 22 means that we want to look for that value, and then remove + # if from the Parquet file + delete_row_filter = Not(delete_filter) with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as overwrite_snapshot: # Potential optimization is where we check if the files actually contain relevant data. files = self._scan(row_filter=delete_filter).plan_files() @@ -480,7 +490,7 @@ def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str tasks=[original_file], table_metadata=self._table.metadata, io=self._table.io, - row_filter=delete_filter, + row_filter=delete_row_filter, projected_schema=self.table_metadata.schema(), ) for data_file in _dataframe_to_data_files( @@ -3100,11 +3110,12 @@ 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_deletes = [_ for entry in entries if entry 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_deletes) == 0: + if len(found_deleted_data_files) == 0: existing_files.append(manifest_file) else: + # We have to rewrite the output_file_location = _new_manifest_path( location=self._transaction.table_metadata.location, num=next(self._manifest_counter), @@ -3128,7 +3139,7 @@ def _existing_manifests(self) -> List[ManifestFile]: ) ) for entry in entries - if entry not in found_deletes + if entry.data_file not in found_deleted_data_files ] existing_files.append(writer.to_manifest_file()) return existing_files diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index 3face11215..4e445062d2 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -15,56 +15,159 @@ # specific language governing permissions and limitations # under the License. # pylint:disable=redefined-outer-name +from typing import List + import pytest -from pyspark.sql import DataFrame, SparkSession +from pyspark.sql import SparkSession from pyiceberg.catalog.rest import RestCatalog from pyiceberg.expressions import EqualTo -@pytest.fixture -def test_deletes_table(spark: SparkSession) -> DataFrame: +def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None: + for sql in sqls: + spark.sql(sql) + + +@pytest.mark.parametrize("format_version", [1, 2]) +def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: identifier = 'default.table_partitioned_delete' - spark.sql(f"DROP TABLE IF EXISTS {identifier}") - - spark.sql( - f""" - CREATE TABLE {identifier} ( - number_partitioned int, - number int - ) - USING iceberg - PARTITIONED BY (number_partitioned) - """ - ) - spark.sql( - f""" - INSERT INTO {identifier} VALUES (10, 20), (10, 30) - """ - ) - spark.sql( - f""" - INSERT INTO {identifier} VALUES (11, 20), (11, 30) - """ + run_spark_commands( + spark, + [ + f"DROP TABLE IF EXISTS {identifier}", + f""" + CREATE TABLE {identifier} ( + number_partitioned int, + number int + ) + USING iceberg + PARTITIONED BY (number_partitioned) + TBLPROPERTIES('format-version' = {format_version}) + """, + f""" + INSERT INTO {identifier} VALUES (10, 20), (10, 30) + """, + f""" + INSERT INTO {identifier} VALUES (11, 20), (11, 30) + """, + ], ) - return spark.table(identifier) + tbl = session_catalog.load_table(identifier) + tbl.delete(EqualTo("number_partitioned", 10)) + + # No overwrite operation + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete'] + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} -def test_partition_deletes(test_deletes_table: DataFrame, session_catalog: RestCatalog) -> None: +@pytest.mark.parametrize("format_version", [1, 2]) +def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: identifier = 'default.table_partitioned_delete' + run_spark_commands( + spark, + [ + f"DROP TABLE IF EXISTS {identifier}", + f""" + CREATE TABLE {identifier} ( + number_partitioned int, + number int + ) + USING iceberg + PARTITIONED BY (number_partitioned) + TBLPROPERTIES('format-version' = {format_version}) + """, + f""" + INSERT INTO {identifier} VALUES (10, 20), (10, 30) + """, + f""" + INSERT INTO {identifier} VALUES (11, 20), (11, 30) + """, + ], + ) + tbl = session_catalog.load_table(identifier) - tbl.delete(EqualTo("number_partitioned", 10)) + tbl.delete(EqualTo("number", 20)) + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete', 'overwrite'] assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} -def test_deletes(test_deletes_table: DataFrame, session_catalog: RestCatalog) -> None: +@pytest.mark.parametrize("format_version", [1, 2]) +def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: identifier = 'default.table_partitioned_delete' + run_spark_commands( + spark, + [ + f"DROP TABLE IF EXISTS {identifier}", + f""" + CREATE TABLE {identifier} ( + number_partitioned int, + number int + ) + USING iceberg + PARTITIONED BY (number_partitioned) + TBLPROPERTIES('format-version' = {format_version}) + """, + f""" + INSERT INTO {identifier} VALUES (10, 20), (10, 30) + """, + ], + ) + + tbl = session_catalog.load_table(identifier) + tbl.delete(EqualTo("number_partitioned", 22)) # Does not affect any data + + # Open for discussion, do we want to create a new snapshot? + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'delete'] + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10, 10], 'number': [20, 30]} + + +def test_partitioned_table_positional_deletes(spark: SparkSession, session_catalog: RestCatalog) -> None: + identifier = 'default.table_partitioned_delete' + + run_spark_commands( + spark, + [ + f"DROP TABLE IF EXISTS {identifier}", + f""" + CREATE TABLE {identifier} ( + number_partitioned int, + number int + ) + USING iceberg + PARTITIONED BY (number_partitioned) + TBLPROPERTIES( + 'format-version' = 2, + 'write.delete.mode'='merge-on-read', + 'write.update.mode'='merge-on-read', + 'write.merge.mode'='merge-on-read' + ) + """, + f""" + INSERT INTO {identifier} VALUES (10, 20), (10, 30), (10, 40) + """, + # Generate a positional delete + f""" + DELETE FROM {identifier} WHERE number = 30 + """, + ], + ) + tbl = session_catalog.load_table(identifier) - tbl.delete(EqualTo("number", 30)) - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 20]} + # Assert that there is just a single Parquet file + assert len(list(tbl.scan().plan_files())) == 1 + + # Will rewrite a data file with a positional delete + tbl.delete(EqualTo("number", 40)) + + # Yet another wrong status by Spark + # One positional delete has been added, but an OVERWRITE status is set + # Related issue https://github.com/apache/iceberg/issues/9995 + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'delete'] + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10], 'number': [20]} From e474fda7a3c01a1c7ac5caedc1e7e4f1da3f3a19 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 11 Apr 2024 13:11:25 +0200 Subject: [PATCH 09/27] Oops --- pyiceberg/table/__init__.py | 1 + tests/integration/test_deletes.py | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index e008791399..4cce25f2cb 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -3063,6 +3063,7 @@ def _existing_manifests(self) -> List[ManifestFile]: def _deleted_entries(self) -> List[ManifestEntry]: return self._compute_deletes[1] + @property def rewrites_needed(self) -> bool: """Indicate if data files need to be rewritten.""" return self._compute_deletes[2] diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index 4e445062d2..8cb6370b6f 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -93,7 +93,7 @@ def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCat tbl.delete(EqualTo("number", 20)) assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete', 'overwrite'] - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 10], 'number': [30, 30]} @pytest.mark.parametrize("format_version", [1, 2]) @@ -169,5 +169,5 @@ def test_partitioned_table_positional_deletes(spark: SparkSession, session_catal # Yet another wrong status by Spark # One positional delete has been added, but an OVERWRITE status is set # Related issue https://github.com/apache/iceberg/issues/9995 - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'delete'] + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'delete', 'overwrite'] assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10], 'number': [20]} From 172f9c0a622b291dba86b89fadb94bc9a3f5b4e0 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 16 Apr 2024 09:34:29 +0200 Subject: [PATCH 10/27] Cleanup --- pyiceberg/table/__init__.py | 96 ++++++++++++++++---------- tests/integration/test_deletes.py | 111 ++++++++++++++++++++++++++++-- 2 files changed, 163 insertions(+), 44 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 4cce25f2cb..ded191d262 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -2967,6 +2967,13 @@ def __init__( super().__init__(operation, transaction, io, commit_uuid, snapshot_properties) self._predicate = AlwaysFalse() + def _commit(self) -> UpdatesAndRequirements: + # Only produce a commit when there is something to delete + if self.files_affected: + return super()._commit() + else: + return (), () + def _build_partition_projection(self, spec_id: int) -> BooleanExpression: schema = self._transaction.table_metadata.schema() spec = self._transaction.table_metadata.specs()[spec_id] @@ -2996,6 +3003,13 @@ def delete_by_predicate(self, predicate: BooleanExpression) -> None: @cached_property def _compute_deletes(self) -> Tuple[List[ManifestFile], List[ManifestEntry], bool]: + """Computes all the delete operation and cache it when nothing changes. + + Returns: + - List of existing manifests that are not affected by the delete operation. + - The manifest-entries that are deleted based on the metadata. + - Flag indicating that rewrites of data-files are needed. + """ schema = self._transaction.table_metadata.schema() def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> ManifestEntry: @@ -3016,44 +3030,47 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> partial_rewrites_needed = False if snapshot := self._transaction.table_metadata.current_snapshot(): for manifest_file in snapshot.manifests(io=self._io): - 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: - deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED)) - elif inclusive_metrics_evaluator(entry.data_file) == ROWS_CANNOT_MATCH: - existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING)) - else: - # Based on the metadata, it is unsure to say if the file can be deleted - partial_rewrites_needed = True - - if len(deleted_entries) > 0: - total_deleted_entries += deleted_entries - - # Rewrite the manifest - if len(existing_entries) > 0: - output_file_location = _new_manifest_path( - location=self._transaction.table_metadata.location, - num=next(self._manifest_counter), - commit_uuid=self.commit_uuid, - ) - 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._io.new_output(output_file_location), - snapshot_id=self._snapshot_id, - ) as writer: - for existing_entry in existing_entries: - writer.add_entry(existing_entry) - existing_manifests.append(writer.to_manifest_file()) - else: + 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: + deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED)) + elif inclusive_metrics_evaluator(entry.data_file) == ROWS_CANNOT_MATCH: + existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING)) + else: + # Based on the metadata, it is unsure to say if the file can be deleted + partial_rewrites_needed = True + + if len(deleted_entries) > 0: + total_deleted_entries += deleted_entries + + # Rewrite the manifest + if len(existing_entries) > 0: + output_file_location = _new_manifest_path( + location=self._transaction.table_metadata.location, + num=next(self._manifest_counter), + commit_uuid=self.commit_uuid, + ) + 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._io.new_output(output_file_location), + snapshot_id=self._snapshot_id, + ) 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 @@ -3068,6 +3085,11 @@ def rewrites_needed(self) -> bool: """Indicate if data files need to be rewritten.""" return self._compute_deletes[2] + @property + def files_affected(self) -> bool: + """Indicate if any manifest-entries can be dropped.""" + return len(self._deleted_entries()) > 0 + class FastAppendFiles(_MergingSnapshotProducer): def _existing_manifests(self) -> List[ManifestFile]: diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index 8cb6370b6f..47086a48c7 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -22,6 +22,7 @@ from pyiceberg.catalog.rest import RestCatalog from pyiceberg.expressions import EqualTo +from pyiceberg.table.snapshots import Operation, Summary def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None: @@ -29,6 +30,7 @@ def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None: spark.sql(sql) +@pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: identifier = 'default.table_partitioned_delete' @@ -63,6 +65,7 @@ def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} +@pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: identifier = 'default.table_partitioned_delete' @@ -92,10 +95,12 @@ def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCat tbl = session_catalog.load_table(identifier) tbl.delete(EqualTo("number", 20)) - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete', 'overwrite'] + # We don't delete a whole partition, so there is only a overwrite + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'overwrite'] assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 10], 'number': [30, 30]} +@pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: identifier = 'default.table_partitioned_delete' @@ -123,10 +128,11 @@ def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCa tbl.delete(EqualTo("number_partitioned", 22)) # Does not affect any data # Open for discussion, do we want to create a new snapshot? - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'delete'] + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append'] assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10, 10], 'number': [20, 30]} +@pytest.mark.integration def test_partitioned_table_positional_deletes(spark: SparkSession, session_catalog: RestCatalog) -> None: identifier = 'default.table_partitioned_delete' @@ -160,14 +166,105 @@ def test_partitioned_table_positional_deletes(spark: SparkSession, session_catal tbl = session_catalog.load_table(identifier) - # Assert that there is just a single Parquet file - assert len(list(tbl.scan().plan_files())) == 1 + # Assert that there is just a single Parquet file, that has one merge on read file + files = list(tbl.scan().plan_files()) + assert len(files) == 1 + assert len(files[0].delete_files) == 1 # Will rewrite a data file with a positional delete tbl.delete(EqualTo("number", 40)) - # Yet another wrong status by Spark # One positional delete has been added, but an OVERWRITE status is set - # Related issue https://github.com/apache/iceberg/issues/9995 - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'delete', 'overwrite'] + # https://github.com/apache/iceberg/issues/10122 + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'overwrite'] assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10], 'number': [20]} + + +@pytest.mark.integration +def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSession, session_catalog: RestCatalog) -> None: + identifier = 'default.table_partitioned_delete_sequence_number' + + # This test case is a bit more complex. Here we run a MoR delete on a file, we make sure that + # the manifest gets rewritten (but not the data file with a MoR), and check if the delete is still there + # to assure that the sequence numbers are maintained + + run_spark_commands( + spark, + [ + f"DROP TABLE IF EXISTS {identifier}", + f""" + CREATE TABLE {identifier} ( + number_partitioned int, + number int + ) + USING iceberg + PARTITIONED BY (number_partitioned) + TBLPROPERTIES( + 'format-version' = 2, + 'write.delete.mode'='merge-on-read', + 'write.update.mode'='merge-on-read', + 'write.merge.mode'='merge-on-read' + ) + """, + f""" + INSERT INTO {identifier} VALUES (10, 100), (10, 101), (20, 200), (20, 201), (20, 202) + """, + # Generate a positional delete + f""" + DELETE FROM {identifier} WHERE number = 101 + """, + ], + ) + + tbl = session_catalog.load_table(identifier) + + files = list(tbl.scan().plan_files()) + assert len(files) == 2 + + # Will rewrite a data file with a positional delete + tbl.delete(EqualTo("number", 201)) + + # One positional delete has been added, but an OVERWRITE status is set + # https://github.com/apache/iceberg/issues/10122 + snapshots = tbl.snapshots() + assert len(snapshots) == 4 + + # Snapshots produced by Spark + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()[0:2]] == ['append', 'overwrite'] + + # Snapshots produced by PyIceberg + # This is a no-op since nothing has been added or deleted (because the predicate cannot drop a whole file) + assert tbl.snapshots()[2].summary == Summary( + Operation.DELETE, + **{ + 'total-data-files': '2', + 'total-delete-files': '1', + 'total-records': '5', + 'total-files-size': tbl.snapshots()[2].summary['total-files-size'], + 'total-position-deletes': '1', + 'total-equality-deletes': '0', + }, + ) + # Will rewrite one parquet file + assert tbl.snapshots()[3].summary == Summary( + Operation.OVERWRITE, + **{ + 'added-files-size': '1145', + 'added-data-files': '1', + 'added-records': '2', + 'changed-partition-count': '1', + 'total-files-size': tbl.snapshots()[3].summary['total-files-size'], + 'total-delete-files': '0', + 'total-data-files': '1', + 'total-position-deletes': '0', + 'total-records': '2', + 'total-equality-deletes': '0', + 'deleted-data-files': '2', + 'removed-delete-files': '1', + 'deleted-records': '5', + 'removed-files-size': '3088', + 'removed-position-deletes': '1', + }, + ) + + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [20, 20, 10], 'number': [200, 202, 100]} From a97c45aea808fa1ac51adaff8d4c30b2af6ad03d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 16 Apr 2024 23:31:23 +0200 Subject: [PATCH 11/27] WIP --- pyiceberg/table/__init__.py | 3 +- tests/conftest.py | 62 +++++++++---------- .../test_writes/test_partitioned_writes.py | 18 +++--- tests/integration/test_writes/test_writes.py | 44 ++++++------- 4 files changed, 66 insertions(+), 61 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index ded191d262..3751793be7 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -1956,8 +1956,7 @@ def union_by_name(self, new_schema: Union[Schema, "pa.Schema"]) -> UpdateSchema: visit_with_partner( Catalog._convert_schema_if_needed(new_schema), -1, - UnionByNameVisitor(update_schema=self, existing_schema=self._schema, case_sensitive=self._case_sensitive), - # type: ignore + UnionByNameVisitor(update_schema=self, existing_schema=self._schema, case_sensitive=self._case_sensitive), # type: ignore PartnerIdByNameAccessor(partner_schema=self._schema, case_sensitive=self._case_sensitive), ) return self diff --git a/tests/conftest.py b/tests/conftest.py index 7da0a0a85a..5a2fceb8b9 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -2064,36 +2064,6 @@ def spark() -> "SparkSession": return spark -TEST_DATA_WITH_NULL = { - 'bool': [False, None, True], - 'string': ['a', None, 'z'], - # Go over the 16 bytes to kick in truncation - 'string_long': ['a' * 22, None, 'z' * 22], - 'int': [1, None, 9], - 'long': [1, None, 9], - 'float': [0.0, None, 0.9], - 'double': [0.0, None, 0.9], - # 'time': [1_000_000, None, 3_000_000], # Example times: 1s, none, and 3s past midnight #Spark does not support time fields - 'timestamp': [datetime(2023, 1, 1, 19, 25, 00), None, datetime(2023, 3, 1, 19, 25, 00)], - 'timestamptz': [ - datetime(2023, 1, 1, 19, 25, 00, tzinfo=timezone.utc), - None, - datetime(2023, 3, 1, 19, 25, 00, tzinfo=timezone.utc), - ], - 'date': [date(2023, 1, 1), None, date(2023, 3, 1)], - # Not supported by Spark - # 'time': [time(1, 22, 0), None, time(19, 25, 0)], - # Not natively supported by Arrow - # 'uuid': [uuid.UUID('00000000-0000-0000-0000-000000000000').bytes, None, uuid.UUID('11111111-1111-1111-1111-111111111111').bytes], - 'binary': [b'\01', None, b'\22'], - 'fixed': [ - uuid.UUID('00000000-0000-0000-0000-000000000000').bytes, - None, - uuid.UUID('11111111-1111-1111-1111-111111111111').bytes, - ], -} - - @pytest.fixture(scope="session") def pa_schema() -> "pa.Schema": import pyarrow as pa @@ -2125,7 +2095,37 @@ def arrow_table_with_null(pa_schema: "pa.Schema") -> "pa.Table": """Pyarrow table with all kinds of columns.""" import pyarrow as pa - return pa.Table.from_pydict(TEST_DATA_WITH_NULL, schema=pa_schema) + return pa.Table.from_pydict( + { + 'bool': [False, None, True], + 'string': ['a', None, 'z'], + # Go over the 16 bytes to kick in truncation + 'string_long': ['a' * 22, None, 'z' * 22], + 'int': [1, None, 9], + 'long': [1, None, 9], + 'float': [0.0, None, 0.9], + 'double': [0.0, None, 0.9], + # 'time': [1_000_000, None, 3_000_000], # Example times: 1s, none, and 3s past midnight #Spark does not support time fields + 'timestamp': [datetime(2023, 1, 1, 19, 25, 00), None, datetime(2023, 3, 1, 19, 25, 00)], + 'timestamptz': [ + datetime(2023, 1, 1, 19, 25, 00, tzinfo=timezone.utc), + None, + datetime(2023, 3, 1, 19, 25, 00, tzinfo=timezone.utc), + ], + 'date': [date(2023, 1, 1), None, date(2023, 3, 1)], + # Not supported by Spark + # 'time': [time(1, 22, 0), None, time(19, 25, 0)], + # Not natively supported by Arrow + # 'uuid': [uuid.UUID('00000000-0000-0000-0000-000000000000').bytes, None, uuid.UUID('11111111-1111-1111-1111-111111111111').bytes], + 'binary': [b'\01', None, b'\22'], + 'fixed': [ + uuid.UUID('00000000-0000-0000-0000-000000000000').bytes, + None, + uuid.UUID('11111111-1111-1111-1111-111111111111').bytes, + ], + }, + schema=pa_schema, + ) @pytest.fixture(scope="session") diff --git a/tests/integration/test_writes/test_partitioned_writes.py b/tests/integration/test_writes/test_partitioned_writes.py index d84b9745a7..b003af3dea 100644 --- a/tests/integration/test_writes/test_partitioned_writes.py +++ b/tests/integration/test_writes/test_partitioned_writes.py @@ -32,7 +32,6 @@ TruncateTransform, YearTransform, ) -from tests.conftest import TEST_DATA_WITH_NULL from utils import TABLE_SCHEMA, _create_table @@ -64,7 +63,7 @@ def test_query_filter_null_partitioned( assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}" df = spark.table(identifier) assert df.count() == 3, f"Expected 3 total rows for {identifier}" - for col in TEST_DATA_WITH_NULL.keys(): + for col in arrow_table_with_null.column_names: assert df.where(f"{col} is not null").count() == 2, f"Expected 2 non-null rows for {col}" assert df.where(f"{col} is null").count() == 1, f"Expected 1 null row for {col} is null" @@ -75,7 +74,12 @@ def test_query_filter_null_partitioned( ) @pytest.mark.parametrize("format_version", [1, 2]) def test_query_filter_without_data_partitioned( - session_catalog: Catalog, spark: SparkSession, arrow_table_without_data: pa.Table, part_col: str, format_version: int + session_catalog: Catalog, + spark: SparkSession, + arrow_table_without_data: pa.Table, + part_col: str, + arrow_table_with_null: pa.Table, + format_version: int, ) -> None: # Given identifier = f"default.arrow_table_v{format_version}_without_data_partitioned_on_col_{part_col}" @@ -96,7 +100,7 @@ def test_query_filter_without_data_partitioned( # Then assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}" df = spark.table(identifier) - for col in TEST_DATA_WITH_NULL.keys(): + for col in arrow_table_with_null.column_names: assert df.where(f"{col} is null").count() == 0, f"Expected 0 row for {col}" assert df.where(f"{col} is not null").count() == 0, f"Expected 0 row for {col}" @@ -128,7 +132,7 @@ def test_query_filter_only_nulls_partitioned( # Then assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}" df = spark.table(identifier) - for col in TEST_DATA_WITH_NULL.keys(): + for col in arrow_table_with_only_nulls.column_names: assert df.where(f"{col} is null").count() == 2, f"Expected 2 row for {col}" assert df.where(f"{col} is not null").count() == 0, f"Expected 0 rows for {col}" @@ -163,7 +167,7 @@ def test_query_filter_appended_null_partitioned( # Then assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}" df = spark.table(identifier) - for col in TEST_DATA_WITH_NULL.keys(): + for col in arrow_table_with_null.column_names: df = spark.table(identifier) assert df.where(f"{col} is not null").count() == 6, f"Expected 6 non-null rows for {col}" assert df.where(f"{col} is null").count() == 3, f"Expected 3 null rows for {col}" @@ -207,7 +211,7 @@ def test_query_filter_v1_v2_append_null( # Then assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}" - for col in TEST_DATA_WITH_NULL.keys(): # type: ignore + for col in arrow_table_with_null.column_names: # type: ignore df = spark.table(identifier) assert df.where(f"{col} is not null").count() == 4, f"Expected 4 non-null rows for {col}" assert df.where(f"{col} is null").count() == 2, f"Expected 2 null rows for {col}" diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py index 65e92521b7..56158d8795 100644 --- a/tests/integration/test_writes/test_writes.py +++ b/tests/integration/test_writes/test_writes.py @@ -38,7 +38,6 @@ from pyiceberg.exceptions import NoSuchTableError from pyiceberg.io.pyarrow import _dataframe_to_data_files from pyiceberg.table import TableProperties -from tests.conftest import TEST_DATA_WITH_NULL from utils import _create_table @@ -120,52 +119,55 @@ def test_query_count(spark: SparkSession, format_version: int) -> None: @pytest.mark.integration -@pytest.mark.parametrize("col", TEST_DATA_WITH_NULL.keys()) @pytest.mark.parametrize("format_version", [1, 2]) -def test_query_filter_null(spark: SparkSession, col: str, format_version: int) -> None: +def test_query_filter_null(spark: SparkSession, arrow_table_with_null: pa.Table, format_version: int) -> None: identifier = f"default.arrow_table_v{format_version}_with_null" df = spark.table(identifier) - assert df.where(f"{col} is null").count() == 1, f"Expected 1 row for {col}" - assert df.where(f"{col} is not null").count() == 2, f"Expected 2 rows for {col}" + for col in arrow_table_with_null.column_names: + assert df.where(f"{col} is null").count() == 1, f"Expected 1 row for {col}" + assert df.where(f"{col} is not null").count() == 2, f"Expected 2 rows for {col}" @pytest.mark.integration -@pytest.mark.parametrize("col", TEST_DATA_WITH_NULL.keys()) @pytest.mark.parametrize("format_version", [1, 2]) -def test_query_filter_without_data(spark: SparkSession, col: str, format_version: int) -> None: +def test_query_filter_without_data(spark: SparkSession, arrow_table_with_null: pa.Table, format_version: int) -> None: identifier = f"default.arrow_table_v{format_version}_without_data" df = spark.table(identifier) - assert df.where(f"{col} is null").count() == 0, f"Expected 0 row for {col}" - assert df.where(f"{col} is not null").count() == 0, f"Expected 0 row for {col}" + for col in arrow_table_with_null.column_names: + assert df.where(f"{col} is null").count() == 0, f"Expected 0 row for {col}" + assert df.where(f"{col} is not null").count() == 0, f"Expected 0 row for {col}" @pytest.mark.integration -@pytest.mark.parametrize("col", TEST_DATA_WITH_NULL.keys()) @pytest.mark.parametrize("format_version", [1, 2]) -def test_query_filter_only_nulls(spark: SparkSession, col: str, format_version: int) -> None: +def test_query_filter_only_nulls(spark: SparkSession, arrow_table_with_null: pa.Table, format_version: int) -> None: identifier = f"default.arrow_table_v{format_version}_with_only_nulls" df = spark.table(identifier) - assert df.where(f"{col} is null").count() == 2, f"Expected 2 rows for {col}" - assert df.where(f"{col} is not null").count() == 0, f"Expected 0 row for {col}" + for col in arrow_table_with_null.column_names: + assert df.where(f"{col} is null").count() == 2, f"Expected 2 rows for {col}" + assert df.where(f"{col} is not null").count() == 0, f"Expected 0 row for {col}" @pytest.mark.integration -@pytest.mark.parametrize("col", TEST_DATA_WITH_NULL.keys()) @pytest.mark.parametrize("format_version", [1, 2]) -def test_query_filter_appended_null(spark: SparkSession, col: str, format_version: int) -> None: +def test_query_filter_appended_null(spark: SparkSession, arrow_table_with_null: pa.Table, format_version: int) -> None: identifier = f"default.arrow_table_v{format_version}_appended_with_null" df = spark.table(identifier) - assert df.where(f"{col} is null").count() == 2, f"Expected 1 row for {col}" - assert df.where(f"{col} is not null").count() == 4, f"Expected 2 rows for {col}" + for col in arrow_table_with_null.column_names: + assert df.where(f"{col} is null").count() == 2, f"Expected 1 row for {col}" + assert df.where(f"{col} is not null").count() == 4, f"Expected 2 rows for {col}" @pytest.mark.integration -@pytest.mark.parametrize("col", TEST_DATA_WITH_NULL.keys()) -def test_query_filter_v1_v2_append_null(spark: SparkSession, col: str) -> None: +def test_query_filter_v1_v2_append_null( + spark: SparkSession, + arrow_table_with_null: pa.Table, +) -> None: identifier = "default.arrow_table_v1_v2_appended_with_null" df = spark.table(identifier) - assert df.where(f"{col} is null").count() == 2, f"Expected 1 row for {col}" - assert df.where(f"{col} is not null").count() == 4, f"Expected 2 rows for {col}" + for col in arrow_table_with_null.column_names: + assert df.where(f"{col} is null").count() == 2, f"Expected 1 row for {col}" + assert df.where(f"{col} is not null").count() == 4, f"Expected 2 rows for {col}" @pytest.mark.integration From 9c6724e48ba684114c535746839a4f5be1ad7285 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 17 Apr 2024 23:39:55 +0200 Subject: [PATCH 12/27] WIP --- pyiceberg/table/__init__.py | 6 +++--- tests/integration/test_deletes.py | 21 ++++----------------- 2 files changed, 7 insertions(+), 20 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 448242aafa..3cbcd830c5 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -439,9 +439,6 @@ def overwrite( if not isinstance(df, pa.Table): raise ValueError(f"Expected PyArrow table, got: {df}") - if overwrite_filter != AlwaysTrue(): - raise NotImplementedError("Cannot overwrite a subset of a table") - if len(self._table.spec().fields) > 0: raise ValueError("Cannot write to partitioned tables") @@ -451,6 +448,9 @@ def overwrite( if table_arrow_schema != df.schema: df = df.cast(table_arrow_schema) + with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: + delete_snapshot.delete_by_predicate(overwrite_filter) + with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as update_snapshot: # skip writing data files if the dataframe is empty if df.shape[0] > 0: diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index 47086a48c7..ebfab3a04e 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -227,33 +227,20 @@ def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSessio # One positional delete has been added, but an OVERWRITE status is set # https://github.com/apache/iceberg/issues/10122 snapshots = tbl.snapshots() - assert len(snapshots) == 4 + assert len(snapshots) == 3 # Snapshots produced by Spark assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()[0:2]] == ['append', 'overwrite'] - # Snapshots produced by PyIceberg - # This is a no-op since nothing has been added or deleted (because the predicate cannot drop a whole file) - assert tbl.snapshots()[2].summary == Summary( - Operation.DELETE, - **{ - 'total-data-files': '2', - 'total-delete-files': '1', - 'total-records': '5', - 'total-files-size': tbl.snapshots()[2].summary['total-files-size'], - 'total-position-deletes': '1', - 'total-equality-deletes': '0', - }, - ) # Will rewrite one parquet file - assert tbl.snapshots()[3].summary == Summary( + assert snapshots[2].summary == Summary( Operation.OVERWRITE, **{ 'added-files-size': '1145', 'added-data-files': '1', 'added-records': '2', 'changed-partition-count': '1', - 'total-files-size': tbl.snapshots()[3].summary['total-files-size'], + 'total-files-size': snapshots[2].summary['total-files-size'], 'total-delete-files': '0', 'total-data-files': '1', 'total-position-deletes': '0', @@ -262,7 +249,7 @@ def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSessio 'deleted-data-files': '2', 'removed-delete-files': '1', 'deleted-records': '5', - 'removed-files-size': '3088', + 'removed-files-size': snapshots[2].summary['removed-files-size'], 'removed-position-deletes': '1', }, ) From 7dae07191bed2dfe6045b5b8dd7b2f3078493d02 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 18 Apr 2024 15:29:06 +0200 Subject: [PATCH 13/27] Fix summary generation --- pyiceberg/table/__init__.py | 11 +++++++++++ tests/integration/test_inspect_table.py | 18 ++++++++++++++++-- 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 3cbcd830c5..626cea5227 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -2888,6 +2888,15 @@ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary: schema=self._transaction.table_metadata.schema(), ) + if len(self._deleted_data_files) > 0: + specs = self._transaction.table_metadata.specs() + for data_file in self._deleted_data_files: + ssc.remove_file( + data_file=data_file, + partition_spec=specs.get(data_file.spec_id), + schema=self._transaction.table_metadata.schema(), + ) + previous_snapshot = ( self._transaction.table_metadata.snapshot_by_id(self._parent_snapshot_id) if self._parent_snapshot_id is not None @@ -3028,6 +3037,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> existing_manifests = [] 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: @@ -3041,6 +3051,7 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> for entry in manifest_file.fetch_manifest_entry(io=self._io, discard_deleted=True): if strict_metrics_evaluator(entry.data_file) == ROWS_MUST_MATCH: deleted_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.DELETED)) + self._deleted_data_files.add(entry.data_file) elif inclusive_metrics_evaluator(entry.data_file) == ROWS_CANNOT_MATCH: existing_entries.append(_copy_with_new_status(entry, ManifestEntryStatus.EXISTING)) else: diff --git a/tests/integration/test_inspect_table.py b/tests/integration/test_inspect_table.py index a884f9d4c0..e25e3f117d 100644 --- a/tests/integration/test_inspect_table.py +++ b/tests/integration/test_inspect_table.py @@ -103,13 +103,14 @@ def test_inspect_snapshots( assert isinstance(snapshot_id.as_py(), int) assert df['parent_id'][0].as_py() is None - assert df['parent_id'][1:] == df['snapshot_id'][:2] + assert df['parent_id'][1:].to_pylist() == df['snapshot_id'][:-1].to_pylist() - assert [operation.as_py() for operation in df['operation']] == ['append', 'overwrite', 'append'] + assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'overwrite', 'append'] for manifest_list in df['manifest_list']: assert manifest_list.as_py().startswith("s3://") + # Append assert df['summary'][0].as_py() == [ ('added-files-size', '5459'), ('added-data-files', '1'), @@ -122,6 +123,19 @@ def test_inspect_snapshots( ('total-equality-deletes', '0'), ] + # Delete + assert df['summary'][1].as_py() == [ + ('removed-files-size', '5459'), + ('deleted-data-files', '1'), + ('deleted-records', '3'), + ('total-data-files', '0'), + ('total-delete-files', '0'), + ('total-records', '0'), + ('total-files-size', '0'), + ('total-position-deletes', '0'), + ('total-equality-deletes', '0'), + ] + lhs = spark.table(f"{identifier}.snapshots").toPandas() rhs = df.to_pandas() for column in df.column_names: From 5e871fba037f5f4e62234631a019b1247c4b286a Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 18 Apr 2024 16:34:47 +0200 Subject: [PATCH 14/27] Last few bits --- pyiceberg/io/pyarrow.py | 5 +- pyiceberg/manifest.py | 2 +- pyiceberg/table/__init__.py | 18 +++--- tests/integration/test_writes/test_writes.py | 62 ++++++++++++++++---- 4 files changed, 60 insertions(+), 27 deletions(-) diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py index 46d0bd93e8..a7d0ea0d38 100644 --- a/pyiceberg/io/pyarrow.py +++ b/pyiceberg/io/pyarrow.py @@ -1772,12 +1772,9 @@ def data_file_statistics_from_parquet_metadata( ) -def write_file(io: FileIO, table_metadata: TableMetadata, tasks: Iterable["WriteTask"]) -> Iterator[DataFile]: +def write_file(io: FileIO, table_metadata: TableMetadata, tasks: Iterator[WriteTask]) -> Iterator[DataFile]: from pyiceberg.table import PropertyUtil, TableProperties - schema = table_metadata.schema() - arrow_file_schema = schema.as_arrow() - parquet_writer_kwargs = _get_parquet_writer_kwargs(table_metadata.properties) row_group_size = PropertyUtil.property_as_int( properties=table_metadata.properties, diff --git a/pyiceberg/manifest.py b/pyiceberg/manifest.py index 3b8138b61a..19c9452223 100644 --- a/pyiceberg/manifest.py +++ b/pyiceberg/manifest.py @@ -338,7 +338,7 @@ class DataFile(Record): split_offsets: Optional[List[int]] equality_ids: Optional[List[int]] sort_order_id: Optional[int] - spec_id: Optional[int] + spec_id: int def __setattr__(self, name: str, value: Any) -> None: """Assign a key/value to a DataFile.""" diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 626cea5227..abdc62151a 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -465,10 +465,10 @@ def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str warnings.warn("PyIceberg only supports copy on write") with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: - delete_snapshot.delete_by_predicate(delete_filter) # type: ignore + delete_snapshot.delete_by_predicate(delete_filter) # Check if there are any files that require an actual rewrite of a data file - if delete_snapshot.rewrites_needed is True: # type: ignore + if delete_snapshot.rewrites_needed is True: # When we want to filter out certain rows, we want to invert the expression # delete id = 22 means that we want to look for that value, and then remove # if from the Parquet file @@ -2767,7 +2767,7 @@ def _parquet_files_to_data_files(table_metadata: TableMetadata, file_paths: List yield from parquet_files_to_data_files(io=io, table_metadata=table_metadata, file_paths=iter(file_paths)) -class _MergingSnapshotProducer(UpdateTableMetadata["_MergingSnapshotProducer"]): +class _MergingSnapshotProducer(UpdateTableMetadata[U], Generic[U]): commit_uuid: uuid.UUID _operation: Operation _snapshot_id: int @@ -2798,11 +2798,11 @@ def __init__( self.snapshot_properties = snapshot_properties self._manifest_counter = itertools.count(0) - def append_data_file(self, data_file: DataFile) -> _MergingSnapshotProducer: + def append_data_file(self, data_file: DataFile) -> _MergingSnapshotProducer[U]: self._added_data_files.append(data_file) return self - def delete_data_file(self, data_file: DataFile) -> _MergingSnapshotProducer: + def delete_data_file(self, data_file: DataFile) -> _MergingSnapshotProducer[U]: self._deleted_data_files.add(data_file) return self @@ -2893,7 +2893,7 @@ def _summary(self, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> Summary: for data_file in self._deleted_data_files: ssc.remove_file( data_file=data_file, - partition_spec=specs.get(data_file.spec_id), + partition_spec=specs[data_file.spec_id], schema=self._transaction.table_metadata.schema(), ) @@ -2953,7 +2953,7 @@ def _commit(self) -> UpdatesAndRequirements: ) -class DeleteFiles(_MergingSnapshotProducer): +class DeleteFiles(_MergingSnapshotProducer["DeleteFiles"]): """Will delete manifest entries from the current snapshot based on the predicate. This will produce a DELETE snapshot: @@ -3102,7 +3102,7 @@ def files_affected(self) -> bool: return len(self._deleted_entries()) > 0 -class FastAppendFiles(_MergingSnapshotProducer): +class FastAppendFiles(_MergingSnapshotProducer["FastAppendFiles"]): def _existing_manifests(self) -> List[ManifestFile]: """To determine if there are any existing manifest files. @@ -3131,7 +3131,7 @@ def _deleted_entries(self) -> List[ManifestEntry]: return [] -class OverwriteFiles(_MergingSnapshotProducer): +class OverwriteFiles(_MergingSnapshotProducer["OverwriteFiles"]): """Overwrites data from the table. This will produce an OVERWRITE snapshot. Data and delete files were added and removed in a logical overwrite operation. diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py index c06ec37597..a9bc15e42e 100644 --- a/tests/integration/test_writes/test_writes.py +++ b/tests/integration/test_writes/test_writes.py @@ -185,10 +185,11 @@ def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_wi ).collect() operations = [row.operation for row in rows] - assert operations == ['append', 'append', 'overwrite'] + assert operations == ['append', 'append', 'delete', 'overwrite'] summaries = [row.summary for row in rows] + # Append assert summaries[0] == { 'added-data-files': '1', 'added-files-size': '5459', @@ -201,6 +202,7 @@ def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_wi 'total-records': '3', } + # Append assert summaries[1] == { 'added-data-files': '1', 'added-files-size': '5459', @@ -213,13 +215,24 @@ def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_wi 'total-records': '6', } + # Delete assert summaries[2] == { - 'added-data-files': '1', - 'added-files-size': '5459', - 'added-records': '3', 'deleted-data-files': '2', 'deleted-records': '6', 'removed-files-size': '10918', + 'total-data-files': '0', + 'total-delete-files': '0', + 'total-equality-deletes': '0', + 'total-files-size': '0', + 'total-position-deletes': '0', + 'total-records': '0', + } + + # Overwrite + assert summaries[3] == { + 'added-data-files': '1', + 'added-files-size': '5459', + 'added-records': '3', 'total-data-files': '1', 'total-delete-files': '0', 'total-equality-deletes': '0', @@ -247,9 +260,9 @@ def test_data_files(spark: SparkSession, session_catalog: Catalog, arrow_table_w """ ).collect() - assert [row.added_data_files_count for row in rows] == [1, 1, 0, 1, 1] - assert [row.existing_data_files_count for row in rows] == [0, 0, 0, 0, 0] - assert [row.deleted_data_files_count for row in rows] == [0, 0, 1, 0, 0] + assert [row.added_data_files_count for row in rows] == [1, 0, 1, 0, 1, 1] + assert [row.existing_data_files_count for row in rows] == [0, 0, 0, 0, 0, 0] + assert [row.deleted_data_files_count for row in rows] == [0, 1, 0, 1, 0, 0] @pytest.mark.integration @@ -476,7 +489,7 @@ def test_summaries_with_only_nulls( ).collect() operations = [row.operation for row in rows] - assert operations == ['append', 'append', 'overwrite'] + assert operations == ['append', 'append', 'delete', 'overwrite'] summaries = [row.summary for row in rows] @@ -502,14 +515,23 @@ def test_summaries_with_only_nulls( } assert summaries[2] == { + 'deleted-data-files': '1', + 'deleted-records': '2', 'removed-files-size': '4239', + 'total-data-files': '0', + 'total-delete-files': '0', 'total-equality-deletes': '0', + 'total-files-size': '0', 'total-position-deletes': '0', - 'deleted-data-files': '1', + 'total-records': '0', + } + + assert summaries[3] == { + 'total-data-files': '0', 'total-delete-files': '0', + 'total-equality-deletes': '0', 'total-files-size': '0', - 'deleted-records': '2', - 'total-data-files': '0', + 'total-position-deletes': '0', 'total-records': '0', } @@ -731,13 +753,14 @@ def test_inspect_snapshots( assert isinstance(snapshot_id.as_py(), int) assert df['parent_id'][0].as_py() is None - assert df['parent_id'][1:] == df['snapshot_id'][:2] + assert df['parent_id'][1:].to_pylist() == df['snapshot_id'][:-1].to_pylist() - assert [operation.as_py() for operation in df['operation']] == ['append', 'overwrite', 'append'] + assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'overwrite', 'append'] for manifest_list in df['manifest_list']: assert manifest_list.as_py().startswith("s3://") + # Append assert df['summary'][0].as_py() == [ ('added-files-size', '5459'), ('added-data-files', '1'), @@ -750,6 +773,19 @@ def test_inspect_snapshots( ('total-equality-deletes', '0'), ] + # Delete + assert df['summary'][1].as_py() == [ + ('removed-files-size', '5459'), + ('deleted-data-files', '1'), + ('deleted-records', '3'), + ('total-data-files', '0'), + ('total-delete-files', '0'), + ('total-records', '0'), + ('total-files-size', '0'), + ('total-position-deletes', '0'), + ('total-equality-deletes', '0'), + ] + lhs = spark.table(f"{identifier}.snapshots").toPandas() rhs = df.to_pandas() for column in df.column_names: From edfadd9621b775288b5863113b6a552f06a01c52 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 25 Apr 2024 12:22:08 +0200 Subject: [PATCH 15/27] Fix the requirement --- pyiceberg/table/__init__.py | 16 ++++++++++++---- tests/integration/test_rest_schema.py | 19 +++++++++++-------- 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index abdc62151a..06068e3236 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -243,6 +243,8 @@ class TableProperties: WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT = 0 DELETE_MODE = "write.delete.mode" + DELETE_MODE_COPY_ON_WRITE = "merge-on-read" + DELETE_MODE_MERGE_ON_READ = "copy-on-write" DEFAULT_NAME_MAPPING = "schema.name-mapping.default" FORMAT_VERSION = "format-version" @@ -301,7 +303,13 @@ def _apply(self, updates: Tuple[TableUpdate, ...], requirements: Tuple[TableRequ requirement.validate(self.table_metadata) self._updates += updates - self._requirements += requirements + + # For the requirements, it does not make sense to add a requirement more than once + # For example, you cannot assert that the current schema has two different IDs + existing_requirements = {type(requirement) for requirement in self._requirements} + for new_requirement in requirements: + if type(new_requirement) not in existing_requirements: + self._requirements = self._requirements + requirements self.table_metadata = update_table_metadata(self.table_metadata, updates) @@ -461,8 +469,8 @@ def overwrite( update_snapshot.append_data_file(data_file) def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: - if (mode := self.table_metadata.properties.get(TableProperties.DELETE_MODE)) and mode != 'copy-on-write': - warnings.warn("PyIceberg only supports copy on write") + if self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_COPY_ON_WRITE) == TableProperties.DELETE_MODE_MERGE_ON_READ: + raise NotImplementedError("Merge on read is not yet supported") with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: delete_snapshot.delete_by_predicate(delete_filter) @@ -2948,7 +2956,7 @@ def _commit(self) -> UpdatesAndRequirements: ), ( AssertTableUUID(uuid=self._transaction.table_metadata.table_uuid), - # AssertRefSnapshotId(snapshot_id=self._parent_snapshot_id, ref="main"), + AssertRefSnapshotId(snapshot_id=self._transaction.table_metadata.current_snapshot_id, ref="main"), ), ) diff --git a/tests/integration/test_rest_schema.py b/tests/integration/test_rest_schema.py index ac5d1ce050..2aaa31652a 100644 --- a/tests/integration/test_rest_schema.py +++ b/tests/integration/test_rest_schema.py @@ -2512,15 +2512,18 @@ def test_two_add_schemas_in_a_single_transaction(catalog: Catalog) -> None: ), ) - with pytest.raises(CommitFailedException) as exc_info: - with tbl.transaction() as tr: - with tr.update_schema() as update: - update.add_column("bar", field_type=StringType()) - with tr.update_schema() as update: - update.add_column("baz", field_type=StringType()) - - assert "CommitFailedException: Requirement failed: current schema changed: expected id 1 != 0" in str(exc_info.value) + with tbl.transaction() as tr: + with tr.update_schema() as update: + update.add_column("bar", field_type=StringType()) + with tr.update_schema() as update: + update.add_column("baz", field_type=StringType()) + assert tbl.schema().schema_id == 2 + assert tbl.schema() == Schema( + NestedField(field_id=1, name="foo", field_type=StringType()), + NestedField(field_id=2, name="bar", field_type=StringType()), + NestedField(field_id=3, name="baz", field_type=StringType()), + ) @pytest.mark.integration def test_create_table_integrity_after_fresh_assignment(catalog: Catalog) -> None: From d65a8a42c4c3dc1d12c7bc146d19a7831cb752e8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 25 Apr 2024 12:29:49 +0200 Subject: [PATCH 16/27] Make ruff happy --- pyiceberg/table/__init__.py | 5 ++++- tests/integration/test_rest_schema.py | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 06068e3236..63bf4ab687 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -469,7 +469,10 @@ def overwrite( update_snapshot.append_data_file(data_file) def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: - if self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_COPY_ON_WRITE) == TableProperties.DELETE_MODE_MERGE_ON_READ: + if ( + self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_COPY_ON_WRITE) + == TableProperties.DELETE_MODE_MERGE_ON_READ + ): raise NotImplementedError("Merge on read is not yet supported") with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: diff --git a/tests/integration/test_rest_schema.py b/tests/integration/test_rest_schema.py index 2aaa31652a..eeb1e30ae6 100644 --- a/tests/integration/test_rest_schema.py +++ b/tests/integration/test_rest_schema.py @@ -2525,6 +2525,7 @@ def test_two_add_schemas_in_a_single_transaction(catalog: Catalog) -> None: NestedField(field_id=3, name="baz", field_type=StringType()), ) + @pytest.mark.integration def test_create_table_integrity_after_fresh_assignment(catalog: Catalog) -> None: schema = Schema( From 3c98eef788b883af8bc3cc1831c1fbb1161d1ab3 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 2 May 2024 22:51:38 +0200 Subject: [PATCH 17/27] Comments, thanks Kevin! --- pyiceberg/table/__init__.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 63bf4ab687..a358f17216 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -243,8 +243,8 @@ class TableProperties: WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT = 0 DELETE_MODE = "write.delete.mode" - DELETE_MODE_COPY_ON_WRITE = "merge-on-read" - DELETE_MODE_MERGE_ON_READ = "copy-on-write" + DELETE_MODE_COPY_ON_WRITE = "copy-on-write" + DELETE_MODE_MERGE_ON_READ = "merge-on-read" DEFAULT_NAME_MAPPING = "schema.name-mapping.default" FORMAT_VERSION = "format-version" @@ -310,6 +310,8 @@ def _apply(self, updates: Tuple[TableUpdate, ...], requirements: Tuple[TableRequ for new_requirement in requirements: if type(new_requirement) not in existing_requirements: self._requirements = self._requirements + requirements + else: + warnings.warn(f"Dropped duplicate requirement: {new_requirement}") self.table_metadata = update_table_metadata(self.table_metadata, updates) From 179fa2792406321a92f10de09b4c1107ad3d4478 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 9 May 2024 10:53:24 -0600 Subject: [PATCH 18/27] Comments --- pyiceberg/table/__init__.py | 124 +++++++++++++++++------------- tests/integration/test_deletes.py | 84 +++++++++++++++++++- 2 files changed, 154 insertions(+), 54 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index a358f17216..f9a29dccc5 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -64,12 +64,13 @@ ROWS_MUST_MATCH, _InclusiveMetricsEvaluator, _StrictMetricsEvaluator, + bind, expression_evaluator, inclusive_projection, manifest_evaluator, ) from pyiceberg.io import FileIO, load_file_io -from pyiceberg.io.pyarrow import _dataframe_to_data_files, project_table +from pyiceberg.io.pyarrow import _dataframe_to_data_files, expression_to_pyarrow, project_table from pyiceberg.manifest import ( POSITIONAL_DELETE_SCHEMA, DataFile, @@ -310,8 +311,6 @@ def _apply(self, updates: Tuple[TableUpdate, ...], requirements: Tuple[TableRequ for new_requirement in requirements: if type(new_requirement) not in existing_requirements: self._requirements = self._requirements + requirements - else: - warnings.warn(f"Dropped duplicate requirement: {new_requirement}") self.table_metadata = update_table_metadata(self.table_metadata, updates) @@ -430,7 +429,10 @@ def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT) update_snapshot.append_data_file(data_file) def overwrite( - self, df: pa.Table, overwrite_filter: BooleanExpression = ALWAYS_TRUE, snapshot_properties: Dict[str, str] = EMPTY_DICT + self, + df: pa.Table, + overwrite_filter: Union[BooleanExpression, str] = ALWAYS_TRUE, + snapshot_properties: Dict[str, str] = EMPTY_DICT, ) -> None: """ Shorthand for adding a table overwrite with a PyArrow table to the transaction. @@ -458,8 +460,7 @@ def overwrite( if table_arrow_schema != df.schema: df = df.cast(table_arrow_schema) - with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: - delete_snapshot.delete_by_predicate(overwrite_filter) + self.delete(delete_filter=overwrite_filter, snapshot_properties=snapshot_properties) with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as update_snapshot: # skip writing data files if the dataframe is empty @@ -470,53 +471,73 @@ def overwrite( for data_file in data_files: update_snapshot.append_data_file(data_file) - def delete(self, delete_filter: BooleanExpression, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: + def delete(self, delete_filter: Union[str, BooleanExpression], snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: if ( self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_COPY_ON_WRITE) == TableProperties.DELETE_MODE_MERGE_ON_READ ): raise NotImplementedError("Merge on read is not yet supported") + if isinstance(delete_filter, str): + delete_filter = _parse_row_filter(delete_filter) + with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot: delete_snapshot.delete_by_predicate(delete_filter) # Check if there are any files that require an actual rewrite of a data file if delete_snapshot.rewrites_needed is True: - # When we want to filter out certain rows, we want to invert the expression - # delete id = 22 means that we want to look for that value, and then remove - # if from the Parquet file - delete_row_filter = Not(delete_filter) - with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as overwrite_snapshot: - # Potential optimization is where we check if the files actually contain relevant data. - files = self._scan(row_filter=delete_filter).plan_files() - - counter = itertools.count(0) - - # This will load the Parquet file into memory, including: - # - Filter out the rows based on the delete filter - # - Projecting it to the current schema - # - Applying the positional deletes if they are there - # When writing - # - Apply the latest partition-spec - # - And sort order when added - for original_file in files: - df = project_table( - tasks=[original_file], - table_metadata=self._table.metadata, - io=self._table.io, - row_filter=delete_row_filter, - projected_schema=self.table_metadata.schema(), - ) - for data_file in _dataframe_to_data_files( - io=self._table.io, - df=df, - table_metadata=self._table.metadata, - write_uuid=overwrite_snapshot.commit_uuid, - counter=counter, - ): - overwrite_snapshot.append_data_file(data_file) + bound_delete_filter = bind(self._table.schema(), delete_filter, case_sensitive=True) + preserve_row_filter = expression_to_pyarrow(Not(bound_delete_filter)) + commit_uuid = uuid.uuid4() + + files = self._scan(row_filter=delete_filter).plan_files() + + counter = itertools.count(0) + + replaced_files: List[Tuple[DataFile, List[DataFile]]] = [] + # This will load the Parquet file into memory, including: + # - Filter out the rows based on the delete filter + # - Projecting it to the current schema + # - Applying the positional deletes if they are there + # When writing + # - Apply the latest partition-spec + # - And sort order when added + for original_file in files: + df = project_table( + tasks=[original_file], + table_metadata=self._table.metadata, + io=self._table.io, + row_filter=AlwaysTrue(), + projected_schema=self.table_metadata.schema(), + ) + filtered_df = df.filter(preserve_row_filter) + + # Only rewrite if there are records being deleted + if len(df) != len(filtered_df): + replaced_files.append(( + original_file.file, + list( + _dataframe_to_data_files( + io=self._table.io, + df=filtered_df, + table_metadata=self._table.metadata, + write_uuid=commit_uuid, + counter=counter, + ) + ), + )) - overwrite_snapshot.delete_data_file(original_file.file) + if len(replaced_files) > 0: + with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite( + commit_uuid=commit_uuid + ) as overwrite_snapshot: + for original_data_file, replaced_data_files in replaced_files: + overwrite_snapshot.delete_data_file(original_data_file) + for replaced_data_file in replaced_data_files: + overwrite_snapshot.append_data_file(replaced_data_file) + + if not delete_snapshot.files_affected and not delete_snapshot.rewrites_needed: + warnings.warn("Delete operation did not match any records") def add_files(self, file_paths: List[str]) -> None: """ @@ -1405,7 +1426,10 @@ def append(self, df: pa.Table, snapshot_properties: Dict[str, str] = EMPTY_DICT) tx.append(df=df, snapshot_properties=snapshot_properties) def overwrite( - self, df: pa.Table, overwrite_filter: BooleanExpression = ALWAYS_TRUE, snapshot_properties: Dict[str, str] = EMPTY_DICT + self, + df: pa.Table, + overwrite_filter: Union[BooleanExpression, str] = ALWAYS_TRUE, + snapshot_properties: Dict[str, str] = EMPTY_DICT, ) -> None: """ Shorthand for overwriting the table with a PyArrow table. @@ -1419,7 +1443,9 @@ def overwrite( with self.transaction() as tx: tx.overwrite(df=df, overwrite_filter=overwrite_filter, snapshot_properties=snapshot_properties) - def delete(self, delete_filter: BooleanExpression = ALWAYS_TRUE, snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: + def delete( + self, delete_filter: Union[BooleanExpression, str] = ALWAYS_TRUE, snapshot_properties: Dict[str, str] = EMPTY_DICT + ) -> None: """ Shorthand for deleting rows from the table. @@ -3011,15 +3037,6 @@ def _build_manifest_evaluator(self, spec_id: int) -> Callable[[ManifestFile], bo spec = self._transaction.table_metadata.specs()[spec_id] return manifest_evaluator(spec, schema, self.partition_filters[spec_id], case_sensitive=True) - def _build_partition_evaluator(self, spec_id: int) -> Callable[[DataFile], bool]: - schema = self._transaction.table_metadata.schema() - spec = self._transaction.table_metadata.specs()[spec_id] - partition_type = spec.partition_type(schema) - partition_schema = Schema(*partition_type.fields) - partition_expr = self.partition_filters[spec_id] - - return lambda data_file: expression_evaluator(partition_schema, partition_expr, case_sensitive=True)(data_file.partition) - def delete_by_predicate(self, predicate: BooleanExpression) -> None: self._predicate = Or(self._predicate, predicate) @@ -3240,8 +3257,9 @@ def fast_append(self) -> FastAppendFiles: operation=Operation.APPEND, transaction=self._transaction, io=self._io, snapshot_properties=self._snapshot_properties ) - def overwrite(self) -> OverwriteFiles: + 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 else Operation.APPEND, diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index ebfab3a04e..a781415ca2 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -17,12 +17,16 @@ # pylint:disable=redefined-outer-name from typing import List +import pyarrow as pa import pytest from pyspark.sql import SparkSession from pyiceberg.catalog.rest import RestCatalog +from pyiceberg.exceptions import NoSuchTableError from pyiceberg.expressions import EqualTo +from pyiceberg.schema import Schema from pyiceberg.table.snapshots import Operation, Summary +from pyiceberg.types import IntegerType, NestedField def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None: @@ -127,7 +131,6 @@ def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCa tbl = session_catalog.load_table(identifier) tbl.delete(EqualTo("number_partitioned", 22)) # Does not affect any data - # Open for discussion, do we want to create a new snapshot? assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append'] assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10, 10], 'number': [20, 30]} @@ -255,3 +258,82 @@ def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSessio ) assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [20, 20, 10], 'number': [200, 202, 100]} + + +@pytest.mark.integration +def test_delete_no_match(session_catalog: RestCatalog) -> None: + arrow_schema = pa.schema([pa.field("ints", pa.int32())]) + arrow_tbl = pa.Table.from_pylist( + [ + { + 'ints': 1, + }, + {'ints': 3}, + ], + schema=arrow_schema, + ) + + iceberg_schema = Schema(NestedField(1, "ints", IntegerType())) + + tbl_identifier = "default.test_delete_no_match" + + try: + session_catalog.drop_table(tbl_identifier) + except NoSuchTableError: + pass + + tbl = session_catalog.create_table(tbl_identifier, iceberg_schema) + tbl.append(arrow_tbl) + + assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [Operation.APPEND] + + tbl.delete('ints == 2') # Only 1 and 3 in the file, but is between the lower and upper bound + + assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [Operation.APPEND] + + +@pytest.mark.integration +def test_delete_overwrite(session_catalog: RestCatalog) -> None: + arrow_schema = pa.schema([pa.field("ints", pa.int32())]) + arrow_tbl = pa.Table.from_pylist( + [ + { + 'ints': 1, + }, + {'ints': 2}, + ], + schema=arrow_schema, + ) + + iceberg_schema = Schema(NestedField(1, "ints", IntegerType())) + + tbl_identifier = "default.test_delete_overwrite" + + try: + session_catalog.drop_table(tbl_identifier) + except NoSuchTableError: + pass + + tbl = session_catalog.create_table(tbl_identifier, iceberg_schema) + tbl.append(arrow_tbl) + + assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [Operation.APPEND] + + arrow_tbl_overwrite = pa.Table.from_pylist( + [ + { + 'ints': 3, + }, + {'ints': 4}, + ], + schema=arrow_schema, + ) + tbl.overwrite(arrow_tbl_overwrite, 'ints == 2') # Should rewrite one file + + assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [ + Operation.APPEND, + Operation.OVERWRITE, + Operation.OVERWRITE, + ] + + assert tbl.scan().to_arrow()['ints'] == [[3, 4], [1]] From 2ea157ea59c36154c292290210d182daa89e34e8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 10 May 2024 21:26:05 +0200 Subject: [PATCH 19/27] Append rather than truncate --- pyiceberg/table/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index f9a29dccc5..d917caf806 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -462,7 +462,7 @@ def overwrite( self.delete(delete_filter=overwrite_filter, snapshot_properties=snapshot_properties) - with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as update_snapshot: + with self.update_snapshot(snapshot_properties=snapshot_properties).fast_append() as update_snapshot: # skip writing data files if the dataframe is empty if df.shape[0] > 0: data_files = _dataframe_to_data_files( From 7d036b1b81c9ce194c76cb93dbb763b2b4f711b6 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 10 May 2024 21:29:49 +0200 Subject: [PATCH 20/27] Fix merge conflicts --- pyiceberg/table/__init__.py | 6 +----- tests/integration/test_deletes.py | 2 +- tests/integration/test_writes/test_writes.py | 19 +++++++------------ 3 files changed, 9 insertions(+), 18 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 5ede4b32c4..dbf79bd85f 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -476,7 +476,7 @@ def delete(self, delete_filter: Union[str, BooleanExpression], snapshot_properti self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_COPY_ON_WRITE) == TableProperties.DELETE_MODE_MERGE_ON_READ ): - raise NotImplementedError("Merge on read is not yet supported") + warnings.warn("Merge on read is not yet supported, falling back to copy-on-write") if isinstance(delete_filter, str): delete_filter = _parse_row_filter(delete_filter) @@ -1443,7 +1443,6 @@ def overwrite( with self.transaction() as tx: tx.overwrite(df=df, overwrite_filter=overwrite_filter, snapshot_properties=snapshot_properties) -<<<<<<< HEAD def delete( self, delete_filter: Union[BooleanExpression, str] = ALWAYS_TRUE, snapshot_properties: Dict[str, str] = EMPTY_DICT ) -> None: @@ -1457,10 +1456,7 @@ def delete( with self.transaction() as tx: tx.delete(delete_filter=delete_filter, snapshot_properties=snapshot_properties) - def add_files(self, file_paths: List[str]) -> None: -======= def add_files(self, file_paths: List[str], snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: ->>>>>>> aa361d1485f4a914bc0bbc2e574becaec9a773ac """ Shorthand API for adding files as data files to the table. diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index a781415ca2..15acd3fcd6 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -333,7 +333,7 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [ Operation.APPEND, Operation.OVERWRITE, - Operation.OVERWRITE, + Operation.APPEND, ] assert tbl.scan().to_arrow()['ints'] == [[3, 4], [1]] diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py index 62b7fd57f7..3a14300e24 100644 --- a/tests/integration/test_writes/test_writes.py +++ b/tests/integration/test_writes/test_writes.py @@ -36,17 +36,12 @@ from pyiceberg.catalog.hive import HiveCatalog from pyiceberg.catalog.sql import SqlCatalog from pyiceberg.exceptions import NoSuchTableError -<<<<<<< HEAD from pyiceberg.io.pyarrow import _dataframe_to_data_files -from pyiceberg.table import TableProperties -======= from pyiceberg.partitioning import PartitionField, PartitionSpec from pyiceberg.schema import Schema -from pyiceberg.table import TableProperties, _dataframe_to_data_files +from pyiceberg.table import TableProperties from pyiceberg.transforms import IdentityTransform from pyiceberg.types import IntegerType, NestedField -from tests.conftest import TEST_DATA_WITH_NULL ->>>>>>> aa361d1485f4a914bc0bbc2e574becaec9a773ac from utils import _create_table @@ -194,7 +189,7 @@ def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_wi ).collect() operations = [row.operation for row in rows] - assert operations == ['append', 'append', 'delete', 'overwrite'] + assert operations == ['append', 'append', 'delete', 'append'] summaries = [row.summary for row in rows] @@ -269,9 +264,9 @@ def test_data_files(spark: SparkSession, session_catalog: Catalog, arrow_table_w """ ).collect() - assert [row.added_data_files_count for row in rows] == [1, 0, 1, 0, 1, 1] - assert [row.existing_data_files_count for row in rows] == [0, 0, 0, 0, 0, 0] - assert [row.deleted_data_files_count for row in rows] == [0, 1, 0, 1, 0, 0] + assert [row.added_data_files_count for row in rows] == [1, 0, 1, 1, 1] + assert [row.existing_data_files_count for row in rows] == [0, 0, 0, 0, 0] + assert [row.deleted_data_files_count for row in rows] == [0, 1, 0, 0, 0] @pytest.mark.integration @@ -522,7 +517,7 @@ def test_summaries_with_only_nulls( ).collect() operations = [row.operation for row in rows] - assert operations == ['append', 'append', 'delete', 'overwrite'] + assert operations == ['append', 'append', 'delete', 'append'] summaries = [row.summary for row in rows] @@ -788,7 +783,7 @@ def test_inspect_snapshots( assert df['parent_id'][0].as_py() is None assert df['parent_id'][1:].to_pylist() == df['snapshot_id'][:-1].to_pylist() - assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'overwrite', 'append'] + assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'append', 'append'] for manifest_list in df['manifest_list']: assert manifest_list.as_py().startswith("s3://") From 5adf3f0bbe4de03482bf3c5968f04e8b447e6864 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 13 May 2024 14:27:31 +0200 Subject: [PATCH 21/27] Make the tests pass --- tests/integration/test_deletes.py | 2 +- tests/integration/test_inspect_table.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index 15acd3fcd6..85e76125f1 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -336,4 +336,4 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: Operation.APPEND, ] - assert tbl.scan().to_arrow()['ints'] == [[3, 4], [1]] + assert tbl.scan().to_arrow()['ints'].to_pylist() == [3, 4, 1] diff --git a/tests/integration/test_inspect_table.py b/tests/integration/test_inspect_table.py index e25e3f117d..78e83b033e 100644 --- a/tests/integration/test_inspect_table.py +++ b/tests/integration/test_inspect_table.py @@ -105,7 +105,7 @@ def test_inspect_snapshots( assert df['parent_id'][0].as_py() is None assert df['parent_id'][1:].to_pylist() == df['snapshot_id'][:-1].to_pylist() - assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'overwrite', 'append'] + assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'append', 'append'] for manifest_list in df['manifest_list']: assert manifest_list.as_py().startswith("s3://") From 4ceacb8c44e4692db67900da6d1f3c5641a2bede Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 30 May 2024 06:59:07 +0200 Subject: [PATCH 22/27] Add another test --- pyiceberg/table/__init__.py | 4 ++- tests/integration/test_deletes.py | 49 ++++++++++++++++++++++++------- 2 files changed, 42 insertions(+), 11 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 86b68bf583..64af7d927a 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -499,10 +499,10 @@ def delete(self, delete_filter: Union[str, BooleanExpression], snapshot_properti if delete_snapshot.rewrites_needed is True: bound_delete_filter = bind(self._table.schema(), delete_filter, case_sensitive=True) preserve_row_filter = expression_to_pyarrow(Not(bound_delete_filter)) - commit_uuid = uuid.uuid4() files = self._scan(row_filter=delete_filter).plan_files() + commit_uuid = uuid.uuid4() counter = itertools.count(0) replaced_files: List[Tuple[DataFile, List[DataFile]]] = [] @@ -3119,6 +3119,8 @@ def _copy_with_new_status(entry: ManifestEntry, status: ManifestEntryStatus) -> for existing_entry in existing_entries: writer.add_entry(existing_entry) existing_manifests.append(writer.to_manifest_file()) + # else: + # deleted_manifests.append() else: existing_manifests.append(manifest_file) else: diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index 85e76125f1..175d03888c 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -23,7 +23,8 @@ from pyiceberg.catalog.rest import RestCatalog from pyiceberg.exceptions import NoSuchTableError -from pyiceberg.expressions import EqualTo +from pyiceberg.expressions import AlwaysTrue, EqualTo +from pyiceberg.manifest import ManifestEntryStatus from pyiceberg.schema import Schema from pyiceberg.table.snapshots import Operation, Summary from pyiceberg.types import IntegerType, NestedField @@ -265,9 +266,7 @@ def test_delete_no_match(session_catalog: RestCatalog) -> None: arrow_schema = pa.schema([pa.field("ints", pa.int32())]) arrow_tbl = pa.Table.from_pylist( [ - { - 'ints': 1, - }, + {'ints': 1}, {'ints': 3}, ], schema=arrow_schema, @@ -297,9 +296,7 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: arrow_schema = pa.schema([pa.field("ints", pa.int32())]) arrow_tbl = pa.Table.from_pylist( [ - { - 'ints': 1, - }, + {'ints': 1}, {'ints': 2}, ], schema=arrow_schema, @@ -321,9 +318,7 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: arrow_tbl_overwrite = pa.Table.from_pylist( [ - { - 'ints': 3, - }, + {'ints': 3}, {'ints': 4}, ], schema=arrow_schema, @@ -337,3 +332,37 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: ] assert tbl.scan().to_arrow()['ints'].to_pylist() == [3, 4, 1] + + +@pytest.mark.integration +def test_delete_truncate(session_catalog: RestCatalog) -> None: + arrow_schema = pa.schema([pa.field("ints", pa.int32())]) + arrow_tbl = pa.Table.from_pylist( + [ + {'ints': 1}, + ], + schema=arrow_schema, + ) + + iceberg_schema = Schema(NestedField(1, "ints", IntegerType())) + + tbl_identifier = "default.test_delete_overwrite" + + try: + session_catalog.drop_table(tbl_identifier) + except NoSuchTableError: + pass + + tbl = session_catalog.create_table(tbl_identifier, iceberg_schema) + tbl.append(arrow_tbl) + + # Effectively a truncate + tbl.delete(delete_filter=AlwaysTrue()) + + manifests = tbl.current_snapshot().manifests(tbl.io) + assert len(manifests) == 1 + + entries = manifests[0].fetch_manifest_entry(tbl.io, discard_deleted=False) + assert len(entries) == 1 + + assert entries[0].status == ManifestEntryStatus.DELETED From 4cd67ac741b41adf607d2d388a14947cf16d4cdf Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 30 May 2024 16:31:02 +0200 Subject: [PATCH 23/27] Conflicts --- tests/conftest.py | 28 ++--- tests/integration/test_deletes.py | 80 ++++++------- tests/integration/test_inspect_table.py | 46 ++++---- tests/integration/test_writes/test_writes.py | 116 +++++++++---------- 4 files changed, 135 insertions(+), 135 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index 879b154701..493684162c 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -2143,31 +2143,31 @@ def arrow_table_with_null(pa_schema: "pa.Schema") -> "pa.Table": return pa.Table.from_pydict( { - 'bool': [False, None, True], - 'string': ['a', None, 'z'], + "bool": [False, None, True], + "string": ["a", None, "z"], # Go over the 16 bytes to kick in truncation - 'string_long': ['a' * 22, None, 'z' * 22], - 'int': [1, None, 9], - 'long': [1, None, 9], - 'float': [0.0, None, 0.9], - 'double': [0.0, None, 0.9], + "string_long": ["a" * 22, None, "z" * 22], + "int": [1, None, 9], + "long": [1, None, 9], + "float": [0.0, None, 0.9], + "double": [0.0, None, 0.9], # 'time': [1_000_000, None, 3_000_000], # Example times: 1s, none, and 3s past midnight #Spark does not support time fields - 'timestamp': [datetime(2023, 1, 1, 19, 25, 00), None, datetime(2023, 3, 1, 19, 25, 00)], - 'timestamptz': [ + "timestamp": [datetime(2023, 1, 1, 19, 25, 00), None, datetime(2023, 3, 1, 19, 25, 00)], + "timestamptz": [ datetime(2023, 1, 1, 19, 25, 00, tzinfo=timezone.utc), None, datetime(2023, 3, 1, 19, 25, 00, tzinfo=timezone.utc), ], - 'date': [date(2023, 1, 1), None, date(2023, 3, 1)], + "date": [date(2023, 1, 1), None, date(2023, 3, 1)], # Not supported by Spark # 'time': [time(1, 22, 0), None, time(19, 25, 0)], # Not natively supported by Arrow # 'uuid': [uuid.UUID('00000000-0000-0000-0000-000000000000').bytes, None, uuid.UUID('11111111-1111-1111-1111-111111111111').bytes], - 'binary': [b'\01', None, b'\22'], - 'fixed': [ - uuid.UUID('00000000-0000-0000-0000-000000000000').bytes, + "binary": [b"\01", None, b"\22"], + "fixed": [ + uuid.UUID("00000000-0000-0000-0000-000000000000").bytes, None, - uuid.UUID('11111111-1111-1111-1111-111111111111').bytes, + uuid.UUID("11111111-1111-1111-1111-111111111111").bytes, ], }, schema=pa_schema, diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index 175d03888c..b5202a5526 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -38,7 +38,7 @@ def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None: @pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: - identifier = 'default.table_partitioned_delete' + identifier = "default.table_partitioned_delete" run_spark_commands( spark, @@ -66,14 +66,14 @@ def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog tbl.delete(EqualTo("number_partitioned", 10)) # No overwrite operation - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete'] - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ["append", "append", "delete"] + assert tbl.scan().to_arrow().to_pydict() == {"number_partitioned": [11, 11], "number": [20, 30]} @pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: - identifier = 'default.table_partitioned_delete' + identifier = "default.table_partitioned_delete" run_spark_commands( spark, @@ -101,14 +101,14 @@ def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCat tbl.delete(EqualTo("number", 20)) # We don't delete a whole partition, so there is only a overwrite - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'overwrite'] - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 10], 'number': [30, 30]} + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ["append", "append", "overwrite"] + assert tbl.scan().to_arrow().to_pydict() == {"number_partitioned": [11, 10], "number": [30, 30]} @pytest.mark.integration @pytest.mark.parametrize("format_version", [1, 2]) def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: - identifier = 'default.table_partitioned_delete' + identifier = "default.table_partitioned_delete" run_spark_commands( spark, @@ -132,13 +132,13 @@ def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCa tbl = session_catalog.load_table(identifier) tbl.delete(EqualTo("number_partitioned", 22)) # Does not affect any data - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append'] - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10, 10], 'number': [20, 30]} + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ["append"] + assert tbl.scan().to_arrow().to_pydict() == {"number_partitioned": [10, 10], "number": [20, 30]} @pytest.mark.integration def test_partitioned_table_positional_deletes(spark: SparkSession, session_catalog: RestCatalog) -> None: - identifier = 'default.table_partitioned_delete' + identifier = "default.table_partitioned_delete" run_spark_commands( spark, @@ -180,13 +180,13 @@ def test_partitioned_table_positional_deletes(spark: SparkSession, session_catal # One positional delete has been added, but an OVERWRITE status is set # https://github.com/apache/iceberg/issues/10122 - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'overwrite'] - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10], 'number': [20]} + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ["append", "overwrite", "overwrite"] + assert tbl.scan().to_arrow().to_pydict() == {"number_partitioned": [10], "number": [20]} @pytest.mark.integration def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSession, session_catalog: RestCatalog) -> None: - identifier = 'default.table_partitioned_delete_sequence_number' + identifier = "default.table_partitioned_delete_sequence_number" # This test case is a bit more complex. Here we run a MoR delete on a file, we make sure that # the manifest gets rewritten (but not the data file with a MoR), and check if the delete is still there @@ -234,31 +234,31 @@ def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSessio assert len(snapshots) == 3 # Snapshots produced by Spark - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()[0:2]] == ['append', 'overwrite'] + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()[0:2]] == ["append", "overwrite"] # Will rewrite one parquet file assert snapshots[2].summary == Summary( Operation.OVERWRITE, **{ - 'added-files-size': '1145', - 'added-data-files': '1', - 'added-records': '2', - 'changed-partition-count': '1', - 'total-files-size': snapshots[2].summary['total-files-size'], - 'total-delete-files': '0', - 'total-data-files': '1', - 'total-position-deletes': '0', - 'total-records': '2', - 'total-equality-deletes': '0', - 'deleted-data-files': '2', - 'removed-delete-files': '1', - 'deleted-records': '5', - 'removed-files-size': snapshots[2].summary['removed-files-size'], - 'removed-position-deletes': '1', + "added-files-size": "1145", + "added-data-files": "1", + "added-records": "2", + "changed-partition-count": "1", + "total-files-size": snapshots[2].summary["total-files-size"], + "total-delete-files": "0", + "total-data-files": "1", + "total-position-deletes": "0", + "total-records": "2", + "total-equality-deletes": "0", + "deleted-data-files": "2", + "removed-delete-files": "1", + "deleted-records": "5", + "removed-files-size": snapshots[2].summary["removed-files-size"], + "removed-position-deletes": "1", }, ) - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [20, 20, 10], 'number': [200, 202, 100]} + assert tbl.scan().to_arrow().to_pydict() == {"number_partitioned": [20, 20, 10], "number": [200, 202, 100]} @pytest.mark.integration @@ -266,8 +266,8 @@ def test_delete_no_match(session_catalog: RestCatalog) -> None: arrow_schema = pa.schema([pa.field("ints", pa.int32())]) arrow_tbl = pa.Table.from_pylist( [ - {'ints': 1}, - {'ints': 3}, + {"ints": 1}, + {"ints": 3}, ], schema=arrow_schema, ) @@ -286,7 +286,7 @@ def test_delete_no_match(session_catalog: RestCatalog) -> None: assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [Operation.APPEND] - tbl.delete('ints == 2') # Only 1 and 3 in the file, but is between the lower and upper bound + tbl.delete("ints == 2") # Only 1 and 3 in the file, but is between the lower and upper bound assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [Operation.APPEND] @@ -296,8 +296,8 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: arrow_schema = pa.schema([pa.field("ints", pa.int32())]) arrow_tbl = pa.Table.from_pylist( [ - {'ints': 1}, - {'ints': 2}, + {"ints": 1}, + {"ints": 2}, ], schema=arrow_schema, ) @@ -318,12 +318,12 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: arrow_tbl_overwrite = pa.Table.from_pylist( [ - {'ints': 3}, - {'ints': 4}, + {"ints": 3}, + {"ints": 4}, ], schema=arrow_schema, ) - tbl.overwrite(arrow_tbl_overwrite, 'ints == 2') # Should rewrite one file + tbl.overwrite(arrow_tbl_overwrite, "ints == 2") # Should rewrite one file assert [snapshot.summary.operation for snapshot in tbl.snapshots()] == [ Operation.APPEND, @@ -331,7 +331,7 @@ def test_delete_overwrite(session_catalog: RestCatalog) -> None: Operation.APPEND, ] - assert tbl.scan().to_arrow()['ints'].to_pylist() == [3, 4, 1] + assert tbl.scan().to_arrow()["ints"].to_pylist() == [3, 4, 1] @pytest.mark.integration @@ -339,7 +339,7 @@ def test_delete_truncate(session_catalog: RestCatalog) -> None: arrow_schema = pa.schema([pa.field("ints", pa.int32())]) arrow_tbl = pa.Table.from_pylist( [ - {'ints': 1}, + {"ints": 1}, ], schema=arrow_schema, ) diff --git a/tests/integration/test_inspect_table.py b/tests/integration/test_inspect_table.py index ee83880a39..753d976ac2 100644 --- a/tests/integration/test_inspect_table.py +++ b/tests/integration/test_inspect_table.py @@ -102,38 +102,38 @@ def test_inspect_snapshots( for snapshot_id in df["snapshot_id"]: assert isinstance(snapshot_id.as_py(), int) - assert df['parent_id'][0].as_py() is None - assert df['parent_id'][1:].to_pylist() == df['snapshot_id'][:-1].to_pylist() + assert df["parent_id"][0].as_py() is None + assert df["parent_id"][1:].to_pylist() == df["snapshot_id"][:-1].to_pylist() - assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'append', 'append'] + assert [operation.as_py() for operation in df["operation"]] == ["append", "delete", "append", "append"] for manifest_list in df["manifest_list"]: assert manifest_list.as_py().startswith("s3://") # Append - assert df['summary'][0].as_py() == [ - ('added-files-size', '5459'), - ('added-data-files', '1'), - ('added-records', '3'), - ('total-data-files', '1'), - ('total-delete-files', '0'), - ('total-records', '3'), - ('total-files-size', '5459'), - ('total-position-deletes', '0'), - ('total-equality-deletes', '0'), + assert df["summary"][0].as_py() == [ + ("added-files-size", "5459"), + ("added-data-files", "1"), + ("added-records", "3"), + ("total-data-files", "1"), + ("total-delete-files", "0"), + ("total-records", "3"), + ("total-files-size", "5459"), + ("total-position-deletes", "0"), + ("total-equality-deletes", "0"), ] # Delete - assert df['summary'][1].as_py() == [ - ('removed-files-size', '5459'), - ('deleted-data-files', '1'), - ('deleted-records', '3'), - ('total-data-files', '0'), - ('total-delete-files', '0'), - ('total-records', '0'), - ('total-files-size', '0'), - ('total-position-deletes', '0'), - ('total-equality-deletes', '0'), + assert df["summary"][1].as_py() == [ + ("removed-files-size", "5459"), + ("deleted-data-files", "1"), + ("deleted-records", "3"), + ("total-data-files", "0"), + ("total-delete-files", "0"), + ("total-records", "0"), + ("total-files-size", "0"), + ("total-position-deletes", "0"), + ("total-equality-deletes", "0"), ] lhs = spark.table(f"{identifier}.snapshots").toPandas() diff --git a/tests/integration/test_writes/test_writes.py b/tests/integration/test_writes/test_writes.py index b23283a492..bc1bd93231 100644 --- a/tests/integration/test_writes/test_writes.py +++ b/tests/integration/test_writes/test_writes.py @@ -189,7 +189,7 @@ def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_wi ).collect() operations = [row.operation for row in rows] - assert operations == ['append', 'append', 'delete', 'append'] + assert operations == ["append", "append", "delete", "append"] summaries = [row.summary for row in rows] @@ -221,28 +221,28 @@ def test_summaries(spark: SparkSession, session_catalog: Catalog, arrow_table_wi # Delete assert summaries[2] == { - 'deleted-data-files': '2', - 'deleted-records': '6', - 'removed-files-size': '10918', - 'total-data-files': '0', - 'total-delete-files': '0', - 'total-equality-deletes': '0', - 'total-files-size': '0', - 'total-position-deletes': '0', - 'total-records': '0', + "deleted-data-files": "2", + "deleted-records": "6", + "removed-files-size": "10918", + "total-data-files": "0", + "total-delete-files": "0", + "total-equality-deletes": "0", + "total-files-size": "0", + "total-position-deletes": "0", + "total-records": "0", } # Overwrite assert summaries[3] == { - 'added-data-files': '1', - 'added-files-size': '5459', - 'added-records': '3', - 'total-data-files': '1', - 'total-delete-files': '0', - 'total-equality-deletes': '0', - 'total-files-size': '5459', - 'total-position-deletes': '0', - 'total-records': '3', + "added-data-files": "1", + "added-files-size": "5459", + "added-records": "3", + "total-data-files": "1", + "total-delete-files": "0", + "total-equality-deletes": "0", + "total-files-size": "5459", + "total-position-deletes": "0", + "total-records": "3", } @@ -517,7 +517,7 @@ def test_summaries_with_only_nulls( ).collect() operations = [row.operation for row in rows] - assert operations == ['append', 'append', 'delete', 'append'] + assert operations == ["append", "append", "delete", "append"] summaries = [row.summary for row in rows] @@ -543,24 +543,24 @@ def test_summaries_with_only_nulls( } assert summaries[2] == { - 'deleted-data-files': '1', - 'deleted-records': '2', - 'removed-files-size': '4239', - 'total-data-files': '0', - 'total-delete-files': '0', - 'total-equality-deletes': '0', - 'total-files-size': '0', - 'total-position-deletes': '0', - 'total-records': '0', + "deleted-data-files": "1", + "deleted-records": "2", + "removed-files-size": "4239", + "total-data-files": "0", + "total-delete-files": "0", + "total-equality-deletes": "0", + "total-files-size": "0", + "total-position-deletes": "0", + "total-records": "0", } assert summaries[3] == { - 'total-data-files': '0', - 'total-delete-files': '0', - 'total-equality-deletes': '0', - 'total-files-size': '0', - 'total-position-deletes': '0', - 'total-records': '0', + "total-data-files": "0", + "total-delete-files": "0", + "total-equality-deletes": "0", + "total-files-size": "0", + "total-position-deletes": "0", + "total-records": "0", } @@ -780,38 +780,38 @@ def test_inspect_snapshots( for snapshot_id in df["snapshot_id"]: assert isinstance(snapshot_id.as_py(), int) - assert df['parent_id'][0].as_py() is None - assert df['parent_id'][1:].to_pylist() == df['snapshot_id'][:-1].to_pylist() + assert df["parent_id"][0].as_py() is None + assert df["parent_id"][1:].to_pylist() == df["snapshot_id"][:-1].to_pylist() - assert [operation.as_py() for operation in df['operation']] == ['append', 'delete', 'append', 'append'] + assert [operation.as_py() for operation in df["operation"]] == ["append", "delete", "append", "append"] for manifest_list in df["manifest_list"]: assert manifest_list.as_py().startswith("s3://") # Append - assert df['summary'][0].as_py() == [ - ('added-files-size', '5459'), - ('added-data-files', '1'), - ('added-records', '3'), - ('total-data-files', '1'), - ('total-delete-files', '0'), - ('total-records', '3'), - ('total-files-size', '5459'), - ('total-position-deletes', '0'), - ('total-equality-deletes', '0'), + assert df["summary"][0].as_py() == [ + ("added-files-size", "5459"), + ("added-data-files", "1"), + ("added-records", "3"), + ("total-data-files", "1"), + ("total-delete-files", "0"), + ("total-records", "3"), + ("total-files-size", "5459"), + ("total-position-deletes", "0"), + ("total-equality-deletes", "0"), ] # Delete - assert df['summary'][1].as_py() == [ - ('removed-files-size', '5459'), - ('deleted-data-files', '1'), - ('deleted-records', '3'), - ('total-data-files', '0'), - ('total-delete-files', '0'), - ('total-records', '0'), - ('total-files-size', '0'), - ('total-position-deletes', '0'), - ('total-equality-deletes', '0'), + assert df["summary"][1].as_py() == [ + ("removed-files-size", "5459"), + ("deleted-data-files", "1"), + ("deleted-records", "3"), + ("total-data-files", "0"), + ("total-delete-files", "0"), + ("total-records", "0"), + ("total-files-size", "0"), + ("total-position-deletes", "0"), + ("total-equality-deletes", "0"), ] lhs = spark.table(f"{identifier}.snapshots").toPandas() From 5cdb363acd2dabebacb38a850c83e202fe6bad13 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+syun64@users.noreply.github.com> Date: Sat, 15 Jun 2024 15:11:28 -0400 Subject: [PATCH 24/27] Add docs (#33) * docs * docs --- mkdocs/docs/api.md | 21 +++++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/mkdocs/docs/api.md b/mkdocs/docs/api.md index 70b5fd62eb..06cc6ade54 100644 --- a/mkdocs/docs/api.md +++ b/mkdocs/docs/api.md @@ -331,12 +331,25 @@ df = pa.Table.from_pylist( table.append(df) ``` - +You can delete some of the data from the table by calling `tbl.delete()` with a desired `delete_filter`. + +```python +tbl.delete(delete_filter="city == 'Paris'") +``` -!!! example "Under development" - Writing using PyIceberg is still under development. Support for [partial overwrites](https://github.com/apache/iceberg-python/issues/268) and writing to [partitioned tables](https://github.com/apache/iceberg-python/issues/208) is planned and being worked on. +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: - +``` +pyarrow.Table +city: string +lat: double +long: double +---- +city: [["Amsterdam","San Francisco","Drachten"],["Groningen"]] +lat: [[52.371807,37.773972,53.11254],[53.21917]] +long: [[4.896029,-122.431297,6.0989],[6.56667]] +``` ## Inspecting tables From 1ccb31de247516a37b8e505ea9f2fc73224bcd31 Mon Sep 17 00:00:00 2001 From: Fokko Date: Fri, 5 Jul 2024 17:10:16 +0200 Subject: [PATCH 25/27] Add a partitioned overwrite test --- pyiceberg/io/pyarrow.py | 4 +-- pyiceberg/table/__init__.py | 32 ++++++++++++++--- tests/integration/test_deletes.py | 57 +++++++++++++++++++++++++++++-- tests/table/test_init.py | 6 ++-- 4 files changed, 87 insertions(+), 12 deletions(-) diff --git a/pyiceberg/io/pyarrow.py b/pyiceberg/io/pyarrow.py index 3d3a7624f8..50406972a7 100644 --- a/pyiceberg/io/pyarrow.py +++ b/pyiceberg/io/pyarrow.py @@ -2084,9 +2084,9 @@ def _dataframe_to_data_files( ]), ) else: - from pyiceberg.table import determine_partitions + from pyiceberg.table import _determine_partitions - partitions = determine_partitions(spec=table_metadata.spec(), schema=table_metadata.schema(), arrow_table=df) + partitions = _determine_partitions(spec=table_metadata.spec(), schema=table_metadata.schema(), arrow_table=df) yield from write_file( io=io, table_metadata=table_metadata, diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index b03bd1a69d..233fbf31ac 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -249,6 +249,7 @@ class TableProperties: DELETE_MODE = "write.delete.mode" DELETE_MODE_COPY_ON_WRITE = "copy-on-write" DELETE_MODE_MERGE_ON_READ = "merge-on-read" + DELETE_MODE_DEFAULT = DELETE_MODE_COPY_ON_WRITE DEFAULT_NAME_MAPPING = "schema.name-mapping.default" FORMAT_VERSION = "format-version" @@ -533,6 +534,12 @@ def overwrite( """ Shorthand for adding a table overwrite with a PyArrow table to the transaction. + An overwrite may produce zero or more snapshots based on the operation: + + - DELETE: In case existing Parquet files can be dropped completely. + - REPLACE: In case existing Parquet files need to be rewritten. + - APPEND: In case new data is being inserted into the table. + Args: df: The Arrow dataframe that will be used to overwrite the table overwrite_filter: ALWAYS_TRUE when you overwrite all the data, @@ -547,8 +554,12 @@ def overwrite( if not isinstance(df, pa.Table): raise ValueError(f"Expected PyArrow table, got: {df}") - if len(self._table.spec().fields) > 0: - raise ValueError("Cannot write to partitioned tables") + if unsupported_partitions := [ + field for field in self.table_metadata.spec().fields if not field.transform.supports_pyarrow_transform + ]: + raise ValueError( + f"Not all partition types are supported for writes. Following partitions cannot be written using pyarrow: {unsupported_partitions}." + ) _check_schema_compatible(self._table.schema(), other_schema=df.schema) # cast if the two schemas are compatible but not equal @@ -568,8 +579,15 @@ def overwrite( update_snapshot.append_data_file(data_file) def delete(self, delete_filter: Union[str, BooleanExpression], snapshot_properties: Dict[str, str] = EMPTY_DICT) -> None: + """ + Shorthand for deleting record from a table. + + Args: + delete_filter: A boolean expression to delete rows from a table + snapshot_properties: Custom properties to be added to the snapshot summary + """ if ( - self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_COPY_ON_WRITE) + self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_DEFAULT) == TableProperties.DELETE_MODE_MERGE_ON_READ ): warnings.warn("Merge on read is not yet supported, falling back to copy-on-write") @@ -1562,6 +1580,12 @@ def overwrite( """ Shorthand for overwriting the table with a PyArrow table. + An overwrite may produce zero or more snapshots based on the operation: + + - DELETE: In case existing Parquet files can be dropped completely. + - REPLACE: In case existing Parquet files need to be rewritten. + - APPEND: In case new data is being inserted into the table. + Args: df: The Arrow dataframe that will be used to overwrite the table overwrite_filter: ALWAYS_TRUE when you overwrite all the data, @@ -4326,7 +4350,7 @@ def _get_table_partitions( return table_partitions -def determine_partitions(spec: PartitionSpec, schema: Schema, arrow_table: pa.Table) -> List[TablePartition]: +def _determine_partitions(spec: PartitionSpec, schema: Schema, arrow_table: pa.Table) -> List[TablePartition]: """Based on the iceberg table partition spec, slice the arrow table into partitions with their keys. Example: diff --git a/tests/integration/test_deletes.py b/tests/integration/test_deletes.py index b5202a5526..ad3adedeca 100644 --- a/tests/integration/test_deletes.py +++ b/tests/integration/test_deletes.py @@ -137,7 +137,7 @@ def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCa @pytest.mark.integration -def test_partitioned_table_positional_deletes(spark: SparkSession, session_catalog: RestCatalog) -> None: +def test_delete_partitioned_table_positional_deletes(spark: SparkSession, session_catalog: RestCatalog) -> None: identifier = "default.table_partitioned_delete" run_spark_commands( @@ -175,7 +175,7 @@ def test_partitioned_table_positional_deletes(spark: SparkSession, session_catal assert len(files) == 1 assert len(files[0].delete_files) == 1 - # Will rewrite a data file with a positional delete + # Will rewrite a data file without the positional delete tbl.delete(EqualTo("number", 40)) # One positional delete has been added, but an OVERWRITE status is set @@ -184,6 +184,57 @@ def test_partitioned_table_positional_deletes(spark: SparkSession, session_catal assert tbl.scan().to_arrow().to_pydict() == {"number_partitioned": [10], "number": [20]} +@pytest.mark.integration +def test_overwrite_partitioned_table(spark: SparkSession, session_catalog: RestCatalog) -> None: + identifier = "default.table_partitioned_delete" + + run_spark_commands( + spark, + [ + f"DROP TABLE IF EXISTS {identifier}", + f""" + CREATE TABLE {identifier} ( + number_partitioned int, + number int + ) + USING iceberg + PARTITIONED BY (number_partitioned) + TBLPROPERTIES( + 'format-version' = 2, + 'write.delete.mode'='merge-on-read', + 'write.update.mode'='merge-on-read', + 'write.merge.mode'='merge-on-read' + ) + """, + f""" + INSERT INTO {identifier} VALUES (10, 1), (10, 2), (20, 3) + """, + ], + ) + + tbl = session_catalog.load_table(identifier) + + files = list(tbl.scan().plan_files()) + assert len(files) == 2 + + arrow_schema = pa.schema([pa.field("number_partitioned", pa.int32()), pa.field("number", pa.int32())]) + arrow_tbl = pa.Table.from_pylist( + [ + {"number_partitioned": 10, "number": 4}, + {"number_partitioned": 10, "number": 5}, + ], + schema=arrow_schema, + ) + + # Will rewrite a data file without the positional delete + tbl.overwrite(arrow_tbl, "number_partitioned == 10") + + # One positional delete has been added, but an OVERWRITE status is set + # https://github.com/apache/iceberg/issues/10122 + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ["append", "delete", "append"] + assert tbl.scan().to_arrow().to_pydict() == {"number_partitioned": [10, 10, 20], "number": [4, 5, 3]} + + @pytest.mark.integration def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSession, session_catalog: RestCatalog) -> None: identifier = "default.table_partitioned_delete_sequence_number" @@ -225,7 +276,7 @@ def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSessio files = list(tbl.scan().plan_files()) assert len(files) == 2 - # Will rewrite a data file with a positional delete + # Will rewrite a data file without a positional delete tbl.delete(EqualTo("number", 201)) # One positional delete has been added, but an OVERWRITE status is set diff --git a/tests/table/test_init.py b/tests/table/test_init.py index 2925f8bcc3..d7c4ffeeaf 100644 --- a/tests/table/test_init.py +++ b/tests/table/test_init.py @@ -64,9 +64,9 @@ UpdateSchema, _apply_table_update, _check_schema_compatible, + _determine_partitions, _match_deletes_to_data_file, _TableMetadataUpdateContext, - determine_partitions, update_table_metadata, ) from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadataUtil, TableMetadataV2, _generate_snapshot_id @@ -1270,7 +1270,7 @@ def test_partition_for_demo() -> None: PartitionField(source_id=2, field_id=1002, transform=IdentityTransform(), name="n_legs_identity"), PartitionField(source_id=1, field_id=1001, transform=IdentityTransform(), name="year_identity"), ) - result = determine_partitions(partition_spec, test_schema, arrow_table) + result = _determine_partitions(partition_spec, test_schema, arrow_table) assert {table_partition.partition_key.partition for table_partition in result} == { Record(n_legs_identity=2, year_identity=2020), Record(n_legs_identity=100, year_identity=2021), @@ -1320,7 +1320,7 @@ def test_identity_partition_on_multi_columns() -> None: } arrow_table = pa.Table.from_pydict(test_data, schema=test_pa_schema) - result = determine_partitions(partition_spec, test_schema, arrow_table) + result = _determine_partitions(partition_spec, test_schema, arrow_table) assert {table_partition.partition_key.partition for table_partition in result} == expected concatenated_arrow_table = pa.concat_tables([table_partition.arrow_table_partition for table_partition in result]) From 96d6392dddf37c4f59a9931190290aa6c9439be8 Mon Sep 17 00:00:00 2001 From: Fokko Date: Tue, 9 Jul 2024 08:49:18 +0200 Subject: [PATCH 26/27] Fix comment --- pyiceberg/table/__init__.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 233fbf31ac..6ca37ca12b 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -582,6 +582,11 @@ def delete(self, delete_filter: Union[str, BooleanExpression], snapshot_properti """ Shorthand for deleting record from a table. + An deletee may produce zero or more snapshots based on the operation: + + - DELETE: In case existing Parquet files can be dropped completely. + - REPLACE: In case existing Parquet files need to be rewritten + Args: delete_filter: A boolean expression to delete rows from a table snapshot_properties: Custom properties to be added to the snapshot summary From 301bc82467f977389135781a10bb43b74b2c402a Mon Sep 17 00:00:00 2001 From: Fokko Date: Tue, 9 Jul 2024 11:23:00 +0200 Subject: [PATCH 27/27] Skip empty manifests --- pyiceberg/table/__init__.py | 41 +++++++++++++++++++------------------ 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/pyiceberg/table/__init__.py b/pyiceberg/table/__init__.py index 6ca37ca12b..8eea9859bc 100644 --- a/pyiceberg/table/__init__.py +++ b/pyiceberg/table/__init__.py @@ -3437,27 +3437,28 @@ def _existing_manifests(self) -> List[ManifestFile]: num=next(self._manifest_counter), commit_uuid=self.commit_uuid, ) - 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._io.new_output(output_file_location), - snapshot_id=self._snapshot_id, - ) as writer: - [ - writer.add_entry( - ManifestEntry( - status=ManifestEntryStatus.EXISTING, - snapshot_id=entry.snapshot_id, - data_sequence_number=entry.data_sequence_number, - file_sequence_number=entry.file_sequence_number, - data_file=entry.data_file, + 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.spec(), + schema=self._transaction.table_metadata.schema(), + output_file=self._io.new_output(output_file_location), + snapshot_id=self._snapshot_id, + ) as writer: + [ + writer.add_entry( + ManifestEntry( + status=ManifestEntryStatus.EXISTING, + snapshot_id=entry.snapshot_id, + data_sequence_number=entry.data_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()) + for entry in entries + if entry.data_file not in found_deleted_data_files + ] + existing_files.append(writer.to_manifest_file()) return existing_files def _deleted_entries(self) -> List[ManifestEntry]: