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
Add a partitioned overwrite test
  • Loading branch information
Fokko committed Jul 5, 2024
commit 1ccb31de247516a37b8e505ea9f2fc73224bcd31
4 changes: 2 additions & 2 deletions pyiceberg/io/pyarrow.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
32 changes: 28 additions & 4 deletions pyiceberg/table/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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,
Expand All @@ -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
Expand All @@ -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")
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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:
Expand Down
57 changes: 54 additions & 3 deletions tests/integration/test_deletes.py
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -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
Expand All @@ -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"
Expand Down Expand Up @@ -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
Expand Down
6 changes: 3 additions & 3 deletions tests/table/test_init.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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),
Expand Down Expand Up @@ -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])
Expand Down