|
15 | 15 | # specific language governing permissions and limitations
|
16 | 16 | # under the License.
|
17 | 17 | # pylint:disable=redefined-outer-name
|
| 18 | +from datetime import datetime |
18 | 19 | from typing import List
|
19 | 20 |
|
20 | 21 | import pyarrow as pa
|
|
25 | 26 | from pyiceberg.exceptions import NoSuchTableError
|
26 | 27 | from pyiceberg.expressions import AlwaysTrue, EqualTo
|
27 | 28 | from pyiceberg.manifest import ManifestEntryStatus
|
| 29 | +from pyiceberg.partitioning import PartitionField, PartitionSpec |
28 | 30 | from pyiceberg.schema import Schema
|
29 | 31 | from pyiceberg.table.snapshots import Operation, Summary
|
30 |
| -from pyiceberg.types import FloatType, IntegerType, NestedField |
| 32 | +from pyiceberg.transforms import IdentityTransform |
| 33 | +from pyiceberg.types import FloatType, IntegerType, LongType, NestedField, TimestampType |
31 | 34 |
|
32 | 35 |
|
33 | 36 | def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None:
|
@@ -556,3 +559,89 @@ def test_delete_overwrite_table_with_nan(session_catalog: RestCatalog) -> None:
|
556 | 559 | assert 2.0 in result
|
557 | 560 | assert 3.0 in result
|
558 | 561 | assert 4.0 in result
|
| 562 | + |
| 563 | + |
| 564 | +@pytest.mark.integration |
| 565 | +def test_delete_after_partition_evolution_from_unpartitioned(session_catalog: RestCatalog) -> None: |
| 566 | + identifier = "default.test_delete_after_partition_evolution_from_unpartitioned" |
| 567 | + |
| 568 | + arrow_table = pa.Table.from_arrays( |
| 569 | + [ |
| 570 | + pa.array([2, 3, 4, 5, 6]), |
| 571 | + ], |
| 572 | + names=["idx"], |
| 573 | + ) |
| 574 | + |
| 575 | + try: |
| 576 | + session_catalog.drop_table(identifier) |
| 577 | + except NoSuchTableError: |
| 578 | + pass |
| 579 | + |
| 580 | + tbl = session_catalog.create_table( |
| 581 | + identifier, |
| 582 | + schema=Schema( |
| 583 | + NestedField(1, "idx", LongType()), |
| 584 | + ), |
| 585 | + ) |
| 586 | + |
| 587 | + tbl.append(arrow_table) |
| 588 | + |
| 589 | + with tbl.transaction() as tx: |
| 590 | + with tx.update_schema() as schema: |
| 591 | + schema.rename_column("idx", "id") |
| 592 | + with tx.update_spec() as spec: |
| 593 | + spec.add_field("id", IdentityTransform()) |
| 594 | + |
| 595 | + # Append one more time to create data files with two partition specs |
| 596 | + tbl.append(arrow_table.rename_columns(["id"])) |
| 597 | + |
| 598 | + tbl.delete("id == 4") |
| 599 | + |
| 600 | + # Expect 8 records: 10 records - 2 |
| 601 | + assert len(tbl.scan().to_arrow()) == 8 |
| 602 | + |
| 603 | + |
| 604 | +@pytest.mark.integration |
| 605 | +def test_delete_after_partition_evolution_from_partitioned(session_catalog: RestCatalog) -> None: |
| 606 | + identifier = "default.test_delete_after_partition_evolution_from_partitioned" |
| 607 | + |
| 608 | + arrow_table = pa.Table.from_arrays( |
| 609 | + [ |
| 610 | + pa.array([2, 3, 4, 5, 6]), |
| 611 | + pa.array([ |
| 612 | + datetime(2021, 5, 19), |
| 613 | + datetime(2022, 7, 25), |
| 614 | + datetime(2023, 3, 22), |
| 615 | + datetime(2024, 7, 17), |
| 616 | + datetime(2025, 2, 22), |
| 617 | + ]), |
| 618 | + ], |
| 619 | + names=["idx", "ts"], |
| 620 | + ) |
| 621 | + |
| 622 | + try: |
| 623 | + session_catalog.drop_table(identifier) |
| 624 | + except NoSuchTableError: |
| 625 | + pass |
| 626 | + |
| 627 | + tbl = session_catalog.create_table( |
| 628 | + identifier, |
| 629 | + schema=Schema(NestedField(1, "idx", LongType()), NestedField(2, "ts", TimestampType())), |
| 630 | + partition_spec=PartitionSpec(PartitionField(source_id=2, field_id=1000, transform=IdentityTransform(), name="ts")), |
| 631 | + ) |
| 632 | + |
| 633 | + tbl.append(arrow_table) |
| 634 | + |
| 635 | + with tbl.transaction() as tx: |
| 636 | + with tx.update_schema() as schema: |
| 637 | + schema.rename_column("idx", "id") |
| 638 | + with tx.update_spec() as spec: |
| 639 | + spec.add_field("id", IdentityTransform()) |
| 640 | + |
| 641 | + # Append one more time to create data files with two partition specs |
| 642 | + tbl.append(arrow_table.rename_columns(["id", "ts"])) |
| 643 | + |
| 644 | + tbl.delete("id == 4") |
| 645 | + |
| 646 | + # Expect 8 records: 10 records - 2 |
| 647 | + assert len(tbl.scan().to_arrow()) == 8 |
0 commit comments