datafusion-physical-plan 54.0.0

Physical (ExecutionPlan) implementations for DataFusion query engine
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! Defines the spilling functions

pub(crate) mod in_progress_spill_file;
pub(crate) mod replayable_spill_input;
pub(crate) mod spill_manager;
pub mod spill_pool;

// Moved for refactor, re-export to keep the public API stable
pub use datafusion_common::utils::memory::get_record_batch_memory_size;
// Re-export SpillManager for doctests only (hidden from public docs)
#[doc(hidden)]
pub use spill_manager::SpillManager;

use std::fs::File;
use std::io::BufReader;
use std::path::{Path, PathBuf};
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};

use arrow::array::{
    Array, ArrayRef, BinaryViewArray, BufferSpec, GenericByteViewArray, StringViewArray,
    layout, make_array,
};
use arrow::datatypes::DataType;
use arrow::datatypes::{ByteViewType, Schema, SchemaRef};
use arrow::ipc::{
    MetadataVersion,
    reader::StreamReader,
    writer::{IpcWriteOptions, StreamWriter},
};
use arrow::record_batch::RecordBatch;
use arrow_data::ArrayDataBuilder;
use arrow_ipc::CompressionType;

use datafusion_common::config::SpillCompression;
use datafusion_common::{DataFusionError, Result, exec_datafusion_err, exec_err};
use datafusion_common_runtime::SpawnedTask;
use datafusion_execution::RecordBatchStream;
use datafusion_execution::disk_manager::RefCountedTempFile;
use futures::{FutureExt as _, Stream};
use log::debug;

/// Stream that reads spill files from disk where each batch is read in a spawned blocking task
/// It will read one batch at a time and will not do any buffering, to buffer data use [`crate::common::spawn_buffered`]
///
/// A simpler solution would be spawning a long-running blocking task for each
/// file read (instead of each batch). This approach does not work because when
/// the number of concurrent reads exceeds the Tokio thread pool limit,
/// deadlocks can occur and block progress.
struct SpillReaderStream {
    schema: SchemaRef,
    state: SpillReaderStreamState,
    /// Maximum memory size observed among spilling sorted record batches.
    /// This is used for validation purposes during reading each RecordBatch from spill.
    /// For context on why this value is recorded and validated,
    /// see `physical_plan/sort/multi_level_merge.rs`.
    max_record_batch_memory: Option<usize>,
}

// Small margin allowed to accommodate slight memory accounting variation
const SPILL_BATCH_MEMORY_MARGIN: usize = 4096;

/// When we poll for the next batch, we will get back both the batch and the reader,
/// so we can call `next` again.
type NextRecordBatchResult = Result<(StreamReader<BufReader<File>>, Option<RecordBatch>)>;

enum SpillReaderStreamState {
    /// Initial state: the stream was not initialized yet
    /// and the file was not opened
    Uninitialized(RefCountedTempFile),

    /// A read is in progress in a spawned blocking task for which we hold the handle.
    ReadInProgress(SpawnedTask<NextRecordBatchResult>),

    /// A read has finished and we wait for being polled again in order to start reading the next batch.
    Waiting(StreamReader<BufReader<File>>),

    /// The stream has finished, successfully or not.
    Done,
}

impl SpillReaderStream {
    fn new(
        schema: SchemaRef,
        spill_file: RefCountedTempFile,
        max_record_batch_memory: Option<usize>,
    ) -> Self {
        Self {
            schema,
            state: SpillReaderStreamState::Uninitialized(spill_file),
            max_record_batch_memory,
        }
    }

    fn poll_next_inner(
        &mut self,
        cx: &mut Context<'_>,
    ) -> Poll<Option<Result<RecordBatch>>> {
        match &mut self.state {
            SpillReaderStreamState::Uninitialized(_) => {
                // Temporarily replace with `Done` to be able to pass the file to the task.
                let SpillReaderStreamState::Uninitialized(spill_file) =
                    std::mem::replace(&mut self.state, SpillReaderStreamState::Done)
                else {
                    unreachable!()
                };

                let expected_schema = Arc::clone(&self.schema);
                let task = SpawnedTask::spawn_blocking(move || {
                    let file = BufReader::new(File::open(spill_file.path())?);
                    // SAFETY: DataFusion's spill writer strictly follows Arrow IPC specifications
                    // with validated schemas and buffers. Skip redundant validation during read
                    // to speedup read operation. This is safe for DataFusion as input guaranteed to be correct when written.
                    let mut reader = unsafe {
                        StreamReader::try_new(file, None)?.with_skip_validation(true)
                    };

                    // Validate the schema read from Arrow IPC file is the same as the
                    // schema of the current `SpillManager`
                    let actual_schema = reader.schema();

                    if actual_schema != expected_schema {
                        return exec_err!(
                            "Spill file schema mismatch: expected {}, got {}. \
                            The caller must use the same SpillManager that created the spill file to read it.",
                            expected_schema,
                            actual_schema
                        );
                    }

                    // TODO: Same-schema reads from a different SpillManager still pass today.
                    // Add a SpillManager UID to IPC metadata and validate it here as well.
                    let next_batch = reader.next().transpose()?;

                    Ok((reader, next_batch))
                });

                self.state = SpillReaderStreamState::ReadInProgress(task);

                // Poll again immediately so the inner task is polled and the waker is
                // registered.
                self.poll_next_inner(cx)
            }

            SpillReaderStreamState::ReadInProgress(task) => {
                let result = futures::ready!(task.poll_unpin(cx))
                    .unwrap_or_else(|err| Err(DataFusionError::External(Box::new(err))));

                match result {
                    Ok((reader, batch)) => {
                        match batch {
                            Some(batch) => {
                                if let Some(max_record_batch_memory) =
                                    self.max_record_batch_memory
                                {
                                    let actual_size =
                                        get_record_batch_memory_size(&batch);
                                    if actual_size
                                        > max_record_batch_memory
                                            + SPILL_BATCH_MEMORY_MARGIN
                                    {
                                        debug!(
                                            "Record batch memory usage ({actual_size} bytes) exceeds the expected limit ({max_record_batch_memory} bytes) \n\
                                                by more than the allowed tolerance ({SPILL_BATCH_MEMORY_MARGIN} bytes).\n\
                                                This likely indicates a bug in memory accounting during spilling.\n\
                                                Please report this issue in https://github.com/apache/datafusion/issues/17340."
                                        );
                                    }
                                }
                                self.state = SpillReaderStreamState::Waiting(reader);

                                Poll::Ready(Some(Ok(batch)))
                            }
                            None => {
                                // Stream is done
                                self.state = SpillReaderStreamState::Done;

                                Poll::Ready(None)
                            }
                        }
                    }
                    Err(err) => {
                        self.state = SpillReaderStreamState::Done;

                        Poll::Ready(Some(Err(err)))
                    }
                }
            }

            SpillReaderStreamState::Waiting(_) => {
                // Temporarily replace with `Done` to be able to pass the file to the task.
                let SpillReaderStreamState::Waiting(mut reader) =
                    std::mem::replace(&mut self.state, SpillReaderStreamState::Done)
                else {
                    unreachable!()
                };

                let task = SpawnedTask::spawn_blocking(move || {
                    let next_batch = reader.next().transpose()?;

                    Ok((reader, next_batch))
                });

                self.state = SpillReaderStreamState::ReadInProgress(task);

                // Poll again immediately so the inner task is polled and the waker is
                // registered.
                self.poll_next_inner(cx)
            }

            SpillReaderStreamState::Done => Poll::Ready(None),
        }
    }
}

impl Stream for SpillReaderStream {
    type Item = Result<RecordBatch>;

    fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
        self.get_mut().poll_next_inner(cx)
    }
}

impl RecordBatchStream for SpillReaderStream {
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }
}

/// Spill the `RecordBatch` to disk as smaller batches
/// split by `batch_size_rows`
#[deprecated(
    since = "46.0.0",
    note = "This method is deprecated. Use `SpillManager::spill_record_batch_by_size` instead."
)]
#[expect(clippy::needless_pass_by_value)]
pub fn spill_record_batch_by_size(
    batch: &RecordBatch,
    path: PathBuf,
    schema: SchemaRef,
    batch_size_rows: usize,
) -> Result<()> {
    let mut offset = 0;
    let total_rows = batch.num_rows();
    let mut writer =
        IPCStreamWriter::new(&path, schema.as_ref(), SpillCompression::Uncompressed)?;

    while offset < total_rows {
        let length = std::cmp::min(total_rows - offset, batch_size_rows);
        let batch = batch.slice(offset, length);
        offset += batch.num_rows();
        writer.write(&batch)?;
    }
    writer.finish()?;

    Ok(())
}

/// Write in Arrow IPC Stream format to a file.
///
/// Stream format is used for spill because it supports dictionary replacement, and the random
/// access of IPC File format is not needed (IPC File format doesn't support dictionary replacement).
struct IPCStreamWriter {
    /// Inner writer
    pub writer: StreamWriter<File>,
    /// Batches written
    pub num_batches: usize,
    /// Rows written
    pub num_rows: usize,
    /// Bytes written
    pub num_bytes: usize,
}

impl IPCStreamWriter {
    /// Create new writer
    ///
    /// # Codec contract
    ///
    /// `arrow-ipc` must be compiled with the `lz4` and `zstd` features
    /// (declared explicitly in `datafusion-physical-plan/Cargo.toml`). If
    /// those features are absent, `try_with_compression` will return an
    /// error at runtime for [`SpillCompression::Lz4Frame`] and
    /// [`SpillCompression::Zstd`] variants. The Cargo dependency keeps this
    /// contract local and build-visible during Cargo feature resolution,
    /// rather than relying solely on workspace-level feature unification;
    /// see #21917.
    pub fn new(
        path: &Path,
        schema: &Schema,
        spill_compression: SpillCompression,
    ) -> Result<Self> {
        let file = File::create(path).map_err(|e| {
            exec_datafusion_err!("(Hint: you may increase the file descriptor limit with shell command 'ulimit -n 4096') Failed to create partition file at {path:?}: {e:?}")
        })?;

        let metadata_version = MetadataVersion::V5;
        // Depending on the schema, some array types such as StringViewArray require larger (16 byte in this case) alignment.
        // If the actual buffer layout after IPC read does not satisfy the alignment requirement,
        // Arrow ArrayBuilder will copy the buffer into a newly allocated, properly aligned buffer.
        // This copying may lead to memory blowup during IPC read due to duplicated buffers.
        // To avoid this, we compute the maximum required alignment based on the schema and configure the IPCStreamWriter accordingly.
        let alignment = get_max_alignment_for_schema(schema);
        let mut write_options =
            IpcWriteOptions::try_new(alignment, false, metadata_version)?;
        let compression_type = Option::<CompressionType>::from(spill_compression);
        write_options = write_options.try_with_compression(compression_type)?;

        let writer = StreamWriter::try_new_with_options(file, schema, write_options)?;
        Ok(Self {
            num_batches: 0,
            num_rows: 0,
            num_bytes: 0,
            writer,
        })
    }

    /// Writes a single batch to the IPC stream and updates the internal counters.
    ///
    /// Returns a tuple containing the change in the number of rows and bytes written.
    pub fn write(&mut self, batch: &RecordBatch) -> Result<(usize, usize)> {
        self.writer.write(batch)?;
        self.num_batches += 1;
        let delta_num_rows = batch.num_rows();
        self.num_rows += delta_num_rows;
        let delta_num_bytes: usize = batch.get_array_memory_size();
        self.num_bytes += delta_num_bytes;
        Ok((delta_num_rows, delta_num_bytes))
    }

    pub fn flush(&mut self) -> Result<()> {
        self.writer.flush()?;
        Ok(())
    }

    /// Finish the writer
    pub fn finish(&mut self) -> Result<()> {
        self.writer.finish().map_err(Into::into)
    }
}

// Returns the maximum byte alignment required by any field in the schema (>= 8), derived from Arrow buffer layouts.
fn get_max_alignment_for_schema(schema: &Schema) -> usize {
    let minimum_alignment = 8;
    let mut max_alignment = minimum_alignment;
    for field in schema.fields() {
        let layout = layout(field.data_type());
        let required_alignment = layout
            .buffers
            .iter()
            .map(|buffer_spec| {
                if let BufferSpec::FixedWidth { alignment, .. } = buffer_spec {
                    *alignment
                } else {
                    minimum_alignment
                }
            })
            .max()
            .unwrap_or(minimum_alignment);
        max_alignment = std::cmp::max(max_alignment, required_alignment);
    }
    max_alignment
}

/// Size of a single view structure in StringView/BinaryView arrays (in bytes).
/// Each view is 16 bytes: 4 bytes length + 4 bytes prefix + 8 bytes buffer ID/offset.
const VIEW_SIZE_BYTES: usize = 16;

/// Performs garbage collection on StringView and BinaryView arrays before spilling to reduce memory usage.
///
/// # Why GC is needed
///
/// StringView and BinaryView arrays can accumulate significant memory waste when sliced.
/// When a large array is sliced (e.g., taking first 100 rows of 1000), the view array
/// still references the original data buffers containing all 1000 rows of data.
///
/// For example, in the ClickBench benchmark (issue #19414), repeated slicing of StringView
/// arrays resulted in 820MB of spill files that could be reduced to just 33MB after GC -
/// a 96% reduction in size.
///
/// # How it works
///
/// The GC process:
/// 1. Identifies view arrays (StringView/BinaryView) in the batch
/// 2. Checks if their data buffers exceed a memory threshold
/// 3. If exceeded, calls the Arrow `gc()` method which creates new compact buffers
///    containing only the data referenced by the current views
/// 4. Returns a new batch with GC'd arrays (or original arrays if GC not needed)
///
/// # When GC is triggered
///
/// GC is only performed when data buffers exceed a threshold (currently 10KB).
/// This balances memory savings against the CPU overhead of garbage collection.
/// Small arrays are passed through unchanged since the GC overhead would exceed
/// any memory savings.
///
/// # Performance considerations
///
/// - If no view arrays need compaction, the original batch is cloned cheaply
/// - GC is skipped for small buffers to avoid unnecessary CPU overhead
/// - Nested container types are traversed recursively so view arrays inside
///   `List`, `Map`, `Union`, `Dictionary`, and other child-bearing arrays are compacted too
/// - The Arrow `gc()` method itself is optimized and only copies referenced data
pub(crate) fn gc_view_arrays(batch: &RecordBatch) -> Result<RecordBatch> {
    let mut mutated = false;
    let mut new_columns: Vec<Arc<dyn Array>> = Vec::with_capacity(batch.num_columns());

    for array in batch.columns() {
        let (gc_array, array_mutated) = gc_array(array)?;
        mutated |= array_mutated;
        new_columns.push(gc_array);
    }

    if mutated {
        Ok(RecordBatch::try_new(batch.schema(), new_columns)?)
    } else {
        Ok(batch.clone())
    }
}

fn gc_array(array: &ArrayRef) -> Result<(ArrayRef, bool)> {
    match array.data_type() {
        DataType::Utf8View => {
            let string_view = array
                .as_any()
                .downcast_ref::<StringViewArray>()
                .expect("Utf8View array should downcast to StringViewArray");
            if should_gc_view_array(string_view) {
                Ok((Arc::new(string_view.gc()) as ArrayRef, true))
            } else {
                Ok((Arc::clone(array), false))
            }
        }
        DataType::BinaryView => {
            let binary_view = array
                .as_any()
                .downcast_ref::<BinaryViewArray>()
                .expect("BinaryView array should downcast to BinaryViewArray");
            if should_gc_view_array(binary_view) {
                Ok((Arc::new(binary_view.gc()) as ArrayRef, true))
            } else {
                Ok((Arc::clone(array), false))
            }
        }
        _ => gc_array_children(array),
    }
}

fn gc_array_children(array: &ArrayRef) -> Result<(ArrayRef, bool)> {
    let data = array.to_data();
    if data.child_data().is_empty() {
        return Ok((Arc::clone(array), false));
    }

    let mut mutated = false;
    let mut child_data = Vec::with_capacity(data.child_data().len());
    for child in data.child_data() {
        let child_array = make_array(child.clone());
        let (gc_child, child_mutated) = gc_array(&child_array)?;
        mutated |= child_mutated;
        child_data.push(gc_child.to_data());
    }

    if !mutated {
        return Ok((Arc::clone(array), false));
    }

    let rebuilt = ArrayDataBuilder::new(data.data_type().clone())
        .len(data.len())
        .offset(data.offset())
        .nulls(data.nulls().cloned())
        .buffers(data.buffers().to_vec())
        .child_data(child_data)
        .build()?;

    Ok((make_array(rebuilt), true))
}

/// Determines whether a view array should be garbage collected before spilling.
///
/// Arrow's `gc()` always allocates new compact buffers (it is never a no-op), so we
/// check here to skip the allocation cost when data buffers are small. We subtract
/// the views buffer (16 bytes × n_rows) from `get_buffer_memory_size()` so the
/// threshold tracks non-inline string data rather than row count.
fn should_gc_view_array<T: ByteViewType>(array: &GenericByteViewArray<T>) -> bool {
    const MIN_BUFFER_SIZE_FOR_GC: usize = 10 * 1024; // 10KB threshold

    if array.data_buffers().is_empty() {
        return false;
    }

    let data_buffer_size = array
        .get_buffer_memory_size()
        .saturating_sub(array.len() * VIEW_SIZE_BYTES);
    data_buffer_size > MIN_BUFFER_SIZE_FOR_GC
}

#[cfg(test)]
fn calculate_string_view_waste_ratio(array: &StringViewArray) -> f64 {
    use arrow_data::MAX_INLINE_VIEW_LEN;
    calculate_view_waste_ratio(array.len(), array.data_buffers(), |i| {
        if !array.is_null(i) {
            let value = array.value(i);
            if value.len() > MAX_INLINE_VIEW_LEN as usize {
                return value.len();
            }
        }
        0
    })
}

#[cfg(test)]
fn calculate_view_waste_ratio<F>(
    len: usize,
    data_buffers: &[arrow::buffer::Buffer],
    get_value_size: F,
) -> f64
where
    F: Fn(usize) -> usize,
{
    let total_buffer_size: usize = data_buffers.iter().map(|b| b.capacity()).sum();
    if total_buffer_size == 0 {
        return 0.0;
    }

    let mut actual_used_size = (0..len).map(get_value_size).sum::<usize>();
    actual_used_size += len * VIEW_SIZE_BYTES;

    let waste = total_buffer_size.saturating_sub(actual_used_size);
    waste as f64 / total_buffer_size as f64
}

#[cfg(test)]
mod tests {
    use super::in_progress_spill_file::InProgressSpillFile;
    use super::*;
    use crate::common::collect;
    use crate::metrics::ExecutionPlanMetricsSet;
    use crate::metrics::SpillMetrics;
    use crate::spill::spill_manager::SpillManager;
    use crate::test::build_table_i32;
    use arrow::array::{ArrayRef, Int32Array, StringArray};
    use arrow::compute::cast;
    use arrow::datatypes::{DataType, Field};
    use datafusion_execution::runtime_env::RuntimeEnv;
    use futures::StreamExt as _;

    #[tokio::test]
    async fn test_batch_spill_and_read() -> Result<()> {
        let batch1 = build_table_i32(
            ("a2", &vec![0, 1, 2]),
            ("b2", &vec![3, 4, 5]),
            ("c2", &vec![4, 5, 6]),
        );

        let batch2 = build_table_i32(
            ("a2", &vec![10, 11, 12]),
            ("b2", &vec![13, 14, 15]),
            ("c2", &vec![14, 15, 16]),
        );

        let schema = batch1.schema();
        let num_rows = batch1.num_rows() + batch2.num_rows();

        // Construct SpillManager
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let spill_manager = SpillManager::new(env, metrics, Arc::clone(&schema));

        let spill_file = spill_manager
            .spill_record_batch_and_finish(&[batch1, batch2], "Test")?
            .unwrap();
        assert!(spill_file.path().exists());
        let spilled_rows = spill_manager.metrics.spilled_rows.value();
        assert_eq!(spilled_rows, num_rows);

        let stream = spill_manager.read_spill_as_stream(spill_file, None)?;
        assert_eq!(stream.schema(), schema);

        let batches = collect(stream).await?;
        assert_eq!(batches.len(), 2);

        Ok(())
    }

    #[tokio::test]
    async fn test_batch_spill_and_read_dictionary_arrays() -> Result<()> {
        // See https://github.com/apache/datafusion/issues/4658

        let batch1 = build_table_i32(
            ("a2", &vec![0, 1, 2]),
            ("b2", &vec![3, 4, 5]),
            ("c2", &vec![4, 5, 6]),
        );

        let batch2 = build_table_i32(
            ("a2", &vec![10, 11, 12]),
            ("b2", &vec![13, 14, 15]),
            ("c2", &vec![14, 15, 16]),
        );

        // Dictionary encode the arrays
        let dict_type =
            DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Int32));
        let dict_schema = Arc::new(Schema::new(vec![
            Field::new("a2", dict_type.clone(), true),
            Field::new("b2", dict_type.clone(), true),
            Field::new("c2", dict_type.clone(), true),
        ]));

        let batch1 = RecordBatch::try_new(
            Arc::clone(&dict_schema),
            batch1
                .columns()
                .iter()
                .map(|array| cast(array, &dict_type))
                .collect::<Result<_, _>>()?,
        )?;

        let batch2 = RecordBatch::try_new(
            Arc::clone(&dict_schema),
            batch2
                .columns()
                .iter()
                .map(|array| cast(array, &dict_type))
                .collect::<Result<_, _>>()?,
        )?;

        // Construct SpillManager
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let spill_manager = SpillManager::new(env, metrics, Arc::clone(&dict_schema));

        let num_rows = batch1.num_rows() + batch2.num_rows();
        let spill_file = spill_manager
            .spill_record_batch_and_finish(&[batch1, batch2], "Test")?
            .unwrap();
        let spilled_rows = spill_manager.metrics.spilled_rows.value();
        assert_eq!(spilled_rows, num_rows);

        let stream = spill_manager.read_spill_as_stream(spill_file, None)?;
        assert_eq!(stream.schema(), dict_schema);
        let batches = collect(stream).await?;
        assert_eq!(batches.len(), 2);

        Ok(())
    }

    #[tokio::test]
    async fn test_batch_spill_by_size() -> Result<()> {
        let batch1 = build_table_i32(
            ("a2", &vec![0, 1, 2, 3]),
            ("b2", &vec![3, 4, 5, 6]),
            ("c2", &vec![4, 5, 6, 7]),
        );

        let schema = batch1.schema();
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let spill_manager = SpillManager::new(env, metrics, Arc::clone(&schema));

        let row_batches: Vec<RecordBatch> =
            (0..batch1.num_rows()).map(|i| batch1.slice(i, 1)).collect();
        let (spill_file, max_batch_mem) = spill_manager
            .spill_record_batch_iter_and_return_max_batch_memory(
                row_batches.iter().map(Ok),
                "Test Spill",
            )?
            .unwrap();
        assert!(spill_file.path().exists());
        assert!(max_batch_mem > 0);

        let stream = spill_manager.read_spill_as_stream(spill_file, None)?;
        assert_eq!(stream.schema(), schema);

        let batches = collect(stream).await?;
        assert_eq!(batches.len(), 4);

        Ok(())
    }

    fn build_compressible_batch() -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Utf8, false),
            Field::new("b", DataType::Int32, false),
            Field::new("c", DataType::Int32, true),
        ]));

        let a: ArrayRef = Arc::new(StringArray::from_iter_values(std::iter::repeat_n(
            "repeated", 100,
        )));
        let b: ArrayRef = Arc::new(Int32Array::from(vec![1; 100]));
        let c: ArrayRef = Arc::new(Int32Array::from(vec![2; 100]));

        RecordBatch::try_new(schema, vec![a, b, c]).unwrap()
    }

    async fn validate(
        spill_manager: &SpillManager,
        spill_file: RefCountedTempFile,
        num_rows: usize,
        schema: SchemaRef,
        batch_count: usize,
    ) -> Result<()> {
        let spilled_rows = spill_manager.metrics.spilled_rows.value();
        assert_eq!(spilled_rows, num_rows);

        let stream = spill_manager.read_spill_as_stream(spill_file, None)?;
        assert_eq!(stream.schema(), schema);

        let batches = collect(stream).await?;
        assert_eq!(batches.len(), batch_count);

        Ok(())
    }

    #[tokio::test]
    async fn test_spill_compression() -> Result<()> {
        let batch = build_compressible_batch();
        let num_rows = batch.num_rows();
        let schema = batch.schema();
        let batch_count = 1;
        let batches = [batch];

        // Construct SpillManager
        let env = Arc::new(RuntimeEnv::default());
        let uncompressed_metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let lz4_metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let zstd_metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let uncompressed_spill_manager = SpillManager::new(
            Arc::clone(&env),
            uncompressed_metrics,
            Arc::clone(&schema),
        );
        let lz4_spill_manager =
            SpillManager::new(Arc::clone(&env), lz4_metrics, Arc::clone(&schema))
                .with_compression_type(SpillCompression::Lz4Frame);
        let zstd_spill_manager =
            SpillManager::new(env, zstd_metrics, Arc::clone(&schema))
                .with_compression_type(SpillCompression::Zstd);
        let uncompressed_spill_file = uncompressed_spill_manager
            .spill_record_batch_and_finish(&batches, "Test")?
            .unwrap();
        let lz4_spill_file = lz4_spill_manager
            .spill_record_batch_and_finish(&batches, "Lz4_Test")?
            .unwrap();
        let zstd_spill_file = zstd_spill_manager
            .spill_record_batch_and_finish(&batches, "ZSTD_Test")?
            .unwrap();
        assert!(uncompressed_spill_file.path().exists());
        assert!(lz4_spill_file.path().exists());
        assert!(zstd_spill_file.path().exists());

        let lz4_spill_size = std::fs::metadata(lz4_spill_file.path())?.len();
        let zstd_spill_size = std::fs::metadata(zstd_spill_file.path())?.len();
        let uncompressed_spill_size =
            std::fs::metadata(uncompressed_spill_file.path())?.len();

        assert!(uncompressed_spill_size > lz4_spill_size);
        assert!(uncompressed_spill_size > zstd_spill_size);

        validate(
            &lz4_spill_manager,
            lz4_spill_file,
            num_rows,
            Arc::clone(&schema),
            batch_count,
        )
        .await?;
        validate(
            &zstd_spill_manager,
            zstd_spill_file,
            num_rows,
            Arc::clone(&schema),
            batch_count,
        )
        .await?;
        validate(
            &uncompressed_spill_manager,
            uncompressed_spill_file,
            num_rows,
            schema,
            batch_count,
        )
        .await?;
        Ok(())
    }

    // ==== Spill manager tests ====

    #[test]
    fn test_spill_manager_spill_record_batch_and_finish() -> Result<()> {
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Utf8, false),
        ]));

        let spill_manager = SpillManager::new(env, metrics, Arc::clone(&schema));

        let batch = RecordBatch::try_new(
            schema,
            vec![
                Arc::new(Int32Array::from(vec![1, 2, 3])),
                Arc::new(StringArray::from(vec!["a", "b", "c"])),
            ],
        )?;

        let temp_file = spill_manager.spill_record_batch_and_finish(&[batch], "Test")?;
        assert!(temp_file.is_some());
        assert!(temp_file.unwrap().path().exists());
        Ok(())
    }

    fn verify_metrics(
        in_progress_file: &InProgressSpillFile,
        expected_spill_file_count: usize,
        expected_spilled_bytes: usize,
        expected_spilled_rows: usize,
    ) -> Result<()> {
        let actual_spill_file_count = in_progress_file
            .spill_writer
            .metrics
            .spill_file_count
            .value();
        let actual_spilled_bytes =
            in_progress_file.spill_writer.metrics.spilled_bytes.value();
        let actual_spilled_rows =
            in_progress_file.spill_writer.metrics.spilled_rows.value();

        assert_eq!(
            actual_spill_file_count, expected_spill_file_count,
            "Spill file count mismatch"
        );
        assert_eq!(
            actual_spilled_bytes, expected_spilled_bytes,
            "Spilled bytes mismatch"
        );
        assert_eq!(
            actual_spilled_rows, expected_spilled_rows,
            "Spilled rows mismatch"
        );

        Ok(())
    }

    #[test]
    fn test_in_progress_spill_file_append_and_finish() -> Result<()> {
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Utf8, false),
        ]));

        let spill_manager =
            Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema)));
        let mut in_progress_file = spill_manager.create_in_progress_file("Test")?;

        let batch1 = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![
                Arc::new(Int32Array::from(vec![1, 2, 3])),
                Arc::new(StringArray::from(vec!["a", "b", "c"])),
            ],
        )?;

        let batch2 = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![
                Arc::new(Int32Array::from(vec![4, 5, 6])),
                Arc::new(StringArray::from(vec!["d", "e", "f"])),
            ],
        )?;
        // After appending each batch, spilled_rows and spilled_bytes should increase incrementally,
        // while spill_file_count remains 1 (since we're writing to the same file)
        in_progress_file.append_batch(&batch1)?;
        verify_metrics(&in_progress_file, 1, 440, 3)?;

        in_progress_file.append_batch(&batch2)?;
        verify_metrics(&in_progress_file, 1, 704, 6)?;

        let completed_file = in_progress_file.finish()?;
        assert!(completed_file.is_some());
        assert!(completed_file.unwrap().path().exists());
        verify_metrics(&in_progress_file, 1, 712, 6)?;
        // Double finish produce error
        let result = in_progress_file.finish();
        assert!(result.is_err());

        Ok(())
    }

    // Test write no batches
    #[test]
    fn test_in_progress_spill_file_write_no_batches() -> Result<()> {
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Utf8, false),
        ]));

        let spill_manager =
            Arc::new(SpillManager::new(env, metrics, Arc::clone(&schema)));

        // Test write empty batch with interface `InProgressSpillFile` and `append_batch()`
        let mut in_progress_file = spill_manager.create_in_progress_file("Test")?;
        let completed_file = in_progress_file.finish()?;
        assert!(completed_file.is_none());

        // Test write empty batch with interface `spill_record_batch_and_finish()`
        let completed_file = spill_manager.spill_record_batch_and_finish(&[], "Test")?;
        assert!(completed_file.is_none());

        // Test write empty batch with interface `spill_record_batch_iter_and_return_max_batch_memory()`
        let empty_batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![
                Arc::new(Int32Array::from(Vec::<Option<i32>>::new())),
                Arc::new(StringArray::from(Vec::<Option<&str>>::new())),
            ],
        )?;
        let completed_file = spill_manager
            .spill_record_batch_iter_and_return_max_batch_memory(
                std::iter::once(Ok(&empty_batch)),
                "Test",
            )?;
        assert!(completed_file.is_none());

        Ok(())
    }

    #[test]
    fn test_reading_more_spills_than_tokio_blocking_threads() -> Result<()> {
        tokio::runtime::Builder::new_current_thread()
            .enable_all()
            .max_blocking_threads(1)
            .build()
            .unwrap()
            .block_on(async {
                let batch = build_table_i32(
                    ("a2", &vec![0, 1, 2]),
                    ("b2", &vec![3, 4, 5]),
                    ("c2", &vec![4, 5, 6]),
                );

                let schema = batch.schema();

                // Construct SpillManager
                let env = Arc::new(RuntimeEnv::default());
                let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
                let spill_manager = SpillManager::new(env, metrics, Arc::clone(&schema));
                let batches: [_; 10] = std::array::from_fn(|_| batch.clone());

                let spill_file_1 = spill_manager
                    .spill_record_batch_and_finish(&batches, "Test1")?
                    .unwrap();
                let spill_file_2 = spill_manager
                    .spill_record_batch_and_finish(&batches, "Test2")?
                    .unwrap();

                let mut stream_1 =
                    spill_manager.read_spill_as_stream(spill_file_1, None)?;
                let mut stream_2 =
                    spill_manager.read_spill_as_stream(spill_file_2, None)?;
                stream_1.next().await;
                stream_2.next().await;

                Ok(())
            })
    }

    #[test]
    fn test_alignment_for_schema() -> Result<()> {
        let schema = Schema::new(vec![Field::new("strings", DataType::Utf8View, false)]);
        let alignment = get_max_alignment_for_schema(&schema);
        assert_eq!(alignment, 16);

        let schema = Schema::new(vec![
            Field::new("int32", DataType::Int32, false),
            Field::new("int64", DataType::Int64, false),
        ]);
        let alignment = get_max_alignment_for_schema(&schema);
        assert_eq!(alignment, 8);
        Ok(())
    }
    #[tokio::test]
    async fn test_real_time_spill_metrics() -> Result<()> {
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let schema = Arc::new(Schema::new(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Utf8, false),
        ]));

        let spill_manager = Arc::new(SpillManager::new(
            Arc::clone(&env),
            metrics.clone(),
            Arc::clone(&schema),
        ));
        let mut in_progress_file = spill_manager.create_in_progress_file("Test")?;

        let batch1 = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![
                Arc::new(Int32Array::from(vec![1, 2, 3])),
                Arc::new(StringArray::from(vec!["a", "b", "c"])),
            ],
        )?;

        // Before any batch, metrics should be 0
        assert_eq!(metrics.spilled_bytes.value(), 0);
        assert_eq!(metrics.spill_file_count.value(), 0);

        // Append first batch
        in_progress_file.append_batch(&batch1)?;

        // Metrics should be updated immediately (at least schema and first batch)
        let bytes_after_batch1 = metrics.spilled_bytes.value();
        assert_eq!(bytes_after_batch1, 440);
        assert_eq!(metrics.spill_file_count.value(), 1);

        // Check global progress
        let progress = env.spilling_progress();
        assert_eq!(progress.current_bytes, bytes_after_batch1 as u64);
        assert_eq!(progress.active_files_count, 1);

        // Append another batch
        in_progress_file.append_batch(&batch1)?;
        let bytes_after_batch2 = metrics.spilled_bytes.value();
        assert!(bytes_after_batch2 > bytes_after_batch1);

        // Check global progress again
        let progress = env.spilling_progress();
        assert_eq!(progress.current_bytes, bytes_after_batch2 as u64);

        // Finish the file
        let spilled_file = in_progress_file.finish()?;
        let final_bytes = metrics.spilled_bytes.value();
        assert!(final_bytes > bytes_after_batch2);

        // Even after finish, file is still "active" until dropped
        let progress = env.spilling_progress();
        assert!(progress.current_bytes > 0);
        assert_eq!(progress.active_files_count, 1);

        drop(spilled_file);
        assert_eq!(env.spilling_progress().active_files_count, 0);
        assert_eq!(env.spilling_progress().current_bytes, 0);

        Ok(())
    }

    #[test]
    fn test_gc_string_view_before_spill() -> Result<()> {
        use arrow::array::StringViewArray;

        let strings: Vec<String> = (0..200)
            .map(|i| {
                if i % 2 == 0 {
                    "short_string".to_string()
                } else {
                    "this_is_a_much_longer_string_that_will_not_be_inlined".to_string()
                }
            })
            .collect();

        let string_array = StringViewArray::from(strings);
        let schema = Arc::new(Schema::new(vec![Field::new(
            "strings",
            DataType::Utf8View,
            false,
        )]));

        let batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![Arc::new(string_array) as ArrayRef],
        )?;
        let sliced_batch = batch.slice(0, 20);
        let gc_batch = gc_view_arrays(&sliced_batch)?;

        assert_eq!(gc_batch.num_rows(), sliced_batch.num_rows());
        assert_eq!(gc_batch.num_columns(), sliced_batch.num_columns());

        Ok(())
    }

    #[test]
    fn test_gc_binary_view_before_spill() -> Result<()> {
        use arrow::array::BinaryViewArray;

        let binaries: Vec<Vec<u8>> = (0..200)
            .map(|i| {
                if i % 2 == 0 {
                    vec![1, 2, 3, 4]
                } else {
                    vec![1; 50]
                }
            })
            .collect();

        let binary_array =
            BinaryViewArray::from_iter(binaries.iter().map(|b| Some(b.as_slice())));
        let schema = Arc::new(Schema::new(vec![Field::new(
            "binaries",
            DataType::BinaryView,
            false,
        )]));

        let batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![Arc::new(binary_array) as ArrayRef],
        )?;
        let sliced_batch = batch.slice(0, 20);
        let gc_batch = gc_view_arrays(&sliced_batch)?;

        assert_eq!(gc_batch.num_rows(), sliced_batch.num_rows());
        assert_eq!(gc_batch.num_columns(), sliced_batch.num_columns());

        Ok(())
    }

    #[test]
    fn test_gc_skips_small_arrays() -> Result<()> {
        use arrow::array::StringViewArray;

        let strings: Vec<String> = (0..10).map(|i| format!("string_{i}")).collect();

        let string_array = StringViewArray::from(strings);
        let array_ref: ArrayRef = Arc::new(string_array);

        let schema = Arc::new(Schema::new(vec![Field::new(
            "strings",
            DataType::Utf8View,
            false,
        )]));

        let batch = RecordBatch::try_new(Arc::clone(&schema), vec![array_ref])?;

        // GC should return the original batch for small arrays
        let should_gc = should_gc_view_array(
            batch
                .column(0)
                .as_any()
                .downcast_ref::<StringViewArray>()
                .unwrap(),
        );
        let gc_batch = gc_view_arrays(&batch)?;

        assert!(!should_gc);
        assert_eq!(gc_batch.num_rows(), batch.num_rows());
        assert!(Arc::ptr_eq(batch.column(0), gc_batch.column(0)));

        Ok(())
    }

    #[test]
    fn test_gc_with_mixed_columns() -> Result<()> {
        use arrow::array::{Int32Array, StringViewArray};

        let strings: Vec<String> = (0..200)
            .map(|i| format!("long_string_for_gc_testing_{i}"))
            .collect();

        let string_array = StringViewArray::from(strings);
        let int_array = Int32Array::from((0..200).collect::<Vec<i32>>());

        let schema = Arc::new(Schema::new(vec![
            Field::new("strings", DataType::Utf8View, false),
            Field::new("ints", DataType::Int32, false),
        ]));

        let batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![
                Arc::new(string_array) as ArrayRef,
                Arc::new(int_array) as ArrayRef,
            ],
        )?;

        let sliced_batch = batch.slice(0, 50);
        let gc_batch = gc_view_arrays(&sliced_batch)?;

        assert_eq!(gc_batch.num_columns(), 2);
        assert_eq!(gc_batch.num_rows(), 50);

        Ok(())
    }

    #[test]
    fn test_verify_gc_triggers_for_sliced_arrays() -> Result<()> {
        let strings: Vec<String> = (0..200)
            .map(|i| {
                format!(
                    "http://example.com/very/long/path/that/exceeds/inline/threshold/{i}"
                )
            })
            .collect();

        let string_array = StringViewArray::from(strings);
        let schema = Arc::new(Schema::new(vec![Field::new(
            "url",
            DataType::Utf8View,
            false,
        )]));

        let batch = RecordBatch::try_new(
            schema,
            vec![Arc::new(string_array.clone()) as ArrayRef],
        )?;

        let sliced = batch.slice(0, 20);

        let sliced_array = sliced
            .column(0)
            .as_any()
            .downcast_ref::<StringViewArray>()
            .unwrap();
        let should_gc = should_gc_view_array(sliced_array);
        let waste_ratio = calculate_string_view_waste_ratio(sliced_array);

        assert!(
            waste_ratio > 0.8,
            "Waste ratio should be > 0.8 for sliced array"
        );
        assert!(
            should_gc,
            "GC should trigger for sliced array with high waste"
        );

        Ok(())
    }

    #[test]
    fn test_reproduce_issue_19414_string_view_spill_without_gc() -> Result<()> {
        use arrow::array::StringViewArray;
        use std::fs;

        let num_rows = 1000;
        let mut strings = Vec::with_capacity(num_rows);

        for i in 0..num_rows {
            let url = match i % 5 {
                0 => format!(
                    "http://irr.ru/index.php?showalbum/login-leniya7777294,938303130/{i}"
                ),
                1 => format!("http://komme%2F27.0.1453.116/very/long/path/{i}"),
                2 => format!("https://produkty%2Fproduct/category/item/{i}"),
                3 => format!(
                    "http://irr.ru/index.php?showalbum/login-kapusta-advert2668/{i}"
                ),
                4 => format!(
                    "http://irr.ru/index.php?showalbum/login-kapustic/product/{i}"
                ),
                _ => unreachable!(),
            };
            strings.push(url);
        }

        let string_array = StringViewArray::from(strings);
        let schema = Arc::new(Schema::new(vec![Field::new(
            "URL",
            DataType::Utf8View,
            false,
        )]));

        let original_batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![Arc::new(string_array.clone()) as ArrayRef],
        )?;

        let total_buffer_size: usize = string_array
            .data_buffers()
            .iter()
            .map(|buffer| buffer.capacity())
            .sum();

        let mut sliced_batches = Vec::new();
        let slice_size = 100;

        for i in (0..num_rows).step_by(slice_size) {
            let len = std::cmp::min(slice_size, num_rows - i);
            let sliced = original_batch.slice(i, len);
            sliced_batches.push(sliced);
        }

        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let spill_manager = SpillManager::new(env, metrics, schema);

        let mut in_progress_file = spill_manager.create_in_progress_file("Test GC")?;

        for batch in &sliced_batches {
            in_progress_file.append_batch(batch)?;
        }

        let spill_file = in_progress_file.finish()?.unwrap();
        let file_size = fs::metadata(spill_file.path())?.len() as usize;

        let theoretical_without_gc = total_buffer_size * sliced_batches.len();
        let reduction_percent = ((theoretical_without_gc - file_size) as f64
            / theoretical_without_gc as f64)
            * 100.0;

        assert!(
            reduction_percent > 80.0,
            "GC should reduce spill file size by >80%, got {reduction_percent:.1}%"
        );

        Ok(())
    }

    #[test]
    fn test_spill_with_and_without_gc_comparison() -> Result<()> {
        let num_rows = 400;
        let strings: Vec<String> = (0..num_rows)
            .map(|i| {
                format!(
                    "http://example.com/this/is/a/long/url/path/that/wont/be/inlined/{i}"
                )
            })
            .collect();

        let string_array = StringViewArray::from(strings);
        let schema = Arc::new(Schema::new(vec![Field::new(
            "url",
            DataType::Utf8View,
            false,
        )]));

        let batch =
            RecordBatch::try_new(schema, vec![Arc::new(string_array) as ArrayRef])?;

        let sliced_batch = batch.slice(0, 40);

        let array_without_gc = sliced_batch
            .column(0)
            .as_any()
            .downcast_ref::<StringViewArray>()
            .unwrap();
        let size_without_gc: usize = array_without_gc
            .data_buffers()
            .iter()
            .map(|buffer| buffer.capacity())
            .sum();

        let gc_batch = gc_view_arrays(&sliced_batch)?;
        let array_with_gc = gc_batch
            .column(0)
            .as_any()
            .downcast_ref::<StringViewArray>()
            .unwrap();
        let size_with_gc: usize = array_with_gc
            .data_buffers()
            .iter()
            .map(|buffer| buffer.capacity())
            .sum();

        let reduction_percent =
            ((size_without_gc - size_with_gc) as f64 / size_without_gc as f64) * 100.0;

        assert!(
            reduction_percent > 85.0,
            "Expected >85% reduction for 10% slice, got {reduction_percent:.1}%"
        );

        Ok(())
    }

    #[test]
    fn test_gc_recurses_into_nested_view_arrays() -> Result<()> {
        use arrow::array::{DictionaryArray, Int32Array};
        use arrow::buffer::Buffer;

        let strings: Vec<String> = (0..200)
            .map(|i| format!("http://example.com/nested/path/that/is/not/inlined/{i}"))
            .collect();
        let string_values = Arc::new(StringViewArray::from(strings)) as ArrayRef;

        let list_data = ArrayDataBuilder::new(DataType::List(Arc::new(
            Field::new_list_field(DataType::Utf8View, true),
        )))
        .len(20)
        .buffers(vec![Buffer::from_iter((0..=20).map(|i| i * 5_i32))])
        .child_data(vec![string_values.slice(0, 100).to_data()])
        .build()?;
        let list_array = make_array(list_data);

        let keys = Int32Array::from_iter_values(0..20);
        let dictionary = DictionaryArray::new(keys, string_values.slice(0, 20));
        let dictionary_array = Arc::new(dictionary) as ArrayRef;

        let schema = Arc::new(Schema::new(vec![
            Field::new(
                "list_strings",
                DataType::List(Arc::new(Field::new_list_field(DataType::Utf8View, true))),
                false,
            ),
            Field::new(
                "dictionary_strings",
                DataType::Dictionary(
                    Box::new(DataType::Int32),
                    Box::new(DataType::Utf8View),
                ),
                false,
            ),
        ]));
        let batch = RecordBatch::try_new(schema, vec![list_array, dictionary_array])?;
        let gc_batch = gc_view_arrays(&batch)?;

        let gc_list_values = gc_batch.column(0).to_data().child_data()[0].clone();
        let gc_list_values = make_array(gc_list_values);
        let gc_list_values = gc_list_values
            .as_any()
            .downcast_ref::<StringViewArray>()
            .unwrap();
        assert!(
            calculate_string_view_waste_ratio(gc_list_values) < 0.2,
            "GC should compact nested List child views"
        );

        let gc_dictionary_values = gc_batch.column(1).to_data().child_data()[0].clone();
        let gc_dictionary_values = make_array(gc_dictionary_values);
        let gc_dictionary_values = gc_dictionary_values
            .as_any()
            .downcast_ref::<StringViewArray>()
            .unwrap();
        assert!(
            calculate_string_view_waste_ratio(gc_dictionary_values) < 0.2,
            "GC should compact nested Dictionary values"
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_spill_file_size_gc_verification_string_view() -> Result<()> {
        use arrow::array::StringViewArray;
        use std::fs;

        // 1. Setup bloated data (large buffers)
        let num_rows = 1000;
        let string_array: StringViewArray = (0..num_rows)
            .map(|i| Some(format!("this_is_a_long_string_to_ensure_it_is_not_inlined_and_causes_waste_{i}")))
            .collect();
        let schema = Arc::new(Schema::new(vec![Field::new(
            "s",
            DataType::Utf8View,
            false,
        )]));
        let batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![Arc::new(string_array.clone()) as ArrayRef],
        )?;

        // 2. Slice it heavily (1% of the data)
        let sliced_batch = batch.slice(0, 10);

        // 3. Spill to disk using SpillManager
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let spill_manager = SpillManager::new(env, metrics, schema);
        let spill_file = spill_manager
            .spill_record_batch_and_finish(&[sliced_batch], "TestGC")?
            .unwrap();

        // 4. Check file size on disk
        let file_size = fs::metadata(spill_file.path())?.len();

        // The original buffer size is around 70KB.
        // Without GC, the spill file would be > 70KB.
        // With GC, it should be much smaller (only 10 rows of ~70 bytes each + metadata).
        assert!(
            file_size < 10 * 1024,
            "Spill file is too large ({file_size} bytes)! GC might not be working."
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_spill_file_size_gc_verification_binary_view() -> Result<()> {
        use arrow::array::BinaryViewArray;
        use std::fs;

        // 1. Setup bloated data (large buffers)
        let num_rows = 1000;
        let binary_array: BinaryViewArray =
            (0..num_rows).map(|i| Some(vec![i as u8; 100])).collect();
        let schema = Arc::new(Schema::new(vec![Field::new(
            "b",
            DataType::BinaryView,
            false,
        )]));
        let batch = RecordBatch::try_new(
            Arc::clone(&schema),
            vec![Arc::new(binary_array.clone()) as ArrayRef],
        )?;

        // 2. Slice it heavily (1% of the data)
        let sliced_batch = batch.slice(0, 10);

        // 3. Spill to disk using SpillManager
        let env = Arc::new(RuntimeEnv::default());
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        let spill_manager = SpillManager::new(env, metrics, schema);
        let spill_file = spill_manager
            .spill_record_batch_and_finish(&[sliced_batch], "TestGCBinary")?
            .unwrap();

        // 4. Check file size on disk
        let file_size = fs::metadata(spill_file.path())?.len();

        // Original buffer is 100KB.
        // With GC, it should be much smaller.
        assert!(
            file_size < 10 * 1024,
            "Spill file is too large ({file_size} bytes)! GC might not be working."
        );

        Ok(())
    }
}