|
15 | 15 | # specific language governing permissions and limitations
|
16 | 16 | # under the License.
|
17 | 17 | # pylint:disable=redefined-outer-name
|
| 18 | +from typing import List |
| 19 | + |
18 | 20 | import pytest
|
19 |
| -from pyspark.sql import DataFrame, SparkSession |
| 21 | +from pyspark.sql import SparkSession |
20 | 22 |
|
21 | 23 | from pyiceberg.catalog.rest import RestCatalog
|
22 | 24 | from pyiceberg.expressions import EqualTo
|
23 | 25 |
|
24 | 26 |
|
25 |
| -@pytest.fixture |
26 |
| -def test_deletes_table(spark: SparkSession) -> DataFrame: |
| 27 | +def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None: |
| 28 | + for sql in sqls: |
| 29 | + spark.sql(sql) |
| 30 | + |
| 31 | + |
| 32 | +@pytest.mark.parametrize("format_version", [1, 2]) |
| 33 | +def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: |
27 | 34 | identifier = 'default.table_partitioned_delete'
|
28 | 35 |
|
29 |
| - spark.sql(f"DROP TABLE IF EXISTS {identifier}") |
30 |
| - |
31 |
| - spark.sql( |
32 |
| - f""" |
33 |
| - CREATE TABLE {identifier} ( |
34 |
| - number_partitioned int, |
35 |
| - number int |
36 |
| - ) |
37 |
| - USING iceberg |
38 |
| - PARTITIONED BY (number_partitioned) |
39 |
| - """ |
40 |
| - ) |
41 |
| - spark.sql( |
42 |
| - f""" |
43 |
| - INSERT INTO {identifier} VALUES (10, 20), (10, 30) |
44 |
| - """ |
45 |
| - ) |
46 |
| - spark.sql( |
47 |
| - f""" |
48 |
| - INSERT INTO {identifier} VALUES (11, 20), (11, 30) |
49 |
| - """ |
| 36 | + run_spark_commands( |
| 37 | + spark, |
| 38 | + [ |
| 39 | + f"DROP TABLE IF EXISTS {identifier}", |
| 40 | + f""" |
| 41 | + CREATE TABLE {identifier} ( |
| 42 | + number_partitioned int, |
| 43 | + number int |
| 44 | + ) |
| 45 | + USING iceberg |
| 46 | + PARTITIONED BY (number_partitioned) |
| 47 | + TBLPROPERTIES('format-version' = {format_version}) |
| 48 | + """, |
| 49 | + f""" |
| 50 | + INSERT INTO {identifier} VALUES (10, 20), (10, 30) |
| 51 | + """, |
| 52 | + f""" |
| 53 | + INSERT INTO {identifier} VALUES (11, 20), (11, 30) |
| 54 | + """, |
| 55 | + ], |
50 | 56 | )
|
51 | 57 |
|
52 |
| - return spark.table(identifier) |
| 58 | + tbl = session_catalog.load_table(identifier) |
| 59 | + tbl.delete(EqualTo("number_partitioned", 10)) |
| 60 | + |
| 61 | + # No overwrite operation |
| 62 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete'] |
| 63 | + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]} |
53 | 64 |
|
54 | 65 |
|
55 |
| -def test_partition_deletes(test_deletes_table: DataFrame, session_catalog: RestCatalog) -> None: |
| 66 | +@pytest.mark.parametrize("format_version", [1, 2]) |
| 67 | +def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: |
56 | 68 | identifier = 'default.table_partitioned_delete'
|
57 | 69 |
|
| 70 | + run_spark_commands( |
| 71 | + spark, |
| 72 | + [ |
| 73 | + f"DROP TABLE IF EXISTS {identifier}", |
| 74 | + f""" |
| 75 | + CREATE TABLE {identifier} ( |
| 76 | + number_partitioned int, |
| 77 | + number int |
| 78 | + ) |
| 79 | + USING iceberg |
| 80 | + PARTITIONED BY (number_partitioned) |
| 81 | + TBLPROPERTIES('format-version' = {format_version}) |
| 82 | + """, |
| 83 | + f""" |
| 84 | + INSERT INTO {identifier} VALUES (10, 20), (10, 30) |
| 85 | + """, |
| 86 | + f""" |
| 87 | + INSERT INTO {identifier} VALUES (11, 20), (11, 30) |
| 88 | + """, |
| 89 | + ], |
| 90 | + ) |
| 91 | + |
58 | 92 | tbl = session_catalog.load_table(identifier)
|
59 |
| - tbl.delete(EqualTo("number_partitioned", 10)) |
| 93 | + tbl.delete(EqualTo("number", 20)) |
60 | 94 |
|
| 95 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete', 'overwrite'] |
61 | 96 | assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]}
|
62 | 97 |
|
63 | 98 |
|
64 |
| -def test_deletes(test_deletes_table: DataFrame, session_catalog: RestCatalog) -> None: |
| 99 | +@pytest.mark.parametrize("format_version", [1, 2]) |
| 100 | +def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None: |
65 | 101 | identifier = 'default.table_partitioned_delete'
|
66 | 102 |
|
| 103 | + run_spark_commands( |
| 104 | + spark, |
| 105 | + [ |
| 106 | + f"DROP TABLE IF EXISTS {identifier}", |
| 107 | + f""" |
| 108 | + CREATE TABLE {identifier} ( |
| 109 | + number_partitioned int, |
| 110 | + number int |
| 111 | + ) |
| 112 | + USING iceberg |
| 113 | + PARTITIONED BY (number_partitioned) |
| 114 | + TBLPROPERTIES('format-version' = {format_version}) |
| 115 | + """, |
| 116 | + f""" |
| 117 | + INSERT INTO {identifier} VALUES (10, 20), (10, 30) |
| 118 | + """, |
| 119 | + ], |
| 120 | + ) |
| 121 | + |
| 122 | + tbl = session_catalog.load_table(identifier) |
| 123 | + tbl.delete(EqualTo("number_partitioned", 22)) # Does not affect any data |
| 124 | + |
| 125 | + # Open for discussion, do we want to create a new snapshot? |
| 126 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'delete'] |
| 127 | + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10, 10], 'number': [20, 30]} |
| 128 | + |
| 129 | + |
| 130 | +def test_partitioned_table_positional_deletes(spark: SparkSession, session_catalog: RestCatalog) -> None: |
| 131 | + identifier = 'default.table_partitioned_delete' |
| 132 | + |
| 133 | + run_spark_commands( |
| 134 | + spark, |
| 135 | + [ |
| 136 | + f"DROP TABLE IF EXISTS {identifier}", |
| 137 | + f""" |
| 138 | + CREATE TABLE {identifier} ( |
| 139 | + number_partitioned int, |
| 140 | + number int |
| 141 | + ) |
| 142 | + USING iceberg |
| 143 | + PARTITIONED BY (number_partitioned) |
| 144 | + TBLPROPERTIES( |
| 145 | + 'format-version' = 2, |
| 146 | + 'write.delete.mode'='merge-on-read', |
| 147 | + 'write.update.mode'='merge-on-read', |
| 148 | + 'write.merge.mode'='merge-on-read' |
| 149 | + ) |
| 150 | + """, |
| 151 | + f""" |
| 152 | + INSERT INTO {identifier} VALUES (10, 20), (10, 30), (10, 40) |
| 153 | + """, |
| 154 | + # Generate a positional delete |
| 155 | + f""" |
| 156 | + DELETE FROM {identifier} WHERE number = 30 |
| 157 | + """, |
| 158 | + ], |
| 159 | + ) |
| 160 | + |
67 | 161 | tbl = session_catalog.load_table(identifier)
|
68 |
| - tbl.delete(EqualTo("number", 30)) |
69 | 162 |
|
70 |
| - assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 20]} |
| 163 | + # Assert that there is just a single Parquet file |
| 164 | + assert len(list(tbl.scan().plan_files())) == 1 |
| 165 | + |
| 166 | + # Will rewrite a data file with a positional delete |
| 167 | + tbl.delete(EqualTo("number", 40)) |
| 168 | + |
| 169 | + # Yet another wrong status by Spark |
| 170 | + # One positional delete has been added, but an OVERWRITE status is set |
| 171 | + # Related issue https://github.com/apache/iceberg/issues/9995 |
| 172 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'delete'] |
| 173 | + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10], 'number': [20]} |
0 commit comments