lance-datafusion 4.0.1

Internal utilities used by other lance modules to simplify working with datafusion
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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::{
    io::{BufReader, BufWriter},
    path::PathBuf,
    sync::{Arc, Mutex},
};

use arrow::ipc::{reader::StreamReader, writer::StreamWriter};
use arrow_array::RecordBatch;
use arrow_schema::{ArrowError, Schema};
use datafusion::{
    execution::SendableRecordBatchStream, physical_plan::stream::RecordBatchStreamAdapter,
};
use datafusion_common::DataFusionError;
use lance_arrow::memory::MemoryAccumulator;
use lance_core::error::LanceOptionExt;

/// Start a spill of Arrow data to a file that can be read later multiple times.
///
/// Up to `memory_limit` bytes of data can be buffered in memory before a spill
/// is created. If the memory limit is never reached before [`SpillSender::finish()`]
/// is called, then the data will simply be kept in memory and no spill will be
/// created.
///
/// `path` is the path to the file that may be created. It should not already
/// exist. It is the responsibility of the caller to delete the file after it is
/// no longer needed.
///
/// The [`SpillSender`] allows you to write batches to the spill.
///
/// The [`SpillReceiver`] can open a [`SendableRecordBatchStream`] that reads
/// batches from the spill. This can be opened before, during, or after batches
/// have been written to the spill.
///
/// Once [`SpillSender`] is dropped, the temporary file is deleted. This will
/// cause the [`SpillReceiver`] to return an error if it is still open.
pub fn create_replay_spill(
    path: std::path::PathBuf,
    schema: Arc<Schema>,
    memory_limit: usize,
) -> (SpillSender, SpillReceiver) {
    let initial_status = WriteStatus::default();
    let (status_sender, status_receiver) = tokio::sync::watch::channel(initial_status);
    let sender = SpillSender {
        memory_limit,
        path: path.clone(),
        schema: schema.clone(),
        state: SpillState::default(),
        status_sender,
    };

    let receiver = SpillReceiver {
        status_receiver,
        path,
        schema,
    };

    (sender, receiver)
}

#[derive(Clone)]
pub struct SpillReceiver {
    status_receiver: tokio::sync::watch::Receiver<WriteStatus>,
    path: PathBuf,
    schema: Arc<Schema>,
}

impl SpillReceiver {
    /// Returns a stream of batches from the spill. The stream will emit
    /// batches as they are written to the spill. If the spill has already
    /// been finished, the stream will emit all batches in the spill.
    ///
    /// The stream will not complete until [`SpillSender::finish()`] is called.
    ///
    /// If the spill has been dropped, an error will be returned.
    pub fn read(&self) -> SendableRecordBatchStream {
        let rx = self.status_receiver.clone();
        let reader = SpillReader::new(rx, self.path.clone());

        let stream = futures::stream::try_unfold(reader, move |mut reader| async move {
            match reader.read().await {
                Ok(None) => Ok(None),
                Ok(Some(batch)) => Ok(Some((batch, reader))),
                Err(err) => Err(err),
            }
        });

        Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream))
    }
}

struct SpillReader {
    pub batches_read: usize,
    receiver: tokio::sync::watch::Receiver<WriteStatus>,
    state: SpillReaderState,
}

enum SpillReaderState {
    Buffered { spill_path: PathBuf },
    Reader { reader: AsyncStreamReader },
}

impl SpillReader {
    fn new(receiver: tokio::sync::watch::Receiver<WriteStatus>, spill_path: PathBuf) -> Self {
        Self {
            batches_read: 0,
            receiver,
            state: SpillReaderState::Buffered { spill_path },
        }
    }

    async fn wait_for_more_data(&mut self) -> Result<Option<Arc<[RecordBatch]>>, DataFusionError> {
        let status = self
            .receiver
            .wait_for(|status| {
                status.error.is_some()
                    || status.finished
                    || status.batches_written() > self.batches_read
            })
            .await
            .map_err(|_| {
                DataFusionError::Execution(
                    "Spill has been dropped before reader has finish.".into(),
                )
            })?;

        if let Some(error) = &status.error {
            let mut guard = error.lock().ok().expect_ok()?;
            return Err(DataFusionError::from(&mut (*guard)));
        }

        if let DataLocation::Buffered { batches } = &status.data_location {
            Ok(Some(batches.clone()))
        } else {
            Ok(None)
        }
    }

    async fn get_reader(&mut self) -> Result<&AsyncStreamReader, ArrowError> {
        if let SpillReaderState::Buffered { spill_path } = &self.state {
            let reader = AsyncStreamReader::open(spill_path.clone()).await?;
            // Skip batches we've already read before the writer started spilling.
            // The read batches were spilled to the file for the benefit of
            // future readers, as the spill is replay-able.
            for _ in 0..self.batches_read {
                reader.read().await?;
            }
            self.state = SpillReaderState::Reader { reader };
        }

        if let SpillReaderState::Reader { reader } = &mut self.state {
            Ok(reader)
        } else {
            unreachable!()
        }
    }

    async fn read(&mut self) -> Result<Option<RecordBatch>, DataFusionError> {
        let maybe_data = self.wait_for_more_data().await?;

        if let Some(batches) = maybe_data {
            if self.batches_read < batches.len() {
                let batch = batches[self.batches_read].clone();
                self.batches_read += 1;
                Ok(Some(batch))
            } else {
                Ok(None)
            }
        } else {
            let reader = self.get_reader().await?;
            let batch = reader.read().await?;
            if batch.is_some() {
                self.batches_read += 1;
            }
            Ok(batch)
        }
    }
}

/// The sender side of the spill. This is used to write batches to the spill.
///
/// Note: this must be kept alive until after the readers are done reading the
/// spill. Otherwise, they will return an error.
pub struct SpillSender {
    memory_limit: usize,
    schema: Arc<Schema>,
    path: PathBuf,
    state: SpillState,
    status_sender: tokio::sync::watch::Sender<WriteStatus>,
}

enum SpillState {
    Buffering {
        batches: Vec<RecordBatch>,
        memory_accumulator: MemoryAccumulator,
    },
    Spilling {
        writer: AsyncStreamWriter,
        batches_written: usize,
    },
    Finished {
        batches: Option<Arc<[RecordBatch]>>,
        batches_written: usize,
    },
    Errored {
        error: Arc<Mutex<SpillError>>,
    },
}

impl Default for SpillState {
    fn default() -> Self {
        Self::Buffering {
            batches: Vec::new(),
            memory_accumulator: MemoryAccumulator::default(),
        }
    }
}

#[derive(Clone, Debug, Default)]
struct WriteStatus {
    error: Option<Arc<Mutex<SpillError>>>,
    finished: bool,
    data_location: DataLocation,
}

impl WriteStatus {
    fn batches_written(&self) -> usize {
        match &self.data_location {
            DataLocation::Buffered { batches } => batches.len(),
            DataLocation::Spilled {
                batches_written, ..
            } => *batches_written,
        }
    }
}

#[derive(Clone, Debug)]
enum DataLocation {
    Buffered { batches: Arc<[RecordBatch]> },
    Spilled { batches_written: usize },
}

impl Default for DataLocation {
    fn default() -> Self {
        Self::Buffered {
            batches: Arc::new([]),
        }
    }
}

/// A DataFusion error that be be emitted multiple times. We provide the
/// Original error first, and subsequent conversions provide a copy with a
/// string representation of the original error.
#[derive(Debug)]
enum SpillError {
    Original(DataFusionError),
    Copy(DataFusionError),
}

impl From<DataFusionError> for SpillError {
    fn from(err: DataFusionError) -> Self {
        Self::Original(err)
    }
}

impl From<&mut SpillError> for DataFusionError {
    fn from(err: &mut SpillError) -> Self {
        match err {
            SpillError::Original(inner) => {
                let copy = Self::Execution(inner.to_string());
                let original = std::mem::replace(err, SpillError::Copy(copy));
                if let SpillError::Original(inner) = original {
                    inner
                } else {
                    unreachable!()
                }
            }
            SpillError::Copy(Self::Execution(message)) => Self::Execution(message.clone()),
            _ => unreachable!(),
        }
    }
}

impl From<&SpillState> for WriteStatus {
    fn from(state: &SpillState) -> Self {
        match state {
            SpillState::Buffering { batches, .. } => Self {
                finished: false,
                data_location: DataLocation::Buffered {
                    batches: batches.clone().into(),
                },
                error: None,
            },
            SpillState::Spilling {
                batches_written, ..
            } => Self {
                finished: false,
                data_location: DataLocation::Spilled {
                    batches_written: *batches_written,
                },
                error: None,
            },
            SpillState::Finished {
                batches_written,
                batches,
            } => {
                let data_location = if let Some(batches) = batches {
                    DataLocation::Buffered {
                        batches: batches.clone(),
                    }
                } else {
                    DataLocation::Spilled {
                        batches_written: *batches_written,
                    }
                };
                Self {
                    finished: true,
                    data_location,
                    error: None,
                }
            }
            SpillState::Errored { error } => Self {
                finished: true,
                data_location: DataLocation::default(), // Doesn't matter.
                error: Some(error.clone()),
            },
        }
    }
}

impl SpillSender {
    /// Write a batch to the spill.  
    ///  
    /// If there is room in the `memory_limit` then the batch is queued.  
    /// If `memory_limit` is first encountered then all queued batches, and this one,  
    /// will be written to disk as part of this call.  
    /// If we are already spilling then the batch will be written to disk as part of this  
    /// call.
    pub async fn write(&mut self, batch: RecordBatch) -> Result<(), DataFusionError> {
        if let SpillState::Finished { .. } = self.state {
            return Err(DataFusionError::Execution(
                "Spill has already been finished".to_string(),
            ));
        }

        if let SpillState::Errored { .. } = &self.state {
            return Err(DataFusionError::Execution(
                "Spill has sent an error".to_string(),
            ));
        }

        let (writer, batches_written) = match &mut self.state {
            SpillState::Buffering {
                batches,
                memory_accumulator,
            } => {
                memory_accumulator.record_batch(&batch);

                if memory_accumulator.total() > self.memory_limit {
                    let writer =
                        AsyncStreamWriter::open(self.path.clone(), self.schema.clone()).await?;
                    let batches_written = batches.len();
                    for batch in batches.drain(..) {
                        writer.write(batch).await?;
                    }
                    self.state = SpillState::Spilling {
                        writer,
                        batches_written,
                    };
                    if let SpillState::Spilling {
                        writer,
                        batches_written,
                    } = &mut self.state
                    {
                        (writer, batches_written)
                    } else {
                        unreachable!()
                    }
                } else {
                    batches.push(batch);
                    self.status_sender
                        .send_replace(WriteStatus::from(&self.state));
                    return Ok(());
                }
            }
            SpillState::Spilling {
                writer,
                batches_written,
            } => (writer, batches_written),
            _ => unreachable!(),
        };

        writer.write(batch).await?;
        *batches_written += 1;
        self.status_sender
            .send_replace(WriteStatus::from(&self.state));

        Ok(())
    }

    /// Send an error to the spill. This will be sent to all readers of the
    /// spill.
    pub fn send_error(&mut self, err: DataFusionError) {
        let error = Arc::new(Mutex::new(err.into()));
        self.state = SpillState::Errored { error };
        self.status_sender
            .send_replace(WriteStatus::from(&self.state));
    }

    /// Complete the spill write. This will finalize the Arrow IPC stream file.
    /// The file will remain available for reading until the spill is dropped.
    pub async fn finish(&mut self) -> Result<(), DataFusionError> {
        // We create a temporary state to get an owned copy of current state.
        // Since we hold an exclusive reference to `self`, no one should be
        // able to see this temporary state.
        let tmp_state = SpillState::Finished {
            batches_written: 0,
            batches: None,
        };
        match std::mem::replace(&mut self.state, tmp_state) {
            SpillState::Buffering { batches, .. } => {
                let batches_written = batches.len();
                self.state = SpillState::Finished {
                    batches_written,
                    batches: Some(batches.into()),
                };
                self.status_sender
                    .send_replace(WriteStatus::from(&self.state));
            }
            SpillState::Spilling {
                writer,
                batches_written,
            } => {
                writer.finish().await?;
                self.state = SpillState::Finished {
                    batches_written,
                    batches: None,
                };
                self.status_sender
                    .send_replace(WriteStatus::from(&self.state));
            }
            SpillState::Finished { .. } => {
                return Err(DataFusionError::Execution(
                    "Spill has already been finished".to_string(),
                ));
            }
            SpillState::Errored { .. } => {
                return Err(DataFusionError::Execution(
                    "Spill has sent an error".to_string(),
                ));
            }
        };

        Ok(())
    }
}

/// An async wrapper around [`StreamWriter`]. Each call uses [`tokio::task::spawn_blocking`]
/// to spawn a blocking task to write the batch.
struct AsyncStreamWriter {
    writer: Arc<Mutex<StreamWriter<BufWriter<std::fs::File>>>>,
}

impl AsyncStreamWriter {
    pub async fn open(path: PathBuf, schema: Arc<Schema>) -> Result<Self, ArrowError> {
        let writer = tokio::task::spawn_blocking(move || {
            let file = std::fs::File::create(&path).map_err(ArrowError::from)?;
            let writer = BufWriter::new(file);
            StreamWriter::try_new(writer, &schema)
        })
        .await
        .unwrap()?;
        let writer = Arc::new(Mutex::new(writer));
        Ok(Self { writer })
    }

    pub async fn write(&self, batch: RecordBatch) -> Result<(), ArrowError> {
        let writer = self.writer.clone();
        tokio::task::spawn_blocking(move || {
            let mut writer = writer.lock().unwrap();
            writer.write(&batch)?;
            writer.flush()
        })
        .await
        .unwrap()
    }

    pub async fn finish(self) -> Result<(), ArrowError> {
        let writer = self.writer.clone();
        tokio::task::spawn_blocking(move || {
            let mut writer = writer.lock().unwrap();
            writer.finish()
        })
        .await
        .unwrap()
    }
}

struct AsyncStreamReader {
    reader: Arc<Mutex<StreamReader<BufReader<std::fs::File>>>>,
}

impl AsyncStreamReader {
    pub async fn open(path: PathBuf) -> Result<Self, ArrowError> {
        let reader = tokio::task::spawn_blocking(move || {
            let file = std::fs::File::open(&path).map_err(ArrowError::from)?;
            let reader = BufReader::new(file);
            StreamReader::try_new(reader, None)
        })
        .await
        .unwrap()?;
        let reader = Arc::new(Mutex::new(reader));
        Ok(Self { reader })
    }

    pub async fn read(&self) -> Result<Option<RecordBatch>, ArrowError> {
        let reader = self.reader.clone();
        tokio::task::spawn_blocking(move || {
            let mut reader = reader.lock().unwrap();
            reader.next()
        })
        .await
        .unwrap()
        .transpose()
    }
}

#[cfg(test)]
mod tests {
    use arrow_array::Int32Array;
    use arrow_schema::{DataType, Field};
    use futures::{StreamExt, TryStreamExt, poll};
    use lance_core::utils::tempfile::{TempStdFile, TempStdPath};

    use super::*;

    #[tokio::test]
    async fn test_spill() {
        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]));
        let batches = [
            RecordBatch::try_new(
                schema.clone(),
                vec![Arc::new(Int32Array::from(vec![1, 2, 3]))],
            )
            .unwrap(),
            RecordBatch::try_new(
                schema.clone(),
                vec![Arc::new(Int32Array::from(vec![4, 5, 6]))],
            )
            .unwrap(),
        ];

        // Create a stream
        let path = TempStdFile::default();
        let (mut spill, receiver) = create_replay_spill(path.to_owned(), schema.clone(), 0);

        // We can open a reader prior to writing any data. No batches will be ready.
        let mut stream_before = receiver.read();
        let mut stream_before_next = stream_before.next();
        let poll_res = poll!(&mut stream_before_next);
        assert!(poll_res.is_pending());

        // If we write a batch, the existing reader can now receive it.
        spill.write(batches[0].clone()).await.unwrap();
        let stream_before_batch1 = stream_before_next
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_before_batch1, &batches[0]);
        let mut stream_before_next = stream_before.next();
        let poll_res = poll!(&mut stream_before_next);
        assert!(poll_res.is_pending());

        // We can also open a ready while the spill is being written to. We can
        // retrieve batches written so far immediately.
        let mut stream_during = receiver.read();
        let stream_during_batch1 = stream_during
            .next()
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_during_batch1, &batches[0]);
        let mut stream_during_next = stream_during.next();
        let poll_res = poll!(&mut stream_during_next);
        assert!(poll_res.is_pending());

        // Once we finish the spill, readers can get remaining batches and will
        // reach the end of the stream.
        spill.write(batches[1].clone()).await.unwrap();
        spill.finish().await.unwrap();

        let stream_before_batch2 = stream_before_next
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_before_batch2, &batches[1]);
        assert!(stream_before.next().await.is_none());

        let stream_during_batch2 = stream_during_next
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_during_batch2, &batches[1]);
        assert!(stream_during.next().await.is_none());

        // Can also start a reader after finishing.
        let stream_after = receiver.read();
        let stream_after_batches = stream_after.try_collect::<Vec<_>>().await.unwrap();
        assert_eq!(&stream_after_batches, &batches);

        std::fs::remove_file(path).unwrap();
    }

    #[tokio::test]
    async fn test_spill_error() {
        // Create a spill
        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]));
        let path = TempStdFile::default();
        let (mut spill, receiver) =
            create_replay_spill(path.as_ref().to_owned(), schema.clone(), 0);
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(Int32Array::from(vec![1, 2, 3]))],
        )
        .unwrap();

        spill.write(batch.clone()).await.unwrap();

        let mut stream = receiver.read();
        let stream_batch = stream
            .next()
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_batch, &batch);

        spill.send_error(DataFusionError::ResourcesExhausted("🥱".into()));
        let stream_error = stream
            .next()
            .await
            .expect("Expected an error")
            .expect_err("Expected an error");
        assert!(matches!(
            stream_error,
            DataFusionError::ResourcesExhausted(message) if message == "🥱"
        ));

        // If we try to write after sending an error, it should return an error.
        let err = spill.write(batch).await;
        assert!(matches!(
            err,
            Err(DataFusionError::Execution(message)) if message == "Spill has sent an error"
        ));

        // If we try to finish after sending an error, it should return an error.
        let err = spill.finish().await;
        assert!(matches!(
            err,
            Err(DataFusionError::Execution(message)) if message == "Spill has sent an error"
        ));

        // If we try to read after sending an error, it should return an error.
        let mut stream = receiver.read();
        let stream_error = stream
            .next()
            .await
            .expect("Expected an error")
            .expect_err("Expected an error");
        assert!(matches!(
            stream_error,
            DataFusionError::Execution(message) if message.contains("🥱")
        ));

        std::fs::remove_file(path).unwrap();
    }

    #[tokio::test]
    async fn test_spill_buffered() {
        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]));
        let path = TempStdPath::default();
        let memory_limit = 1024 * 1024; // 1 MiB
        let (mut spill, receiver) = create_replay_spill(path.clone(), schema.clone(), memory_limit);

        // 0.5 MB batch
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(Int32Array::from(vec![1; (512 * 1024) / 4]))],
        )
        .unwrap();
        spill.write(batch.clone()).await.unwrap();
        assert!(!std::fs::exists(&path).unwrap());

        spill.finish().await.unwrap();
        assert!(!std::fs::exists(&path).unwrap());

        let mut stream = receiver.read();
        let stream_batch = stream
            .next()
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_batch, &batch);

        assert!(!std::fs::exists(&path).unwrap());
    }

    #[tokio::test]
    async fn test_spill_buffered_transition() {
        // Starts as buffered, then spills, then finished.
        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]));
        let path = TempStdPath::default();
        let memory_limit = 1024 * 1024; // 1 MiB
        let (mut spill, receiver) = create_replay_spill(path.clone(), schema.clone(), memory_limit);

        // 0.7 MB batch
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(Int32Array::from(vec![1; (768 * 1024) / 4]))],
        )
        .unwrap();
        spill.write(batch.clone()).await.unwrap();
        assert!(!std::fs::exists(&path).unwrap());

        let mut stream = receiver.read();
        let stream_batch = stream
            .next()
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_batch, &batch);
        assert!(!std::fs::exists(&path).unwrap());

        // 0.5 MB batch
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(Int32Array::from(vec![1; (512 * 1024) / 4]))],
        )
        .unwrap();
        spill.write(batch.clone()).await.unwrap();
        assert!(std::fs::exists(&path).unwrap());

        let stream_batch = stream
            .next()
            .await
            .expect("Expected a batch")
            .expect("Expected no error");
        assert_eq!(&stream_batch, &batch);
        assert!(std::fs::exists(&path).unwrap());

        spill.finish().await.unwrap();

        assert!(stream.next().await.is_none());

        std::fs::remove_file(path).unwrap();
    }
}