lance-index 4.0.1

Lance indices implementation
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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

//! Shuffler is a component that takes a stream of record batches and shuffles them into
//! the corresponding IVF partitions.

use std::ops::Range;
use std::sync::atomic::AtomicU64;
use std::sync::{Arc, Mutex};

use arrow::compute::concat_batches;
use arrow::datatypes::UInt64Type;
use arrow::{array::AsArray, compute::sort_to_indices};
use arrow_array::{RecordBatch, UInt32Array, UInt64Array};
use arrow_schema::{DataType, Field, Schema};
use futures::{future::try_join_all, prelude::*};
use lance_arrow::stream::rechunk_stream_by_size;
use lance_arrow::{RecordBatchExt, SchemaExt};
use lance_core::{
    Error, Result,
    cache::LanceCache,
    utils::tokio::{get_num_compute_intensive_cpus, spawn_cpu},
};
use lance_encoding::decoder::{DecoderPlugins, FilterExpression};
use lance_encoding::version::LanceFileVersion;
use lance_file::reader::{FileReader, FileReaderOptions};
use lance_file::writer::{FileWriter, FileWriterOptions};
use lance_io::{
    ReadBatchParams,
    object_store::ObjectStore,
    scheduler::{ScanScheduler, SchedulerConfig},
    stream::{RecordBatchStream, RecordBatchStreamAdapter},
    utils::CachedFileSize,
};
use object_store::path::Path;

use crate::vector::{LOSS_METADATA_KEY, PART_ID_COLUMN};

#[async_trait::async_trait]
/// A reader that can read the shuffled partitions.
pub trait ShuffleReader: Send + Sync {
    /// Read a partition by partition_id
    /// will return Ok(None) if partition_size is 0
    /// check reader.partition_size(partition_id) before calling this function
    async fn read_partition(
        &self,
        partition_id: usize,
    ) -> Result<Option<Box<dyn RecordBatchStream + Unpin + 'static>>>;

    /// Get the size of the partition by partition_id
    fn partition_size(&self, partition_id: usize) -> Result<usize>;

    /// Get the total loss,
    /// if the loss is not available, return None,
    /// in such case, the caller should sum up the losses from each batch's metadata.
    /// Must be called after all partitions are read.
    fn total_loss(&self) -> Option<f64>;
}

#[async_trait::async_trait]
/// A shuffler that can shuffle the incoming stream of record batches into IVF partitions.
/// Returns a IvfShuffleReader that can be used to read the shuffled partitions.
pub trait Shuffler: Send + Sync {
    /// Shuffle the incoming stream of record batches into IVF partitions.
    /// Returns a IvfShuffleReader that can be used to read the shuffled partitions.
    async fn shuffle(
        &self,
        data: Box<dyn RecordBatchStream + Unpin + 'static>,
    ) -> Result<Box<dyn ShuffleReader>>;
}

pub struct IvfShuffler {
    object_store: Arc<ObjectStore>,
    output_dir: Path,
    num_partitions: usize,
    format_version: LanceFileVersion,

    progress: Arc<dyn crate::progress::IndexBuildProgress>,
}

impl IvfShuffler {
    pub fn new(output_dir: Path, num_partitions: usize) -> Self {
        Self {
            object_store: Arc::new(ObjectStore::local()),
            output_dir,
            num_partitions,
            format_version: LanceFileVersion::V2_0,
            progress: crate::progress::noop_progress(),
        }
    }

    pub fn with_format_version(mut self, format_version: LanceFileVersion) -> Self {
        self.format_version = format_version;
        self
    }

    pub fn with_progress(mut self, progress: Arc<dyn crate::progress::IndexBuildProgress>) -> Self {
        self.progress = progress;
        self
    }
}

#[async_trait::async_trait]
impl Shuffler for IvfShuffler {
    async fn shuffle(
        &self,
        data: Box<dyn RecordBatchStream + Unpin + 'static>,
    ) -> Result<Box<dyn ShuffleReader>> {
        let num_partitions = self.num_partitions;
        let mut partition_sizes = vec![0; num_partitions];
        let schema = data.schema().without_column(PART_ID_COLUMN);
        let mut writers = stream::iter(0..num_partitions)
            .map(|partition_id| {
                let part_path = self.output_dir.child(format!("ivf_{}.lance", partition_id));
                let spill_path = self.output_dir.child(format!("ivf_{}.spill", partition_id));
                let object_store = self.object_store.clone();
                let schema = schema.clone();
                let format_version = self.format_version;
                async move {
                    let writer = object_store.create(&part_path).await?;
                    let file_writer = FileWriter::try_new(
                        writer,
                        lance_core::datatypes::Schema::try_from(&schema)?,
                        FileWriterOptions {
                            format_version: Some(format_version),
                            ..Default::default()
                        },
                    )?
                    .with_page_metadata_spill(object_store.clone(), spill_path);
                    Result::Ok(file_writer)
                }
            })
            .buffered(self.object_store.io_parallelism())
            .try_collect::<Vec<_>>()
            .await?;
        let mut parallel_sort_stream = data
            .map(|batch| {
                spawn_cpu(move || {
                    let batch = batch?;

                    let loss = batch
                        .metadata()
                        .get(LOSS_METADATA_KEY)
                        .map(|s| s.parse::<f64>().unwrap_or_default())
                        .unwrap_or_default();

                    let part_ids: &UInt32Array = batch[PART_ID_COLUMN].as_primitive();

                    let indices = sort_to_indices(&part_ids, None, None)?;
                    let batch = batch.take(&indices)?;

                    let part_ids: &UInt32Array = batch[PART_ID_COLUMN].as_primitive();
                    let batch = batch.drop_column(PART_ID_COLUMN)?;

                    let mut partition_buffers = vec![Vec::new(); num_partitions];

                    let mut start = 0;
                    while start < batch.num_rows() {
                        let part_id: u32 = part_ids.value(start);
                        let mut end = start + 1;
                        while end < batch.num_rows() && part_ids.value(end) == part_id {
                            end += 1;
                        }

                        let part_batches = &mut partition_buffers[part_id as usize];
                        part_batches.push(batch.slice(start, end - start));
                        start = end;
                    }

                    Ok::<(Vec<Vec<RecordBatch>>, f64), Error>((partition_buffers, loss))
                })
            })
            .buffered(get_num_compute_intensive_cpus());

        let mut total_loss = 0.0;
        let mut num_rows = 0u64;
        while let Some(shuffled) = parallel_sort_stream.next().await {
            let (shuffled, loss) = shuffled?;
            total_loss += loss;

            let mut futs = Vec::new();
            for (part_id, (writer, batches)) in writers.iter_mut().zip(shuffled.iter()).enumerate()
            {
                if !batches.is_empty() {
                    let rows = batches.iter().map(|b| b.num_rows()).sum::<usize>();
                    partition_sizes[part_id] += rows;
                    num_rows += rows as u64;
                    futs.push(writer.write_batches(batches.iter()));
                }
            }
            try_join_all(futs).await?;

            self.progress.stage_progress("shuffle", num_rows).await?;
        }

        // finish all writers
        for writer in writers.iter_mut() {
            writer.finish().await?;
        }

        Ok(Box::new(IvfShufflerReader::new(
            self.object_store.clone(),
            self.output_dir.clone(),
            partition_sizes,
            total_loss,
        )))
    }
}

pub struct IvfShufflerReader {
    scheduler: Arc<ScanScheduler>,
    output_dir: Path,
    partition_sizes: Vec<usize>,
    loss: f64,
}

impl IvfShufflerReader {
    pub fn new(
        object_store: Arc<ObjectStore>,
        output_dir: Path,
        partition_sizes: Vec<usize>,
        loss: f64,
    ) -> Self {
        let scheduler_config = SchedulerConfig::max_bandwidth(&object_store);
        let scheduler = ScanScheduler::new(object_store, scheduler_config);
        Self {
            scheduler,
            output_dir,
            partition_sizes,
            loss,
        }
    }
}

#[async_trait::async_trait]
impl ShuffleReader for IvfShufflerReader {
    async fn read_partition(
        &self,
        partition_id: usize,
    ) -> Result<Option<Box<dyn RecordBatchStream + Unpin + 'static>>> {
        if partition_id >= self.partition_sizes.len() {
            return Ok(None);
        }

        let partition_path = self.output_dir.child(format!("ivf_{}.lance", partition_id));

        let reader = FileReader::try_open(
            self.scheduler
                .open_file(&partition_path, &CachedFileSize::unknown())
                .await?,
            None,
            Arc::<DecoderPlugins>::default(),
            &LanceCache::no_cache(),
            FileReaderOptions::default(),
        )
        .await?;
        let schema: Schema = reader.schema().as_ref().into();
        Ok(Some(Box::new(RecordBatchStreamAdapter::new(
            Arc::new(schema),
            reader.read_stream(
                lance_io::ReadBatchParams::RangeFull,
                u32::MAX,
                16,
                FilterExpression::no_filter(),
            )?,
        ))))
    }

    fn partition_size(&self, partition_id: usize) -> Result<usize> {
        Ok(self.partition_sizes.get(partition_id).copied().unwrap_or(0))
    }

    fn total_loss(&self) -> Option<f64> {
        Some(self.loss)
    }
}

pub struct EmptyReader;

#[async_trait::async_trait]
impl ShuffleReader for EmptyReader {
    async fn read_partition(
        &self,
        _partition_id: usize,
    ) -> Result<Option<Box<dyn RecordBatchStream + Unpin + 'static>>> {
        Ok(None)
    }

    fn partition_size(&self, _partition_id: usize) -> Result<usize> {
        Ok(0)
    }

    fn total_loss(&self) -> Option<f64> {
        None
    }
}

/// Create an IVF shuffler. Uses [`TwoFileShuffler`] by default, which writes
/// all data to just two files (data + offsets) instead of one file per partition.
/// Set `LANCE_LEGACY_SHUFFLER=1` to fall back to [`IvfShuffler`], which opens
/// one file per partition.
///
/// An optional `progress` callback can be provided to receive shuffle progress
/// updates.
pub fn create_ivf_shuffler(
    output_dir: Path,
    num_partitions: usize,
    format_version: LanceFileVersion,
    progress: Option<Arc<dyn crate::progress::IndexBuildProgress>>,
) -> Box<dyn Shuffler> {
    let use_legacy = std::env::var("LANCE_LEGACY_SHUFFLER")
        .map(|v| v == "1" || v.eq_ignore_ascii_case("true"))
        .unwrap_or(false);
    if use_legacy {
        let mut shuffler =
            IvfShuffler::new(output_dir, num_partitions).with_format_version(format_version);
        if let Some(progress) = progress {
            shuffler = shuffler.with_progress(progress);
        }
        Box::new(shuffler)
    } else {
        let mut shuffler = TwoFileShuffler::new(output_dir, num_partitions);
        if let Some(progress) = progress {
            shuffler = shuffler.with_progress(progress);
        }
        Box::new(shuffler)
    }
}

const DEFAULT_SHUFFLE_BATCH_BYTES: usize = 128 * 1024 * 1024;

/// Limit of how much transformed data we accumulate before spilling to disk.
///
/// A larger value will use more RAM but require less random access during the
/// read phase.
///
/// This default is likely to be fine for most use cases.
fn shuffle_batch_bytes() -> usize {
    let batch_size = std::env::var("LANCE_SHUFFLE_BATCH_BYTES")
        .ok()
        .and_then(|s| s.parse().ok())
        .unwrap_or(DEFAULT_SHUFFLE_BATCH_BYTES);
    if batch_size == 0 {
        log::warn!(
            "LANCE_SHUFFLE_BATCH_BYTES is 0, using default of {}",
            DEFAULT_SHUFFLE_BATCH_BYTES
        );
        DEFAULT_SHUFFLE_BATCH_BYTES
    } else {
        batch_size
    }
}

/// A shuffler that writes all data to just two files (data + offsets) instead
/// of one file per partition. This avoids hitting OS file descriptor limits
/// when there are many partitions.
///
/// First we accumulate data in memory until we reach the batch size limit.
/// Then we sort the data by partition ID and compute an offset per partition.
/// Then we write the data to a data file and the offsets to an offsets file.
///
/// To read the data back, we read every Nth value from the offsets file to get
/// the start and end of each partition.
///
/// Then we read those ranges from the data file.
pub struct TwoFileShuffler {
    object_store: Arc<ObjectStore>,
    output_dir: Path,
    num_partitions: usize,
    batch_size_bytes: usize,

    progress: Arc<dyn crate::progress::IndexBuildProgress>,
}

impl TwoFileShuffler {
    pub fn new(output_dir: Path, num_partitions: usize) -> Self {
        Self {
            object_store: Arc::new(ObjectStore::local()),
            output_dir,
            num_partitions,
            batch_size_bytes: shuffle_batch_bytes(),
            progress: crate::progress::noop_progress(),
        }
    }

    pub fn with_progress(mut self, progress: Arc<dyn crate::progress::IndexBuildProgress>) -> Self {
        self.progress = progress;
        self
    }

    #[cfg(test)]
    fn with_batch_size_bytes(mut self, batch_size_bytes: usize) -> Self {
        self.batch_size_bytes = batch_size_bytes;
        self
    }
}

#[async_trait::async_trait]
impl Shuffler for TwoFileShuffler {
    async fn shuffle(
        &self,
        data: Box<dyn RecordBatchStream + Unpin + 'static>,
    ) -> Result<Box<dyn ShuffleReader>> {
        let num_partitions = self.num_partitions;
        let full_schema = Arc::new(data.schema().as_ref().clone());
        // No need to write partition ids since we can infer this
        let schema = data.schema().without_column(PART_ID_COLUMN);
        let offsets_schema = Arc::new(Schema::new(vec![Field::new(
            "offset",
            DataType::UInt64,
            false,
        )]));
        let batch_size_bytes = self.batch_size_bytes;

        // Extract loss from batch metadata before rechunking (concat_batches drops metadata)
        let total_loss = Arc::new(Mutex::new(0.0f64));
        let loss_ref = total_loss.clone();
        let loss_stream = data.map(move |result| {
            result.inspect(|batch| {
                let loss = batch
                    .metadata()
                    .get(LOSS_METADATA_KEY)
                    .and_then(|s| s.parse::<f64>().ok())
                    .unwrap_or(0.0);
                *loss_ref.lock().unwrap() += loss;
            })
        });

        // Rechunk to target batch size
        let rechunked = rechunk_stream_by_size(
            loss_stream,
            full_schema,
            batch_size_bytes,
            batch_size_bytes * 2,
        );

        // Create data file writer
        let data_path = self.output_dir.child("shuffle_data.lance");
        let spill_path = self.output_dir.child("shuffle_data.spill");
        let writer = self.object_store.create(&data_path).await?;
        let mut file_writer = FileWriter::try_new(
            writer,
            lance_core::datatypes::Schema::try_from(&schema)?,
            Default::default(),
        )?
        .with_page_metadata_spill(self.object_store.clone(), spill_path);

        // Create offsets file writer
        let offsets_path = self.output_dir.child("shuffle_offsets.lance");
        let spill_path = self.output_dir.child("shuffle_offsets.spill");
        let writer = self.object_store.create(&offsets_path).await?;
        let mut offsets_writer = FileWriter::try_new(
            writer,
            lance_core::datatypes::Schema::try_from(offsets_schema.as_ref())?,
            Default::default(),
        )?
        .with_page_metadata_spill(self.object_store.clone(), spill_path);

        let num_batches = Arc::new(AtomicU64::new(0));
        let num_batches_ref = num_batches.clone();
        let mut partition_counts: Vec<u64> = vec![0; num_partitions];
        let mut global_row_count: u64 = 0;
        let mut rows_processed: u64 = 0;

        let mut rechunked = std::pin::pin!(rechunked);
        while let Some(batch) = rechunked.next().await {
            num_batches_ref.fetch_add(1, std::sync::atomic::Ordering::Relaxed);
            let batch = batch?;
            let np = num_partitions;
            let num_rows = batch.num_rows() as u64;

            // Sort by partition ID and compute offsets on CPU
            let (sorted_batch, batch_offsets) = spawn_cpu(move || {
                let part_ids: &UInt32Array = batch[PART_ID_COLUMN].as_primitive();
                let indices = sort_to_indices(part_ids, None, None)?;
                let batch = batch.take(&indices)?;

                let part_ids: &UInt32Array = batch[PART_ID_COLUMN].as_primitive();
                let batch = batch.drop_column(PART_ID_COLUMN)?;

                // Count rows per partition by scanning sorted part IDs
                let mut partition_counts = vec![0u64; np];
                for i in 0..part_ids.len() {
                    let pid = part_ids.value(i) as usize;
                    if pid < np {
                        partition_counts[pid] += 1;
                    } else {
                        log::warn!("Partition ID {} is out of range [0, {})", pid, np);
                    }
                }

                // Build cumulative offsets (end positions) for this batch
                let mut batch_offsets = Vec::with_capacity(np);
                let mut running = 0u64;
                for count in &partition_counts {
                    running += count;
                    batch_offsets.push(running);
                }

                Ok::<(RecordBatch, Vec<u64>), Error>((batch, batch_offsets))
            })
            .await?;

            // Write sorted batch to data file
            file_writer.write_batch(&sorted_batch).await?;

            // Record offsets adjusted by global row count
            let mut adjusted_offsets = Vec::with_capacity(batch_offsets.len());
            let mut last_offset = 0;
            for (idx, offset) in batch_offsets.iter().enumerate() {
                adjusted_offsets.push(global_row_count + offset);
                partition_counts[idx] += offset - last_offset;
                last_offset = *offset;
            }
            global_row_count += sorted_batch.num_rows() as u64;

            // Write offsets to offsets file
            let offsets_batch = RecordBatch::try_new(
                offsets_schema.clone(),
                vec![Arc::new(UInt64Array::from(adjusted_offsets))],
            )?;
            offsets_writer.write_batch(&offsets_batch).await?;

            rows_processed += num_rows;
            self.progress
                .stage_progress("shuffle", rows_processed)
                .await?;
        }

        // Finish files
        file_writer.finish().await?;
        offsets_writer.finish().await?;

        let num_batches = num_batches.load(std::sync::atomic::Ordering::Relaxed);

        let total_loss_val = *total_loss.lock().unwrap();

        TwoFileShuffleReader::try_new(
            self.object_store.clone(),
            self.output_dir.clone(),
            num_partitions,
            num_batches,
            partition_counts,
            total_loss_val,
        )
        .await
    }
}

pub struct TwoFileShuffleReader {
    _scheduler: Arc<ScanScheduler>,
    file_reader: FileReader,
    offsets_reader: FileReader,
    num_partitions: usize,
    num_batches: u64,
    partition_counts: Vec<u64>,
    total_loss: f64,
}

impl TwoFileShuffleReader {
    async fn try_new(
        object_store: Arc<ObjectStore>,
        output_dir: Path,
        num_partitions: usize,
        num_batches: u64,
        partition_counts: Vec<u64>,
        total_loss: f64,
    ) -> Result<Box<dyn ShuffleReader>> {
        if num_batches == 0 {
            return Ok(Box::new(EmptyReader));
        }

        let scheduler_config = SchedulerConfig::max_bandwidth(&object_store);
        let scheduler = ScanScheduler::new(object_store, scheduler_config);

        let data_path = output_dir.child("shuffle_data.lance");
        let file_reader = FileReader::try_open(
            scheduler
                .open_file(&data_path, &CachedFileSize::unknown())
                .await?,
            None,
            Arc::<DecoderPlugins>::default(),
            &LanceCache::no_cache(),
            FileReaderOptions::default(),
        )
        .await?;

        let offsets_path = output_dir.child("shuffle_offsets.lance");
        let offsets_reader = FileReader::try_open(
            scheduler
                .open_file(&offsets_path, &CachedFileSize::unknown())
                .await?,
            None,
            Arc::<DecoderPlugins>::default(),
            &LanceCache::no_cache(),
            FileReaderOptions::default(),
        )
        .await?;

        Ok(Box::new(Self {
            _scheduler: scheduler,
            file_reader,
            offsets_reader,
            num_partitions,
            num_batches,
            partition_counts,
            total_loss,
        }))
    }

    async fn partition_ranges(&self, partition_id: usize) -> Result<Vec<Range<u64>>> {
        let mut positions = Vec::with_capacity(self.num_batches as usize * 2);
        for batch_idx in 0..self.num_batches {
            let end_pos = u32::try_from(batch_idx as usize * self.num_partitions + partition_id)
                .map_err(|_| Error::invalid_input("There are more than 2^32 partition offsets in the spill file.  Need to support 64-bit take"))?;
            if end_pos != 0 {
                positions.push(end_pos - 1);
            }
            positions.push(end_pos);
        }
        let positions = UInt32Array::from(positions);
        let num_positions = positions.len() as u32;
        let offsets_stream = self.offsets_reader.read_stream(
            ReadBatchParams::Indices(positions),
            num_positions,
            1,
            FilterExpression::no_filter(),
        )?;
        let schema = offsets_stream.schema().clone();
        let offsets = offsets_stream.try_collect::<Vec<_>>().await?;
        let offsets = if offsets.is_empty() {
            // We should not hit this path if there is no batches
            unreachable!()
        } else if offsets.len() == 1 {
            offsets.into_iter().next().unwrap()
        } else {
            concat_batches(&schema, &offsets)?
        };

        let offsets = offsets.column(0).as_primitive::<UInt64Type>();
        let mut offsets_iter = offsets.values().iter().copied();

        let mut ranges = Vec::with_capacity(self.num_batches as usize);
        for batch_idx in 0..self.num_batches {
            if batch_idx == 0 && partition_id == 0 {
                // Implicit 0 for start-of-file
                ranges.push(0..offsets_iter.next().unwrap());
            } else {
                ranges.push(offsets_iter.next().unwrap()..offsets_iter.next().unwrap());
            }
        }
        Ok(ranges)
    }
}

#[async_trait::async_trait]
impl ShuffleReader for TwoFileShuffleReader {
    async fn read_partition(
        &self,
        partition_id: usize,
    ) -> Result<Option<Box<dyn RecordBatchStream + Unpin + 'static>>> {
        if partition_id >= self.num_partitions {
            return Ok(None);
        }
        if self.partition_counts[partition_id] == 0 {
            return Ok(None);
        }

        let ranges = self.partition_ranges(partition_id).await?;
        if ranges.is_empty() {
            return Ok(None);
        }

        let schema: Schema = self.file_reader.schema().as_ref().into();
        Ok(Some(Box::new(RecordBatchStreamAdapter::new(
            Arc::new(schema),
            self.file_reader.read_stream(
                ReadBatchParams::Ranges(ranges.into()),
                u32::MAX,
                16,
                FilterExpression::no_filter(),
            )?,
        ))))
    }

    fn partition_size(&self, partition_id: usize) -> Result<usize> {
        Ok(self
            .partition_counts
            .get(partition_id)
            .copied()
            .unwrap_or(0) as usize)
    }

    fn total_loss(&self) -> Option<f64> {
        Some(self.total_loss)
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    use arrow_array::{Int32Array, RecordBatch, UInt32Array};
    use arrow_schema::{DataType, Field, Schema as ArrowSchema};
    use futures::stream;
    use lance_arrow::RecordBatchExt;
    use lance_core::utils::tempfile::TempStrDir;
    use lance_io::stream::RecordBatchStreamAdapter;

    use crate::vector::{LOSS_METADATA_KEY, PART_ID_COLUMN};

    /// Create a test batch with partition IDs, an int column, and optional loss metadata.
    fn make_batch(part_ids: &[u32], values: &[i32], loss: Option<f64>) -> RecordBatch {
        let schema = Arc::new(ArrowSchema::new(vec![
            Field::new(PART_ID_COLUMN, DataType::UInt32, false),
            Field::new("val", DataType::Int32, false),
        ]));
        let batch = RecordBatch::try_new(
            schema,
            vec![
                Arc::new(UInt32Array::from(part_ids.to_vec())),
                Arc::new(Int32Array::from(values.to_vec())),
            ],
        )
        .unwrap();
        if let Some(loss_val) = loss {
            batch
                .add_metadata(LOSS_METADATA_KEY.to_owned(), loss_val.to_string())
                .unwrap()
        } else {
            batch
        }
    }

    fn batches_to_stream(
        batches: Vec<RecordBatch>,
    ) -> Box<dyn RecordBatchStream + Unpin + 'static> {
        let schema = batches[0].schema();
        let stream = stream::iter(batches.into_iter().map(Ok));
        Box::new(RecordBatchStreamAdapter::new(schema, stream))
    }

    /// Collect all rows from a partition into a single RecordBatch.
    async fn collect_partition(
        reader: &dyn ShuffleReader,
        partition_id: usize,
    ) -> Option<RecordBatch> {
        let stream = reader.read_partition(partition_id).await.unwrap()?;
        let batches: Vec<RecordBatch> = stream.try_collect().await.unwrap();
        if batches.is_empty() {
            return None;
        }
        Some(arrow::compute::concat_batches(&batches[0].schema(), &batches).unwrap())
    }

    #[tokio::test]
    async fn test_two_file_shuffler_round_trip() {
        let dir = TempStrDir::default();
        let output_dir = Path::from(dir.as_ref());
        let num_partitions = 3;

        // Partition 0: rows with values 10, 40
        // Partition 1: rows with values 20, 50
        // Partition 2: rows with values 30
        let batch = make_batch(&[0, 1, 2, 0, 1], &[10, 20, 30, 40, 50], None);

        let shuffler = TwoFileShuffler::new(output_dir, num_partitions);
        let stream = batches_to_stream(vec![batch]);
        let reader = shuffler.shuffle(stream).await.unwrap();

        // Verify partition sizes
        assert_eq!(reader.partition_size(0).unwrap(), 2);
        assert_eq!(reader.partition_size(1).unwrap(), 2);
        assert_eq!(reader.partition_size(2).unwrap(), 1);

        // Verify partition 0 data
        let p0 = collect_partition(reader.as_ref(), 0).await.unwrap();
        let vals: &Int32Array = p0.column_by_name("val").unwrap().as_primitive();
        let mut v: Vec<i32> = vals.iter().map(|x| x.unwrap()).collect();
        v.sort();
        assert_eq!(v, vec![10, 40]);

        // Verify partition 1 data
        let p1 = collect_partition(reader.as_ref(), 1).await.unwrap();
        let vals: &Int32Array = p1.column_by_name("val").unwrap().as_primitive();
        let mut v: Vec<i32> = vals.iter().map(|x| x.unwrap()).collect();
        v.sort();
        assert_eq!(v, vec![20, 50]);

        // Verify partition 2 data
        let p2 = collect_partition(reader.as_ref(), 2).await.unwrap();
        let vals: &Int32Array = p2.column_by_name("val").unwrap().as_primitive();
        let v: Vec<i32> = vals.iter().map(|x| x.unwrap()).collect();
        assert_eq!(v, vec![30]);

        // Out of range partition returns None
        assert!(reader.read_partition(3).await.unwrap().is_none());
    }

    #[tokio::test]
    async fn test_two_file_shuffler_empty_partitions() {
        let dir = TempStrDir::default();
        let output_dir = Path::from(dir.as_ref());
        let num_partitions = 5;

        // Only use partitions 0 and 3, leaving 1, 2, 4 empty
        let batch = make_batch(&[0, 3, 0, 3], &[10, 20, 30, 40], None);

        let shuffler = TwoFileShuffler::new(output_dir, num_partitions);
        let stream = batches_to_stream(vec![batch]);
        let reader = shuffler.shuffle(stream).await.unwrap();

        assert_eq!(reader.partition_size(0).unwrap(), 2);
        assert_eq!(reader.partition_size(1).unwrap(), 0);
        assert_eq!(reader.partition_size(2).unwrap(), 0);
        assert_eq!(reader.partition_size(3).unwrap(), 2);
        assert_eq!(reader.partition_size(4).unwrap(), 0);

        assert!(reader.read_partition(1).await.unwrap().is_none());
        assert!(reader.read_partition(2).await.unwrap().is_none());
        assert!(reader.read_partition(4).await.unwrap().is_none());

        let p0 = collect_partition(reader.as_ref(), 0).await.unwrap();
        assert_eq!(p0.num_rows(), 2);
        let p3 = collect_partition(reader.as_ref(), 3).await.unwrap();
        assert_eq!(p3.num_rows(), 2);
    }

    #[tokio::test]
    async fn test_two_file_shuffler_loss_tracking() {
        let dir = TempStrDir::default();
        let output_dir = Path::from(dir.as_ref());
        let num_partitions = 2;

        let batch1 = make_batch(&[0, 1], &[10, 20], Some(1.5));
        let batch2 = make_batch(&[0, 1], &[30, 40], Some(2.5));
        let batch3 = make_batch(&[0], &[50], Some(0.25));

        let shuffler = TwoFileShuffler::new(output_dir, num_partitions);
        let stream = batches_to_stream(vec![batch1, batch2, batch3]);
        let reader = shuffler.shuffle(stream).await.unwrap();

        let loss = reader.total_loss().unwrap();
        assert!((loss - 4.25).abs() < 1e-10, "expected 4.25, got {}", loss);
    }

    #[tokio::test]
    async fn test_two_file_shuffler_single_batch() {
        let dir = TempStrDir::default();
        let output_dir = Path::from(dir.as_ref());
        let num_partitions = 2;

        let batch = make_batch(&[1, 0], &[100, 200], Some(3.0));

        let shuffler = TwoFileShuffler::new(output_dir, num_partitions);
        let stream = batches_to_stream(vec![batch]);
        let reader = shuffler.shuffle(stream).await.unwrap();

        assert_eq!(reader.partition_size(0).unwrap(), 1);
        assert_eq!(reader.partition_size(1).unwrap(), 1);

        let p0 = collect_partition(reader.as_ref(), 0).await.unwrap();
        let vals: &Int32Array = p0.column_by_name("val").unwrap().as_primitive();
        assert_eq!(vals.value(0), 200);

        let p1 = collect_partition(reader.as_ref(), 1).await.unwrap();
        let vals: &Int32Array = p1.column_by_name("val").unwrap().as_primitive();
        assert_eq!(vals.value(0), 100);

        assert!((reader.total_loss().unwrap() - 3.0).abs() < 1e-10);
    }

    #[tokio::test]
    async fn test_two_file_shuffler_multiple_batches() {
        let dir = TempStrDir::default();
        let output_dir = Path::from(dir.as_ref());
        let num_partitions = 3;

        // Use a very small batch size to force multiple write batches
        // Each i32 is 4 bytes, each u32 is 4 bytes, so ~8 bytes/row.
        // With a small batch_size_bytes, we get multiple rechunked batches.
        let batch1 = make_batch(&[0, 1, 2], &[10, 20, 30], Some(1.0));
        let batch2 = make_batch(&[2, 0, 1], &[40, 50, 60], Some(2.0));
        let batch3 = make_batch(&[1, 2, 0], &[70, 80, 90], Some(3.0));

        let shuffler = TwoFileShuffler::new(output_dir, num_partitions)
            // Set very small batch size to force multiple batches
            .with_batch_size_bytes(16);
        let stream = batches_to_stream(vec![batch1, batch2, batch3]);
        let reader = shuffler.shuffle(stream).await.unwrap();

        // Partition 0 should have values: 10, 50, 90
        assert_eq!(reader.partition_size(0).unwrap(), 3);
        let p0 = collect_partition(reader.as_ref(), 0).await.unwrap();
        let vals: &Int32Array = p0.column_by_name("val").unwrap().as_primitive();
        let mut v: Vec<i32> = vals.iter().map(|x| x.unwrap()).collect();
        v.sort();
        assert_eq!(v, vec![10, 50, 90]);

        // Partition 1 should have values: 20, 60, 70
        assert_eq!(reader.partition_size(1).unwrap(), 3);
        let p1 = collect_partition(reader.as_ref(), 1).await.unwrap();
        let vals: &Int32Array = p1.column_by_name("val").unwrap().as_primitive();
        let mut v: Vec<i32> = vals.iter().map(|x| x.unwrap()).collect();
        v.sort();
        assert_eq!(v, vec![20, 60, 70]);

        // Partition 2 should have values: 30, 40, 80
        assert_eq!(reader.partition_size(2).unwrap(), 3);
        let p2 = collect_partition(reader.as_ref(), 2).await.unwrap();
        let vals: &Int32Array = p2.column_by_name("val").unwrap().as_primitive();
        let mut v: Vec<i32> = vals.iter().map(|x| x.unwrap()).collect();
        v.sort();
        assert_eq!(v, vec![30, 40, 80]);

        assert!((reader.total_loss().unwrap() - 6.0).abs() < 1e-10);
    }
}