lance_file/v2/
reader.rs

1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright The Lance Authors
3
4use std::{
5    collections::{BTreeMap, BTreeSet},
6    io::Cursor,
7    ops::Range,
8    pin::Pin,
9    sync::Arc,
10};
11
12use arrow_array::RecordBatchReader;
13use arrow_schema::Schema as ArrowSchema;
14use byteorder::{ByteOrder, LittleEndian, ReadBytesExt};
15use bytes::{Bytes, BytesMut};
16use deepsize::{Context, DeepSizeOf};
17use futures::{stream::BoxStream, Stream, StreamExt};
18use lance_encoding::{
19    decoder::{
20        schedule_and_decode, schedule_and_decode_blocking, ColumnInfo, DecoderPlugins,
21        FilterExpression, PageEncoding, PageInfo, ReadBatchTask, RequestedRows,
22        SchedulerDecoderConfig,
23    },
24    encoder::EncodedBatch,
25    version::LanceFileVersion,
26    EncodingsIo,
27};
28use log::debug;
29use object_store::path::Path;
30use prost::{Message, Name};
31use snafu::location;
32
33use lance_core::{
34    cache::FileMetadataCache,
35    datatypes::{Field, Schema},
36    Error, Result,
37};
38use lance_encoding::format::pb as pbenc;
39use lance_io::{
40    scheduler::FileScheduler,
41    stream::{RecordBatchStream, RecordBatchStreamAdapter},
42    ReadBatchParams,
43};
44
45use crate::{
46    datatypes::{Fields, FieldsWithMeta},
47    format::{pb, pbfile, MAGIC, MAJOR_VERSION, MINOR_VERSION},
48    v2::writer::PAGE_BUFFER_ALIGNMENT,
49};
50
51use super::io::LanceEncodingsIo;
52
53// For now, we don't use global buffers for anything other than schema.  If we
54// use these later we should make them lazily loaded and then cached once loaded.
55//
56// We store their position / length for debugging purposes
57#[derive(Debug, DeepSizeOf)]
58pub struct BufferDescriptor {
59    pub position: u64,
60    pub size: u64,
61}
62
63/// Statistics summarize some of the file metadata for quick summary info
64#[derive(Debug)]
65pub struct FileStatistics {
66    /// Statistics about each of the columns in the file
67    pub columns: Vec<ColumnStatistics>,
68}
69
70/// Summary information describing a column
71#[derive(Debug)]
72pub struct ColumnStatistics {
73    /// The number of pages in the column
74    pub num_pages: usize,
75    /// The total number of data & metadata bytes in the column
76    ///
77    /// This is the compressed on-disk size
78    pub size_bytes: u64,
79}
80
81// TODO: Caching
82#[derive(Debug)]
83pub struct CachedFileMetadata {
84    /// The schema of the file
85    pub file_schema: Arc<Schema>,
86    /// The column metadatas
87    pub column_metadatas: Vec<pbfile::ColumnMetadata>,
88    pub column_infos: Vec<Arc<ColumnInfo>>,
89    /// The number of rows in the file
90    pub num_rows: u64,
91    pub file_buffers: Vec<BufferDescriptor>,
92    /// The number of bytes contained in the data page section of the file
93    pub num_data_bytes: u64,
94    /// The number of bytes contained in the column metadata (not including buffers
95    /// referenced by the metadata)
96    pub num_column_metadata_bytes: u64,
97    /// The number of bytes contained in global buffers
98    pub num_global_buffer_bytes: u64,
99    /// The number of bytes contained in the CMO and GBO tables
100    pub num_footer_bytes: u64,
101    pub major_version: u16,
102    pub minor_version: u16,
103}
104
105impl DeepSizeOf for CachedFileMetadata {
106    // TODO: include size for `column_metadatas` and `column_infos`.
107    fn deep_size_of_children(&self, context: &mut Context) -> usize {
108        self.file_schema.deep_size_of_children(context)
109            + self
110                .file_buffers
111                .iter()
112                .map(|file_buffer| file_buffer.deep_size_of_children(context))
113                .sum::<usize>()
114    }
115}
116
117impl CachedFileMetadata {
118    pub fn version(&self) -> LanceFileVersion {
119        match (self.major_version, self.minor_version) {
120            (0, 3) => LanceFileVersion::V2_0,
121            (2, 1) => LanceFileVersion::V2_1,
122            _ => panic!(
123                "Unsupported version: {}.{}",
124                self.major_version, self.minor_version
125            ),
126        }
127    }
128}
129
130/// Selecting columns from a lance file requires specifying both the
131/// index of the column and the data type of the column
132///
133/// Partly, this is because it is not strictly required that columns
134/// be read into the same type.  For example, a string column may be
135/// read as a string, large_string or string_view type.
136///
137/// A read will only succeed if the decoder for a column is capable
138/// of decoding into the requested type.
139///
140/// Note that this should generally be limited to different in-memory
141/// representations of the same semantic type.  An encoding could
142/// theoretically support "casting" (e.g. int to string,  etc.) but
143/// there is little advantage in doing so here.
144///
145/// Note: in order to specify a projection the user will need some way
146/// to figure out the column indices.  In the table format we do this
147/// using field IDs and keeping track of the field id->column index mapping.
148///
149/// If users are not using the table format then they will need to figure
150/// out some way to do this themselves.
151#[derive(Debug, Clone)]
152pub struct ReaderProjection {
153    /// The data types (schema) of the selected columns.  The names
154    /// of the schema are arbitrary and ignored.
155    pub schema: Arc<Schema>,
156    /// The indices of the columns to load.
157    ///
158    /// The mapping should be as follows:
159    ///
160    /// - Primitive: the index of the column in the schema
161    /// - List: the index of the list column in the schema
162    ///         followed by the column indices of the children
163    /// - FixedSizeList (of primitive): the index of the column in the schema
164    ///         (this case is not nested)
165    /// - FixedSizeList (of non-primitive): not yet implemented
166    /// - Dictionary: same as primitive
167    /// - Struct: the index of the struct column in the schema
168    ///          followed by the column indices of the children
169    ///
170    /// In other words, this should be a DFS listing of the desired schema.
171    ///
172    /// For example, if the goal is to load:
173    ///
174    ///   x: int32
175    ///   y: struct<z: int32, w: string>
176    ///   z: list<int32>
177    ///
178    /// and the schema originally used to store the data was:
179    ///
180    ///   a: struct<x: int32>
181    ///   b: int64
182    ///   y: struct<z: int32, c: int64, w: string>
183    ///   z: list<int32>
184    ///
185    /// Then the column_indices should be [1, 3, 4, 6, 7, 8]
186    pub column_indices: Vec<u32>,
187}
188
189impl ReaderProjection {
190    fn from_field_ids_helper<'a>(
191        reader: &FileReader,
192        fields: impl Iterator<Item = &'a Field>,
193        field_id_to_column_index: &BTreeMap<u32, u32>,
194        column_indices: &mut Vec<u32>,
195    ) -> Result<()> {
196        for field in fields {
197            let is_structural = reader.metadata.version() >= LanceFileVersion::V2_1;
198            // In the 2.0 system we needed ids for intermediate fields.  In 2.1+
199            // we only need ids for leaf fields.
200            if !is_structural || field.children.is_empty() {
201                if let Some(column_idx) = field_id_to_column_index.get(&(field.id as u32)).copied()
202                {
203                    column_indices.push(column_idx);
204                }
205            }
206            Self::from_field_ids_helper(
207                reader,
208                field.children.iter(),
209                field_id_to_column_index,
210                column_indices,
211            )?;
212        }
213        Ok(())
214    }
215
216    /// Creates a projection using a mapping from field IDs to column indices
217    ///
218    /// You can obtain such a mapping when the file is written using the
219    /// [`crate::v2::writer::FileWriter::field_id_to_column_indices`] method.
220    pub fn from_field_ids(
221        reader: &FileReader,
222        schema: &Schema,
223        field_id_to_column_index: &BTreeMap<u32, u32>,
224    ) -> Result<Self> {
225        let mut column_indices = Vec::new();
226        Self::from_field_ids_helper(
227            reader,
228            schema.fields.iter(),
229            field_id_to_column_index,
230            &mut column_indices,
231        )?;
232        Ok(Self {
233            schema: Arc::new(schema.clone()),
234            column_indices,
235        })
236    }
237
238    /// Creates a projection that reads the entire file
239    ///
240    /// If the schema provided is not the schema of the entire file then
241    /// the projection will be invalid and the read will fail.
242    /// If the field is a `struct datatype` with `packed` set to true in the field metadata,
243    /// the whole struct has one column index.
244    /// To support nested `packed-struct encoding`, this method need to be further adjusted.
245    pub fn from_whole_schema(schema: &Schema, version: LanceFileVersion) -> Self {
246        let schema = Arc::new(schema.clone());
247        let is_structural = version >= LanceFileVersion::V2_1;
248        let mut column_indices = vec![];
249        let mut curr_column_idx = 0;
250        let mut packed_struct_fields_num = 0;
251        for field in schema.fields_pre_order() {
252            if packed_struct_fields_num > 0 {
253                packed_struct_fields_num -= 1;
254                continue;
255            }
256            if field.is_packed_struct() {
257                column_indices.push(curr_column_idx);
258                curr_column_idx += 1;
259                packed_struct_fields_num = field.children.len();
260            } else if field.children.is_empty() || !is_structural {
261                column_indices.push(curr_column_idx);
262                curr_column_idx += 1;
263            }
264        }
265        Self {
266            schema,
267            column_indices,
268        }
269    }
270
271    /// Creates a projection that reads the specified columns provided by name
272    ///
273    /// The syntax for column names is the same as [`lance_core::datatypes::Schema::project`]
274    ///
275    /// If the schema provided is not the schema of the entire file then
276    /// the projection will be invalid and the read will fail.
277    pub fn from_column_names(schema: &Schema, column_names: &[&str]) -> Result<Self> {
278        let field_id_to_column_index = schema
279            .fields_pre_order()
280            .enumerate()
281            .map(|(idx, field)| (field.id as u32, idx as u32))
282            .collect::<BTreeMap<_, _>>();
283        let projected = schema.project(column_names)?;
284        let column_indices = projected
285            .fields_pre_order()
286            .map(|f| field_id_to_column_index[&(f.id as u32)])
287            .collect::<Vec<_>>();
288        Ok(Self {
289            schema: Arc::new(projected),
290            column_indices,
291        })
292    }
293}
294
295#[derive(Clone, Debug, Default)]
296pub struct FileReaderOptions {
297    validate_on_decode: bool,
298}
299
300#[derive(Debug)]
301pub struct FileReader {
302    scheduler: Arc<dyn EncodingsIo>,
303    // The default projection to be applied to all reads
304    base_projection: ReaderProjection,
305    num_rows: u64,
306    metadata: Arc<CachedFileMetadata>,
307    decoder_plugins: Arc<DecoderPlugins>,
308    cache: Arc<FileMetadataCache>,
309    options: FileReaderOptions,
310}
311#[derive(Debug)]
312struct Footer {
313    #[allow(dead_code)]
314    column_meta_start: u64,
315    // We don't use this today because we always load metadata for every column
316    // and don't yet support "metadata projection"
317    #[allow(dead_code)]
318    column_meta_offsets_start: u64,
319    global_buff_offsets_start: u64,
320    num_global_buffers: u32,
321    num_columns: u32,
322    major_version: u16,
323    minor_version: u16,
324}
325
326const FOOTER_LEN: usize = 40;
327
328impl FileReader {
329    pub fn with_scheduler(&self, scheduler: Arc<dyn EncodingsIo>) -> Self {
330        Self {
331            scheduler,
332            base_projection: self.base_projection.clone(),
333            cache: self.cache.clone(),
334            decoder_plugins: self.decoder_plugins.clone(),
335            metadata: self.metadata.clone(),
336            options: self.options.clone(),
337            num_rows: self.num_rows,
338        }
339    }
340
341    pub fn num_rows(&self) -> u64 {
342        self.num_rows
343    }
344
345    pub fn metadata(&self) -> &Arc<CachedFileMetadata> {
346        &self.metadata
347    }
348
349    pub fn file_statistics(&self) -> FileStatistics {
350        let column_metadatas = &self.metadata().column_metadatas;
351
352        let column_stats = column_metadatas
353            .iter()
354            .map(|col_metadata| {
355                let num_pages = col_metadata.pages.len();
356                let size_bytes = col_metadata
357                    .pages
358                    .iter()
359                    .map(|page| page.buffer_sizes.iter().sum::<u64>())
360                    .sum::<u64>();
361                ColumnStatistics {
362                    num_pages,
363                    size_bytes,
364                }
365            })
366            .collect();
367
368        FileStatistics {
369            columns: column_stats,
370        }
371    }
372
373    pub async fn read_global_buffer(&self, index: u32) -> Result<Bytes> {
374        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!()))?;
375        self.scheduler
376            .submit_single(
377                buffer_desc.position..buffer_desc.position + buffer_desc.size,
378                0,
379            )
380            .await
381    }
382
383    async fn read_tail(scheduler: &FileScheduler) -> Result<(Bytes, u64)> {
384        let file_size = scheduler.reader().size().await? as u64;
385        let begin = if file_size < scheduler.reader().block_size() as u64 {
386            0
387        } else {
388            file_size - scheduler.reader().block_size() as u64
389        };
390        let tail_bytes = scheduler.submit_single(begin..file_size, 0).await?;
391        Ok((tail_bytes, file_size))
392    }
393
394    // Checks to make sure the footer is written correctly and returns the
395    // position of the file descriptor (which comes from the footer)
396    fn decode_footer(footer_bytes: &Bytes) -> Result<Footer> {
397        let len = footer_bytes.len();
398        if len < FOOTER_LEN {
399            return Err(Error::io(
400                format!(
401                    "does not have sufficient data, len: {}, bytes: {:?}",
402                    len, footer_bytes
403                ),
404                location!(),
405            ));
406        }
407        let mut cursor = Cursor::new(footer_bytes.slice(len - FOOTER_LEN..));
408
409        let column_meta_start = cursor.read_u64::<LittleEndian>()?;
410        let column_meta_offsets_start = cursor.read_u64::<LittleEndian>()?;
411        let global_buff_offsets_start = cursor.read_u64::<LittleEndian>()?;
412        let num_global_buffers = cursor.read_u32::<LittleEndian>()?;
413        let num_columns = cursor.read_u32::<LittleEndian>()?;
414        let major_version = cursor.read_u16::<LittleEndian>()?;
415        let minor_version = cursor.read_u16::<LittleEndian>()?;
416
417        if major_version == MAJOR_VERSION as u16 && minor_version == MINOR_VERSION as u16 {
418            return Err(Error::version_conflict(
419                "Attempt to use the lance v2 reader to read a legacy file".to_string(),
420                major_version,
421                minor_version,
422                location!(),
423            ));
424        }
425
426        let magic_bytes = footer_bytes.slice(len - 4..);
427        if magic_bytes.as_ref() != MAGIC {
428            return Err(Error::io(
429                format!(
430                    "file does not appear to be a Lance file (invalid magic: {:?})",
431                    MAGIC
432                ),
433                location!(),
434            ));
435        }
436        Ok(Footer {
437            column_meta_start,
438            column_meta_offsets_start,
439            global_buff_offsets_start,
440            num_global_buffers,
441            num_columns,
442            major_version,
443            minor_version,
444        })
445    }
446
447    // TODO: Once we have coalesced I/O we should only read the column metadatas that we need
448    fn read_all_column_metadata(
449        column_metadata_bytes: Bytes,
450        footer: &Footer,
451    ) -> Result<Vec<pbfile::ColumnMetadata>> {
452        let column_metadata_start = footer.column_meta_start;
453        // cmo == column_metadata_offsets
454        let cmo_table_size = 16 * footer.num_columns as usize;
455        let cmo_table = column_metadata_bytes.slice(column_metadata_bytes.len() - cmo_table_size..);
456
457        (0..footer.num_columns)
458            .map(|col_idx| {
459                let offset = (col_idx * 16) as usize;
460                let position = LittleEndian::read_u64(&cmo_table[offset..offset + 8]);
461                let length = LittleEndian::read_u64(&cmo_table[offset + 8..offset + 16]);
462                let normalized_position = (position - column_metadata_start) as usize;
463                let normalized_end = normalized_position + (length as usize);
464                Ok(pbfile::ColumnMetadata::decode(
465                    &column_metadata_bytes[normalized_position..normalized_end],
466                )?)
467            })
468            .collect::<Result<Vec<_>>>()
469    }
470
471    async fn optimistic_tail_read(
472        data: &Bytes,
473        start_pos: u64,
474        scheduler: &FileScheduler,
475        file_len: u64,
476    ) -> Result<Bytes> {
477        let num_bytes_needed = (file_len - start_pos) as usize;
478        if data.len() >= num_bytes_needed {
479            Ok(data.slice((data.len() - num_bytes_needed)..))
480        } else {
481            let num_bytes_missing = (num_bytes_needed - data.len()) as u64;
482            let start = file_len - num_bytes_needed as u64;
483            let missing_bytes = scheduler
484                .submit_single(start..start + num_bytes_missing, 0)
485                .await?;
486            let mut combined = BytesMut::with_capacity(data.len() + num_bytes_missing as usize);
487            combined.extend(missing_bytes);
488            combined.extend(data);
489            Ok(combined.freeze())
490        }
491    }
492
493    fn do_decode_gbo_table(
494        gbo_bytes: &Bytes,
495        footer: &Footer,
496        version: LanceFileVersion,
497    ) -> Result<Vec<BufferDescriptor>> {
498        let mut global_bufs_cursor = Cursor::new(gbo_bytes);
499
500        let mut global_buffers = Vec::with_capacity(footer.num_global_buffers as usize);
501        for _ in 0..footer.num_global_buffers {
502            let buf_pos = global_bufs_cursor.read_u64::<LittleEndian>()?;
503            assert!(
504                version < LanceFileVersion::V2_1 || buf_pos % PAGE_BUFFER_ALIGNMENT as u64 == 0
505            );
506            let buf_size = global_bufs_cursor.read_u64::<LittleEndian>()?;
507            global_buffers.push(BufferDescriptor {
508                position: buf_pos,
509                size: buf_size,
510            });
511        }
512
513        Ok(global_buffers)
514    }
515
516    async fn decode_gbo_table(
517        tail_bytes: &Bytes,
518        file_len: u64,
519        scheduler: &FileScheduler,
520        footer: &Footer,
521        version: LanceFileVersion,
522    ) -> Result<Vec<BufferDescriptor>> {
523        // This could, in theory, trigger another IOP but the GBO table should never be large
524        // enough for that to happen
525        let gbo_bytes = Self::optimistic_tail_read(
526            tail_bytes,
527            footer.global_buff_offsets_start,
528            scheduler,
529            file_len,
530        )
531        .await?;
532        Self::do_decode_gbo_table(&gbo_bytes, footer, version)
533    }
534
535    fn decode_schema(schema_bytes: Bytes) -> Result<(u64, lance_core::datatypes::Schema)> {
536        let file_descriptor = pb::FileDescriptor::decode(schema_bytes)?;
537        let pb_schema = file_descriptor.schema.unwrap();
538        let num_rows = file_descriptor.length;
539        let fields_with_meta = FieldsWithMeta {
540            fields: Fields(pb_schema.fields),
541            metadata: pb_schema.metadata,
542        };
543        let schema = lance_core::datatypes::Schema::from(fields_with_meta);
544        Ok((num_rows, schema))
545    }
546
547    // TODO: Support late projection.  Currently, if we want to perform a
548    // projected read of a file, we load all of the column metadata, and then
549    // only read the column data that is requested.  This is fine for most cases.
550    //
551    // However, if there are many columns then loading all of the column metadata
552    // may be expensive.  We should support a mode where we only load the column
553    // metadata for the columns that are requested (the file format supports this).
554    //
555    // The main challenge is that we either need to ignore the column metadata cache
556    // or have a more sophisticated cache that can cache per-column metadata.
557    //
558    // Also, if the number of columns is fairly small, it's faster to read them as a
559    // single IOP, but we can fix this through coalescing.
560    pub async fn read_all_metadata(scheduler: &FileScheduler) -> Result<CachedFileMetadata> {
561        // 1. read the footer
562        let (tail_bytes, file_len) = Self::read_tail(scheduler).await?;
563        let footer = Self::decode_footer(&tail_bytes)?;
564
565        let file_version = LanceFileVersion::try_from_major_minor(
566            footer.major_version as u32,
567            footer.minor_version as u32,
568        )?;
569
570        let gbo_table =
571            Self::decode_gbo_table(&tail_bytes, file_len, scheduler, &footer, file_version).await?;
572        if gbo_table.is_empty() {
573            return Err(Error::Internal {
574                message: "File did not contain any global buffers, schema expected".to_string(),
575                location: location!(),
576            });
577        }
578        let schema_start = gbo_table[0].position;
579        let schema_size = gbo_table[0].size;
580
581        let num_footer_bytes = file_len - schema_start;
582
583        // By default we read all column metadatas.  We do NOT read the column metadata buffers
584        // at this point.  We only want to read the column metadata for columns we are actually loading.
585        let all_metadata_bytes =
586            Self::optimistic_tail_read(&tail_bytes, schema_start, scheduler, file_len).await?;
587
588        let schema_bytes = all_metadata_bytes.slice(0..schema_size as usize);
589        let (num_rows, schema) = Self::decode_schema(schema_bytes)?;
590
591        // Next, read the metadata for the columns
592        // This is both the column metadata and the CMO table
593        let column_metadata_start = (footer.column_meta_start - schema_start) as usize;
594        let column_metadata_end = (footer.global_buff_offsets_start - schema_start) as usize;
595        let column_metadata_bytes =
596            all_metadata_bytes.slice(column_metadata_start..column_metadata_end);
597        let column_metadatas = Self::read_all_column_metadata(column_metadata_bytes, &footer)?;
598
599        let num_global_buffer_bytes = gbo_table.iter().map(|buf| buf.size).sum::<u64>();
600        let num_data_bytes = footer.column_meta_start - num_global_buffer_bytes;
601        let num_column_metadata_bytes = footer.global_buff_offsets_start - footer.column_meta_start;
602
603        let column_infos = Self::meta_to_col_infos(column_metadatas.as_slice(), file_version);
604
605        Ok(CachedFileMetadata {
606            file_schema: Arc::new(schema),
607            column_metadatas,
608            column_infos,
609            num_rows,
610            num_data_bytes,
611            num_column_metadata_bytes,
612            num_global_buffer_bytes,
613            num_footer_bytes,
614            file_buffers: gbo_table,
615            major_version: footer.major_version,
616            minor_version: footer.minor_version,
617        })
618    }
619
620    fn fetch_encoding<M: Default + Name + Sized>(encoding: &pbfile::Encoding) -> M {
621        match &encoding.location {
622            Some(pbfile::encoding::Location::Indirect(_)) => todo!(),
623            Some(pbfile::encoding::Location::Direct(encoding)) => {
624                let encoding_buf = Bytes::from(encoding.encoding.clone());
625                let encoding_any = prost_types::Any::decode(encoding_buf).unwrap();
626                encoding_any.to_msg::<M>().unwrap()
627            }
628            Some(pbfile::encoding::Location::None(_)) => panic!(),
629            None => panic!(),
630        }
631    }
632
633    fn meta_to_col_infos(
634        column_metadatas: &[pbfile::ColumnMetadata],
635        file_version: LanceFileVersion,
636    ) -> Vec<Arc<ColumnInfo>> {
637        column_metadatas
638            .iter()
639            .enumerate()
640            .map(|(col_idx, col_meta)| {
641                let page_infos = col_meta
642                    .pages
643                    .iter()
644                    .map(|page| {
645                        let num_rows = page.length;
646                        let encoding = match file_version {
647                            LanceFileVersion::V2_0 => {
648                                PageEncoding::Legacy(Self::fetch_encoding::<pbenc::ArrayEncoding>(
649                                    page.encoding.as_ref().unwrap(),
650                                ))
651                            }
652                            _ => {
653                                PageEncoding::Structural(Self::fetch_encoding::<pbenc::PageLayout>(
654                                    page.encoding.as_ref().unwrap(),
655                                ))
656                            }
657                        };
658                        let buffer_offsets_and_sizes = Arc::from(
659                            page.buffer_offsets
660                                .iter()
661                                .zip(page.buffer_sizes.iter())
662                                .map(|(offset, size)| {
663                                    // Starting with version 2.1 we can assert that page buffers are aligned
664                                    assert!(
665                                        file_version < LanceFileVersion::V2_1
666                                            || offset % PAGE_BUFFER_ALIGNMENT as u64 == 0
667                                    );
668                                    (*offset, *size)
669                                })
670                                .collect::<Vec<_>>(),
671                        );
672                        PageInfo {
673                            buffer_offsets_and_sizes,
674                            encoding,
675                            num_rows,
676                            priority: page.priority,
677                        }
678                    })
679                    .collect::<Vec<_>>();
680                let buffer_offsets_and_sizes = Arc::from(
681                    col_meta
682                        .buffer_offsets
683                        .iter()
684                        .zip(col_meta.buffer_sizes.iter())
685                        .map(|(offset, size)| (*offset, *size))
686                        .collect::<Vec<_>>(),
687                );
688                Arc::new(ColumnInfo {
689                    index: col_idx as u32,
690                    page_infos: Arc::from(page_infos),
691                    buffer_offsets_and_sizes,
692                    encoding: Self::fetch_encoding(col_meta.encoding.as_ref().unwrap()),
693                })
694            })
695            .collect::<Vec<_>>()
696    }
697
698    fn validate_projection(
699        projection: &ReaderProjection,
700        metadata: &CachedFileMetadata,
701    ) -> Result<()> {
702        if projection.schema.fields.is_empty() {
703            return Err(Error::invalid_input(
704                "Attempt to read zero columns from the file, at least one column must be specified"
705                    .to_string(),
706                location!(),
707            ));
708        }
709        let mut column_indices_seen = BTreeSet::new();
710        for column_index in &projection.column_indices {
711            if !column_indices_seen.insert(*column_index) {
712                return Err(Error::invalid_input(
713                    format!(
714                        "The projection specified the column index {} more than once",
715                        column_index
716                    ),
717                    location!(),
718                ));
719            }
720            if *column_index >= metadata.column_infos.len() as u32 {
721                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!()));
722            }
723        }
724        Ok(())
725    }
726
727    /// Opens a new file reader without any pre-existing knowledge
728    ///
729    /// This will read the file schema from the file itself and thus requires a bit more I/O
730    ///
731    /// A `base_projection` can also be provided.  If provided, then the projection will apply
732    /// to all reads from the file that do not specify their own projection.
733    pub async fn try_open(
734        scheduler: FileScheduler,
735        base_projection: Option<ReaderProjection>,
736        decoder_plugins: Arc<DecoderPlugins>,
737        cache: &FileMetadataCache,
738        options: FileReaderOptions,
739    ) -> Result<Self> {
740        let file_metadata = Arc::new(Self::read_all_metadata(&scheduler).await?);
741        let path = scheduler.reader().path().clone();
742        Self::try_open_with_file_metadata(
743            Arc::new(LanceEncodingsIo(scheduler)),
744            path,
745            base_projection,
746            decoder_plugins,
747            file_metadata,
748            cache,
749            options,
750        )
751        .await
752    }
753
754    /// Same as `try_open` but with the file metadata already loaded.
755    ///
756    /// This method also can accept any kind of `EncodingsIo` implementation allowing
757    /// for custom strategies to be used for I/O scheduling (e.g. for takes on fast
758    /// disks it may be better to avoid asynchronous overhead).
759    pub async fn try_open_with_file_metadata(
760        scheduler: Arc<dyn EncodingsIo>,
761        path: Path,
762        base_projection: Option<ReaderProjection>,
763        decoder_plugins: Arc<DecoderPlugins>,
764        file_metadata: Arc<CachedFileMetadata>,
765        cache: &FileMetadataCache,
766        options: FileReaderOptions,
767    ) -> Result<Self> {
768        let cache = Arc::new(cache.with_base_path(path));
769
770        if let Some(base_projection) = base_projection.as_ref() {
771            Self::validate_projection(base_projection, &file_metadata)?;
772        }
773        let num_rows = file_metadata.num_rows;
774        Ok(Self {
775            scheduler,
776            base_projection: base_projection.unwrap_or(ReaderProjection::from_whole_schema(
777                file_metadata.file_schema.as_ref(),
778                file_metadata.version(),
779            )),
780            num_rows,
781            metadata: file_metadata,
782            decoder_plugins,
783            cache,
784            options,
785        })
786    }
787
788    // The actual decoder needs all the column infos that make up a type.  In other words, if
789    // the first type in the schema is Struct<i32, i32> then the decoder will need 3 column infos.
790    //
791    // This is a file reader concern because the file reader needs to support late projection of columns
792    // and so it will need to figure this out anyways.
793    //
794    // It's a bit of a tricky process though because the number of column infos may depend on the
795    // encoding.  Considering the above example, if we wrote it with a packed encoding, then there would
796    // only be a single column in the file (and not 3).
797    //
798    // At the moment this method words because our rules are simple and we just repeat them here.  See
799    // Self::default_projection for a similar problem.  In the future this is something the encodings
800    // registry will need to figure out.
801    fn collect_columns_from_projection(
802        &self,
803        _projection: &ReaderProjection,
804    ) -> Result<Vec<Arc<ColumnInfo>>> {
805        Ok(self.metadata.column_infos.to_vec())
806    }
807
808    #[allow(clippy::too_many_arguments)]
809    fn do_read_range(
810        column_infos: Vec<Arc<ColumnInfo>>,
811        io: Arc<dyn EncodingsIo>,
812        cache: Arc<FileMetadataCache>,
813        num_rows: u64,
814        decoder_plugins: Arc<DecoderPlugins>,
815        range: Range<u64>,
816        batch_size: u32,
817        projection: ReaderProjection,
818        filter: FilterExpression,
819        should_validate: bool,
820    ) -> Result<BoxStream<'static, ReadBatchTask>> {
821        debug!(
822            "Reading range {:?} with batch_size {} from file with {} rows and {} columns into schema with {} columns",
823            range,
824            batch_size,
825            num_rows,
826            column_infos.len(),
827            projection.schema.fields.len(),
828        );
829
830        let config = SchedulerDecoderConfig {
831            batch_size,
832            cache,
833            decoder_plugins,
834            io,
835            should_validate,
836        };
837
838        let requested_rows = RequestedRows::Ranges(vec![range]);
839
840        Ok(schedule_and_decode(
841            column_infos,
842            requested_rows,
843            filter,
844            projection.column_indices,
845            projection.schema,
846            config,
847        ))
848    }
849
850    fn read_range(
851        &self,
852        range: Range<u64>,
853        batch_size: u32,
854        projection: ReaderProjection,
855        filter: FilterExpression,
856    ) -> Result<BoxStream<'static, ReadBatchTask>> {
857        // Create and initialize the stream
858        Self::do_read_range(
859            self.collect_columns_from_projection(&projection)?,
860            self.scheduler.clone(),
861            self.cache.clone(),
862            self.num_rows,
863            self.decoder_plugins.clone(),
864            range,
865            batch_size,
866            projection,
867            filter,
868            self.options.validate_on_decode,
869        )
870    }
871
872    #[allow(clippy::too_many_arguments)]
873    fn do_take_rows(
874        column_infos: Vec<Arc<ColumnInfo>>,
875        io: Arc<dyn EncodingsIo>,
876        cache: Arc<FileMetadataCache>,
877        decoder_plugins: Arc<DecoderPlugins>,
878        indices: Vec<u64>,
879        batch_size: u32,
880        projection: ReaderProjection,
881        filter: FilterExpression,
882        should_validate: bool,
883    ) -> Result<BoxStream<'static, ReadBatchTask>> {
884        debug!(
885            "Taking {} rows spread across range {}..{} with batch_size {} from columns {:?}",
886            indices.len(),
887            indices[0],
888            indices[indices.len() - 1],
889            batch_size,
890            column_infos.iter().map(|ci| ci.index).collect::<Vec<_>>()
891        );
892
893        let config = SchedulerDecoderConfig {
894            batch_size,
895            cache,
896            decoder_plugins,
897            io,
898            should_validate,
899        };
900
901        let requested_rows = RequestedRows::Indices(indices);
902
903        Ok(schedule_and_decode(
904            column_infos,
905            requested_rows,
906            filter,
907            projection.column_indices,
908            projection.schema,
909            config,
910        ))
911    }
912
913    fn take_rows(
914        &self,
915        indices: Vec<u64>,
916        batch_size: u32,
917        projection: ReaderProjection,
918    ) -> Result<BoxStream<'static, ReadBatchTask>> {
919        // Create and initialize the stream
920        Self::do_take_rows(
921            self.collect_columns_from_projection(&projection)?,
922            self.scheduler.clone(),
923            self.cache.clone(),
924            self.decoder_plugins.clone(),
925            indices,
926            batch_size,
927            projection,
928            FilterExpression::no_filter(),
929            self.options.validate_on_decode,
930        )
931    }
932
933    /// Creates a stream of "read tasks" to read the data from the file
934    ///
935    /// The arguments are similar to [`Self::read_stream_projected`] but instead of returning a stream
936    /// of record batches it returns a stream of "read tasks".
937    ///
938    /// The tasks should be consumed with some kind of `buffered` argument if CPU parallelism is desired.
939    ///
940    /// Note that "read task" is probably a bit imprecise.  The tasks are actually "decode tasks".  The
941    /// reading happens asynchronously in the background.  In other words, a single read task may map to
942    /// multiple I/O operations or a single I/O operation may map to multiple read tasks.
943    pub fn read_tasks(
944        &self,
945        params: ReadBatchParams,
946        batch_size: u32,
947        projection: Option<ReaderProjection>,
948        filter: FilterExpression,
949    ) -> Result<Pin<Box<dyn Stream<Item = ReadBatchTask> + Send>>> {
950        let projection = projection.unwrap_or_else(|| self.base_projection.clone());
951        Self::validate_projection(&projection, &self.metadata)?;
952        let verify_bound = |params: &ReadBatchParams, bound: u64, inclusive: bool| {
953            if bound > self.num_rows || bound == self.num_rows && inclusive {
954                Err(Error::invalid_input(
955                    format!(
956                        "cannot read {:?} from file with {} rows",
957                        params, self.num_rows
958                    ),
959                    location!(),
960                ))
961            } else {
962                Ok(())
963            }
964        };
965        match &params {
966            ReadBatchParams::Indices(indices) => {
967                for idx in indices {
968                    match idx {
969                        None => {
970                            return Err(Error::invalid_input(
971                                "Null value in indices array",
972                                location!(),
973                            ));
974                        }
975                        Some(idx) => {
976                            verify_bound(&params, idx as u64, true)?;
977                        }
978                    }
979                }
980                let indices = indices.iter().map(|idx| idx.unwrap() as u64).collect();
981                self.take_rows(indices, batch_size, projection)
982            }
983            ReadBatchParams::Range(range) => {
984                verify_bound(&params, range.end as u64, false)?;
985                self.read_range(
986                    range.start as u64..range.end as u64,
987                    batch_size,
988                    projection,
989                    filter,
990                )
991            }
992            ReadBatchParams::RangeFrom(range) => {
993                verify_bound(&params, range.start as u64, true)?;
994                self.read_range(
995                    range.start as u64..self.num_rows,
996                    batch_size,
997                    projection,
998                    filter,
999                )
1000            }
1001            ReadBatchParams::RangeTo(range) => {
1002                verify_bound(&params, range.end as u64, false)?;
1003                self.read_range(0..range.end as u64, batch_size, projection, filter)
1004            }
1005            ReadBatchParams::RangeFull => {
1006                self.read_range(0..self.num_rows, batch_size, projection, filter)
1007            }
1008        }
1009    }
1010
1011    /// Reads data from the file as a stream of record batches
1012    ///
1013    /// * `params` - Specifies the range (or indices) of data to read
1014    /// * `batch_size` - The maximum size of a single batch.  A batch may be smaller
1015    ///   if it is the last batch or if it is not possible to create a batch of the
1016    ///   requested size.
1017    ///
1018    ///   For example, if the batch size is 1024 and one of the columns is a string
1019    ///   column then there may be some ranges of 1024 rows that contain more than
1020    ///   2^31 bytes of string data (which is the maximum size of a string column
1021    ///   in Arrow).  In this case smaller batches may be emitted.
1022    /// * `batch_readahead` - The number of batches to read ahead.  This controls the
1023    ///   amount of CPU parallelism of the read.  In other words it controls how many
1024    ///   batches will be decoded in parallel.  It has no effect on the I/O parallelism
1025    ///   of the read (how many I/O requests are in flight at once).
1026    ///
1027    ///   This parameter also is also related to backpressure.  If the consumer of the
1028    ///   stream is slow then the reader will build up RAM.
1029    /// * `projection` - A projection to apply to the read.  This controls which columns
1030    ///   are read from the file.  The projection is NOT applied on top of the base
1031    ///   projection.  The projection is applied directly to the file schema.
1032    pub fn read_stream_projected(
1033        &self,
1034        params: ReadBatchParams,
1035        batch_size: u32,
1036        batch_readahead: u32,
1037        projection: ReaderProjection,
1038        filter: FilterExpression,
1039    ) -> Result<Pin<Box<dyn RecordBatchStream>>> {
1040        let arrow_schema = Arc::new(ArrowSchema::from(projection.schema.as_ref()));
1041        let tasks_stream = self.read_tasks(params, batch_size, Some(projection), filter)?;
1042        let batch_stream = tasks_stream
1043            .map(|task| task.task)
1044            .buffered(batch_readahead as usize)
1045            .boxed();
1046        Ok(Box::pin(RecordBatchStreamAdapter::new(
1047            arrow_schema,
1048            batch_stream,
1049        )))
1050    }
1051
1052    fn take_rows_blocking(
1053        &self,
1054        indices: Vec<u64>,
1055        batch_size: u32,
1056        projection: ReaderProjection,
1057        filter: FilterExpression,
1058    ) -> Result<Box<dyn RecordBatchReader + Send + 'static>> {
1059        let column_infos = self.collect_columns_from_projection(&projection)?;
1060        debug!(
1061            "Taking {} rows spread across range {}..{} with batch_size {} from columns {:?}",
1062            indices.len(),
1063            indices[0],
1064            indices[indices.len() - 1],
1065            batch_size,
1066            column_infos.iter().map(|ci| ci.index).collect::<Vec<_>>()
1067        );
1068
1069        let config = SchedulerDecoderConfig {
1070            batch_size,
1071            cache: self.cache.clone(),
1072            decoder_plugins: self.decoder_plugins.clone(),
1073            io: self.scheduler.clone(),
1074            should_validate: self.options.validate_on_decode,
1075        };
1076
1077        let requested_rows = RequestedRows::Indices(indices);
1078
1079        schedule_and_decode_blocking(
1080            column_infos,
1081            requested_rows,
1082            filter,
1083            projection.column_indices,
1084            projection.schema,
1085            config,
1086        )
1087    }
1088
1089    fn read_range_blocking(
1090        &self,
1091        range: Range<u64>,
1092        batch_size: u32,
1093        projection: ReaderProjection,
1094        filter: FilterExpression,
1095    ) -> Result<Box<dyn RecordBatchReader + Send + 'static>> {
1096        let column_infos = self.collect_columns_from_projection(&projection)?;
1097        let num_rows = self.num_rows;
1098
1099        debug!(
1100            "Reading range {:?} with batch_size {} from file with {} rows and {} columns into schema with {} columns",
1101            range,
1102            batch_size,
1103            num_rows,
1104            column_infos.len(),
1105            projection.schema.fields.len(),
1106        );
1107
1108        let config = SchedulerDecoderConfig {
1109            batch_size,
1110            cache: self.cache.clone(),
1111            decoder_plugins: self.decoder_plugins.clone(),
1112            io: self.scheduler.clone(),
1113            should_validate: self.options.validate_on_decode,
1114        };
1115
1116        let requested_rows = RequestedRows::Ranges(vec![range]);
1117
1118        schedule_and_decode_blocking(
1119            column_infos,
1120            requested_rows,
1121            filter,
1122            projection.column_indices,
1123            projection.schema,
1124            config,
1125        )
1126    }
1127
1128    /// Read data from the file as an iterator of record batches
1129    ///
1130    /// This is a blocking variant of [`Self::read_stream_projected`] that runs entirely in the
1131    /// calling thread.  It will block on I/O if the decode is faster than the I/O.  It is useful
1132    /// for benchmarking and potentially from "take"ing small batches from fast disks.
1133    ///
1134    /// Large scans of in-memory data will still benefit from threading (and should therefore not
1135    /// use this method) because we can parallelize the decode.
1136    ///
1137    /// Note: calling this from within a tokio runtime will panic.  It is acceptable to call this
1138    /// from a spawn_blocking context.
1139    pub fn read_stream_projected_blocking(
1140        &self,
1141        params: ReadBatchParams,
1142        batch_size: u32,
1143        projection: Option<ReaderProjection>,
1144        filter: FilterExpression,
1145    ) -> Result<Box<dyn RecordBatchReader + Send + 'static>> {
1146        let projection = projection.unwrap_or_else(|| self.base_projection.clone());
1147        Self::validate_projection(&projection, &self.metadata)?;
1148        let verify_bound = |params: &ReadBatchParams, bound: u64, inclusive: bool| {
1149            if bound > self.num_rows || bound == self.num_rows && inclusive {
1150                Err(Error::invalid_input(
1151                    format!(
1152                        "cannot read {:?} from file with {} rows",
1153                        params, self.num_rows
1154                    ),
1155                    location!(),
1156                ))
1157            } else {
1158                Ok(())
1159            }
1160        };
1161        match &params {
1162            ReadBatchParams::Indices(indices) => {
1163                for idx in indices {
1164                    match idx {
1165                        None => {
1166                            return Err(Error::invalid_input(
1167                                "Null value in indices array",
1168                                location!(),
1169                            ));
1170                        }
1171                        Some(idx) => {
1172                            verify_bound(&params, idx as u64, true)?;
1173                        }
1174                    }
1175                }
1176                let indices = indices.iter().map(|idx| idx.unwrap() as u64).collect();
1177                self.take_rows_blocking(indices, batch_size, projection, filter)
1178            }
1179            ReadBatchParams::Range(range) => {
1180                verify_bound(&params, range.end as u64, false)?;
1181                self.read_range_blocking(
1182                    range.start as u64..range.end as u64,
1183                    batch_size,
1184                    projection,
1185                    filter,
1186                )
1187            }
1188            ReadBatchParams::RangeFrom(range) => {
1189                verify_bound(&params, range.start as u64, true)?;
1190                self.read_range_blocking(
1191                    range.start as u64..self.num_rows,
1192                    batch_size,
1193                    projection,
1194                    filter,
1195                )
1196            }
1197            ReadBatchParams::RangeTo(range) => {
1198                verify_bound(&params, range.end as u64, false)?;
1199                self.read_range_blocking(0..range.end as u64, batch_size, projection, filter)
1200            }
1201            ReadBatchParams::RangeFull => {
1202                self.read_range_blocking(0..self.num_rows, batch_size, projection, filter)
1203            }
1204        }
1205    }
1206
1207    /// Reads data from the file as a stream of record batches
1208    ///
1209    /// This is similar to [`Self::read_stream_projected`] but uses the base projection
1210    /// provided when the file was opened (or reads all columns if the file was
1211    /// opened without a base projection)
1212    pub fn read_stream(
1213        &self,
1214        params: ReadBatchParams,
1215        batch_size: u32,
1216        batch_readahead: u32,
1217        filter: FilterExpression,
1218    ) -> Result<Pin<Box<dyn RecordBatchStream>>> {
1219        self.read_stream_projected(
1220            params,
1221            batch_size,
1222            batch_readahead,
1223            self.base_projection.clone(),
1224            filter,
1225        )
1226    }
1227
1228    pub fn schema(&self) -> &Arc<Schema> {
1229        &self.metadata.file_schema
1230    }
1231}
1232
1233/// Inspects a page and returns a String describing the page's encoding
1234pub fn describe_encoding(page: &pbfile::column_metadata::Page) -> String {
1235    if let Some(encoding) = &page.encoding {
1236        if let Some(style) = &encoding.location {
1237            match style {
1238                pbfile::encoding::Location::Indirect(indirect) => {
1239                    format!(
1240                        "IndirectEncoding(pos={},size={})",
1241                        indirect.buffer_location, indirect.buffer_length
1242                    )
1243                }
1244                pbfile::encoding::Location::Direct(direct) => {
1245                    let encoding_any =
1246                        prost_types::Any::decode(Bytes::from(direct.encoding.clone()))
1247                            .expect("failed to deserialize encoding as protobuf");
1248                    if encoding_any.type_url == "/lance.encodings.ArrayEncoding" {
1249                        let encoding = encoding_any.to_msg::<pbenc::ArrayEncoding>();
1250                        match encoding {
1251                            Ok(encoding) => {
1252                                format!("{:#?}", encoding)
1253                            }
1254                            Err(err) => {
1255                                format!("Unsupported(decode_err={})", err)
1256                            }
1257                        }
1258                    } else if encoding_any.type_url == "/lance.encodings.PageLayout" {
1259                        let encoding = encoding_any.to_msg::<pbenc::PageLayout>();
1260                        match encoding {
1261                            Ok(encoding) => {
1262                                format!("{:#?}", encoding)
1263                            }
1264                            Err(err) => {
1265                                format!("Unsupported(decode_err={})", err)
1266                            }
1267                        }
1268                    } else {
1269                        format!("Unrecognized(type_url={})", encoding_any.type_url)
1270                    }
1271                }
1272                pbfile::encoding::Location::None(_) => "NoEncodingDescription".to_string(),
1273            }
1274        } else {
1275            "MISSING STYLE".to_string()
1276        }
1277    } else {
1278        "MISSING".to_string()
1279    }
1280}
1281
1282pub trait EncodedBatchReaderExt {
1283    fn try_from_mini_lance(
1284        bytes: Bytes,
1285        schema: &Schema,
1286        version: LanceFileVersion,
1287    ) -> Result<Self>
1288    where
1289        Self: Sized;
1290    fn try_from_self_described_lance(bytes: Bytes) -> Result<Self>
1291    where
1292        Self: Sized;
1293}
1294
1295impl EncodedBatchReaderExt for EncodedBatch {
1296    fn try_from_mini_lance(
1297        bytes: Bytes,
1298        schema: &Schema,
1299        file_version: LanceFileVersion,
1300    ) -> Result<Self>
1301    where
1302        Self: Sized,
1303    {
1304        let projection = ReaderProjection::from_whole_schema(schema, file_version);
1305        let footer = FileReader::decode_footer(&bytes)?;
1306
1307        // Next, read the metadata for the columns
1308        // This is both the column metadata and the CMO table
1309        let column_metadata_start = footer.column_meta_start as usize;
1310        let column_metadata_end = footer.global_buff_offsets_start as usize;
1311        let column_metadata_bytes = bytes.slice(column_metadata_start..column_metadata_end);
1312        let column_metadatas =
1313            FileReader::read_all_column_metadata(column_metadata_bytes, &footer)?;
1314
1315        let file_version = LanceFileVersion::try_from_major_minor(
1316            footer.major_version as u32,
1317            footer.minor_version as u32,
1318        )?;
1319
1320        let page_table = FileReader::meta_to_col_infos(&column_metadatas, file_version);
1321
1322        Ok(Self {
1323            data: bytes,
1324            num_rows: page_table
1325                .first()
1326                .map(|col| col.page_infos.iter().map(|page| page.num_rows).sum::<u64>())
1327                .unwrap_or(0),
1328            page_table,
1329            top_level_columns: projection.column_indices,
1330            schema: Arc::new(schema.clone()),
1331        })
1332    }
1333
1334    fn try_from_self_described_lance(bytes: Bytes) -> Result<Self>
1335    where
1336        Self: Sized,
1337    {
1338        let footer = FileReader::decode_footer(&bytes)?;
1339        let file_version = LanceFileVersion::try_from_major_minor(
1340            footer.major_version as u32,
1341            footer.minor_version as u32,
1342        )?;
1343
1344        let gbo_table = FileReader::do_decode_gbo_table(
1345            &bytes.slice(footer.global_buff_offsets_start as usize..),
1346            &footer,
1347            file_version,
1348        )?;
1349        if gbo_table.is_empty() {
1350            return Err(Error::Internal {
1351                message: "File did not contain any global buffers, schema expected".to_string(),
1352                location: location!(),
1353            });
1354        }
1355        let schema_start = gbo_table[0].position as usize;
1356        let schema_size = gbo_table[0].size as usize;
1357
1358        let schema_bytes = bytes.slice(schema_start..(schema_start + schema_size));
1359        let (_, schema) = FileReader::decode_schema(schema_bytes)?;
1360        let projection = ReaderProjection::from_whole_schema(&schema, file_version);
1361
1362        // Next, read the metadata for the columns
1363        // This is both the column metadata and the CMO table
1364        let column_metadata_start = footer.column_meta_start as usize;
1365        let column_metadata_end = footer.global_buff_offsets_start as usize;
1366        let column_metadata_bytes = bytes.slice(column_metadata_start..column_metadata_end);
1367        let column_metadatas =
1368            FileReader::read_all_column_metadata(column_metadata_bytes, &footer)?;
1369
1370        let page_table = FileReader::meta_to_col_infos(&column_metadatas, file_version);
1371
1372        Ok(Self {
1373            data: bytes,
1374            num_rows: page_table
1375                .first()
1376                .map(|col| col.page_infos.iter().map(|page| page.num_rows).sum::<u64>())
1377                .unwrap_or(0),
1378            page_table,
1379            top_level_columns: projection.column_indices,
1380            schema: Arc::new(schema),
1381        })
1382    }
1383}
1384
1385#[cfg(test)]
1386pub mod tests {
1387    use std::{collections::BTreeMap, pin::Pin, sync::Arc};
1388
1389    use arrow_array::{
1390        types::{Float64Type, Int32Type},
1391        RecordBatch, UInt32Array,
1392    };
1393    use arrow_schema::{DataType, Field, Fields, Schema as ArrowSchema};
1394    use bytes::Bytes;
1395    use futures::{prelude::stream::TryStreamExt, StreamExt};
1396    use lance_arrow::RecordBatchExt;
1397    use lance_core::{datatypes::Schema, ArrowResult};
1398    use lance_datagen::{array, gen, BatchCount, ByteCount, RowCount};
1399    use lance_encoding::{
1400        decoder::{decode_batch, DecodeBatchScheduler, DecoderPlugins, FilterExpression},
1401        encoder::{encode_batch, CoreFieldEncodingStrategy, EncodedBatch, EncodingOptions},
1402        version::LanceFileVersion,
1403    };
1404    use lance_io::stream::RecordBatchStream;
1405    use log::debug;
1406    use tokio::sync::mpsc;
1407
1408    use crate::v2::{
1409        reader::{EncodedBatchReaderExt, FileReader, FileReaderOptions, ReaderProjection},
1410        testing::{test_cache, write_lance_file, FsFixture, WrittenFile},
1411        writer::{EncodedBatchWriteExt, FileWriter, FileWriterOptions},
1412    };
1413
1414    async fn create_some_file(fs: &FsFixture, version: LanceFileVersion) -> WrittenFile {
1415        let location_type = DataType::Struct(Fields::from(vec![
1416            Field::new("x", DataType::Float64, true),
1417            Field::new("y", DataType::Float64, true),
1418        ]));
1419        let categories_type = DataType::List(Arc::new(Field::new("item", DataType::Utf8, true)));
1420
1421        let mut reader = gen()
1422            .col("score", array::rand::<Float64Type>())
1423            .col("location", array::rand_type(&location_type))
1424            .col("categories", array::rand_type(&categories_type))
1425            .col("binary", array::rand_type(&DataType::Binary));
1426        if version <= LanceFileVersion::V2_0 {
1427            reader = reader.col("large_bin", array::rand_type(&DataType::LargeBinary));
1428        }
1429        let reader = reader.into_reader_rows(RowCount::from(1000), BatchCount::from(100));
1430
1431        write_lance_file(
1432            reader,
1433            fs,
1434            FileWriterOptions {
1435                format_version: Some(version),
1436                ..Default::default()
1437            },
1438        )
1439        .await
1440    }
1441
1442    type Transformer = Box<dyn Fn(&RecordBatch) -> RecordBatch>;
1443
1444    async fn verify_expected(
1445        expected: &[RecordBatch],
1446        mut actual: Pin<Box<dyn RecordBatchStream>>,
1447        read_size: u32,
1448        transform: Option<Transformer>,
1449    ) {
1450        let mut remaining = expected.iter().map(|batch| batch.num_rows()).sum::<usize>() as u32;
1451        let mut expected_iter = expected.iter().map(|batch| {
1452            if let Some(transform) = &transform {
1453                transform(batch)
1454            } else {
1455                batch.clone()
1456            }
1457        });
1458        let mut next_expected = expected_iter.next().unwrap().clone();
1459        while let Some(actual) = actual.next().await {
1460            let mut actual = actual.unwrap();
1461            let mut rows_to_verify = actual.num_rows() as u32;
1462            let expected_length = remaining.min(read_size);
1463            assert_eq!(expected_length, rows_to_verify);
1464
1465            while rows_to_verify > 0 {
1466                let next_slice_len = (next_expected.num_rows() as u32).min(rows_to_verify);
1467                assert_eq!(
1468                    next_expected.slice(0, next_slice_len as usize),
1469                    actual.slice(0, next_slice_len as usize)
1470                );
1471                remaining -= next_slice_len;
1472                rows_to_verify -= next_slice_len;
1473                if remaining > 0 {
1474                    if next_slice_len == next_expected.num_rows() as u32 {
1475                        next_expected = expected_iter.next().unwrap().clone();
1476                    } else {
1477                        next_expected = next_expected.slice(
1478                            next_slice_len as usize,
1479                            next_expected.num_rows() - next_slice_len as usize,
1480                        );
1481                    }
1482                }
1483                if rows_to_verify > 0 {
1484                    actual = actual.slice(
1485                        next_slice_len as usize,
1486                        actual.num_rows() - next_slice_len as usize,
1487                    );
1488                }
1489            }
1490        }
1491        assert_eq!(remaining, 0);
1492    }
1493
1494    #[tokio::test]
1495    async fn test_round_trip() {
1496        let fs = FsFixture::default();
1497
1498        let WrittenFile { data, .. } = create_some_file(&fs, LanceFileVersion::V2_0).await;
1499
1500        for read_size in [32, 1024, 1024 * 1024] {
1501            let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1502            let file_reader = FileReader::try_open(
1503                file_scheduler,
1504                None,
1505                Arc::<DecoderPlugins>::default(),
1506                &test_cache(),
1507                FileReaderOptions::default(),
1508            )
1509            .await
1510            .unwrap();
1511
1512            let schema = file_reader.schema();
1513            assert_eq!(schema.metadata.get("foo").unwrap(), "bar");
1514
1515            let batch_stream = file_reader
1516                .read_stream(
1517                    lance_io::ReadBatchParams::RangeFull,
1518                    read_size,
1519                    16,
1520                    FilterExpression::no_filter(),
1521                )
1522                .unwrap();
1523
1524            verify_expected(&data, batch_stream, read_size, None).await;
1525        }
1526    }
1527
1528    #[test_log::test(tokio::test)]
1529    async fn test_encoded_batch_round_trip() {
1530        let data = gen()
1531            .col("x", array::rand::<Int32Type>())
1532            .col("y", array::rand_utf8(ByteCount::from(16), false))
1533            .into_batch_rows(RowCount::from(10000))
1534            .unwrap();
1535
1536        let lance_schema = Arc::new(Schema::try_from(data.schema().as_ref()).unwrap());
1537
1538        let encoding_options = EncodingOptions {
1539            cache_bytes_per_column: 4096,
1540            max_page_bytes: 32 * 1024 * 1024,
1541            keep_original_array: true,
1542            buffer_alignment: 64,
1543        };
1544        let encoded_batch = encode_batch(
1545            &data,
1546            lance_schema.clone(),
1547            &CoreFieldEncodingStrategy::default(),
1548            &encoding_options,
1549        )
1550        .await
1551        .unwrap();
1552
1553        // Test self described
1554        let bytes = encoded_batch.try_to_self_described_lance().unwrap();
1555
1556        let decoded_batch = EncodedBatch::try_from_self_described_lance(bytes).unwrap();
1557
1558        let decoded = decode_batch(
1559            &decoded_batch,
1560            &FilterExpression::no_filter(),
1561            Arc::<DecoderPlugins>::default(),
1562            false,
1563            LanceFileVersion::default(),
1564            None,
1565        )
1566        .await
1567        .unwrap();
1568
1569        assert_eq!(data, decoded);
1570
1571        // Test mini
1572        let bytes = encoded_batch.try_to_mini_lance().unwrap();
1573        let decoded_batch =
1574            EncodedBatch::try_from_mini_lance(bytes, lance_schema.as_ref(), LanceFileVersion::V2_0)
1575                .unwrap();
1576        let decoded = decode_batch(
1577            &decoded_batch,
1578            &FilterExpression::no_filter(),
1579            Arc::<DecoderPlugins>::default(),
1580            false,
1581            LanceFileVersion::default(),
1582            None,
1583        )
1584        .await
1585        .unwrap();
1586
1587        assert_eq!(data, decoded);
1588    }
1589
1590    #[test_log::test(tokio::test)]
1591    async fn test_projection() {
1592        let fs = FsFixture::default();
1593
1594        let written_file = create_some_file(&fs, LanceFileVersion::V2_0).await;
1595        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1596
1597        let field_id_mapping = written_file
1598            .field_id_mapping
1599            .iter()
1600            .copied()
1601            .collect::<BTreeMap<_, _>>();
1602
1603        let empty_projection = ReaderProjection {
1604            column_indices: Vec::default(),
1605            schema: Arc::new(Schema::default()),
1606        };
1607
1608        for columns in [
1609            vec!["score"],
1610            vec!["location"],
1611            vec!["categories"],
1612            vec!["score.x"],
1613            vec!["score", "categories"],
1614            vec!["score", "location"],
1615            vec!["location", "categories"],
1616            vec!["score.y", "location", "categories"],
1617        ] {
1618            debug!("Testing round trip with projection {:?}", columns);
1619            // We can specify the projection as part of the read operation via read_stream_projected
1620            let file_reader = FileReader::try_open(
1621                file_scheduler.clone(),
1622                None,
1623                Arc::<DecoderPlugins>::default(),
1624                &test_cache(),
1625                FileReaderOptions::default(),
1626            )
1627            .await
1628            .unwrap();
1629
1630            let projected_schema = written_file.schema.project(&columns).unwrap();
1631            let projection = ReaderProjection::from_field_ids(
1632                &file_reader,
1633                &projected_schema,
1634                &field_id_mapping,
1635            )
1636            .unwrap();
1637
1638            let batch_stream = file_reader
1639                .read_stream_projected(
1640                    lance_io::ReadBatchParams::RangeFull,
1641                    1024,
1642                    16,
1643                    projection.clone(),
1644                    FilterExpression::no_filter(),
1645                )
1646                .unwrap();
1647
1648            let projection_arrow = ArrowSchema::from(projection.schema.as_ref());
1649            verify_expected(
1650                &written_file.data,
1651                batch_stream,
1652                1024,
1653                Some(Box::new(move |batch: &RecordBatch| {
1654                    batch.project_by_schema(&projection_arrow).unwrap()
1655                })),
1656            )
1657            .await;
1658
1659            // We can also specify the projection as a base projection when we open the file
1660            let file_reader = FileReader::try_open(
1661                file_scheduler.clone(),
1662                Some(projection.clone()),
1663                Arc::<DecoderPlugins>::default(),
1664                &test_cache(),
1665                FileReaderOptions::default(),
1666            )
1667            .await
1668            .unwrap();
1669
1670            let batch_stream = file_reader
1671                .read_stream(
1672                    lance_io::ReadBatchParams::RangeFull,
1673                    1024,
1674                    16,
1675                    FilterExpression::no_filter(),
1676                )
1677                .unwrap();
1678
1679            let projection_arrow = ArrowSchema::from(projection.schema.as_ref());
1680            verify_expected(
1681                &written_file.data,
1682                batch_stream,
1683                1024,
1684                Some(Box::new(move |batch: &RecordBatch| {
1685                    batch.project_by_schema(&projection_arrow).unwrap()
1686                })),
1687            )
1688            .await;
1689
1690            assert!(file_reader
1691                .read_stream_projected(
1692                    lance_io::ReadBatchParams::RangeFull,
1693                    1024,
1694                    16,
1695                    empty_projection.clone(),
1696                    FilterExpression::no_filter(),
1697                )
1698                .is_err());
1699        }
1700
1701        assert!(FileReader::try_open(
1702            file_scheduler.clone(),
1703            Some(empty_projection),
1704            Arc::<DecoderPlugins>::default(),
1705            &test_cache(),
1706            FileReaderOptions::default(),
1707        )
1708        .await
1709        .is_err());
1710
1711        let arrow_schema = ArrowSchema::new(vec![
1712            Field::new("x", DataType::Int32, true),
1713            Field::new("y", DataType::Int32, true),
1714        ]);
1715        let schema = Schema::try_from(&arrow_schema).unwrap();
1716
1717        let projection_with_dupes = ReaderProjection {
1718            column_indices: vec![0, 0],
1719            schema: Arc::new(schema),
1720        };
1721
1722        assert!(FileReader::try_open(
1723            file_scheduler.clone(),
1724            Some(projection_with_dupes),
1725            Arc::<DecoderPlugins>::default(),
1726            &test_cache(),
1727            FileReaderOptions::default(),
1728        )
1729        .await
1730        .is_err());
1731    }
1732
1733    #[test_log::test(tokio::test)]
1734    async fn test_compressing_buffer() {
1735        let fs = FsFixture::default();
1736
1737        let written_file = create_some_file(&fs, LanceFileVersion::V2_0).await;
1738        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1739
1740        // We can specify the projection as part of the read operation via read_stream_projected
1741        let file_reader = FileReader::try_open(
1742            file_scheduler.clone(),
1743            None,
1744            Arc::<DecoderPlugins>::default(),
1745            &test_cache(),
1746            FileReaderOptions::default(),
1747        )
1748        .await
1749        .unwrap();
1750
1751        let mut projection = written_file.schema.project(&["score"]).unwrap();
1752        for field in projection.fields.iter_mut() {
1753            field
1754                .metadata
1755                .insert("lance:compression".to_string(), "zstd".to_string());
1756        }
1757        let projection = ReaderProjection {
1758            column_indices: projection.fields.iter().map(|f| f.id as u32).collect(),
1759            schema: Arc::new(projection),
1760        };
1761
1762        let batch_stream = file_reader
1763            .read_stream_projected(
1764                lance_io::ReadBatchParams::RangeFull,
1765                1024,
1766                16,
1767                projection.clone(),
1768                FilterExpression::no_filter(),
1769            )
1770            .unwrap();
1771
1772        let projection_arrow = Arc::new(ArrowSchema::from(projection.schema.as_ref()));
1773        verify_expected(
1774            &written_file.data,
1775            batch_stream,
1776            1024,
1777            Some(Box::new(move |batch: &RecordBatch| {
1778                batch.project_by_schema(&projection_arrow).unwrap()
1779            })),
1780        )
1781        .await;
1782    }
1783
1784    #[tokio::test]
1785    async fn test_read_all() {
1786        let fs = FsFixture::default();
1787        let WrittenFile { data, .. } = create_some_file(&fs, LanceFileVersion::V2_0).await;
1788        let total_rows = data.iter().map(|batch| batch.num_rows()).sum::<usize>();
1789
1790        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1791        let file_reader = FileReader::try_open(
1792            file_scheduler.clone(),
1793            None,
1794            Arc::<DecoderPlugins>::default(),
1795            &test_cache(),
1796            FileReaderOptions::default(),
1797        )
1798        .await
1799        .unwrap();
1800
1801        let batches = file_reader
1802            .read_stream(
1803                lance_io::ReadBatchParams::RangeFull,
1804                total_rows as u32,
1805                16,
1806                FilterExpression::no_filter(),
1807            )
1808            .unwrap()
1809            .try_collect::<Vec<_>>()
1810            .await
1811            .unwrap();
1812        assert_eq!(batches.len(), 1);
1813        assert_eq!(batches[0].num_rows(), total_rows);
1814    }
1815
1816    #[tokio::test]
1817    async fn test_blocking_take() {
1818        let fs = FsFixture::default();
1819        let WrittenFile { data, schema, .. } = create_some_file(&fs, LanceFileVersion::V2_1).await;
1820        let total_rows = data.iter().map(|batch| batch.num_rows()).sum::<usize>();
1821
1822        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1823        let file_reader = FileReader::try_open(
1824            file_scheduler.clone(),
1825            Some(ReaderProjection::from_column_names(&schema, &["score"]).unwrap()),
1826            Arc::<DecoderPlugins>::default(),
1827            &test_cache(),
1828            FileReaderOptions::default(),
1829        )
1830        .await
1831        .unwrap();
1832
1833        let batches = tokio::task::spawn_blocking(move || {
1834            file_reader
1835                .read_stream_projected_blocking(
1836                    lance_io::ReadBatchParams::Indices(UInt32Array::from(vec![0, 1, 2, 3, 4])),
1837                    total_rows as u32,
1838                    None,
1839                    FilterExpression::no_filter(),
1840                )
1841                .unwrap()
1842                .collect::<ArrowResult<Vec<_>>>()
1843                .unwrap()
1844        })
1845        .await
1846        .unwrap();
1847
1848        assert_eq!(batches.len(), 1);
1849        assert_eq!(batches[0].num_rows(), 5);
1850        assert_eq!(batches[0].num_columns(), 1);
1851    }
1852
1853    #[tokio::test(flavor = "multi_thread")]
1854    async fn test_drop_in_progress() {
1855        let fs = FsFixture::default();
1856        let WrittenFile { data, .. } = create_some_file(&fs, LanceFileVersion::V2_0).await;
1857        let total_rows = data.iter().map(|batch| batch.num_rows()).sum::<usize>();
1858
1859        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1860        let file_reader = FileReader::try_open(
1861            file_scheduler.clone(),
1862            None,
1863            Arc::<DecoderPlugins>::default(),
1864            &test_cache(),
1865            FileReaderOptions::default(),
1866        )
1867        .await
1868        .unwrap();
1869
1870        let mut batches = file_reader
1871            .read_stream(
1872                lance_io::ReadBatchParams::RangeFull,
1873                (total_rows / 10) as u32,
1874                16,
1875                FilterExpression::no_filter(),
1876            )
1877            .unwrap();
1878
1879        drop(file_reader);
1880
1881        let batch = batches.next().await.unwrap().unwrap();
1882        assert!(batch.num_rows() > 0);
1883
1884        // Drop in-progress scan
1885        drop(batches);
1886    }
1887
1888    #[tokio::test]
1889    async fn drop_while_scheduling() {
1890        // This is a bit of a white-box test, pokes at the internals.  We want to
1891        // test the case where the read stream is dropped before the scheduling
1892        // thread finishes.  We can't do that in a black-box fashion because the
1893        // scheduling thread runs in the background and there is no easy way to
1894        // pause / gate it.
1895
1896        // It's a regression for a bug where the scheduling thread would panic
1897        // if the stream was dropped before it finished.
1898
1899        let fs = FsFixture::default();
1900        let written_file = create_some_file(&fs, LanceFileVersion::V2_0).await;
1901        let total_rows = written_file
1902            .data
1903            .iter()
1904            .map(|batch| batch.num_rows())
1905            .sum::<usize>();
1906
1907        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1908        let file_reader = FileReader::try_open(
1909            file_scheduler.clone(),
1910            None,
1911            Arc::<DecoderPlugins>::default(),
1912            &test_cache(),
1913            FileReaderOptions::default(),
1914        )
1915        .await
1916        .unwrap();
1917
1918        let projection =
1919            ReaderProjection::from_whole_schema(&written_file.schema, LanceFileVersion::V2_0);
1920        let column_infos = file_reader
1921            .collect_columns_from_projection(&projection)
1922            .unwrap();
1923        let mut decode_scheduler = DecodeBatchScheduler::try_new(
1924            &projection.schema,
1925            &projection.column_indices,
1926            &column_infos,
1927            &vec![],
1928            total_rows as u64,
1929            Arc::<DecoderPlugins>::default(),
1930            file_reader.scheduler.clone(),
1931            test_cache(),
1932            &FilterExpression::no_filter(),
1933        )
1934        .await
1935        .unwrap();
1936
1937        let range = 0..total_rows as u64;
1938
1939        let (tx, rx) = mpsc::unbounded_channel();
1940
1941        // Simulate the stream / decoder being dropped
1942        drop(rx);
1943
1944        // Scheduling should not panic
1945        decode_scheduler.schedule_range(
1946            range,
1947            &FilterExpression::no_filter(),
1948            tx,
1949            file_reader.scheduler.clone(),
1950        )
1951    }
1952
1953    #[tokio::test]
1954    async fn test_global_buffers() {
1955        let fs = FsFixture::default();
1956
1957        let lance_schema =
1958            lance_core::datatypes::Schema::try_from(&ArrowSchema::new(vec![Field::new(
1959                "foo",
1960                DataType::Int32,
1961                true,
1962            )]))
1963            .unwrap();
1964
1965        let mut file_writer = FileWriter::try_new(
1966            fs.object_store.create(&fs.tmp_path).await.unwrap(),
1967            lance_schema.clone(),
1968            FileWriterOptions::default(),
1969        )
1970        .unwrap();
1971
1972        let test_bytes = Bytes::from_static(b"hello");
1973
1974        let buf_index = file_writer
1975            .add_global_buffer(test_bytes.clone())
1976            .await
1977            .unwrap();
1978
1979        assert_eq!(buf_index, 1);
1980
1981        file_writer.finish().await.unwrap();
1982
1983        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
1984        let file_reader = FileReader::try_open(
1985            file_scheduler.clone(),
1986            None,
1987            Arc::<DecoderPlugins>::default(),
1988            &test_cache(),
1989            FileReaderOptions::default(),
1990        )
1991        .await
1992        .unwrap();
1993
1994        let buf = file_reader.read_global_buffer(1).await.unwrap();
1995        assert_eq!(buf, test_bytes);
1996    }
1997}