bharath-techie opened a new issue, #19414:
URL: https://github.com/apache/datafusion/issues/19414
### Describe the bug
Running clickbench queries `q12` and `q33` with low memory [ 4 GB ] and
single target partition , found that spill file was taking more than 100 GB
and resources exhaustion error was thrown.
```
SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10;
```
On further investigation , found that for each sliced record batch , we
write the entire original array's buffer as the string view array was not GC'd.
### To Reproduce
Take one clickbench partitioned parquet file ~120 mb :
```
RUST_LOG=datafusion_physical_plan=debug ./datafusion-cli -m 40m
--disk-spill-path /home/ec2-user/spilldir --disk-limit 75g
SET datafusion.execution.listing_table_ignore_subdirectory = false;
SET datafusion.execution.target_partitions=1;
SET datafusion.execution.parquet.binary_as_string=true;
CREATE EXTERNAL TABLE hits
STORED AS PARQUET
LOCATION '/home/ec2-user/hits_0.parquet';
SELECT "URL", COUNT(*) AS c FROM hits GROUP BY "URL" ORDER BY c DESC LIMIT
10;
[2025-12-19T17:31:48Z DEBUG
datafusion_physical_plan::spill::in_progress_spill_file] [SPILL_FILE]
Finished spill file:
path="/var/folders/xx/b0vrbhv96tl6y9l2xzwr3twh0000gq/T/.tmpfswXa6/.tmpKujQH8",
size=820.56 MB, total_spilled_bytes=820.56 MB, total_spill_files=1
Resources exhausted: Additional allocation failed for
GroupedHashAggregateStream[0] (count(1)) with top memory consumers (across
reservations) as:
GroupedHashAggregateStream[0] (count(1))#5(can spill: true) consumed 16.4
KB, peak 39.5 MB,
DataFusion-Cli#4(can spill: false) consumed 0.0 B, peak 0.0 B,
TopK[0]#6(can spill: false) consumed 0.0 B, peak 0.0 B.
Error: Failed to allocate additional 60.3 MB for
GroupedHashAggregateStream[0] (count(1)) with 0.0 B already
allocated for this reservation - 40.0 MB remain available for the total pool
```
`size=820.56 MB, total_spilled_bytes=820.56 MB, total_spill_files=1` 820 MB
was getting written to spill file.
### Expected behavior
After GC'ing the stringView arrays , writing the compacted record batch
resulted in
```
2025-12-19T17:33:34Z DEBUG
datafusion_physical_plan::spill::in_progress_spill_file] [SPILL_FILE] Finished
spill file:
path="/var/folders/xx/b0vrbhv96tl6y9l2xzwr3twh0000gq/T/.tmprkn7C5/.tmptxEvdP",
size=33.43 MB, total_spilled_bytes=33.43 MB, total_spill_files=1
+--------------------------------------------------------------------------+-------+
| URL |
c |
+--------------------------------------------------------------------------+-------+
| http://irr.ru/index.php?showalbum/login-leniya7777294,938303130 |
58976 |
| http://komme%2F27.0.1453.116 |
29585 |
| https://produkty%2Fproduct |
11464 |
| http://irr.ru/index.php?showalbum/login-kapusta-advert2668]=0&order_by=0 |
10480 |
| http://irr.ru/index.php?showalbum/login-kapustic/product_name |
10128 |
| http://irr.ru/index.php |
7758 |
| https://produkty%2F |
6649 |
| http://irr.ru/index.php?showalbum/login |
6141 |
| https://produkty/kurortmag |
5764 |
| https://produkty%2Fpulove.ru/album/login |
5495 |
+--------------------------------------------------------------------------+-------+
10 row(s) fetched.
Elapsed 0.223 seconds.
```
`size=33.43 MB, total_spilled_bytes=33.43 MB, total_spill_files=1`
Need to write GC'd string view arrays to spill files. [ Unless there are
cases where this will not efficient ? ]
### Additional context
_No response_
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]