Skip to content

Commit 9c6724e

Browse files
committed
WIP
1 parent 47c9de1 commit 9c6724e

File tree

2 files changed

+7
-20
lines changed

2 files changed

+7
-20
lines changed

pyiceberg/table/__init__.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -439,9 +439,6 @@ def overwrite(
439439
if not isinstance(df, pa.Table):
440440
raise ValueError(f"Expected PyArrow table, got: {df}")
441441

442-
if overwrite_filter != AlwaysTrue():
443-
raise NotImplementedError("Cannot overwrite a subset of a table")
444-
445442
if len(self._table.spec().fields) > 0:
446443
raise ValueError("Cannot write to partitioned tables")
447444

@@ -451,6 +448,9 @@ def overwrite(
451448
if table_arrow_schema != df.schema:
452449
df = df.cast(table_arrow_schema)
453450

451+
with self.update_snapshot(snapshot_properties=snapshot_properties).delete() as delete_snapshot:
452+
delete_snapshot.delete_by_predicate(overwrite_filter)
453+
454454
with self.update_snapshot(snapshot_properties=snapshot_properties).overwrite() as update_snapshot:
455455
# skip writing data files if the dataframe is empty
456456
if df.shape[0] > 0:

tests/integration/test_deletes.py

Lines changed: 4 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -227,33 +227,20 @@ def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSessio
227227
# One positional delete has been added, but an OVERWRITE status is set
228228
# https://github.com/apache/iceberg/issues/10122
229229
snapshots = tbl.snapshots()
230-
assert len(snapshots) == 4
230+
assert len(snapshots) == 3
231231

232232
# Snapshots produced by Spark
233233
assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()[0:2]] == ['append', 'overwrite']
234234

235-
# Snapshots produced by PyIceberg
236-
# This is a no-op since nothing has been added or deleted (because the predicate cannot drop a whole file)
237-
assert tbl.snapshots()[2].summary == Summary(
238-
Operation.DELETE,
239-
**{
240-
'total-data-files': '2',
241-
'total-delete-files': '1',
242-
'total-records': '5',
243-
'total-files-size': tbl.snapshots()[2].summary['total-files-size'],
244-
'total-position-deletes': '1',
245-
'total-equality-deletes': '0',
246-
},
247-
)
248235
# Will rewrite one parquet file
249-
assert tbl.snapshots()[3].summary == Summary(
236+
assert snapshots[2].summary == Summary(
250237
Operation.OVERWRITE,
251238
**{
252239
'added-files-size': '1145',
253240
'added-data-files': '1',
254241
'added-records': '2',
255242
'changed-partition-count': '1',
256-
'total-files-size': tbl.snapshots()[3].summary['total-files-size'],
243+
'total-files-size': snapshots[2].summary['total-files-size'],
257244
'total-delete-files': '0',
258245
'total-data-files': '1',
259246
'total-position-deletes': '0',
@@ -262,7 +249,7 @@ def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSessio
262249
'deleted-data-files': '2',
263250
'removed-delete-files': '1',
264251
'deleted-records': '5',
265-
'removed-files-size': '3088',
252+
'removed-files-size': snapshots[2].summary['removed-files-size'],
266253
'removed-position-deletes': '1',
267254
},
268255
)

0 commit comments

Comments
 (0)