|
22 | 22 |
|
23 | 23 | from pyiceberg.catalog.rest import RestCatalog
|
24 | 24 | from pyiceberg.expressions import EqualTo
|
| 25 | +from pyiceberg.table.snapshots import Operation, Summary |
25 | 26 |
|
26 | 27 |
|
27 | 28 | def run_spark_commands(spark: SparkSession, sqls: List[str]) -> None:
|
28 | 29 | for sql in sqls:
|
29 | 30 | spark.sql(sql)
|
30 | 31 |
|
31 | 32 |
|
| 33 | +@pytest.mark.integration |
32 | 34 | @pytest.mark.parametrize("format_version", [1, 2])
|
33 | 35 | def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None:
|
34 | 36 | identifier = 'default.table_partitioned_delete'
|
@@ -63,6 +65,7 @@ def test_partitioned_table_delete_full_file(spark: SparkSession, session_catalog
|
63 | 65 | assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 11], 'number': [20, 30]}
|
64 | 66 |
|
65 | 67 |
|
| 68 | +@pytest.mark.integration |
66 | 69 | @pytest.mark.parametrize("format_version", [1, 2])
|
67 | 70 | def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None:
|
68 | 71 | identifier = 'default.table_partitioned_delete'
|
@@ -92,10 +95,12 @@ def test_partitioned_table_rewrite(spark: SparkSession, session_catalog: RestCat
|
92 | 95 | tbl = session_catalog.load_table(identifier)
|
93 | 96 | tbl.delete(EqualTo("number", 20))
|
94 | 97 |
|
95 |
| - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'delete', 'overwrite'] |
| 98 | + # We don't delete a whole partition, so there is only a overwrite |
| 99 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'append', 'overwrite'] |
96 | 100 | assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [11, 10], 'number': [30, 30]}
|
97 | 101 |
|
98 | 102 |
|
| 103 | +@pytest.mark.integration |
99 | 104 | @pytest.mark.parametrize("format_version", [1, 2])
|
100 | 105 | def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCatalog, format_version: int) -> None:
|
101 | 106 | identifier = 'default.table_partitioned_delete'
|
@@ -123,10 +128,11 @@ def test_partitioned_table_no_match(spark: SparkSession, session_catalog: RestCa
|
123 | 128 | tbl.delete(EqualTo("number_partitioned", 22)) # Does not affect any data
|
124 | 129 |
|
125 | 130 | # Open for discussion, do we want to create a new snapshot?
|
126 |
| - assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'delete'] |
| 131 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append'] |
127 | 132 | assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10, 10], 'number': [20, 30]}
|
128 | 133 |
|
129 | 134 |
|
| 135 | +@pytest.mark.integration |
130 | 136 | def test_partitioned_table_positional_deletes(spark: SparkSession, session_catalog: RestCatalog) -> None:
|
131 | 137 | identifier = 'default.table_partitioned_delete'
|
132 | 138 |
|
@@ -160,14 +166,105 @@ def test_partitioned_table_positional_deletes(spark: SparkSession, session_catal
|
160 | 166 |
|
161 | 167 | tbl = session_catalog.load_table(identifier)
|
162 | 168 |
|
163 |
| - # Assert that there is just a single Parquet file |
164 |
| - assert len(list(tbl.scan().plan_files())) == 1 |
| 169 | + # Assert that there is just a single Parquet file, that has one merge on read file |
| 170 | + files = list(tbl.scan().plan_files()) |
| 171 | + assert len(files) == 1 |
| 172 | + assert len(files[0].delete_files) == 1 |
165 | 173 |
|
166 | 174 | # Will rewrite a data file with a positional delete
|
167 | 175 | tbl.delete(EqualTo("number", 40))
|
168 | 176 |
|
169 |
| - # Yet another wrong status by Spark |
170 | 177 | # 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', 'overwrite'] |
| 178 | + # https://github.com/apache/iceberg/issues/10122 |
| 179 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()] == ['append', 'overwrite', 'overwrite'] |
173 | 180 | assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [10], 'number': [20]}
|
| 181 | + |
| 182 | + |
| 183 | +@pytest.mark.integration |
| 184 | +def test_partitioned_table_positional_deletes_sequence_number(spark: SparkSession, session_catalog: RestCatalog) -> None: |
| 185 | + identifier = 'default.table_partitioned_delete_sequence_number' |
| 186 | + |
| 187 | + # This test case is a bit more complex. Here we run a MoR delete on a file, we make sure that |
| 188 | + # the manifest gets rewritten (but not the data file with a MoR), and check if the delete is still there |
| 189 | + # to assure that the sequence numbers are maintained |
| 190 | + |
| 191 | + run_spark_commands( |
| 192 | + spark, |
| 193 | + [ |
| 194 | + f"DROP TABLE IF EXISTS {identifier}", |
| 195 | + f""" |
| 196 | + CREATE TABLE {identifier} ( |
| 197 | + number_partitioned int, |
| 198 | + number int |
| 199 | + ) |
| 200 | + USING iceberg |
| 201 | + PARTITIONED BY (number_partitioned) |
| 202 | + TBLPROPERTIES( |
| 203 | + 'format-version' = 2, |
| 204 | + 'write.delete.mode'='merge-on-read', |
| 205 | + 'write.update.mode'='merge-on-read', |
| 206 | + 'write.merge.mode'='merge-on-read' |
| 207 | + ) |
| 208 | + """, |
| 209 | + f""" |
| 210 | + INSERT INTO {identifier} VALUES (10, 100), (10, 101), (20, 200), (20, 201), (20, 202) |
| 211 | + """, |
| 212 | + # Generate a positional delete |
| 213 | + f""" |
| 214 | + DELETE FROM {identifier} WHERE number = 101 |
| 215 | + """, |
| 216 | + ], |
| 217 | + ) |
| 218 | + |
| 219 | + tbl = session_catalog.load_table(identifier) |
| 220 | + |
| 221 | + files = list(tbl.scan().plan_files()) |
| 222 | + assert len(files) == 2 |
| 223 | + |
| 224 | + # Will rewrite a data file with a positional delete |
| 225 | + tbl.delete(EqualTo("number", 201)) |
| 226 | + |
| 227 | + # One positional delete has been added, but an OVERWRITE status is set |
| 228 | + # https://github.com/apache/iceberg/issues/10122 |
| 229 | + snapshots = tbl.snapshots() |
| 230 | + assert len(snapshots) == 4 |
| 231 | + |
| 232 | + # Snapshots produced by Spark |
| 233 | + assert [snapshot.summary.operation.value for snapshot in tbl.snapshots()[0:2]] == ['append', 'overwrite'] |
| 234 | + |
| 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 | + ) |
| 248 | + # Will rewrite one parquet file |
| 249 | + assert tbl.snapshots()[3].summary == Summary( |
| 250 | + Operation.OVERWRITE, |
| 251 | + **{ |
| 252 | + 'added-files-size': '1145', |
| 253 | + 'added-data-files': '1', |
| 254 | + 'added-records': '2', |
| 255 | + 'changed-partition-count': '1', |
| 256 | + 'total-files-size': tbl.snapshots()[3].summary['total-files-size'], |
| 257 | + 'total-delete-files': '0', |
| 258 | + 'total-data-files': '1', |
| 259 | + 'total-position-deletes': '0', |
| 260 | + 'total-records': '2', |
| 261 | + 'total-equality-deletes': '0', |
| 262 | + 'deleted-data-files': '2', |
| 263 | + 'removed-delete-files': '1', |
| 264 | + 'deleted-records': '5', |
| 265 | + 'removed-files-size': '3088', |
| 266 | + 'removed-position-deletes': '1', |
| 267 | + }, |
| 268 | + ) |
| 269 | + |
| 270 | + assert tbl.scan().to_arrow().to_pydict() == {'number_partitioned': [20, 20, 10], 'number': [200, 202, 100]} |
0 commit comments