@@ -222,6 +222,73 @@ def test_delete_partitioned_table_positional_deletes(spark: SparkSession, sessio
222222 assert tbl .scan ().to_arrow ().to_pydict () == {"number_partitioned" : [10 ], "number" : [20 ]}
223223
224224
225+ @pytest .mark .integration
226+ def test_delete_partitioned_table_positional_deletes_empty_batch (spark : SparkSession , session_catalog : RestCatalog ) -> None :
227+ identifier = "default.test_delete_partitioned_table_positional_deletes_empty_batch"
228+
229+ run_spark_commands (
230+ spark ,
231+ [
232+ f"DROP TABLE IF EXISTS { identifier } " ,
233+ f"""
234+ CREATE TABLE { identifier } (
235+ number_partitioned int,
236+ number int
237+ )
238+ USING iceberg
239+ PARTITIONED BY (number_partitioned)
240+ TBLPROPERTIES(
241+ 'format-version' = 2,
242+ 'write.delete.mode'='merge-on-read',
243+ 'write.update.mode'='merge-on-read',
244+ 'write.merge.mode'='merge-on-read',
245+ 'write.parquet.row-group-limit'=1
246+ )
247+ """ ,
248+ ],
249+ )
250+
251+ tbl = session_catalog .load_table (identifier )
252+
253+ arrow_table = pa .Table .from_arrays (
254+ [
255+ pa .array ([10 , 10 , 10 ]),
256+ pa .array ([1 , 2 , 3 ]),
257+ ],
258+ schema = pa .schema ([pa .field ("number_partitioned" , pa .int32 ()), pa .field ("number" , pa .int32 ())]),
259+ )
260+
261+ tbl .append (arrow_table )
262+
263+ assert len (tbl .scan ().to_arrow ()) == 3
264+
265+ run_spark_commands (
266+ spark ,
267+ [
268+ # Generate a positional delete
269+ f"""
270+ DELETE FROM { identifier } WHERE number = 1
271+ """ ,
272+ ],
273+ )
274+ # Assert that there is just a single Parquet file, that has one merge on read file
275+ tbl = tbl .refresh ()
276+
277+ files = list (tbl .scan ().plan_files ())
278+ assert len (files ) == 1
279+ assert len (files [0 ].delete_files ) == 1
280+
281+ assert len (tbl .scan ().to_arrow ()) == 2
282+
283+ assert len (tbl .scan (row_filter = "number_partitioned == 10" ).to_arrow ()) == 2
284+
285+ assert len (tbl .scan (row_filter = "number_partitioned == 1" ).to_arrow ()) == 0
286+
287+ reader = tbl .scan (row_filter = "number_partitioned == 1" ).to_arrow_batch_reader ()
288+ assert isinstance (reader , pa .RecordBatchReader )
289+ assert len (reader .read_all ()) == 0
290+
291+
225292@pytest .mark .integration
226293def test_overwrite_partitioned_table (spark : SparkSession , session_catalog : RestCatalog ) -> None :
227294 identifier = "default.table_partitioned_delete"
0 commit comments