Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
49 commits
Select commit Hold shift + click to select a range
8d45920
Add option to delete datafiles
Fokko Apr 2, 2024
f6084a6
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 2, 2024
87cc065
Pull in main
Fokko Apr 2, 2024
bc9c83e
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 3, 2024
284d05a
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 5, 2024
234d55b
WIP
Fokko Apr 5, 2024
aadc89c
Change DataScan to accept Metadata and io
Fokko Apr 5, 2024
7e59342
fix name-mapping issue
HonahX Apr 7, 2024
fbf6492
Merge pull request #1 from HonahX/honahx-update-datascan
Fokko Apr 8, 2024
c3fa7e7
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 8, 2024
a925d69
Merge branch 'fd-update-datascan' of github.com:Fokko/iceberg-python …
Fokko Apr 8, 2024
5cec00a
WIP
Fokko Apr 8, 2024
a5e988a
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 8, 2024
1723819
WIP
Fokko Apr 9, 2024
5025b4a
Moar tests
Fokko Apr 10, 2024
e474fda
Oops
Fokko Apr 11, 2024
172f9c0
Cleanup
Fokko Apr 16, 2024
a97c45a
WIP
Fokko Apr 16, 2024
74497fb
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 17, 2024
47c9de1
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 17, 2024
9c6724e
WIP
Fokko Apr 17, 2024
edff166
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 18, 2024
c443af2
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 18, 2024
7dae071
Fix summary generation
Fokko Apr 18, 2024
5e871fb
Last few bits
Fokko Apr 18, 2024
9910d29
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 18, 2024
cd19f80
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Apr 25, 2024
edfadd9
Fix the requirement
Fokko Apr 25, 2024
d65a8a4
Make ruff happy
Fokko Apr 25, 2024
8849d97
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko May 2, 2024
3c98eef
Comments, thanks Kevin!
Fokko May 2, 2024
179fa27
Comments
Fokko May 9, 2024
2ea157e
Append rather than truncate
Fokko May 10, 2024
18392d1
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko May 10, 2024
7d036b1
Fix merge conflicts
Fokko May 10, 2024
5adf3f0
Make the tests pass
Fokko May 13, 2024
b3fcdcf
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko May 27, 2024
4ceacb8
Add another test
Fokko May 30, 2024
ddf6119
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko May 30, 2024
5b10f25
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko May 30, 2024
4cd67ac
Conflicts
Fokko May 30, 2024
5cdb363
Add docs (#33)
sungwy Jun 15, 2024
2252e71
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Jun 21, 2024
05fcf2d
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Jul 5, 2024
1ccb31d
Add a partitioned overwrite test
Fokko Jul 5, 2024
86432fe
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Jul 8, 2024
259f8c5
Merge branch 'main' of github.com:apache/iceberg-python into fd-add-a…
Fokko Jul 9, 2024
96d6392
Fix comment
Fokko Jul 9, 2024
301bc82
Skip empty manifests
Fokko Jul 9, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Cleanup
  • Loading branch information
Fokko committed Apr 16, 2024
commit 172f9c0a622b291dba86b89fadb94bc9a3f5b4e0
96 changes: 59 additions & 37 deletions pyiceberg/table/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Expand Down Expand Up @@ -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:
Expand All @@ -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

Expand All @@ -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]:
Expand Down
111 changes: 104 additions & 7 deletions tests/integration/test_deletes.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,15 @@

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:
for sql in sqls:
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'
Expand Down Expand Up @@ -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'
Expand Down Expand Up @@ -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'
Expand Down Expand Up @@ -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'

Expand Down Expand Up @@ -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]}