-
Notifications
You must be signed in to change notification settings - Fork 194
/
reader.rs
1390 lines (1255 loc) · 51 KB
/
reader.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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors
use std::{collections::BTreeSet, io::Cursor, ops::Range, pin::Pin, sync::Arc};
use arrow_schema::Schema as ArrowSchema;
use byteorder::{ByteOrder, LittleEndian, ReadBytesExt};
use bytes::{Bytes, BytesMut};
use futures::{stream::BoxStream, Stream, StreamExt};
use lance_encoding::{
decoder::{
BatchDecodeStream, ColumnInfo, DecodeBatchScheduler, DecoderMiddlewareChain,
FilterExpression, PageInfo, ReadBatchTask,
},
encoder::EncodedBatch,
EncodingsIo,
};
use log::debug;
use prost::{Message, Name};
use snafu::{location, Location};
use lance_core::{
datatypes::{Field, Schema},
Error, Result,
};
use lance_encoding::format::pb as pbenc;
use lance_io::{
scheduler::FileScheduler,
stream::{RecordBatchStream, RecordBatchStreamAdapter},
ReadBatchParams,
};
use tokio::sync::mpsc;
use crate::{
datatypes::{Fields, FieldsWithMeta},
format::{pb, pbfile, MAGIC, MAJOR_VERSION, MINOR_VERSION_NEXT},
};
use super::io::LanceEncodingsIo;
// For now, we don't use global buffers for anything other than schema. If we
// use these later we should make them lazily loaded and then cached once loaded.
//
// We store their position / length for debugging purposes
#[derive(Debug)]
pub struct BufferDescriptor {
pub position: u64,
pub size: u64,
}
// TODO: Caching
#[derive(Debug)]
pub struct CachedFileMetadata {
/// The schema of the file
pub file_schema: Arc<Schema>,
/// The column metadatas
pub column_metadatas: Vec<pbfile::ColumnMetadata>,
pub column_infos: Vec<Arc<ColumnInfo>>,
/// The number of rows in the file
pub num_rows: u64,
pub file_buffers: Vec<BufferDescriptor>,
/// The number of bytes contained in the data page section of the file
pub num_data_bytes: u64,
/// The number of bytes contained in the column metadata (not including buffers
/// referenced by the metadata)
pub num_column_metadata_bytes: u64,
/// The number of bytes contained in global buffers
pub num_global_buffer_bytes: u64,
/// The number of bytes contained in the CMO and GBO tables
pub num_footer_bytes: u64,
pub major_version: u16,
pub minor_version: u16,
}
/// Selecting columns from a lance file requires specifying both the
/// index of the column and the data type of the column
///
/// Partly, this is because it is not strictly required that columns
/// be read into the same type. For example, a string column may be
/// read as a string, large_string or string_view type.
///
/// A read will only succeed if the decoder for a column is capable
/// of decoding into the requested type.
///
/// Note that this should generally be limited to different in-memory
/// representations of the same semantic type. An encoding could
/// theoretically support "casting" (e.g. int to string, etc.) but
/// there is little advantage in doing so here.
#[derive(Debug, Clone)]
pub struct ReaderProjection {
/// The data types (schema) of the selected columns. The names
/// of the schema are arbitrary and ignored.
pub schema: Arc<Schema>,
/// The indices of the columns to load. Note, these are the
/// indices of the top level fields only
pub column_indices: Vec<u32>,
}
#[derive(Debug)]
pub struct FileReader {
scheduler: Arc<LanceEncodingsIo>,
// The default projection to be applied to all reads
base_projection: ReaderProjection,
num_rows: u64,
metadata: Arc<CachedFileMetadata>,
decoder_strategy: DecoderMiddlewareChain,
}
#[derive(Debug)]
struct Footer {
#[allow(dead_code)]
column_meta_start: u64,
// We don't use this today because we always load metadata for every column
// and don't yet support "metadata projection"
#[allow(dead_code)]
column_meta_offsets_start: u64,
global_buff_offsets_start: u64,
num_global_buffers: u32,
num_columns: u32,
major_version: u16,
minor_version: u16,
}
const FOOTER_LEN: usize = 40;
impl FileReader {
pub fn metadata(&self) -> &Arc<CachedFileMetadata> {
&self.metadata
}
pub async fn read_global_buffer(&self, index: u32) -> Result<Bytes> {
let buffer_desc = self.metadata.file_buffers.get(index as usize).ok_or_else(||Error::invalid_input(format!("request for global buffer at index {} but there were only {} global buffers in the file", index, self.metadata.file_buffers.len()), location!()))?;
self.scheduler
.submit_single(
buffer_desc.position..buffer_desc.position + buffer_desc.size,
0,
)
.await
}
async fn read_tail(scheduler: &FileScheduler) -> Result<(Bytes, u64)> {
let file_size = scheduler.reader().size().await? as u64;
let begin = if file_size < scheduler.reader().block_size() as u64 {
0
} else {
file_size - scheduler.reader().block_size() as u64
};
let tail_bytes = scheduler.submit_single(begin..file_size, 0).await?;
Ok((tail_bytes, file_size))
}
// Checks to make sure the footer is written correctly and returns the
// position of the file descriptor (which comes from the footer)
fn decode_footer(footer_bytes: &Bytes) -> Result<Footer> {
let len = footer_bytes.len();
if len < FOOTER_LEN {
return Err(Error::io(
format!(
"does not have sufficient data, len: {}, bytes: {:?}",
len, footer_bytes
),
location!(),
));
}
let mut cursor = Cursor::new(footer_bytes.slice(len - FOOTER_LEN..));
let column_meta_start = cursor.read_u64::<LittleEndian>()?;
let column_meta_offsets_start = cursor.read_u64::<LittleEndian>()?;
let global_buff_offsets_start = cursor.read_u64::<LittleEndian>()?;
let num_global_buffers = cursor.read_u32::<LittleEndian>()?;
let num_columns = cursor.read_u32::<LittleEndian>()?;
let major_version = cursor.read_u16::<LittleEndian>()?;
let minor_version = cursor.read_u16::<LittleEndian>()?;
if major_version != MAJOR_VERSION as u16 || minor_version != MINOR_VERSION_NEXT {
return Err(Error::io(
format!(
"Attempt to use the lance v0.2 reader to read a file with version {}.{}",
major_version, minor_version
),
location!(),
));
}
let magic_bytes = footer_bytes.slice(len - 4..);
if magic_bytes.as_ref() != MAGIC {
return Err(Error::io(
format!(
"file does not appear to be a Lance file (invalid magic: {:?})",
MAGIC
),
location!(),
));
}
Ok(Footer {
column_meta_start,
column_meta_offsets_start,
global_buff_offsets_start,
num_global_buffers,
num_columns,
major_version,
minor_version,
})
}
// TODO: Once we have coalesced I/O we should only read the column metadatas that we need
fn read_all_column_metadata(
column_metadata_bytes: Bytes,
footer: &Footer,
) -> Result<Vec<pbfile::ColumnMetadata>> {
let column_metadata_start = footer.column_meta_start;
// cmo == column_metadata_offsets
let cmo_table_size = 16 * footer.num_columns as usize;
let cmo_table = column_metadata_bytes.slice(column_metadata_bytes.len() - cmo_table_size..);
(0..footer.num_columns)
.map(|col_idx| {
let offset = (col_idx * 16) as usize;
let position = LittleEndian::read_u64(&cmo_table[offset..offset + 8]);
let length = LittleEndian::read_u64(&cmo_table[offset + 8..offset + 16]);
let normalized_position = (position - column_metadata_start) as usize;
let normalized_end = normalized_position + (length as usize);
Ok(pbfile::ColumnMetadata::decode(
&column_metadata_bytes[normalized_position..normalized_end],
)?)
})
.collect::<Result<Vec<_>>>()
}
async fn optimistic_tail_read(
data: &Bytes,
start_pos: u64,
scheduler: &FileScheduler,
file_len: u64,
) -> Result<Bytes> {
let num_bytes_needed = (file_len - start_pos) as usize;
if data.len() >= num_bytes_needed {
Ok(data.slice((data.len() - num_bytes_needed)..))
} else {
let num_bytes_missing = (num_bytes_needed - data.len()) as u64;
let start = file_len - num_bytes_needed as u64;
let missing_bytes = scheduler
.submit_single(start..start + num_bytes_missing, 0)
.await?;
let mut combined = BytesMut::with_capacity(data.len() + num_bytes_missing as usize);
combined.extend(missing_bytes);
combined.extend(data);
Ok(combined.freeze())
}
}
fn do_decode_gbo_table(gbo_bytes: &Bytes, footer: &Footer) -> Result<Vec<BufferDescriptor>> {
let mut global_bufs_cursor = Cursor::new(gbo_bytes);
let mut global_buffers = Vec::with_capacity(footer.num_global_buffers as usize);
for _ in 0..footer.num_global_buffers {
let buf_pos = global_bufs_cursor.read_u64::<LittleEndian>()?;
let buf_size = global_bufs_cursor.read_u64::<LittleEndian>()?;
global_buffers.push(BufferDescriptor {
position: buf_pos,
size: buf_size,
});
}
Ok(global_buffers)
}
async fn decode_gbo_table(
tail_bytes: &Bytes,
file_len: u64,
scheduler: &FileScheduler,
footer: &Footer,
) -> Result<Vec<BufferDescriptor>> {
// This could, in theory, trigger another IOP but the GBO table should never be large
// enough for that to happen
let gbo_bytes = Self::optimistic_tail_read(
tail_bytes,
footer.global_buff_offsets_start,
scheduler,
file_len,
)
.await?;
Self::do_decode_gbo_table(&gbo_bytes, footer)
}
fn decode_schema(schema_bytes: Bytes) -> Result<(u64, lance_core::datatypes::Schema)> {
let file_descriptor = pb::FileDescriptor::decode(schema_bytes)?;
let pb_schema = file_descriptor.schema.unwrap();
let num_rows = file_descriptor.length;
let fields_with_meta = FieldsWithMeta {
fields: Fields(pb_schema.fields),
metadata: pb_schema.metadata,
};
let schema = lance_core::datatypes::Schema::from(fields_with_meta);
Ok((num_rows, schema))
}
// TODO: Support late projection. Currently, if we want to perform a
// projected read of a file, we load all of the column metadata, and then
// only read the column data that is requested. This is fine for most cases.
//
// However, if there are many columns then loading all of the column metadata
// may be expensive. We should support a mode where we only load the column
// metadata for the columns that are requested (the file format supports this).
//
// The main challenge is that we either need to ignore the column metadata cache
// or have a more sophisticated cache that can cache per-column metadata.
//
// Also, if the number of columns is fairly small, it's faster to read them as a
// single IOP, but we can fix this through coalescing.
async fn read_all_metadata(scheduler: &FileScheduler) -> Result<CachedFileMetadata> {
// 1. read the footer
let (tail_bytes, file_len) = Self::read_tail(scheduler).await?;
let footer = Self::decode_footer(&tail_bytes)?;
let gbo_table = Self::decode_gbo_table(&tail_bytes, file_len, scheduler, &footer).await?;
if gbo_table.is_empty() {
return Err(Error::Internal {
message: "File did not contain any global buffers, schema expected".to_string(),
location: location!(),
});
}
let schema_start = gbo_table[0].position;
let schema_size = gbo_table[0].size;
let num_footer_bytes = file_len - schema_start;
// By default we read all column metadatas. We do NOT read the column metadata buffers
// at this point. We only want to read the column metadata for columns we are actually loading.
let all_metadata_bytes =
Self::optimistic_tail_read(&tail_bytes, schema_start, scheduler, file_len).await?;
let schema_bytes = all_metadata_bytes.slice(0..schema_size as usize);
let (num_rows, schema) = Self::decode_schema(schema_bytes)?;
// Next, read the metadata for the columns
// This is both the column metadata and the CMO table
let column_metadata_start = (footer.column_meta_start - schema_start) as usize;
let column_metadata_end = (footer.global_buff_offsets_start - schema_start) as usize;
let column_metadata_bytes =
all_metadata_bytes.slice(column_metadata_start..column_metadata_end);
let column_metadatas = Self::read_all_column_metadata(column_metadata_bytes, &footer)?;
let footer_start = file_len - FOOTER_LEN as u64;
let num_data_bytes = footer.column_meta_start;
let num_global_buffer_bytes = gbo_table.iter().map(|buf| buf.size).sum::<u64>()
+ (footer_start - footer.global_buff_offsets_start);
let num_column_metadata_bytes = footer.global_buff_offsets_start - footer.column_meta_start;
let column_infos = Self::meta_to_col_infos(column_metadatas.as_slice());
Ok(CachedFileMetadata {
file_schema: Arc::new(schema),
column_metadatas,
column_infos,
num_rows,
num_data_bytes,
num_column_metadata_bytes,
num_global_buffer_bytes,
num_footer_bytes,
file_buffers: gbo_table,
major_version: footer.major_version,
minor_version: footer.minor_version,
})
}
fn fetch_encoding<M: Default + Name + Sized>(encoding: &pbfile::Encoding) -> M {
match &encoding.location {
Some(pbfile::encoding::Location::Indirect(_)) => todo!(),
Some(pbfile::encoding::Location::Direct(encoding)) => {
let encoding_buf = Bytes::from(encoding.encoding.clone());
let encoding_any = prost_types::Any::decode(encoding_buf).unwrap();
encoding_any.to_msg::<M>().unwrap()
}
Some(pbfile::encoding::Location::None(_)) => panic!(),
None => panic!(),
}
}
fn meta_to_col_infos(column_metadatas: &[pbfile::ColumnMetadata]) -> Vec<Arc<ColumnInfo>> {
column_metadatas
.iter()
.enumerate()
.map(|(col_idx, col_meta)| {
let page_infos = col_meta
.pages
.iter()
.map(|page| {
let num_rows = page.length;
let encoding = Self::fetch_encoding(page.encoding.as_ref().unwrap());
let buffer_offsets_and_sizes = Arc::from(
page.buffer_offsets
.iter()
.zip(page.buffer_sizes.iter())
.map(|(offset, size)| (*offset, *size))
.collect::<Vec<_>>(),
);
PageInfo {
buffer_offsets_and_sizes,
encoding,
num_rows,
}
})
.collect::<Vec<_>>();
let buffer_offsets_and_sizes = Arc::from(
col_meta
.buffer_offsets
.iter()
.zip(col_meta.buffer_sizes.iter())
.map(|(offset, size)| (*offset, *size))
.collect::<Vec<_>>(),
);
Arc::new(ColumnInfo {
index: col_idx as u32,
page_infos: Arc::from(page_infos),
buffer_offsets_and_sizes,
encoding: Self::fetch_encoding(col_meta.encoding.as_ref().unwrap()),
})
})
.collect::<Vec<_>>()
}
fn validate_projection(
projection: &ReaderProjection,
metadata: &CachedFileMetadata,
) -> Result<()> {
if projection.schema.fields.is_empty() {
return Err(Error::invalid_input(
"Attempt to read zero columns from the file, at least one column must be specified"
.to_string(),
location!(),
));
}
if projection.schema.fields.len() != projection.column_indices.len() {
return Err(Error::invalid_input(format!("The projection schema has {} top level fields but only {} column indices were provided", projection.schema.fields.len(), projection.column_indices.len()), location!()));
}
let mut column_indices_seen = BTreeSet::new();
for column_index in &projection.column_indices {
if !column_indices_seen.insert(*column_index) {
return Err(Error::invalid_input(
format!(
"The projection specified the column index {} more than once",
column_index
),
location!(),
));
}
if *column_index >= metadata.column_infos.len() as u32 {
return Err(Error::invalid_input(format!("The projection specified the column index {} but there are only {} columns in the file", column_index, metadata.column_infos.len()), location!()));
}
}
Ok(())
}
// Helper function for `default_projection` to determine how many columns are occupied
// by a lance field.
fn default_column_count(field: &Field) -> u32 {
1 + field
.children
.iter()
.map(Self::default_column_count)
.sum::<u32>()
}
// This function is one of the few spots in the reader where we rely on Lance table
// format and the fact that we wrote a Lance table schema into the global buffers.
//
// TODO: In the future it would probably be better for the "default type" of a column
// to be something that can be provided dynamically via the encodings registry. We
// could pass the pages of the column to some logic that picks a data type based on the
// page encodings.
/// Loads a default projection for all columns in the file, using the data type that
/// was provided when the file was written.
fn default_projection(lance_schema: &Schema) -> ReaderProjection {
let schema = Arc::new(lance_schema.clone());
let mut column_indices = Vec::with_capacity(lance_schema.fields.len());
let mut column_index = 0;
for field in &lance_schema.fields {
column_indices.push(column_index);
column_index += Self::default_column_count(field);
}
ReaderProjection {
schema,
column_indices,
}
}
/// Opens a new file reader without any pre-existing knowledge
///
/// This will read the file schema from the file itself and thus requires a bit more I/O
///
/// A `base_projection` can also be provided. If provided, then the projection will apply
/// to all reads from the file that do not specify their own projection.
pub async fn try_open(
scheduler: FileScheduler,
base_projection: Option<ReaderProjection>,
decoder_strategy: DecoderMiddlewareChain,
) -> Result<Self> {
let file_metadata = Arc::new(Self::read_all_metadata(&scheduler).await?);
if let Some(base_projection) = base_projection.as_ref() {
Self::validate_projection(base_projection, &file_metadata)?;
}
let num_rows = file_metadata.num_rows;
Ok(Self {
scheduler: Arc::new(LanceEncodingsIo(scheduler)),
base_projection: base_projection
.unwrap_or(Self::default_projection(file_metadata.file_schema.as_ref())),
num_rows,
metadata: file_metadata,
decoder_strategy,
})
}
fn collect_columns(
&self,
field: &Field,
column_idx: &mut usize,
column_infos: &mut Vec<Arc<ColumnInfo>>,
) -> Result<()> {
column_infos.push(self.metadata.column_infos[*column_idx].clone());
*column_idx += 1;
for child in &field.children {
self.collect_columns(child, column_idx, column_infos)?;
}
Ok(())
}
// The actual decoder needs all the column infos that make up a type. In other words, if
// the first type in the schema is Struct<i32, i32> then the decoder will need 3 column infos.
//
// This is a file reader concern because the file reader needs to support late projection of columns
// and so it will need to figure this out anyways.
//
// It's a bit of a tricky process though because the number of column infos may depend on the
// encoding. Considering the above example, if we wrote it with a packed encoding, then there would
// only be a single column in the file (and not 3).
//
// At the moment this method words because our rules are simple and we just repeat them here. See
// Self::default_projection for a similar problem. In the future this is something the encodings
// registry will need to figure out.
fn collect_columns_from_projection(
&self,
projection: &ReaderProjection,
) -> Result<Vec<Arc<ColumnInfo>>> {
let mut column_infos = Vec::with_capacity(projection.column_indices.len());
for (field, starting_column) in projection
.schema
.fields
.iter()
.zip(projection.column_indices.iter())
{
let mut starting_column = *starting_column as usize;
self.collect_columns(field, &mut starting_column, &mut column_infos)?;
}
Ok(column_infos)
}
#[allow(clippy::too_many_arguments)]
fn do_read_range(
column_infos: Vec<Arc<ColumnInfo>>,
scheduler: Arc<dyn EncodingsIo>,
num_rows: u64,
decoder_strategy: DecoderMiddlewareChain,
range: Range<u64>,
batch_size: u32,
projection: &ReaderProjection,
filter: FilterExpression,
) -> Result<BoxStream<'static, ReadBatchTask>> {
debug!(
"Reading range {:?} with batch_size {} from columns {:?}",
range,
batch_size,
column_infos.iter().map(|ci| ci.index).collect::<Vec<_>>()
);
let mut decode_scheduler = DecodeBatchScheduler::try_new(
&projection.schema,
&column_infos,
&vec![],
num_rows,
&decoder_strategy,
&scheduler,
)?;
let root_decoder = decode_scheduler.new_root_decoder_ranges(&[range.clone()]);
let (tx, rx) = mpsc::unbounded_channel();
let num_rows_to_read = range.end - range.start;
tokio::task::spawn(async move {
decode_scheduler.schedule_range(range, &filter, tx, scheduler)
});
Ok(BatchDecodeStream::new(rx, batch_size, num_rows_to_read, root_decoder).into_stream())
}
fn read_range(
&self,
range: Range<u64>,
batch_size: u32,
projection: &ReaderProjection,
filter: FilterExpression,
) -> Result<BoxStream<'static, ReadBatchTask>> {
// Grab what we need to initialize the stream
let range = range.clone();
let projection = projection.clone();
let column_infos = self.collect_columns_from_projection(&projection)?;
let scheduler = self.scheduler.clone();
let num_rows = self.num_rows;
let decoder_strategy = self.decoder_strategy.clone();
// Create and initialize the stream
Self::do_read_range(
column_infos,
scheduler,
num_rows,
decoder_strategy,
range,
batch_size,
&projection,
filter,
)
}
fn do_take_rows(
column_infos: Vec<Arc<ColumnInfo>>,
scheduler: Arc<dyn EncodingsIo>,
num_rows: u64,
decoder_strategy: DecoderMiddlewareChain,
indices: Vec<u64>,
batch_size: u32,
projection: &ReaderProjection,
) -> Result<BoxStream<'static, ReadBatchTask>> {
debug!(
"Taking {} rows spread across range {}..{} with batch_size {} from columns {:?}",
indices.len(),
indices[0],
indices[indices.len() - 1],
batch_size,
column_infos.iter().map(|ci| ci.index).collect::<Vec<_>>()
);
let mut decode_scheduler = DecodeBatchScheduler::try_new(
&projection.schema,
&column_infos,
&vec![],
num_rows,
&decoder_strategy,
&scheduler,
)?;
let root_decoder = decode_scheduler.new_root_decoder_indices(&indices);
let (tx, rx) = mpsc::unbounded_channel();
let num_rows_to_read = indices.len() as u64;
tokio::task::spawn(async move {
decode_scheduler.schedule_take(&indices, &FilterExpression::no_filter(), tx, scheduler)
});
Ok(BatchDecodeStream::new(rx, batch_size, num_rows_to_read, root_decoder).into_stream())
}
fn take_rows(
&self,
indices: Vec<u64>,
batch_size: u32,
projection: &ReaderProjection,
) -> Result<BoxStream<'static, ReadBatchTask>> {
// Grab what we need to initialize the stream
let projection = projection.clone();
let column_infos = self.collect_columns_from_projection(&projection)?;
let scheduler = self.scheduler.clone();
let num_rows = self.num_rows;
let decoder_strategy = self.decoder_strategy.clone();
// Create and initialize the stream
Self::do_take_rows(
column_infos,
scheduler,
num_rows,
decoder_strategy,
indices,
batch_size,
&projection,
)
}
/// Creates a stream of "read tasks" to read the data from the file
///
/// The arguments are similar to [`Self::read_stream_projected`] but instead of returning a stream
/// of record batches it returns a stream of "read tasks".
///
/// The tasks should be consumed with some kind of `buffered` argument if CPU parallelism is desired.
///
/// Note that "read task" is probably a bit imprecise. The tasks are actually "decode tasks". The
/// reading happens asynchronously in the background. In other words, a single read task may map to
/// multiple I/O operations or a single I/O operation may map to multiple read tasks.
pub fn read_tasks(
&self,
params: ReadBatchParams,
batch_size: u32,
projection: &ReaderProjection,
filter: FilterExpression,
) -> Result<Pin<Box<dyn Stream<Item = ReadBatchTask> + Send>>> {
Self::validate_projection(projection, &self.metadata)?;
let verify_bound = |params: &ReadBatchParams, bound: u64, inclusive: bool| {
if bound > self.num_rows || bound == self.num_rows && inclusive {
Err(Error::invalid_input(
format!(
"cannot read {:?} from file with {} rows",
params, self.num_rows
),
location!(),
))
} else {
Ok(())
}
};
match ¶ms {
ReadBatchParams::Indices(indices) => {
for idx in indices {
match idx {
None => {
return Err(Error::invalid_input(
"Null value in indices array",
location!(),
));
}
Some(idx) => {
verify_bound(¶ms, idx as u64, true)?;
}
}
}
let indices = indices.iter().map(|idx| idx.unwrap() as u64).collect();
self.take_rows(indices, batch_size, projection)
}
ReadBatchParams::Range(range) => {
verify_bound(¶ms, range.end as u64, false)?;
self.read_range(
range.start as u64..range.end as u64,
batch_size,
projection,
filter,
)
}
ReadBatchParams::RangeFrom(range) => {
verify_bound(¶ms, range.start as u64, true)?;
self.read_range(
range.start as u64..self.num_rows,
batch_size,
projection,
filter,
)
}
ReadBatchParams::RangeTo(range) => {
verify_bound(¶ms, range.end as u64, false)?;
self.read_range(0..range.end as u64, batch_size, projection, filter)
}
ReadBatchParams::RangeFull => {
self.read_range(0..self.num_rows, batch_size, projection, filter)
}
}
}
/// Reads data from the file as a stream of record batches
///
/// * `params` - Specifies the range (or indices) of data to read
/// * `batch_size` - The maximum size of a single batch. A batch may be smaller
/// if it is the last batch or if it is not possible to create a batch of the
/// requested size.
///
/// For example, if the batch size is 1024 and one of the columns is a string
/// column then there may be some ranges of 1024 rows that contain more than
/// 2^31 bytes of string data (which is the maximum size of a string column
/// in Arrow). In this case smaller batches may be emitted.
/// * `batch_readahead` - The number of batches to read ahead. This controls the
/// amount of CPU parallelism of the read. In other words it controlls how many
/// batches will be decoded in parallel. It has no effect on the I/O parallelism
/// of the read (how many I/O requests are in flight at once).
///
/// This parameter also is also related to backpressure. If the consumer of the
/// stream is slow then the reader will build up RAM.
/// * `projection` - A projection to apply to the read. This controls which columns
/// are read from the file. The projection is NOT applied on top of the base
/// projection. The projection is applied directly to the file schema.
pub fn read_stream_projected(
&self,
params: ReadBatchParams,
batch_size: u32,
batch_readahead: u32,
projection: &ReaderProjection,
filter: FilterExpression,
) -> Result<Pin<Box<dyn RecordBatchStream>>> {
let tasks_stream = self.read_tasks(params, batch_size, projection, filter)?;
let batch_stream = tasks_stream
.map(|task| task.task)
.buffered(batch_readahead as usize)
.boxed();
let arrow_schema = Arc::new(ArrowSchema::from(projection.schema.as_ref()));
Ok(Box::pin(RecordBatchStreamAdapter::new(
arrow_schema,
batch_stream,
)))
}
/// Reads data from the file as a stream of record batches
///
/// This is similar to [`Self::read_stream_projected`] but uses the base projection
/// provided when the file was opened (or reads all columns if the file was
/// opened without a base projection)
pub fn read_stream(
&self,
params: ReadBatchParams,
batch_size: u32,
batch_readahead: u32,
filter: FilterExpression,
) -> Result<Pin<Box<dyn RecordBatchStream>>> {
self.read_stream_projected(
params,
batch_size,
batch_readahead,
&self.base_projection,
filter,
)
}
pub fn schema(&self) -> &Arc<Schema> {
&self.metadata.file_schema
}
}
/// Inspects a page and returns a String describing the page's encoding
pub fn describe_encoding(page: &pbfile::column_metadata::Page) -> String {
if let Some(encoding) = &page.encoding {
if let Some(style) = &encoding.location {
match style {
pbfile::encoding::Location::Indirect(indirect) => {
format!(
"IndirectEncoding(pos={},size={})",
indirect.buffer_location, indirect.buffer_length
)
}
pbfile::encoding::Location::Direct(direct) => {
let encoding_any =
prost_types::Any::decode(Bytes::from(direct.encoding.clone()))
.expect("failed to deserialize encoding as protobuf");
if encoding_any.type_url == "/lance.encodings.ArrayEncoding" {
let encoding = encoding_any.to_msg::<pbenc::ArrayEncoding>();
match encoding {
Ok(encoding) => {
format!("{:#?}", encoding)
}
Err(err) => {
format!("Unsupported(decode_err={})", err)
}
}
} else {
format!("Unrecognized(type_url={})", encoding_any.type_url)
}
}
pbfile::encoding::Location::None(_) => "NoEncodingDescription".to_string(),
}
} else {
"MISSING STYLE".to_string()
}
} else {
"MISSING".to_string()
}
}
pub trait EncodedBatchReaderExt {
fn try_from_mini_lance(bytes: Bytes, schema: &Schema) -> Result<Self>
where
Self: Sized;
fn try_from_self_described_lance(bytes: Bytes) -> Result<Self>
where
Self: Sized;
}
impl EncodedBatchReaderExt for EncodedBatch {
fn try_from_mini_lance(bytes: Bytes, schema: &Schema) -> Result<Self>
where
Self: Sized,
{
let footer = FileReader::decode_footer(&bytes)?;
// Next, read the metadata for the columns
// This is both the column metadata and the CMO table
let column_metadata_start = footer.column_meta_start as usize;
let column_metadata_end = footer.global_buff_offsets_start as usize;
let column_metadata_bytes = bytes.slice(column_metadata_start..column_metadata_end);
let column_metadatas =
FileReader::read_all_column_metadata(column_metadata_bytes, &footer)?;
let page_table = FileReader::meta_to_col_infos(&column_metadatas);
Ok(Self {
data: bytes,
num_rows: page_table
.first()
.map(|col| col.page_infos.iter().map(|page| page.num_rows).sum::<u64>())
.unwrap_or(0),
page_table,
schema: Arc::new(schema.clone()),
})
}
fn try_from_self_described_lance(bytes: Bytes) -> Result<Self>
where
Self: Sized,
{
let footer = FileReader::decode_footer(&bytes)?;
let gbo_table = FileReader::do_decode_gbo_table(
&bytes.slice(footer.global_buff_offsets_start as usize..),
&footer,
)?;
if gbo_table.is_empty() {
return Err(Error::Internal {
message: "File did not contain any global buffers, schema expected".to_string(),
location: location!(),
});
}
let schema_start = gbo_table[0].position as usize;
let schema_size = gbo_table[0].size as usize;
let schema_bytes = bytes.slice(schema_start..(schema_start + schema_size));
let (_, schema) = FileReader::decode_schema(schema_bytes)?;
// Next, read the metadata for the columns
// This is both the column metadata and the CMO table
let column_metadata_start = footer.column_meta_start as usize;
let column_metadata_end = footer.global_buff_offsets_start as usize;
let column_metadata_bytes = bytes.slice(column_metadata_start..column_metadata_end);
let column_metadatas =
FileReader::read_all_column_metadata(column_metadata_bytes, &footer)?;
let page_table = FileReader::meta_to_col_infos(&column_metadatas);
Ok(Self {
data: bytes,
num_rows: page_table
.first()
.map(|col| col.page_infos.iter().map(|page| page.num_rows).sum::<u64>())
.unwrap_or(0),
page_table,
schema: Arc::new(schema.clone()),
})
}
}
#[cfg(test)]
pub mod tests {
use std::{pin::Pin, sync::Arc};
use arrow_array::{
types::{Float64Type, Int32Type},
RecordBatch,
};
use arrow_schema::{DataType, Field, Fields, Schema as ArrowSchema};
use bytes::Bytes;
use futures::{prelude::stream::TryStreamExt, StreamExt};
use lance_arrow::RecordBatchExt;
use lance_core::datatypes::Schema;
use lance_datagen::{array, gen, BatchCount, ByteCount, RowCount};
use lance_encoding::{
decoder::{decode_batch, DecoderMiddlewareChain, FilterExpression},
encoder::{encode_batch, CoreFieldEncodingStrategy, EncodedBatch},
};
use lance_io::stream::RecordBatchStream;
use log::debug;
use crate::v2::{
reader::{EncodedBatchReaderExt, FileReader, ReaderProjection},
testing::{write_lance_file, FsFixture},
writer::{EncodedBatchWriteExt, FileWriter, FileWriterOptions},
};
async fn create_some_file(fs: &FsFixture) -> (Arc<Schema>, Vec<RecordBatch>) {
let location_type = DataType::Struct(Fields::from(vec![
Field::new("x", DataType::Float64, true),
Field::new("y", DataType::Float64, true),
]));
let categories_type = DataType::List(Arc::new(Field::new("item", DataType::Utf8, true)));
let reader = gen()
.col("score", array::rand::<Float64Type>())
.col("location", array::rand_type(&location_type))
.col("categories", array::rand_type(&categories_type))
.col("binary", array::rand_type(&DataType::Binary))
.col("large_bin", array::rand_type(&DataType::LargeBinary))
.into_reader_rows(RowCount::from(1000), BatchCount::from(100));
write_lance_file(reader, fs, FileWriterOptions::default()).await
}
type Transformer = Box<dyn Fn(&RecordBatch) -> RecordBatch>;