lance_index/vector/ivf/
shuffler.rs

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

//! Disk-based shuffle a stream of [RecordBatch] into each IVF partition.
//!
//! 1. write the entire stream to a file
//! 2. count the number of rows in each partition
//! 3. read the data back into memory and shuffle into grouped vectors
//!
//! Problems for the future:
//! 1. while groupby column will stay the same, we may want to include extra data columns in the future
//! 2. shuffling into memory is fast but we should add disk buffer to support bigger datasets

use std::collections::HashMap;
use std::sync::Arc;

use arrow::array::{
    ArrayBuilder, FixedSizeListBuilder, StructBuilder, UInt32Builder, UInt64Builder, UInt8Builder,
};
use arrow::buffer::{OffsetBuffer, ScalarBuffer};
use arrow::compute::sort_to_indices;
use arrow::datatypes::UInt32Type;
use arrow_array::{cast::AsArray, types::UInt64Type, Array, RecordBatch, UInt32Array};
use arrow_array::{FixedSizeListArray, UInt8Array};
use arrow_array::{ListArray, StructArray, UInt64Array};
use arrow_schema::{DataType, Field, Fields};
use futures::stream::repeat_with;
use futures::{stream, FutureExt, Stream, StreamExt, TryStreamExt};
use lance_arrow::RecordBatchExt;
use lance_core::cache::{CapacityMode, FileMetadataCache};
use lance_core::utils::tokio::get_num_compute_intensive_cpus;
use lance_core::{datatypes::Schema, Error, Result, ROW_ID};
use lance_encoding::decoder::{DecoderPlugins, FilterExpression};
use lance_file::reader::FileReader;
use lance_file::v2::reader::{FileReader as Lancev2FileReader, FileReaderOptions};
use lance_file::v2::writer::FileWriterOptions;
use lance_file::writer::FileWriter;
use lance_io::object_store::ObjectStore;
use lance_io::scheduler::{ScanScheduler, SchedulerConfig};
use lance_io::stream::RecordBatchStream;
use lance_io::ReadBatchParams;
use lance_table::format::SelfDescribingFileReader;
use lance_table::io::manifest::ManifestDescribing;
use log::info;
use object_store::path::Path;
use snafu::{location, Location};
use tempfile::TempDir;

use crate::vector::ivf::IvfTransformer;
use crate::vector::transform::{KeepFiniteVectors, Transformer};
use crate::vector::PART_ID_COLUMN;

const UNSORTED_BUFFER: &str = "unsorted.lance";
const SHUFFLE_BATCH_SIZE: usize = 1024;

fn get_temp_dir() -> Result<Path> {
    // Note: using into_path here means we will not delete this TempDir automatically
    let dir = TempDir::new()?.into_path();
    let tmp_dir_path = Path::from_filesystem_path(dir).map_err(|e| Error::IO {
        source: Box::new(e),
        location: location!(),
    })?;
    Ok(tmp_dir_path)
}

/// A builder for a partition of data
///
/// After we sort a batch of data into partitions we append those slices into this builder.
///
/// The builder is pre-allocated and so this extend operation should only be a memcpy
#[derive(Debug)]
struct PartitionBuilder {
    builder: StructBuilder,
}

// Fork of arrow_array::builder::make_builder that handles FixedSizeList >_<
//
// Not really suitable for upstreaming because FixedSizeListBuilder<Box<dyn ArrayBuilder>> is
// awkward and the entire make_builder function needs some overhaul (dyn ArrayBuilder should have
// an extend(array: &dyn Array) method).
fn make_builder(datatype: &DataType, capacity: usize) -> Box<dyn ArrayBuilder> {
    if let DataType::FixedSizeList(inner, dim) = datatype {
        let inner_builder =
            arrow_array::builder::make_builder(inner.data_type(), capacity * (*dim) as usize);
        Box::new(FixedSizeListBuilder::new(inner_builder, *dim))
    } else {
        arrow_array::builder::make_builder(datatype, capacity)
    }
}

// Fork of StructBuilder::from_fields that handles FixedSizeList >_<
fn from_fields(fields: impl Into<Fields>, capacity: usize) -> StructBuilder {
    let fields = fields.into();
    let mut builders = Vec::with_capacity(fields.len());
    for field in &fields {
        builders.push(make_builder(field.data_type(), capacity));
    }
    StructBuilder::new(fields, builders)
}

impl PartitionBuilder {
    fn new(schema: &arrow_schema::Schema, initial_capacity: usize) -> Self {
        let builder = from_fields(schema.fields.clone(), initial_capacity);
        Self { builder }
    }

    fn extend(&mut self, batch: &RecordBatch) {
        for _ in 0..batch.num_rows() {
            self.builder.append(true);
        }
        let schema = batch.schema_ref();
        for (field_idx, (col, field)) in batch.columns().iter().zip(schema.fields()).enumerate() {
            match field.data_type() {
                DataType::UInt32 => {
                    let col = col.as_any().downcast_ref::<UInt32Array>().unwrap();
                    self.builder
                        .field_builder::<UInt32Builder>(field_idx)
                        .unwrap()
                        .append_slice(col.values());
                }
                DataType::UInt64 => {
                    let col = col.as_any().downcast_ref::<UInt64Array>().unwrap();
                    self.builder
                        .field_builder::<UInt64Builder>(field_idx)
                        .unwrap()
                        .append_slice(col.values());
                }
                DataType::FixedSizeList(inner, _) => {
                    let col = col.as_any().downcast_ref::<FixedSizeListArray>().unwrap();
                    match inner.data_type() {
                        DataType::UInt8 => {
                            let values =
                                col.values().as_any().downcast_ref::<UInt8Array>().unwrap();
                            let fsl_builder = self
                                .builder
                                .field_builder::<FixedSizeListBuilder<Box<dyn ArrayBuilder>>>(
                                    field_idx,
                                )
                                .unwrap();
                            // TODO: Upstream an append_many to FSL builder
                            for _ in 0..col.len() {
                                fsl_builder.append(true);
                            }
                            fsl_builder
                                .values()
                                .as_any_mut()
                                .downcast_mut::<UInt8Builder>()
                                .unwrap()
                                .append_slice(values.values());
                        }
                        _ => panic!("Unexpected fixed size list item type in shuffled file"),
                    }
                }
                _ => panic!("Unexpected column type in shuffled file"),
            }
        }
    }

    // Convert the partition builder into a list array with 1 row
    fn finish(mut self) -> Result<ListArray> {
        let struct_array = Arc::new(self.builder.finish());

        let item_field = Arc::new(Field::new("item", struct_array.data_type().clone(), true));

        Ok(ListArray::try_new(
            item_field,
            OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![
                0,
                struct_array.len() as i32,
            ])),
            struct_array,
            None,
        )?)
    }
}

struct PartitionListBuilder {
    partitions: Vec<Option<PartitionBuilder>>,
    partition_sizes: Vec<u64>,
}

impl PartitionListBuilder {
    fn new(partition_sizes: Vec<u64>) -> Self {
        Self {
            partitions: Vec::default(),
            partition_sizes,
        }
    }

    fn extend(&mut self, batch: &RecordBatch) {
        if batch.num_rows() == 0 {
            return;
        }

        if self.partitions.is_empty() {
            let schema = batch.schema();
            self.partitions = Vec::from_iter(self.partition_sizes.iter().map(|part_size| {
                if *part_size == 0 {
                    None
                } else {
                    Some(PartitionBuilder::new(schema.as_ref(), *part_size as usize))
                }
            }))
        }

        let part_ids = batch[PART_ID_COLUMN].as_primitive::<UInt32Type>();

        let part_id = part_ids.value(0) as usize;

        let builder = &mut self.partitions[part_id];
        builder
            .as_mut()
            .expect("partition size was zero but received data for partition")
            .extend(batch);
    }

    fn finish(self) -> Result<Vec<ListArray>> {
        self.partitions
            .into_iter()
            .filter_map(|builder| builder.map(|builder| builder.finish()))
            .collect()
    }
}

/// Disk-based shuffle for a stream of [RecordBatch] into each IVF partition.
/// Sub-quantizer will be applied if provided.
///
/// Parameters
/// ----------
///   *data*: input data stream.
///   *column*: column name of the vector column.
///   *ivf*: IVF model.
///   *num_partitions*: number of IVF partitions.
///   *num_sub_vectors*: number of PQ sub-vectors.
///
/// Returns
/// -------
///   Result<Vec<impl Stream<Item = Result<RecordBatch>>>>: a vector of streams
///   of shuffled partitioned data. Each stream corresponds to a partition and
///   is sorted within the stream. Consumer of these streams is expected to merge
///   the streams into a single stream by k-list merge algo.
///
#[allow(clippy::too_many_arguments)]
pub async fn shuffle_dataset(
    data: impl RecordBatchStream + Unpin + 'static,
    column: &str,
    ivf: Arc<IvfTransformer>,
    precomputed_partitions: Option<HashMap<u64, u32>>,
    num_partitions: u32,
    shuffle_partition_batches: usize,
    shuffle_partition_concurrency: usize,
    precomputed_shuffle_buffers: Option<(Path, Vec<String>)>,
) -> Result<Vec<impl Stream<Item = Result<RecordBatch>>>> {
    // step 1: either use precomputed shuffle files or write shuffle data to a file
    let shuffler = if let Some((path, buffers)) = precomputed_shuffle_buffers {
        info!("Precomputed shuffle files provided, skip calculation of IVF partition.");
        let mut shuffler = IvfShuffler::try_new(num_partitions, Some(path), true, None)?;
        unsafe {
            shuffler.set_unsorted_buffers(&buffers);
        }

        shuffler
    } else {
        info!(
            "Calculating IVF partitions for vectors (num_partitions={}, precomputed_partitions={})",
            num_partitions,
            precomputed_partitions.is_some()
        );
        let mut shuffler = IvfShuffler::try_new(num_partitions, None, true, None)?;

        let column = column.to_owned();
        let precomputed_partitions = precomputed_partitions.map(Arc::new);
        let stream = data
            .zip(repeat_with(move || ivf.clone()))
            .map(move |(b, ivf)| {
                // If precomputed_partitions map is provided, use it
                // for fast partitions.
                let partition_map = precomputed_partitions
                    .as_ref()
                    .cloned()
                    .unwrap_or(Arc::new(HashMap::new()));
                let nan_filter = KeepFiniteVectors::new(&column);

                tokio::task::spawn(async move {
                    let mut batch = b?;

                    if !partition_map.is_empty() {
                        let row_ids = batch.column_by_name(ROW_ID).ok_or(Error::Index {
                            message: "column does not exist".to_string(),
                            location: location!(),
                        })?;
                        let part_ids = UInt32Array::from_iter(
                            row_ids
                                .as_primitive::<UInt64Type>()
                                .values()
                                .iter()
                                .map(|row_id| partition_map.get(row_id).copied()),
                        );
                        let part_ids = UInt32Array::from(part_ids);
                        batch = batch
                            .try_with_column(
                                Field::new(PART_ID_COLUMN, part_ids.data_type().clone(), true),
                                Arc::new(part_ids.clone()),
                            )
                            .expect("failed to add part id column");

                        if part_ids.null_count() > 0 {
                            info!(
                                "Filter out rows without valid partition IDs: null_count={}",
                                part_ids.null_count()
                            );
                            let indices = UInt32Array::from_iter(
                                part_ids
                                    .iter()
                                    .enumerate()
                                    .filter_map(|(idx, v)| v.map(|_| idx as u32)),
                            );
                            assert_eq!(indices.len(), batch.num_rows() - part_ids.null_count());
                            batch = batch.take(&indices)?;
                        }
                    }

                    // Filter out NaNs/Infs
                    batch = nan_filter.transform(&batch)?;

                    ivf.transform(&batch)
                })
            })
            .buffer_unordered(get_num_compute_intensive_cpus())
            .map(|res| match res {
                Ok(Ok(batch)) => Ok(batch),
                Ok(Err(err)) => Err(Error::io(err.to_string(), location!())),
                Err(err) => Err(Error::io(err.to_string(), location!())),
            })
            .boxed();

        let start = std::time::Instant::now();
        shuffler.write_unsorted_stream(stream).await?;
        info!(
            "wrote partition assignment to unsorted tmp file in {:?}",
            start.elapsed()
        );

        shuffler
    };

    // step 2: stream in the shuffle data in chunks and write sorted chunks out
    let start = std::time::Instant::now();
    let partition_files = shuffler
        .write_partitioned_shuffles(shuffle_partition_batches, shuffle_partition_concurrency)
        .await?;
    info!("created sorted chunks in {:?}", start.elapsed());

    // step 3: load the sorted chunks, consumers are expect to be responsible for merging the streams
    let start = std::time::Instant::now();
    let stream =
        IvfShuffler::load_partitioned_shuffles(&shuffler.output_dir, partition_files).await?;
    info!("merged partitioned shuffles in {:?}", start.elapsed());

    Ok(stream)
}

pub async fn shuffle_vectors(
    unsorted_filenames: Vec<String>,
    dir_path: Path,
    ivf_centroids: FixedSizeListArray,
    shuffle_output_root_filename: &str,
) -> Result<Vec<String>> {
    let num_partitions = ivf_centroids.len() as u32;
    let shuffle_partition_batches = SHUFFLE_BATCH_SIZE * 10;
    let shuffle_partition_concurrency = 2;
    let mut shuffler = IvfShuffler::try_new(
        num_partitions,
        Some(dir_path),
        false,
        Some(shuffle_output_root_filename.to_string()),
    )?;

    unsafe {
        shuffler.set_unsorted_buffers(&unsorted_filenames);
    }

    let partition_files = shuffler
        .write_partitioned_shuffles(shuffle_partition_batches, shuffle_partition_concurrency)
        .await?;

    Ok(partition_files)
}

#[derive(Clone)]
pub struct IvfShuffler {
    unsorted_buffers: Vec<String>,

    num_partitions: u32,

    output_dir: Path,

    // whether the lance file is v1 (legacy) or v2
    is_legacy: bool,

    shuffle_output_root_filename: String,
}

/// Represents a range of batches in a file that should be shuffled
struct ShuffleInput {
    // the idx of the file in IvfShuffler::unsorted_buffers
    file_idx: usize,
    // the start index of the batch in the file
    start: usize,
    // the end index of the batch in the file
    end: usize,
}

impl IvfShuffler {
    pub fn try_new(
        num_partitions: u32,
        output_dir: Option<Path>,
        is_legacy: bool,
        shuffle_output_root_filename: Option<String>,
    ) -> Result<Self> {
        let output_dir = match output_dir {
            Some(output_dir) => output_dir,
            None => get_temp_dir()?,
        };

        let shuffle_output_root_filename = match shuffle_output_root_filename {
            Some(shuffle_output_root_filename) => shuffle_output_root_filename,
            None => "sorted".to_string(),
        };

        Ok(Self {
            num_partitions,
            output_dir,
            unsorted_buffers: vec![],
            is_legacy,
            shuffle_output_root_filename,
        })
    }

    /// Set the unsorted buffers to be shuffled.
    ///
    /// # Safety
    ///
    /// user must ensure the buffers are valid.
    pub unsafe fn set_unsorted_buffers(&mut self, unsorted_buffers: &[impl ToString]) {
        self.unsorted_buffers = unsorted_buffers.iter().map(|x| x.to_string()).collect();
    }

    pub async fn write_unsorted_stream(
        &mut self,
        data: impl Stream<Item = Result<RecordBatch>>,
    ) -> Result<()> {
        let object_store = ObjectStore::local();
        let path = self.output_dir.child(UNSORTED_BUFFER);
        let writer = object_store.create(&path).await?;

        let mut data = Box::pin(data.peekable());
        let schema = match data.as_mut().peek().await {
            Some(Ok(batch)) => batch.schema(),
            Some(Err(err)) => {
                return Err(Error::io(err.to_string(), location!()));
            }
            None => {
                return Err(Error::io("empty stream".to_string(), location!()));
            }
        };

        // validate the schema,
        // we need to have row ID and partition ID column
        schema
            .column_with_name(ROW_ID)
            .ok_or(Error::io("row ID column not found".to_owned(), location!()))?;
        schema.column_with_name(PART_ID_COLUMN).ok_or(Error::io(
            "partition ID column not found".to_owned(),
            location!(),
        ))?;

        info!("Writing unsorted data to disk at {}", path);
        info!("with schema: {:?}", schema);

        let mut file_writer = FileWriter::<ManifestDescribing>::with_object_writer(
            writer,
            Schema::try_from(schema.as_ref())?,
            &Default::default(),
        )?;

        let mut batches_processed = 0;
        while let Some(batch) = data.next().await {
            if batches_processed % 1000 == 0 {
                info!("Partition assignment progress {}/?", batches_processed);
            }
            batches_processed += 1;
            file_writer.write(&[batch?]).await?;
        }

        file_writer.finish().await?;

        unsafe {
            self.set_unsorted_buffers(&[UNSORTED_BUFFER]);
        }

        Ok(())
    }

    async fn total_batches(&self) -> Result<Vec<usize>> {
        let mut total_batches = vec![];
        for buffer in &self.unsorted_buffers {
            let object_store = ObjectStore::local();
            let path = self.output_dir.child(buffer.as_str());

            if self.is_legacy {
                let reader = FileReader::try_new_self_described(&object_store, &path, None).await?;
                total_batches.push(reader.num_batches());
            } else {
                let scheduler_config = SchedulerConfig::max_bandwidth(&object_store);
                let scheduler = ScanScheduler::new(object_store.into(), scheduler_config);
                let file = scheduler.open_file(&path).await?;
                let cache =
                    FileMetadataCache::with_capacity(128 * 1024 * 1024, CapacityMode::Bytes);

                let reader = Lancev2FileReader::try_open(
                    file,
                    None,
                    Default::default(),
                    &cache,
                    FileReaderOptions::default(),
                )
                .await?;
                let num_batches = reader.metadata().num_rows / (SHUFFLE_BATCH_SIZE as u64);
                total_batches.push(num_batches as usize);
            }
        }
        Ok(total_batches)
    }

    async fn count_partition_size(&self, inputs: &[ShuffleInput]) -> Result<Vec<u64>> {
        let object_store = ObjectStore::local();
        let mut partition_sizes = vec![0; self.num_partitions as usize];
        let scheduler = ScanScheduler::new(
            Arc::new(object_store.clone()),
            SchedulerConfig::max_bandwidth(&object_store),
        );

        for &ShuffleInput {
            file_idx,
            start,
            end,
        } in inputs
        {
            let file_name = &self.unsorted_buffers[file_idx];
            let path = self.output_dir.child(file_name.as_str());

            if self.is_legacy {
                let reader = FileReader::try_new_self_described(&object_store, &path, None).await?;
                let lance_schema = reader
                    .schema()
                    .project(&[PART_ID_COLUMN])
                    .expect("part id should exist");

                let mut stream = stream::iter(start..end)
                    .map(|i| reader.read_batch(i as i32, .., &lance_schema))
                    .buffer_unordered(16);

                while let Some(batch) = stream.next().await {
                    let batch = batch?;
                    let part_ids: &UInt32Array = batch
                        .column_by_name(PART_ID_COLUMN)
                        .expect("Partition ID column not found")
                        .as_primitive();
                    part_ids.values().iter().for_each(|part_id| {
                        partition_sizes[*part_id as usize] += 1;
                    });
                }
            } else {
                let file = scheduler.open_file(&path).await?;
                let reader = Lancev2FileReader::try_open(
                    file,
                    None,
                    Default::default(),
                    &FileMetadataCache::no_cache(),
                    FileReaderOptions::default(),
                )
                .await?;
                let mut stream = reader
                    .read_stream(
                        lance_io::ReadBatchParams::Range(
                            (start * SHUFFLE_BATCH_SIZE)..(end * SHUFFLE_BATCH_SIZE),
                        ),
                        SHUFFLE_BATCH_SIZE as u32,
                        16,
                        FilterExpression::no_filter(),
                    )
                    .unwrap();

                while let Some(batch) = stream.next().await {
                    let batch = batch?;
                    let part_ids: &UInt32Array = batch
                        .column_by_name(PART_ID_COLUMN)
                        .expect("Partition ID column not found")
                        .as_primitive();
                    part_ids.values().iter().for_each(|part_id| {
                        partition_sizes[*part_id as usize] += 1;
                    });
                }
            }
        }

        Ok(partition_sizes)
    }

    async fn shuffle_to_partitions(
        &self,
        inputs: &[ShuffleInput],
        partition_size: Vec<u64>,
        num_batches_to_sort: usize,
    ) -> Result<Vec<ListArray>> {
        info!("Shuffling into memory");

        let mut num_processed = 0;
        let mut partitions_builder = PartitionListBuilder::new(partition_size);

        for &ShuffleInput {
            file_idx,
            start,
            end,
        } in inputs
        {
            let object_store = ObjectStore::local();
            let file_name = &self.unsorted_buffers[file_idx];
            let path = self.output_dir.child(file_name.as_str());
            let mut _reader_handle = None;

            let mut stream = if self.is_legacy {
                _reader_handle =
                    Some(FileReader::try_new_self_described(&object_store, &path, None).await?);

                stream::iter(start..end)
                    .map(|i| {
                        let reader = _reader_handle.as_ref().unwrap();
                        reader.read_batch(i as i32, ReadBatchParams::RangeFull, reader.schema())
                    })
                    .buffered(16)
                    .boxed()
            } else {
                let scheduler_config = SchedulerConfig::max_bandwidth(&object_store);
                let scheduler = ScanScheduler::new(Arc::new(object_store), scheduler_config);
                let file = scheduler.open_file(&path).await?;
                let reader = Lancev2FileReader::try_open(
                    file,
                    None,
                    Default::default(),
                    &FileMetadataCache::no_cache(),
                    FileReaderOptions::default(),
                )
                .await?;
                reader
                    .read_stream(
                        lance_io::ReadBatchParams::Range(
                            (start * SHUFFLE_BATCH_SIZE)..(end * SHUFFLE_BATCH_SIZE),
                        ),
                        SHUFFLE_BATCH_SIZE as u32,
                        16,
                        FilterExpression::no_filter(),
                    )?
                    .boxed()
            };

            while let Some(batch) = stream.next().await {
                if num_processed % 100 == 0 {
                    info!("Shuffle Progress {}/{}", num_processed, num_batches_to_sort);
                }
                num_processed += 1;

                let batch = batch?;

                if batch.num_rows() == 0 {
                    continue;
                }

                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 sorted_part_ids: &UInt32Array = batch[PART_ID_COLUMN].as_primitive();

                let mut start = 0;
                let mut prev_id = sorted_part_ids.value(0);
                for (idx, part_id) in sorted_part_ids.values().iter().enumerate() {
                    if *part_id != prev_id {
                        partitions_builder.extend(&batch.slice(start, idx - start));
                        start = idx;
                        prev_id = *part_id;
                    }
                }
                partitions_builder.extend(&batch.slice(start, sorted_part_ids.len() - start));
            }
        }

        partitions_builder.finish()
    }

    pub async fn write_partitioned_shuffles(
        &self,
        batches_per_partition: usize,
        concurrent_jobs: usize,
    ) -> Result<Vec<String>> {
        let num_batches = self.total_batches().await?;
        let total_batches = num_batches.iter().sum();
        info!(
            "Sorting unsorted data into sorted chunks (batches_per_chunk={} concurrent_jobs={})",
            batches_per_partition, concurrent_jobs
        );
        stream::iter((0..total_batches).step_by(batches_per_partition))
            .zip(stream::repeat(num_batches))
            .map(|(i, num_batches)| {
                let this = self.clone();
                tokio::spawn(async move {
                    // first, calculate which files and ranges needs to be processed
                    let start = i;
                    let end = std::cmp::min(i + batches_per_partition, total_batches);
                    let num_batches_to_sort = end - start;
                    let mut input = vec![];

                    let mut cumulative_size = 0;
                    for (file_idx, partition_size) in num_batches.iter().enumerate() {
                        let cur_start = cumulative_size;
                        let cur_end = cumulative_size + partition_size;

                        cumulative_size += partition_size;

                        let should_include_file = start < cur_end && end > cur_start;

                        if !should_include_file {
                            continue;
                        }

                        // the current part doesn't overlap with the current batch
                        if start >= cur_end {
                            continue;
                        }

                        let local_start = if start < cur_start {
                            0
                        } else {
                            start - cur_start
                        };
                        let local_end = std::cmp::min(end - cur_start, *partition_size);

                        input.push(ShuffleInput {
                            file_idx,
                            start: local_start,
                            end: local_end,
                        });
                    }

                    // second, count the number of rows in each partition
                    let size_counts = this.count_partition_size(&input).await?;

                    // third, shuffle the data into each partition
                    let shuffled = this
                        .shuffle_to_partitions(&input, size_counts, num_batches_to_sort)
                        .await?;

                    // finally, write the shuffled data to disk
                    let object_store = ObjectStore::local();
                    let output_file = format!("{}_{}.lance", this.shuffle_output_root_filename, i);
                    let path = this.output_dir.child(output_file.clone());
                    let writer = object_store.create(&path).await?;

                    info!(
                        "Chunk loaded into memory and sorted, writing to disk at {}",
                        path
                    );

                    let sorted_file_schema = Arc::new(arrow_schema::Schema::new(vec![Field::new(
                        "partitions",
                        shuffled.first().unwrap().data_type().clone(),
                        true,
                    )]));
                    let lance_schema = Schema::try_from(sorted_file_schema.as_ref())?;
                    let mut file_writer = lance_file::v2::writer::FileWriter::try_new(
                        writer,
                        lance_schema,
                        FileWriterOptions::default(),
                    )?;

                    for partition_and_idx in shuffled.into_iter().enumerate() {
                        let (idx, partition) = partition_and_idx;
                        if idx % 1000 == 0 {
                            info!("Writing partition {}/{}", idx, this.num_partitions);
                        }
                        let batch = RecordBatch::try_new(
                            sorted_file_schema.clone(),
                            vec![Arc::new(partition)],
                        )?;
                        file_writer.write_batch(&batch).await?;
                    }

                    file_writer.finish().await?;

                    Ok(output_file) as Result<String>
                })
                .map(|join_res| join_res.unwrap())
            })
            .buffered(concurrent_jobs)
            .try_collect()
            .await
    }

    pub async fn load_partitioned_shuffles(
        basedir: &Path,
        files: Vec<String>,
    ) -> Result<Vec<impl Stream<Item = Result<RecordBatch>>>> {
        // impl RecordBatchStream
        let mut streams = vec![];

        for file in files {
            let object_store = Arc::new(ObjectStore::local());
            let path = basedir.child(file);
            let scheduler_config = SchedulerConfig::max_bandwidth(&object_store);
            let scan_scheduler = ScanScheduler::new(object_store, scheduler_config);
            let file_scheduler = scan_scheduler.open_file(&path).await?;
            let reader = lance_file::v2::reader::FileReader::try_open(
                file_scheduler,
                None,
                Arc::<DecoderPlugins>::default(),
                &FileMetadataCache::no_cache(),
                FileReaderOptions::default(),
            )
            .await?;
            let stream = reader
                .read_stream(
                    ReadBatchParams::RangeFull,
                    /*batch_size=*/ 1,
                    /*batch_readahead=*/ 32,
                    FilterExpression::no_filter(),
                )?
                .and_then(|batch| {
                    let list_array = batch
                        .column(0)
                        .as_any()
                        .downcast_ref::<ListArray>()
                        .expect("ListArray expected");
                    let struct_array = list_array
                        .values()
                        .as_any()
                        .downcast_ref::<StructArray>()
                        .expect("StructArray expected")
                        .clone();
                    let batch: RecordBatch = struct_array.into();
                    std::future::ready(Ok(batch))
                });

            streams.push(stream);
        }

        Ok(streams)
    }
}

#[cfg(test)]
mod test {
    use arrow_array::{
        types::{UInt32Type, UInt8Type},
        FixedSizeListArray, UInt64Array, UInt8Array,
    };
    use arrow_schema::DataType;
    use lance_arrow::FixedSizeListArrayExt;
    use lance_core::ROW_ID_FIELD;
    use lance_io::stream::RecordBatchStreamAdapter;
    use rand::RngCore;

    use crate::vector::PQ_CODE_COLUMN;

    use super::*;

    fn make_schema(pq_dim: u32) -> Arc<arrow_schema::Schema> {
        Arc::new(arrow_schema::Schema::new(vec![
            ROW_ID_FIELD.clone(),
            arrow_schema::Field::new(PART_ID_COLUMN, DataType::UInt32, true),
            arrow_schema::Field::new(
                PQ_CODE_COLUMN,
                DataType::FixedSizeList(
                    Arc::new(arrow_schema::Field::new("item", DataType::UInt8, true)),
                    pq_dim as i32,
                ),
                false,
            ),
        ]))
    }

    fn make_stream_and_shuffler(
        include_empty_batches: bool,
    ) -> (impl RecordBatchStream, IvfShuffler) {
        let schema = make_schema(32);

        let schema2 = schema.clone();

        let stream =
            stream::iter(0..if include_empty_batches { 101 } else { 100 }).map(move |idx| {
                if include_empty_batches && idx == 100 {
                    return Ok(RecordBatch::try_new(
                        schema2.clone(),
                        vec![
                            Arc::new(UInt64Array::from_iter_values([])),
                            Arc::new(UInt32Array::from_iter_values([])),
                            Arc::new(
                                FixedSizeListArray::try_new_from_values(
                                    Arc::new(UInt8Array::from_iter_values([])) as Arc<dyn Array>,
                                    32,
                                )
                                .unwrap(),
                            ),
                        ],
                    )
                    .unwrap());
                }
                let start_idx = idx * (SHUFFLE_BATCH_SIZE as u64);
                let end_idx = (idx + 1) * (SHUFFLE_BATCH_SIZE as u64);
                let row_ids = Arc::new(UInt64Array::from_iter(start_idx..end_idx));

                let part_id = Arc::new(UInt32Array::from_iter(
                    (start_idx..end_idx).map(|_| idx as u32),
                ));

                let values = Arc::new(UInt8Array::from_iter(
                    (0..32 * SHUFFLE_BATCH_SIZE).map(|_| idx as u8),
                ));
                let pq_codes = Arc::new(
                    FixedSizeListArray::try_new_from_values(values as Arc<dyn Array>, 32).unwrap(),
                );

                Ok(
                    RecordBatch::try_new(schema2.clone(), vec![row_ids, part_id, pq_codes])
                        .unwrap(),
                )
            });

        let stream = RecordBatchStreamAdapter::new(schema, stream);

        let shuffler = IvfShuffler::try_new(100, None, true, None).unwrap();

        (stream, shuffler)
    }

    fn check_batch(batch: RecordBatch, idx: usize, num_rows: usize) {
        let row_ids = batch
            .column_by_name(ROW_ID)
            .unwrap()
            .as_primitive::<UInt64Type>();
        let part_ids = batch
            .column_by_name(PART_ID_COLUMN)
            .unwrap()
            .as_primitive::<UInt32Type>();
        let pq_codes = batch
            .column_by_name(PQ_CODE_COLUMN)
            .unwrap()
            .as_fixed_size_list()
            .values()
            .as_primitive::<UInt8Type>();

        assert_eq!(row_ids.len(), num_rows);
        assert_eq!(part_ids.len(), num_rows);
        assert_eq!(pq_codes.len(), num_rows * 32);

        for i in 0..num_rows {
            assert_eq!(part_ids.value(i), idx as u32);
        }

        for v in pq_codes.values() {
            assert_eq!(*v, idx as u8);
        }
    }

    #[tokio::test]
    async fn test_shuffler_single_partition() {
        let (stream, mut shuffler) = make_stream_and_shuffler(false);

        shuffler.write_unsorted_stream(stream).await.unwrap();
        let partition_files = shuffler.write_partitioned_shuffles(100, 1).await.unwrap();

        assert_eq!(partition_files.len(), 1);

        let mut result_stream =
            IvfShuffler::load_partitioned_shuffles(&shuffler.output_dir, partition_files)
                .await
                .unwrap();

        let mut num_batches = 0;
        let mut stream = result_stream.pop().unwrap();

        while let Some(item) = stream.next().await {
            check_batch(item.unwrap(), num_batches, SHUFFLE_BATCH_SIZE);
            num_batches += 1;
        }

        assert_eq!(num_batches, 100);
    }

    #[tokio::test]
    async fn test_shuffler_single_partition_with_empty_batch() {
        let (stream, mut shuffler) = make_stream_and_shuffler(true);

        shuffler.write_unsorted_stream(stream).await.unwrap();
        let partition_files = shuffler.write_partitioned_shuffles(101, 1).await.unwrap();

        assert_eq!(partition_files.len(), 1);

        let mut result_stream =
            IvfShuffler::load_partitioned_shuffles(&shuffler.output_dir, partition_files)
                .await
                .unwrap();

        let mut num_batches = 0;
        let mut stream = result_stream.pop().unwrap();

        while let Some(item) = stream.next().await {
            check_batch(item.unwrap(), num_batches, SHUFFLE_BATCH_SIZE);
            num_batches += 1;
        }

        assert_eq!(num_batches, 100);
    }

    #[tokio::test]
    async fn test_shuffler_multiple_partition() {
        let (stream, mut shuffler) = make_stream_and_shuffler(false);

        shuffler.write_unsorted_stream(stream).await.unwrap();
        let partition_files = shuffler.write_partitioned_shuffles(1, 100).await.unwrap();

        assert_eq!(partition_files.len(), 100);

        let mut result_stream =
            IvfShuffler::load_partitioned_shuffles(&shuffler.output_dir, partition_files)
                .await
                .unwrap();

        let mut num_batches = 0;
        result_stream.reverse();

        while let Some(mut stream) = result_stream.pop() {
            while let Some(item) = stream.next().await {
                check_batch(item.unwrap(), num_batches, SHUFFLE_BATCH_SIZE);
                num_batches += 1
            }
        }

        assert_eq!(num_batches, 100);
    }

    #[tokio::test]
    async fn test_shuffler_multi_buffer_single_partition() {
        let (stream, mut shuffler) = make_stream_and_shuffler(false);
        shuffler.write_unsorted_stream(stream).await.unwrap();

        // set the same buffer twice we should get double the data
        unsafe { shuffler.set_unsorted_buffers(&[UNSORTED_BUFFER, UNSORTED_BUFFER]) }

        let partition_files = shuffler.write_partitioned_shuffles(200, 1).await.unwrap();

        assert_eq!(partition_files.len(), 1);

        let mut result_stream =
            IvfShuffler::load_partitioned_shuffles(&shuffler.output_dir, partition_files)
                .await
                .unwrap();

        let mut num_batches = 0;
        result_stream.reverse();

        while let Some(mut stream) = result_stream.pop() {
            while let Some(item) = stream.next().await {
                check_batch(item.unwrap(), num_batches, 2048);
                num_batches += 1
            }
        }

        assert_eq!(num_batches, 100);
    }

    #[tokio::test]
    async fn test_shuffler_multi_buffer_multi_partition() {
        let (stream, mut shuffler) = make_stream_and_shuffler(false);
        shuffler.write_unsorted_stream(stream).await.unwrap();

        // set the same buffer twice we should get double the data
        unsafe { shuffler.set_unsorted_buffers(&[UNSORTED_BUFFER, UNSORTED_BUFFER]) }

        let partition_files = shuffler.write_partitioned_shuffles(1, 32).await.unwrap();
        assert_eq!(partition_files.len(), 200);

        let mut result_stream =
            IvfShuffler::load_partitioned_shuffles(&shuffler.output_dir, partition_files)
                .await
                .unwrap();

        let mut num_batches = 0;
        result_stream.reverse();

        while let Some(mut stream) = result_stream.pop() {
            while let Some(item) = stream.next().await {
                check_batch(item.unwrap(), num_batches % 100, SHUFFLE_BATCH_SIZE);
                num_batches += 1
            }
        }

        assert_eq!(num_batches, 200);
    }

    fn make_big_stream_and_shuffler(
        num_batches: u32,
        num_partitions: u32,
        pq_dim: u32,
    ) -> (impl RecordBatchStream, IvfShuffler) {
        let schema = make_schema(pq_dim);

        let schema2 = schema.clone();

        let stream = stream::iter(0..num_batches).map(move |idx| {
            let mut rng = rand::thread_rng();
            let row_ids = Arc::new(UInt64Array::from_iter(
                (idx * 1024..(idx + 1) * 1024).map(u64::from),
            ));

            let part_id = Arc::new(UInt32Array::from_iter(
                (idx * 1024..(idx + 1) * 1024).map(|_| rng.next_u32() % num_partitions),
            ));

            let values = Arc::new(UInt8Array::from_iter((0..pq_dim * 1024).map(|_| idx as u8)));
            let pq_codes = Arc::new(
                FixedSizeListArray::try_new_from_values(values as Arc<dyn Array>, pq_dim as i32)
                    .unwrap(),
            );

            Ok(RecordBatch::try_new(schema2.clone(), vec![row_ids, part_id, pq_codes]).unwrap())
        });

        let stream = RecordBatchStreamAdapter::new(schema, stream);

        let shuffler = IvfShuffler::try_new(num_partitions, None, true, None).unwrap();

        (stream, shuffler)
    }

    // Change NUM_BATCHES = 1000 * 1024 and NUM_PARTITIONS to 35000 to test 1B shuffle
    const NUM_BATCHES: u32 = 100;
    const NUM_PARTITIONS: u32 = 1000;
    const PQ_DIM: u32 = 48;
    const BATCHES_PER_PARTITION: u32 = 10200;
    const NUM_CONCURRENT_JOBS: u32 = 16;

    #[test_log::test(tokio::test(flavor = "multi_thread"))]
    async fn test_big_shuffle() {
        let (stream, mut shuffler) =
            make_big_stream_and_shuffler(NUM_BATCHES, NUM_PARTITIONS, PQ_DIM);

        shuffler.write_unsorted_stream(stream).await.unwrap();
        let partition_files = shuffler
            .write_partitioned_shuffles(
                BATCHES_PER_PARTITION as usize,
                NUM_CONCURRENT_JOBS as usize,
            )
            .await
            .unwrap();

        let expected_num_part_files = NUM_BATCHES.div_ceil(BATCHES_PER_PARTITION);

        assert_eq!(partition_files.len(), expected_num_part_files as usize);

        let mut result_stream =
            IvfShuffler::load_partitioned_shuffles(&shuffler.output_dir, partition_files)
                .await
                .unwrap();

        let mut num_batches = 0;
        result_stream.reverse();

        while let Some(mut stream) = result_stream.pop() {
            while (stream.next().await).is_some() {
                num_batches += 1
            }
        }

        assert_eq!(num_batches, NUM_PARTITIONS * expected_num_part_files);
    }
}