@@ -292,6 +292,59 @@ def test_delete_partitioned_table_positional_deletes_empty_batch(spark: SparkSes
292292 assert len (reader .read_all ()) == 0
293293
294294
295+ @pytest .mark .integration
296+ @pytest .mark .filterwarnings ("ignore:Merge on read is not yet supported, falling back to copy-on-write" )
297+ def test_read_multiple_batches_in_task_with_position_deletes (spark : SparkSession , session_catalog : RestCatalog ) -> None :
298+ identifier = "default.test_read_multiple_batches_in_task_with_position_deletes"
299+
300+ run_spark_commands (
301+ spark ,
302+ [
303+ f"DROP TABLE IF EXISTS { identifier } " ,
304+ f"""
305+ CREATE TABLE { identifier } (
306+ number int
307+ )
308+ USING iceberg
309+ TBLPROPERTIES(
310+ 'format-version' = 2,
311+ 'write.delete.mode'='merge-on-read',
312+ 'write.update.mode'='merge-on-read',
313+ 'write.merge.mode'='merge-on-read'
314+ )
315+ """ ,
316+ ],
317+ )
318+
319+ tbl = session_catalog .load_table (identifier )
320+
321+ arrow_table = pa .Table .from_arrays (
322+ [
323+ pa .array (list (range (1 , 1001 )) * 100 ),
324+ ],
325+ schema = pa .schema ([pa .field ("number" , pa .int32 ())]),
326+ )
327+
328+ tbl .append (arrow_table )
329+
330+ run_spark_commands (
331+ spark ,
332+ [
333+ f"""
334+ DELETE FROM { identifier } WHERE number in (1, 2, 3, 4)
335+ """ ,
336+ ],
337+ )
338+
339+ tbl .refresh ()
340+
341+ reader = tbl .scan (row_filter = "number <= 50" ).to_arrow_batch_reader ()
342+ assert isinstance (reader , pa .RecordBatchReader )
343+ pyiceberg_count = len (reader .read_all ())
344+ expected_count = 46 * 100
345+ assert pyiceberg_count == expected_count , f"Failing check. { pyiceberg_count } != { expected_count } "
346+
347+
295348@pytest .mark .integration
296349@pytest .mark .filterwarnings ("ignore:Merge on read is not yet supported, falling back to copy-on-write" )
297350def test_overwrite_partitioned_table (spark : SparkSession , session_catalog : RestCatalog ) -> None :
0 commit comments