lance_encoding/
decoder.rs

1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright The Lance Authors
3
4//! Utilities and traits for scheduling & decoding data
5//!
6//! Reading data involves two steps: scheduling and decoding.  The
7//! scheduling step is responsible for figuring out what data is needed
8//! and issuing the appropriate I/O requests.  The decoding step is
9//! responsible for taking the loaded data and turning it into Arrow
10//! arrays.
11//!
12//! # Scheduling
13//!
14//! Scheduling is split into [`self::FieldScheduler`] and [`self::PageScheduler`].
15//! There is one field scheduler for each output field, which may map to many
16//! columns of actual data.  A field scheduler is responsible for figuring out
17//! the order in which pages should be scheduled.  Field schedulers then delegate
18//! to page schedulers to figure out the I/O requests that need to be made for
19//! the page.
20//!
21//! Page schedulers also create the decoders that will be used to decode the
22//! scheduled data.
23//!
24//! # Decoding
25//!
26//! Decoders are split into [`self::PhysicalPageDecoder`] and
27//! [`self::LogicalPageDecoder`].  Note that both physical and logical decoding
28//! happens on a per-page basis.  There is no concept of a "field decoder" or
29//! "column decoder".
30//!
31//! The physical decoders handle lower level encodings.  They have a few advantages:
32//!
33//!  * They do not need to decode into an Arrow array and so they don't need
34//!    to be enveloped into the Arrow filesystem (e.g. Arrow doesn't have a
35//!    bit-packed type.  We can use variable-length binary but that is kind
36//!    of awkward)
37//!  * They can decode into an existing allocation.  This can allow for "page
38//!    bridging".  If we are trying to decode into a batch of 1024 rows and
39//!    the rows 0..1024 are spread across two pages then we can avoid a memory
40//!    copy by allocating once and decoding each page into the outer allocation.
41//!    (note: page bridging is not actually implemented yet)
42//!
43//! However, there are some limitations for physical decoders:
44//!
45//!  * They are constrained to a single column
46//!  * The API is more complex
47//!
48//! The logical decoders are designed to map one or more columns of Lance
49//! data into an Arrow array.
50//!
51//! Typically, a "logical encoding" will have both a logical decoder and a field scheduler.
52//! Meanwhile, a "physical encoding" will have a physical decoder but no corresponding field
53//! scheduler.git add --all
54//!
55//!
56//! # General notes
57//!
58//! Encodings are typically nested into each other to form a tree.  The top of the tree is
59//! the user requested schema.  Each field in that schema is assigned to one top-level logical
60//! encoding.  That encoding can then contain other logical encodings or physical encodings.
61//! Physical encodings can also contain other physical encodings.
62//!
63//! So, for example, a single field in the Arrow schema might have the type List<UInt32>
64//!
65//! The encoding tree could then be:
66//!
67//! root: List (logical encoding)
68//!  - indices: Primitive (logical encoding)
69//!    - column: Basic (physical encoding)
70//!      - validity: Bitmap (physical encoding)
71//!      - values: RLE (physical encoding)
72//!        - runs: Value (physical encoding)
73//!        - values: Value (physical encoding)
74//!  - items: Primitive (logical encoding)
75//!    - column: Basic (physical encoding)
76//!      - values: Value (physical encoding)
77//!
78//! Note that, in this example, root.items.column does not have a validity because there were
79//! no nulls in the page.
80//!
81//! ## Multiple buffers or multiple columns?
82//!
83//! Note that there are many different ways we can write encodings.  For example, we might
84//! store primitive fields in a single column with two buffers (one for validity and one for
85//! values)
86//!
87//! On the other hand, we could also store a primitive field as two different columns.  One
88//! that yields a non-nullable boolean array and one that yields a non-nullable array of items.
89//! Then we could combine these two arrays into a single array where the boolean array is the
90//! bitmap.  There are a few subtle differences between the approaches:
91//!
92//! * Storing things as multiple buffers within the same column is generally more efficient and
93//!   easier to schedule.  For example, in-batch coalescing is very easy but can only be done
94//!   on data that is in the same page.
95//! * When things are stored in multiple columns you have to worry about their pages not being
96//!   in sync.  In our previous validity / values example this means we might have to do some
97//!   memory copies to get the validity array and values arrays to be the same length as
98//!   decode.
99//! * When things are stored in a single column, projection is impossible.  For example, if we
100//!   tried to store all the struct fields in a single column with lots of buffers then we wouldn't
101//!   be able to read back individual fields of the struct.
102//!
103//! The fixed size list decoding is an interesting example because it is actually both a physical
104//! encoding and a logical encoding.  A fixed size list of a physical encoding is, itself, a physical
105//! encoding (e.g. a fixed size list of doubles).  However, a fixed size list of a logical encoding
106//! is a logical encoding (e.g. a fixed size list of structs).
107//!
108//! # The scheduling loop
109//!
110//! Reading a Lance file involves both scheduling and decoding.  Its generally expected that these
111//! will run as two separate threads.
112//!
113//! ```text
114//!
115//!                                    I/O PARALLELISM
116//!                       Issues
117//!                       Requests   ┌─────────────────┐
118//!                                  │                 │        Wait for
119//!                       ┌──────────►   I/O Service   ├─────►  Enough I/O ◄─┐
120//!                       │          │                 │        For batch    │
121//!                       │          └─────────────────┘             │3      │
122//!                       │                                          │       │
123//!                       │                                          │       │2
124//! ┌─────────────────────┴─┐                              ┌─────────▼───────┴┐
125//! │                       │                              │                  │Poll
126//! │       Batch Decode    │ Decode tasks sent via channel│   Batch Decode   │1
127//! │       Scheduler       ├─────────────────────────────►│   Stream         ◄─────
128//! │                       │                              │                  │
129//! └─────▲─────────────┬───┘                              └─────────┬────────┘
130//!       │             │                                            │4
131//!       │             │                                            │
132//!       └─────────────┘                                   ┌────────┴────────┐
133//!  Caller of schedule_range                Buffer polling │                 │
134//!  will be scheduler thread                to achieve CPU │ Decode Batch    ├────►
135//!  and schedule one decode                 parallelism    │ Task            │
136//!  task (and all needed I/O)               (thread per    │                 │
137//!  per logical page                         batch)        └─────────────────┘
138//! ```
139//!
140//! The scheduling thread will work through the file from the
141//! start to the end as quickly as possible.  Data is scheduled one page at a time in a row-major
142//! fashion.  For example, imagine we have a file with the following page structure:
143//!
144//! ```text
145//! Score (Float32)     | C0P0 |
146//! Id (16-byte UUID)   | C1P0 | C1P1 | C1P2 | C1P3 |
147//! Vector (4096 bytes) | C2P0 | C2P1 | C2P2 | C2P3 | .. | C2P1024 |
148//! ```
149//!
150//! This would be quite common as each of these pages has the same number of bytes.  Let's pretend
151//! each page is 1MiB and so there are 256Ki rows of data.  Each page of `Score` has 256Ki rows.
152//! Each page of `Id` has 64Ki rows.  Each page of `Vector` has 256 rows.  The scheduler would then
153//! schedule in the following order:
154//!
155//! C0 P0
156//! C1 P0
157//! C2 P0
158//! C2 P1
159//! ... (254 pages omitted)
160//! C2 P255
161//! C1 P1
162//! C2 P256
163//! ... (254 pages omitted)
164//! C2 P511
165//! C1 P2
166//! C2 P512
167//! ... (254 pages omitted)
168//! C2 P767
169//! C1 P3
170//! C2 P768
171//! ... (254 pages omitted)
172//! C2 P1024
173//!
174//! This is the ideal scheduling order because it means we can decode complete rows as quickly as possible.
175//! Note that the scheduler thread does not need to wait for I/O to happen at any point.  As soon as it starts
176//! it will start scheduling one page of I/O after another until it has scheduled the entire file's worth of
177//! I/O.  This is slightly different than other file readers which have "row group parallelism" and will
178//! typically only schedule X row groups worth of reads at a time.
179//!
180//! In the near future there will be a backpressure mechanism and so it may need to stop/pause if the compute
181//! falls behind.
182//!
183//! ## Indirect I/O
184//!
185//! Regrettably, there are times where we cannot know exactly what data we need until we have partially decoded
186//! the file.  This happens when we have variable sized list data.  In that case the scheduling task for that
187//! page will only schedule the first part of the read (loading the list offsets).  It will then immediately
188//! spawn a new tokio task to wait for that I/O and decode the list offsets.  That follow-up task is not part
189//! of the scheduling loop or the decode loop.  It is a free task.  Once the list offsets are decoded we submit
190//! a follow-up I/O task.  This task is scheduled at a high priority because the decoder is going to need it soon.
191//!
192//! # The decode loop
193//!
194//! As soon as the scheduler starts we can start decoding.  Each time we schedule a page we
195//! push a decoder for that page's data into a channel.  The decode loop
196//! ([`BatchDecodeStream`]) reads from that channel.  Each time it receives a decoder it
197//! waits until the decoder has all of its data.  Then it grabs the next decoder.  Once it has
198//! enough loaded decoders to complete a batch worth of rows it will spawn a "decode batch task".
199//!
200//! These batch decode tasks perform the actual CPU work of decoding the loaded data into Arrow
201//! arrays.  This may involve signifciant CPU processing like decompression or arithmetic in order
202//! to restore the data to its correct in-memory representation.
203//!
204//! ## Batch size
205//!
206//! The `BatchDecodeStream` is configured with a batch size.  This does not need to have any
207//! relation to the page size(s) used to write the data.  This keeps our compute work completely
208//! independent of our I/O work.  We suggest using small batch sizes:
209//!
210//!  * Batches should fit in CPU cache (at least L3)
211//!  * More batches means more opportunity for parallelism
212//!  * The "batch overhead" is very small in Lance compared to other formats because it has no
213//!    relation to the way the data is stored.
214
215use std::collections::VecDeque;
216use std::sync::Once;
217use std::{ops::Range, sync::Arc};
218
219use arrow_array::cast::AsArray;
220use arrow_array::{ArrayRef, RecordBatch, RecordBatchIterator, RecordBatchReader};
221use arrow_schema::{ArrowError, DataType, Field as ArrowField, Fields, Schema as ArrowSchema};
222use bytes::Bytes;
223use futures::future::{maybe_done, BoxFuture, MaybeDone};
224use futures::stream::{self, BoxStream};
225use futures::{FutureExt, StreamExt};
226use lance_arrow::DataTypeExt;
227use lance_core::cache::{CapacityMode, FileMetadataCache};
228use lance_core::datatypes::{Field, Schema, BLOB_DESC_LANCE_FIELD};
229use log::{debug, trace, warn};
230use snafu::location;
231use tokio::sync::mpsc::error::SendError;
232use tokio::sync::mpsc::{self, unbounded_channel};
233
234use lance_core::{ArrowResult, Error, Result};
235use tracing::instrument;
236
237use crate::buffer::LanceBuffer;
238use crate::data::{DataBlock, FixedWidthDataBlock, VariableWidthBlock};
239use crate::encoder::{values_column_encoding, EncodedBatch};
240use crate::encodings::logical::binary::BinaryFieldScheduler;
241use crate::encodings::logical::blob::BlobFieldScheduler;
242use crate::encodings::logical::list::{
243    ListFieldScheduler, OffsetPageInfo, StructuralListScheduler,
244};
245use crate::encodings::logical::primitive::{
246    PrimitiveFieldScheduler, StructuralPrimitiveFieldScheduler,
247};
248use crate::encodings::logical::r#struct::{
249    SimpleStructDecoder, SimpleStructScheduler, StructuralStructDecoder, StructuralStructScheduler,
250};
251use crate::encodings::physical::binary::{
252    BinaryBlockDecompressor, BinaryMiniBlockDecompressor, VariableDecoder,
253};
254use crate::encodings::physical::bitpack_fastlanes::BitpackMiniBlockDecompressor;
255use crate::encodings::physical::fsst::{FsstMiniBlockDecompressor, FsstPerValueDecompressor};
256use crate::encodings::physical::struct_encoding::PackedStructFixedWidthMiniBlockDecompressor;
257use crate::encodings::physical::value::{ConstantDecompressor, ValueDecompressor};
258use crate::encodings::physical::{ColumnBuffers, FileBuffers};
259use crate::format::pb::{self, column_encoding};
260use crate::repdef::{CompositeRepDefUnraveler, RepDefUnraveler};
261use crate::version::LanceFileVersion;
262use crate::{BufferScheduler, EncodingsIo};
263
264// If users are getting batches over 10MiB large then it's time to reduce the batch size
265const BATCH_SIZE_BYTES_WARNING: u64 = 10 * 1024 * 1024;
266
267/// Top-level encoding message for a page.  Wraps both the
268/// legacy pb::ArrayEncoding and the newer pb::PageLayout
269///
270/// A file should only use one or the other and never both.
271/// 2.0 decoders can always assume this is pb::ArrayEncoding
272/// and 2.1+ decoders can always assume this is pb::PageLayout
273#[derive(Debug)]
274pub enum PageEncoding {
275    Legacy(pb::ArrayEncoding),
276    Structural(pb::PageLayout),
277}
278
279impl PageEncoding {
280    pub fn as_legacy(&self) -> &pb::ArrayEncoding {
281        match self {
282            Self::Legacy(enc) => enc,
283            Self::Structural(_) => panic!("Expected a legacy encoding"),
284        }
285    }
286
287    pub fn as_structural(&self) -> &pb::PageLayout {
288        match self {
289            Self::Structural(enc) => enc,
290            Self::Legacy(_) => panic!("Expected a structural encoding"),
291        }
292    }
293
294    pub fn is_structural(&self) -> bool {
295        matches!(self, Self::Structural(_))
296    }
297}
298
299/// Metadata describing a page in a file
300///
301/// This is typically created by reading the metadata section of a Lance file
302#[derive(Debug)]
303pub struct PageInfo {
304    /// The number of rows in the page
305    pub num_rows: u64,
306    /// The priority (top level row number) of the page
307    ///
308    /// This is only set in 2.1 files and will be 0 for 2.0 files
309    pub priority: u64,
310    /// The encoding that explains the buffers in the page
311    pub encoding: PageEncoding,
312    /// The offsets and sizes of the buffers in the file
313    pub buffer_offsets_and_sizes: Arc<[(u64, u64)]>,
314}
315
316/// Metadata describing a column in a file
317///
318/// This is typically created by reading the metadata section of a Lance file
319#[derive(Debug, Clone)]
320pub struct ColumnInfo {
321    /// The index of the column in the file
322    pub index: u32,
323    /// The metadata for each page in the column
324    pub page_infos: Arc<[PageInfo]>,
325    /// File positions and their sizes of the column-level buffers
326    pub buffer_offsets_and_sizes: Arc<[(u64, u64)]>,
327    pub encoding: pb::ColumnEncoding,
328}
329
330impl ColumnInfo {
331    /// Create a new instance
332    pub fn new(
333        index: u32,
334        page_infos: Arc<[PageInfo]>,
335        buffer_offsets_and_sizes: Vec<(u64, u64)>,
336        encoding: pb::ColumnEncoding,
337    ) -> Self {
338        Self {
339            index,
340            page_infos,
341            buffer_offsets_and_sizes: buffer_offsets_and_sizes.into_boxed_slice().into(),
342            encoding,
343        }
344    }
345
346    pub fn is_structural(&self) -> bool {
347        self.page_infos
348            // Can just look at the first since all should be the same
349            .first()
350            .map(|page| page.encoding.is_structural())
351            .unwrap_or(false)
352    }
353}
354
355enum RootScheduler {
356    Structural(Box<dyn StructuralFieldScheduler>),
357    Legacy(Arc<dyn FieldScheduler>),
358}
359
360impl RootScheduler {
361    fn as_legacy(&self) -> &Arc<dyn FieldScheduler> {
362        match self {
363            Self::Structural(_) => panic!("Expected a legacy scheduler"),
364            Self::Legacy(s) => s,
365        }
366    }
367
368    fn as_structural(&self) -> &dyn StructuralFieldScheduler {
369        match self {
370            Self::Structural(s) => s.as_ref(),
371            Self::Legacy(_) => panic!("Expected a structural scheduler"),
372        }
373    }
374}
375
376/// The scheduler for decoding batches
377///
378/// Lance decoding is done in two steps, scheduling, and decoding.  The
379/// scheduling tends to be lightweight and should quickly figure what data
380/// is needed from the disk issue the appropriate I/O requests.  A decode task is
381/// created to eventually decode the data (once it is loaded) and scheduling
382/// moves on to scheduling the next page.
383///
384/// Meanwhile, it's expected that a decode stream will be setup to run at the
385/// same time.  Decode tasks take the data that is loaded and turn it into
386/// Arrow arrays.
387///
388/// This approach allows us to keep our I/O parallelism and CPU parallelism
389/// completely separate since those are often two very different values.
390///
391/// Backpressure should be achieved via the I/O service.  Requests that are
392/// issued will pile up if the decode stream is not polling quickly enough.
393/// The [`crate::EncodingsIo::submit_request`] function should return a pending
394/// future once there are too many I/O requests in flight.
395///
396/// TODO: Implement backpressure
397pub struct DecodeBatchScheduler {
398    root_scheduler: RootScheduler,
399    pub root_fields: Fields,
400    cache: Arc<FileMetadataCache>,
401}
402
403pub struct ColumnInfoIter<'a> {
404    column_infos: Vec<Arc<ColumnInfo>>,
405    column_indices: &'a [u32],
406    column_info_pos: usize,
407    column_indices_pos: usize,
408}
409
410impl<'a> ColumnInfoIter<'a> {
411    pub fn new(column_infos: Vec<Arc<ColumnInfo>>, column_indices: &'a [u32]) -> Self {
412        let initial_pos = column_indices[0] as usize;
413        Self {
414            column_infos,
415            column_indices,
416            column_info_pos: initial_pos,
417            column_indices_pos: 0,
418        }
419    }
420
421    pub fn peek(&self) -> &Arc<ColumnInfo> {
422        &self.column_infos[self.column_info_pos]
423    }
424
425    pub fn peek_transform(&mut self, transform: impl FnOnce(Arc<ColumnInfo>) -> Arc<ColumnInfo>) {
426        let column_info = self.column_infos[self.column_info_pos].clone();
427        let transformed = transform(column_info);
428        self.column_infos[self.column_info_pos] = transformed;
429    }
430
431    pub fn expect_next(&mut self) -> Result<&Arc<ColumnInfo>> {
432        self.next().ok_or_else(|| {
433            Error::invalid_input(
434                "there were more fields in the schema than provided column indices / infos",
435                location!(),
436            )
437        })
438    }
439
440    fn next(&mut self) -> Option<&Arc<ColumnInfo>> {
441        if self.column_info_pos < self.column_infos.len() {
442            let info = &self.column_infos[self.column_info_pos];
443            self.column_info_pos += 1;
444            Some(info)
445        } else {
446            None
447        }
448    }
449
450    pub(crate) fn next_top_level(&mut self) {
451        self.column_indices_pos += 1;
452        if self.column_indices_pos < self.column_indices.len() {
453            self.column_info_pos = self.column_indices[self.column_indices_pos] as usize;
454        } else {
455            self.column_info_pos = self.column_infos.len();
456        }
457    }
458}
459
460pub trait MiniBlockDecompressor: std::fmt::Debug + Send + Sync {
461    fn decompress(&self, data: LanceBuffer, num_values: u64) -> Result<DataBlock>;
462}
463
464pub trait FixedPerValueDecompressor: std::fmt::Debug + Send + Sync {
465    /// Decompress one or more values
466    fn decompress(&self, data: FixedWidthDataBlock) -> Result<DataBlock>;
467    /// The number of bits in each value
468    ///
469    /// Currently (and probably long term) this must be a multiple of 8
470    fn bits_per_value(&self) -> u64;
471}
472
473pub trait VariablePerValueDecompressor: std::fmt::Debug + Send + Sync {
474    /// Decompress one or more values
475    fn decompress(&self, data: VariableWidthBlock) -> Result<DataBlock>;
476}
477
478pub trait BlockDecompressor: std::fmt::Debug + Send + Sync {
479    fn decompress(&self, data: LanceBuffer) -> Result<DataBlock>;
480}
481
482pub trait DecompressorStrategy: std::fmt::Debug + Send + Sync {
483    fn create_miniblock_decompressor(
484        &self,
485        description: &pb::ArrayEncoding,
486    ) -> Result<Box<dyn MiniBlockDecompressor>>;
487
488    fn create_fixed_per_value_decompressor(
489        &self,
490        description: &pb::ArrayEncoding,
491    ) -> Result<Box<dyn FixedPerValueDecompressor>>;
492
493    fn create_variable_per_value_decompressor(
494        &self,
495        description: &pb::ArrayEncoding,
496    ) -> Result<Box<dyn VariablePerValueDecompressor>>;
497
498    fn create_block_decompressor(
499        &self,
500        description: &pb::ArrayEncoding,
501    ) -> Result<Box<dyn BlockDecompressor>>;
502}
503
504#[derive(Debug, Default)]
505pub struct CoreDecompressorStrategy {}
506
507impl DecompressorStrategy for CoreDecompressorStrategy {
508    fn create_miniblock_decompressor(
509        &self,
510        description: &pb::ArrayEncoding,
511    ) -> Result<Box<dyn MiniBlockDecompressor>> {
512        match description.array_encoding.as_ref().unwrap() {
513            pb::array_encoding::ArrayEncoding::Flat(flat) => {
514                Ok(Box::new(ValueDecompressor::new(flat)))
515            }
516            pb::array_encoding::ArrayEncoding::Bitpack2(description) => {
517                Ok(Box::new(BitpackMiniBlockDecompressor::new(description)))
518            }
519            pb::array_encoding::ArrayEncoding::Variable(_) => {
520                Ok(Box::new(BinaryMiniBlockDecompressor::default()))
521            }
522            pb::array_encoding::ArrayEncoding::Fsst(description) => {
523                Ok(Box::new(FsstMiniBlockDecompressor::new(description)))
524            }
525            pb::array_encoding::ArrayEncoding::PackedStructFixedWidthMiniBlock(description) => {
526                Ok(Box::new(PackedStructFixedWidthMiniBlockDecompressor::new(
527                    description,
528                )))
529            }
530            _ => todo!(),
531        }
532    }
533
534    fn create_fixed_per_value_decompressor(
535        &self,
536        description: &pb::ArrayEncoding,
537    ) -> Result<Box<dyn FixedPerValueDecompressor>> {
538        match description.array_encoding.as_ref().unwrap() {
539            pb::array_encoding::ArrayEncoding::Flat(flat) => {
540                Ok(Box::new(ValueDecompressor::new(flat)))
541            }
542            _ => todo!("fixed-per-value decompressor for {:?}", description),
543        }
544    }
545
546    fn create_variable_per_value_decompressor(
547        &self,
548        description: &pb::ArrayEncoding,
549    ) -> Result<Box<dyn VariablePerValueDecompressor>> {
550        match *description.array_encoding.as_ref().unwrap() {
551            pb::array_encoding::ArrayEncoding::Variable(variable) => {
552                assert!(variable.bits_per_offset < u8::MAX as u32);
553                Ok(Box::new(VariableDecoder::default()))
554            }
555            pb::array_encoding::ArrayEncoding::Fsst(ref fsst) => {
556                Ok(Box::new(FsstPerValueDecompressor::new(
557                    LanceBuffer::from_bytes(fsst.symbol_table.clone(), 1),
558                    Box::new(VariableDecoder::default()),
559                )))
560            }
561            _ => todo!("variable-per-value decompressor for {:?}", description),
562        }
563    }
564
565    fn create_block_decompressor(
566        &self,
567        description: &pb::ArrayEncoding,
568    ) -> Result<Box<dyn BlockDecompressor>> {
569        match description.array_encoding.as_ref().unwrap() {
570            pb::array_encoding::ArrayEncoding::Flat(flat) => {
571                Ok(Box::new(ValueDecompressor::new(flat)))
572            }
573            pb::array_encoding::ArrayEncoding::Constant(constant) => {
574                let scalar = LanceBuffer::from_bytes(constant.value.clone(), 1);
575                Ok(Box::new(ConstantDecompressor::new(
576                    scalar,
577                    constant.num_values,
578                )))
579            }
580            pb::array_encoding::ArrayEncoding::Variable(_) => {
581                Ok(Box::new(BinaryBlockDecompressor::default()))
582            }
583            _ => todo!(),
584        }
585    }
586}
587
588/// The core decoder strategy handles all the various Arrow types
589#[derive(Debug)]
590pub struct CoreFieldDecoderStrategy {
591    pub validate_data: bool,
592    pub decompressor_strategy: Arc<dyn DecompressorStrategy>,
593}
594
595impl Default for CoreFieldDecoderStrategy {
596    fn default() -> Self {
597        Self {
598            validate_data: false,
599            decompressor_strategy: Arc::new(CoreDecompressorStrategy {}),
600        }
601    }
602}
603
604impl CoreFieldDecoderStrategy {
605    /// This is just a sanity check to ensure there is no "wrapped encodings"
606    /// that haven't been handled.
607    fn ensure_values_encoded(column_info: &ColumnInfo, field_name: &str) -> Result<()> {
608        let column_encoding = column_info
609            .encoding
610            .column_encoding
611            .as_ref()
612            .ok_or_else(|| {
613                Error::invalid_input(
614                    format!(
615                        "the column at index {} was missing a ColumnEncoding",
616                        column_info.index
617                    ),
618                    location!(),
619                )
620            })?;
621        if matches!(
622            column_encoding,
623            pb::column_encoding::ColumnEncoding::Values(_)
624        ) {
625            Ok(())
626        } else {
627            Err(Error::invalid_input(format!("the column at index {} mapping to the input field {} has column encoding {:?} and no decoder is registered to handle it", column_info.index, field_name, column_encoding), location!()))
628        }
629    }
630
631    fn is_primitive(data_type: &DataType) -> bool {
632        if data_type.is_primitive() {
633            true
634        } else {
635            match data_type {
636                // DataType::is_primitive doesn't consider these primitive but we do
637                DataType::Boolean | DataType::Null | DataType::FixedSizeBinary(_) => true,
638                DataType::FixedSizeList(inner, _) => Self::is_primitive(inner.data_type()),
639                _ => false,
640            }
641        }
642    }
643
644    fn create_primitive_scheduler(
645        &self,
646        field: &Field,
647        column: &ColumnInfo,
648        buffers: FileBuffers,
649    ) -> Result<Box<dyn FieldScheduler>> {
650        Self::ensure_values_encoded(column, &field.name)?;
651        // Primitive fields map to a single column
652        let column_buffers = ColumnBuffers {
653            file_buffers: buffers,
654            positions_and_sizes: &column.buffer_offsets_and_sizes,
655        };
656        Ok(Box::new(PrimitiveFieldScheduler::new(
657            column.index,
658            field.data_type(),
659            column.page_infos.clone(),
660            column_buffers,
661            self.validate_data,
662        )))
663    }
664
665    /// Helper method to verify the page encoding of a struct header column
666    fn check_simple_struct(column_info: &ColumnInfo, field_name: &str) -> Result<()> {
667        Self::ensure_values_encoded(column_info, field_name)?;
668        if column_info.page_infos.len() != 1 {
669            return Err(Error::InvalidInput { source: format!("Due to schema we expected a struct column but we received a column with {} pages and right now we only support struct columns with 1 page", column_info.page_infos.len()).into(), location: location!() });
670        }
671        let encoding = &column_info.page_infos[0].encoding;
672        match encoding.as_legacy().array_encoding.as_ref().unwrap() {
673            pb::array_encoding::ArrayEncoding::Struct(_) => Ok(()),
674            _ => Err(Error::InvalidInput { source: format!("Expected a struct encoding because we have a struct field in the schema but got the encoding {:?}", encoding).into(), location: location!() }),
675        }
676    }
677
678    fn check_packed_struct(column_info: &ColumnInfo) -> bool {
679        let encoding = &column_info.page_infos[0].encoding;
680        matches!(
681            encoding.as_legacy().array_encoding.as_ref().unwrap(),
682            pb::array_encoding::ArrayEncoding::PackedStruct(_)
683        )
684    }
685
686    fn create_list_scheduler(
687        &self,
688        list_field: &Field,
689        column_infos: &mut ColumnInfoIter,
690        buffers: FileBuffers,
691        offsets_column: &ColumnInfo,
692    ) -> Result<Box<dyn FieldScheduler>> {
693        Self::ensure_values_encoded(offsets_column, &list_field.name)?;
694        let offsets_column_buffers = ColumnBuffers {
695            file_buffers: buffers,
696            positions_and_sizes: &offsets_column.buffer_offsets_and_sizes,
697        };
698        let items_scheduler =
699            self.create_legacy_field_scheduler(&list_field.children[0], column_infos, buffers)?;
700
701        let (inner_infos, null_offset_adjustments): (Vec<_>, Vec<_>) = offsets_column
702            .page_infos
703            .iter()
704            .filter(|offsets_page| offsets_page.num_rows > 0)
705            .map(|offsets_page| {
706                if let Some(pb::array_encoding::ArrayEncoding::List(list_encoding)) =
707                    &offsets_page.encoding.as_legacy().array_encoding
708                {
709                    let inner = PageInfo {
710                        buffer_offsets_and_sizes: offsets_page.buffer_offsets_and_sizes.clone(),
711                        encoding: PageEncoding::Legacy(
712                            list_encoding.offsets.as_ref().unwrap().as_ref().clone(),
713                        ),
714                        num_rows: offsets_page.num_rows,
715                        priority: 0,
716                    };
717                    (
718                        inner,
719                        OffsetPageInfo {
720                            offsets_in_page: offsets_page.num_rows,
721                            null_offset_adjustment: list_encoding.null_offset_adjustment,
722                            num_items_referenced_by_page: list_encoding.num_items,
723                        },
724                    )
725                } else {
726                    // TODO: Should probably return Err here
727                    panic!("Expected a list column");
728                }
729            })
730            .unzip();
731        let inner = Arc::new(PrimitiveFieldScheduler::new(
732            offsets_column.index,
733            DataType::UInt64,
734            Arc::from(inner_infos.into_boxed_slice()),
735            offsets_column_buffers,
736            self.validate_data,
737        )) as Arc<dyn FieldScheduler>;
738        let items_field = match list_field.data_type() {
739            DataType::List(inner) => inner,
740            DataType::LargeList(inner) => inner,
741            _ => unreachable!(),
742        };
743        let offset_type = if matches!(list_field.data_type(), DataType::List(_)) {
744            DataType::Int32
745        } else {
746            DataType::Int64
747        };
748        Ok(Box::new(ListFieldScheduler::new(
749            inner,
750            items_scheduler.into(),
751            items_field,
752            offset_type,
753            null_offset_adjustments,
754        )))
755    }
756
757    fn unwrap_blob(column_info: &ColumnInfo) -> Option<ColumnInfo> {
758        if let column_encoding::ColumnEncoding::Blob(blob) =
759            column_info.encoding.column_encoding.as_ref().unwrap()
760        {
761            let mut column_info = column_info.clone();
762            column_info.encoding = blob.inner.as_ref().unwrap().as_ref().clone();
763            Some(column_info)
764        } else {
765            None
766        }
767    }
768
769    fn items_per_row(data_type: &DataType) -> u64 {
770        match data_type {
771            DataType::FixedSizeList(inner, dimension) => {
772                Self::items_per_row(inner.data_type()) * *dimension as u64
773            }
774            _ => 1,
775        }
776    }
777
778    fn create_structural_field_scheduler(
779        &self,
780        field: &Field,
781        column_infos: &mut ColumnInfoIter,
782    ) -> Result<Box<dyn StructuralFieldScheduler>> {
783        let data_type = field.data_type();
784        if Self::is_primitive(&data_type) {
785            let column_info = column_infos.expect_next()?;
786            let items_per_row = Self::items_per_row(&data_type);
787            let scheduler = Box::new(StructuralPrimitiveFieldScheduler::try_new(
788                column_info.as_ref(),
789                items_per_row,
790                self.decompressor_strategy.as_ref(),
791            )?);
792
793            // advance to the next top level column
794            column_infos.next_top_level();
795
796            return Ok(scheduler);
797        }
798        match &data_type {
799            DataType::Struct(fields) => {
800                if field.is_packed_struct() {
801                    let column_info = column_infos.expect_next()?;
802                    let scheduler = Box::new(StructuralPrimitiveFieldScheduler::try_new(
803                        column_info.as_ref(),
804                        1, // items_per_row is always 1, any FSL will get transposed into 1 row
805                        self.decompressor_strategy.as_ref(),
806                    )?);
807
808                    // advance to the next top level column
809                    column_infos.next_top_level();
810
811                    return Ok(scheduler);
812                }
813                let mut child_schedulers = Vec::with_capacity(field.children.len());
814                for field in field.children.iter() {
815                    let field_scheduler =
816                        self.create_structural_field_scheduler(field, column_infos)?;
817                    child_schedulers.push(field_scheduler);
818                }
819
820                let fields = fields.clone();
821                Ok(
822                    Box::new(StructuralStructScheduler::new(child_schedulers, fields))
823                        as Box<dyn StructuralFieldScheduler>,
824                )
825            }
826            DataType::Binary | DataType::Utf8 => {
827                let column_info = column_infos.expect_next()?;
828                let scheduler = Box::new(StructuralPrimitiveFieldScheduler::try_new(
829                    column_info.as_ref(),
830                    /*items_per_row=*/ 1,
831                    self.decompressor_strategy.as_ref(),
832                )?);
833                column_infos.next_top_level();
834                Ok(scheduler)
835            }
836            DataType::List(_) | DataType::LargeList(_) => {
837                let child = field
838                    .children
839                    .first()
840                    .expect("List field must have a child");
841                let child_scheduler =
842                    self.create_structural_field_scheduler(child, column_infos)?;
843                Ok(Box::new(StructuralListScheduler::new(child_scheduler))
844                    as Box<dyn StructuralFieldScheduler>)
845            }
846            _ => todo!(),
847        }
848    }
849
850    fn create_legacy_field_scheduler(
851        &self,
852        field: &Field,
853        column_infos: &mut ColumnInfoIter,
854        buffers: FileBuffers,
855    ) -> Result<Box<dyn FieldScheduler>> {
856        let data_type = field.data_type();
857        if Self::is_primitive(&data_type) {
858            let column_info = column_infos.expect_next()?;
859            let scheduler = self.create_primitive_scheduler(field, column_info, buffers)?;
860            return Ok(scheduler);
861        } else if data_type.is_binary_like() {
862            let column_info = column_infos.next().unwrap().clone();
863            // Column is blob and user is asking for binary data
864            if let Some(blob_col) = Self::unwrap_blob(column_info.as_ref()) {
865                let desc_scheduler =
866                    self.create_primitive_scheduler(&BLOB_DESC_LANCE_FIELD, &blob_col, buffers)?;
867                let blob_scheduler = Box::new(BlobFieldScheduler::new(desc_scheduler.into()));
868                return Ok(blob_scheduler);
869            }
870            if let Some(page_info) = column_info.page_infos.first() {
871                if matches!(
872                    page_info.encoding.as_legacy(),
873                    pb::ArrayEncoding {
874                        array_encoding: Some(pb::array_encoding::ArrayEncoding::List(..))
875                    }
876                ) {
877                    let list_type = if matches!(data_type, DataType::Utf8 | DataType::Binary) {
878                        DataType::List(Arc::new(ArrowField::new("item", DataType::UInt8, false)))
879                    } else {
880                        DataType::LargeList(Arc::new(ArrowField::new(
881                            "item",
882                            DataType::UInt8,
883                            false,
884                        )))
885                    };
886                    let list_field = Field::try_from(ArrowField::new(
887                        field.name.clone(),
888                        list_type,
889                        field.nullable,
890                    ))
891                    .unwrap();
892                    let list_scheduler = self.create_list_scheduler(
893                        &list_field,
894                        column_infos,
895                        buffers,
896                        &column_info,
897                    )?;
898                    let binary_scheduler = Box::new(BinaryFieldScheduler::new(
899                        list_scheduler.into(),
900                        field.data_type(),
901                    ));
902                    return Ok(binary_scheduler);
903                } else {
904                    let scheduler =
905                        self.create_primitive_scheduler(field, &column_info, buffers)?;
906                    return Ok(scheduler);
907                }
908            } else {
909                return self.create_primitive_scheduler(field, &column_info, buffers);
910            }
911        }
912        match &data_type {
913            DataType::FixedSizeList(inner, _dimension) => {
914                // A fixed size list column could either be a physical or a logical decoder
915                // depending on the child data type.
916                if Self::is_primitive(inner.data_type()) {
917                    let primitive_col = column_infos.expect_next()?;
918                    let scheduler =
919                        self.create_primitive_scheduler(field, primitive_col, buffers)?;
920                    Ok(scheduler)
921                } else {
922                    todo!()
923                }
924            }
925            DataType::Dictionary(_key_type, value_type) => {
926                if Self::is_primitive(value_type) || value_type.is_binary_like() {
927                    let primitive_col = column_infos.expect_next()?;
928                    let scheduler =
929                        self.create_primitive_scheduler(field, primitive_col, buffers)?;
930                    Ok(scheduler)
931                } else {
932                    Err(Error::NotSupported {
933                        source: format!(
934                            "No way to decode into a dictionary field of type {}",
935                            value_type
936                        )
937                        .into(),
938                        location: location!(),
939                    })
940                }
941            }
942            DataType::List(_) | DataType::LargeList(_) => {
943                let offsets_column = column_infos.expect_next()?.clone();
944                column_infos.next_top_level();
945                self.create_list_scheduler(field, column_infos, buffers, &offsets_column)
946            }
947            DataType::Struct(fields) => {
948                let column_info = column_infos.expect_next()?;
949
950                // Column is blob and user is asking for descriptions
951                if let Some(blob_col) = Self::unwrap_blob(column_info.as_ref()) {
952                    // Can use primitive scheduler here since descriptions are always packed struct
953                    return self.create_primitive_scheduler(field, &blob_col, buffers);
954                }
955
956                if Self::check_packed_struct(column_info) {
957                    // use packed struct encoding
958                    self.create_primitive_scheduler(field, column_info, buffers)
959                } else {
960                    // use default struct encoding
961                    Self::check_simple_struct(column_info, &field.name).unwrap();
962                    let mut child_schedulers = Vec::with_capacity(field.children.len());
963                    for field in &field.children {
964                        column_infos.next_top_level();
965                        let field_scheduler =
966                            self.create_legacy_field_scheduler(field, column_infos, buffers)?;
967                        child_schedulers.push(Arc::from(field_scheduler));
968                    }
969
970                    let fields = fields.clone();
971                    Ok(Box::new(SimpleStructScheduler::new(
972                        child_schedulers,
973                        fields,
974                    )))
975                }
976            }
977            // TODO: Still need support for RLE
978            _ => todo!(),
979        }
980    }
981}
982
983/// Create's a dummy ColumnInfo for the root column
984fn root_column(num_rows: u64) -> ColumnInfo {
985    let num_root_pages = num_rows.div_ceil(u32::MAX as u64);
986    let final_page_num_rows = num_rows % (u32::MAX as u64);
987    let root_pages = (0..num_root_pages)
988        .map(|i| PageInfo {
989            num_rows: if i == num_root_pages - 1 {
990                final_page_num_rows
991            } else {
992                u64::MAX
993            },
994            encoding: PageEncoding::Legacy(pb::ArrayEncoding {
995                array_encoding: Some(pb::array_encoding::ArrayEncoding::Struct(
996                    pb::SimpleStruct {},
997                )),
998            }),
999            priority: 0, // not used in legacy scheduler
1000            buffer_offsets_and_sizes: Arc::new([]),
1001        })
1002        .collect::<Vec<_>>();
1003    ColumnInfo {
1004        buffer_offsets_and_sizes: Arc::new([]),
1005        encoding: values_column_encoding(),
1006        index: u32::MAX,
1007        page_infos: Arc::from(root_pages),
1008    }
1009}
1010
1011pub enum RootDecoder {
1012    Structural(StructuralStructDecoder),
1013    Legacy(SimpleStructDecoder),
1014}
1015
1016impl RootDecoder {
1017    pub fn into_structural(self) -> StructuralStructDecoder {
1018        match self {
1019            Self::Structural(decoder) => decoder,
1020            Self::Legacy(_) => panic!("Expected a structural decoder"),
1021        }
1022    }
1023
1024    pub fn into_legacy(self) -> SimpleStructDecoder {
1025        match self {
1026            Self::Legacy(decoder) => decoder,
1027            Self::Structural(_) => panic!("Expected a legacy decoder"),
1028        }
1029    }
1030}
1031
1032impl DecodeBatchScheduler {
1033    /// Creates a new decode scheduler with the expected schema and the column
1034    /// metadata of the file.
1035    #[allow(clippy::too_many_arguments)]
1036    pub async fn try_new<'a>(
1037        schema: &'a Schema,
1038        column_indices: &[u32],
1039        column_infos: &[Arc<ColumnInfo>],
1040        file_buffer_positions_and_sizes: &'a Vec<(u64, u64)>,
1041        num_rows: u64,
1042        _decoder_plugins: Arc<DecoderPlugins>,
1043        io: Arc<dyn EncodingsIo>,
1044        cache: Arc<FileMetadataCache>,
1045        filter: &FilterExpression,
1046    ) -> Result<Self> {
1047        assert!(num_rows > 0);
1048        let buffers = FileBuffers {
1049            positions_and_sizes: file_buffer_positions_and_sizes,
1050        };
1051        let arrow_schema = ArrowSchema::from(schema);
1052        let root_fields = arrow_schema.fields().clone();
1053        let root_type = DataType::Struct(root_fields.clone());
1054        let mut root_field = Field::try_from(&ArrowField::new("root", root_type, false))?;
1055        // root_field.children and schema.fields should be identical at this point but the latter
1056        // has field ids and the former does not.  This line restores that.
1057        // TODO:  Is there another way to create the root field without forcing a trip through arrow?
1058        root_field.children.clone_from(&schema.fields);
1059        root_field
1060            .metadata
1061            .insert("__lance_decoder_root".to_string(), "true".to_string());
1062
1063        if column_infos[0].is_structural() {
1064            let mut column_iter = ColumnInfoIter::new(column_infos.to_vec(), column_indices);
1065
1066            let mut root_scheduler = CoreFieldDecoderStrategy::default()
1067                .create_structural_field_scheduler(&root_field, &mut column_iter)?;
1068
1069            let context = SchedulerContext::new(io, cache.clone());
1070            root_scheduler.initialize(filter, &context).await?;
1071
1072            Ok(Self {
1073                root_scheduler: RootScheduler::Structural(root_scheduler),
1074                root_fields,
1075                cache,
1076            })
1077        } else {
1078            // The old encoding style expected a header column for structs and so we
1079            // need a header column for the top-level struct
1080            let mut columns = Vec::with_capacity(column_infos.len() + 1);
1081            columns.push(Arc::new(root_column(num_rows)));
1082            columns.extend(column_infos.iter().cloned());
1083
1084            let adjusted_column_indices = [0_u32]
1085                .into_iter()
1086                .chain(column_indices.iter().map(|i| i.saturating_add(1)))
1087                .collect::<Vec<_>>();
1088            let mut column_iter = ColumnInfoIter::new(columns, &adjusted_column_indices);
1089            let root_scheduler = CoreFieldDecoderStrategy::default()
1090                .create_legacy_field_scheduler(&root_field, &mut column_iter, buffers)?;
1091
1092            let context = SchedulerContext::new(io, cache.clone());
1093            root_scheduler.initialize(filter, &context).await?;
1094
1095            Ok(Self {
1096                root_scheduler: RootScheduler::Legacy(root_scheduler.into()),
1097                root_fields,
1098                cache,
1099            })
1100        }
1101    }
1102
1103    pub fn from_scheduler(
1104        root_scheduler: Arc<dyn FieldScheduler>,
1105        root_fields: Fields,
1106        cache: Arc<FileMetadataCache>,
1107    ) -> Self {
1108        Self {
1109            root_scheduler: RootScheduler::Legacy(root_scheduler),
1110            root_fields,
1111            cache,
1112        }
1113    }
1114
1115    fn do_schedule_ranges_structural(
1116        &mut self,
1117        ranges: &[Range<u64>],
1118        filter: &FilterExpression,
1119        io: Arc<dyn EncodingsIo>,
1120        mut schedule_action: impl FnMut(Result<DecoderMessage>) -> bool,
1121    ) {
1122        let root_scheduler = self.root_scheduler.as_structural();
1123        let mut context = SchedulerContext::new(io, self.cache.clone());
1124        let maybe_root_job = root_scheduler.schedule_ranges(ranges, filter);
1125        if let Err(schedule_ranges_err) = maybe_root_job {
1126            schedule_action(Err(schedule_ranges_err));
1127            return;
1128        }
1129        let mut root_job = maybe_root_job.unwrap();
1130        let mut num_rows_scheduled = 0;
1131        loop {
1132            let maybe_next_scan_line = root_job.schedule_next(&mut context);
1133            if let Err(err) = maybe_next_scan_line {
1134                schedule_action(Err(err));
1135                return;
1136            }
1137            let next_scan_line = maybe_next_scan_line.unwrap();
1138            match next_scan_line {
1139                Some(next_scan_line) => {
1140                    trace!(
1141                        "Scheduled scan line of {} rows and {} decoders",
1142                        next_scan_line.rows_scheduled,
1143                        next_scan_line.decoders.len()
1144                    );
1145                    num_rows_scheduled += next_scan_line.rows_scheduled;
1146                    if !schedule_action(Ok(DecoderMessage {
1147                        scheduled_so_far: num_rows_scheduled,
1148                        decoders: next_scan_line.decoders,
1149                    })) {
1150                        // Decoder has disconnected
1151                        return;
1152                    }
1153                }
1154                None => return,
1155            }
1156        }
1157    }
1158
1159    fn do_schedule_ranges_legacy(
1160        &mut self,
1161        ranges: &[Range<u64>],
1162        filter: &FilterExpression,
1163        io: Arc<dyn EncodingsIo>,
1164        mut schedule_action: impl FnMut(Result<DecoderMessage>) -> bool,
1165        // If specified, this will be used as the top_level_row for all scheduling
1166        // tasks.  This is used by list scheduling to ensure all items scheduling
1167        // tasks are scheduled at the same top level row.
1168        priority: Option<Box<dyn PriorityRange>>,
1169    ) {
1170        let root_scheduler = self.root_scheduler.as_legacy();
1171        let rows_requested = ranges.iter().map(|r| r.end - r.start).sum::<u64>();
1172        trace!(
1173            "Scheduling {} ranges across {}..{} ({} rows){}",
1174            ranges.len(),
1175            ranges.first().unwrap().start,
1176            ranges.last().unwrap().end,
1177            rows_requested,
1178            priority
1179                .as_ref()
1180                .map(|p| format!(" (priority={:?})", p))
1181                .unwrap_or_default()
1182        );
1183
1184        let mut context = SchedulerContext::new(io, self.cache.clone());
1185        let maybe_root_job = root_scheduler.schedule_ranges(ranges, filter);
1186        if let Err(schedule_ranges_err) = maybe_root_job {
1187            schedule_action(Err(schedule_ranges_err));
1188            return;
1189        }
1190        let mut root_job = maybe_root_job.unwrap();
1191        let mut num_rows_scheduled = 0;
1192        let mut rows_to_schedule = root_job.num_rows();
1193        let mut priority = priority.unwrap_or(Box::new(SimplePriorityRange::new(0)));
1194        trace!("Scheduled ranges refined to {} rows", rows_to_schedule);
1195        while rows_to_schedule > 0 {
1196            let maybe_next_scan_line = root_job.schedule_next(&mut context, priority.as_ref());
1197            if let Err(schedule_next_err) = maybe_next_scan_line {
1198                schedule_action(Err(schedule_next_err));
1199                return;
1200            }
1201            let next_scan_line = maybe_next_scan_line.unwrap();
1202            priority.advance(next_scan_line.rows_scheduled);
1203            num_rows_scheduled += next_scan_line.rows_scheduled;
1204            rows_to_schedule -= next_scan_line.rows_scheduled;
1205            trace!(
1206                "Scheduled scan line of {} rows and {} decoders",
1207                next_scan_line.rows_scheduled,
1208                next_scan_line.decoders.len()
1209            );
1210            if !schedule_action(Ok(DecoderMessage {
1211                scheduled_so_far: num_rows_scheduled,
1212                decoders: next_scan_line.decoders,
1213            })) {
1214                // Decoder has disconnected
1215                return;
1216            }
1217
1218            trace!("Finished scheduling {} ranges", ranges.len());
1219        }
1220    }
1221
1222    fn do_schedule_ranges(
1223        &mut self,
1224        ranges: &[Range<u64>],
1225        filter: &FilterExpression,
1226        io: Arc<dyn EncodingsIo>,
1227        schedule_action: impl FnMut(Result<DecoderMessage>) -> bool,
1228        // If specified, this will be used as the top_level_row for all scheduling
1229        // tasks.  This is used by list scheduling to ensure all items scheduling
1230        // tasks are scheduled at the same top level row.
1231        priority: Option<Box<dyn PriorityRange>>,
1232    ) {
1233        match &self.root_scheduler {
1234            RootScheduler::Legacy(_) => {
1235                self.do_schedule_ranges_legacy(ranges, filter, io, schedule_action, priority)
1236            }
1237            RootScheduler::Structural(_) => {
1238                self.do_schedule_ranges_structural(ranges, filter, io, schedule_action)
1239            }
1240        }
1241    }
1242
1243    // This method is similar to schedule_ranges but instead of
1244    // sending the decoders to a channel it collects them all into a vector
1245    pub fn schedule_ranges_to_vec(
1246        &mut self,
1247        ranges: &[Range<u64>],
1248        filter: &FilterExpression,
1249        io: Arc<dyn EncodingsIo>,
1250        priority: Option<Box<dyn PriorityRange>>,
1251    ) -> Result<Vec<DecoderMessage>> {
1252        let mut decode_messages = Vec::new();
1253        self.do_schedule_ranges(
1254            ranges,
1255            filter,
1256            io,
1257            |msg| {
1258                decode_messages.push(msg);
1259                true
1260            },
1261            priority,
1262        );
1263        decode_messages.into_iter().collect::<Result<Vec<_>>>()
1264    }
1265
1266    /// Schedules the load of multiple ranges of rows
1267    ///
1268    /// Ranges must be non-overlapping and in sorted order
1269    ///
1270    /// # Arguments
1271    ///
1272    /// * `ranges` - The ranges of rows to load
1273    /// * `sink` - A channel to send the decode tasks
1274    /// * `scheduler` An I/O scheduler to issue I/O requests
1275    #[instrument(skip_all)]
1276    pub fn schedule_ranges(
1277        &mut self,
1278        ranges: &[Range<u64>],
1279        filter: &FilterExpression,
1280        sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
1281        scheduler: Arc<dyn EncodingsIo>,
1282    ) {
1283        self.do_schedule_ranges(
1284            ranges,
1285            filter,
1286            scheduler,
1287            |msg| {
1288                match sink.send(msg) {
1289                    Ok(_) => true,
1290                    Err(SendError { .. }) => {
1291                        // The receiver has gone away.  We can't do anything about it
1292                        // so just ignore the error.
1293                        debug!(
1294                        "schedule_ranges aborting early since decoder appears to have been dropped"
1295                    );
1296                        false
1297                    }
1298                }
1299            },
1300            None,
1301        )
1302    }
1303
1304    /// Schedules the load of a range of rows
1305    ///
1306    /// # Arguments
1307    ///
1308    /// * `range` - The range of rows to load
1309    /// * `sink` - A channel to send the decode tasks
1310    /// * `scheduler` An I/O scheduler to issue I/O requests
1311    #[instrument(skip_all)]
1312    pub fn schedule_range(
1313        &mut self,
1314        range: Range<u64>,
1315        filter: &FilterExpression,
1316        sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
1317        scheduler: Arc<dyn EncodingsIo>,
1318    ) {
1319        self.schedule_ranges(&[range], filter, sink, scheduler)
1320    }
1321
1322    /// Schedules the load of selected rows
1323    ///
1324    /// # Arguments
1325    ///
1326    /// * `indices` - The row indices to load (these must be in ascending order!)
1327    /// * `sink` - A channel to send the decode tasks
1328    /// * `scheduler` An I/O scheduler to issue I/O requests
1329    pub fn schedule_take(
1330        &mut self,
1331        indices: &[u64],
1332        filter: &FilterExpression,
1333        sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
1334        scheduler: Arc<dyn EncodingsIo>,
1335    ) {
1336        debug_assert!(indices.windows(2).all(|w| w[0] <= w[1]));
1337        if indices.is_empty() {
1338            return;
1339        }
1340        trace!("Scheduling take of {} rows", indices.len());
1341        let ranges = indices
1342            .iter()
1343            .map(|&idx| idx..(idx + 1))
1344            .collect::<Vec<_>>();
1345        self.schedule_ranges(&ranges, filter, sink, scheduler)
1346    }
1347}
1348
1349pub struct ReadBatchTask {
1350    pub task: BoxFuture<'static, Result<RecordBatch>>,
1351    pub num_rows: u32,
1352}
1353
1354/// A stream that takes scheduled jobs and generates decode tasks from them.
1355pub struct BatchDecodeStream {
1356    context: DecoderContext,
1357    root_decoder: SimpleStructDecoder,
1358    rows_remaining: u64,
1359    rows_per_batch: u32,
1360    rows_scheduled: u64,
1361    rows_drained: u64,
1362    scheduler_exhausted: bool,
1363    emitted_batch_size_warning: Arc<Once>,
1364}
1365
1366impl BatchDecodeStream {
1367    /// Create a new instance of a batch decode stream
1368    ///
1369    /// # Arguments
1370    ///
1371    /// * `scheduled` - an incoming stream of decode tasks from a
1372    ///   [`crate::decode::DecodeBatchScheduler`]
1373    /// * `schema` - the schema of the data to create
1374    /// * `rows_per_batch` the number of rows to create before making a batch
1375    /// * `num_rows` the total number of rows scheduled
1376    /// * `num_columns` the total number of columns in the file
1377    pub fn new(
1378        scheduled: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
1379        rows_per_batch: u32,
1380        num_rows: u64,
1381        root_decoder: SimpleStructDecoder,
1382    ) -> Self {
1383        Self {
1384            context: DecoderContext::new(scheduled),
1385            root_decoder,
1386            rows_remaining: num_rows,
1387            rows_per_batch,
1388            rows_scheduled: 0,
1389            rows_drained: 0,
1390            scheduler_exhausted: false,
1391            emitted_batch_size_warning: Arc::new(Once::new()),
1392        }
1393    }
1394
1395    fn accept_decoder(&mut self, decoder: DecoderReady) -> Result<()> {
1396        if decoder.path.is_empty() {
1397            // The root decoder we can ignore
1398            Ok(())
1399        } else {
1400            self.root_decoder.accept_child(decoder)
1401        }
1402    }
1403
1404    async fn wait_for_scheduled(&mut self, scheduled_need: u64) -> Result<u64> {
1405        if self.scheduler_exhausted {
1406            return Ok(self.rows_scheduled);
1407        }
1408        while self.rows_scheduled < scheduled_need {
1409            let next_message = self.context.source.recv().await;
1410            match next_message {
1411                Some(scan_line) => {
1412                    let scan_line = scan_line?;
1413                    self.rows_scheduled = scan_line.scheduled_so_far;
1414                    for message in scan_line.decoders {
1415                        self.accept_decoder(message.into_legacy())?;
1416                    }
1417                }
1418                None => {
1419                    // Schedule ended before we got all the data we expected.  This probably
1420                    // means some kind of pushdown filter was applied and we didn't load as
1421                    // much data as we thought we would.
1422                    self.scheduler_exhausted = true;
1423                    return Ok(self.rows_scheduled);
1424                }
1425            }
1426        }
1427        Ok(scheduled_need)
1428    }
1429
1430    #[instrument(level = "debug", skip_all)]
1431    async fn next_batch_task(&mut self) -> Result<Option<NextDecodeTask>> {
1432        trace!(
1433            "Draining batch task (rows_remaining={} rows_drained={} rows_scheduled={})",
1434            self.rows_remaining,
1435            self.rows_drained,
1436            self.rows_scheduled,
1437        );
1438        if self.rows_remaining == 0 {
1439            return Ok(None);
1440        }
1441
1442        let mut to_take = self.rows_remaining.min(self.rows_per_batch as u64);
1443        self.rows_remaining -= to_take;
1444
1445        let scheduled_need = (self.rows_drained + to_take).saturating_sub(self.rows_scheduled);
1446        trace!("scheduled_need = {} because rows_drained = {} and to_take = {} and rows_scheduled = {}", scheduled_need, self.rows_drained, to_take, self.rows_scheduled);
1447        if scheduled_need > 0 {
1448            let desired_scheduled = scheduled_need + self.rows_scheduled;
1449            trace!(
1450                "Draining from scheduler (desire at least {} scheduled rows)",
1451                desired_scheduled
1452            );
1453            let actually_scheduled = self.wait_for_scheduled(desired_scheduled).await?;
1454            if actually_scheduled < desired_scheduled {
1455                let under_scheduled = desired_scheduled - actually_scheduled;
1456                to_take -= under_scheduled;
1457            }
1458        }
1459
1460        if to_take == 0 {
1461            return Ok(None);
1462        }
1463
1464        // wait_for_loaded waits for *>* loaded_need (not >=) so we do a -1 here
1465        let loaded_need = self.rows_drained + to_take - 1;
1466        trace!(
1467            "Waiting for I/O (desire at least {} fully loaded rows)",
1468            loaded_need
1469        );
1470        self.root_decoder.wait_for_loaded(loaded_need).await?;
1471
1472        let next_task = self.root_decoder.drain(to_take)?;
1473        self.rows_drained += to_take;
1474        Ok(Some(next_task))
1475    }
1476
1477    pub fn into_stream(self) -> BoxStream<'static, ReadBatchTask> {
1478        let stream = futures::stream::unfold(self, |mut slf| async move {
1479            let next_task = slf.next_batch_task().await;
1480            let next_task = next_task.transpose().map(|next_task| {
1481                let num_rows = next_task.as_ref().map(|t| t.num_rows).unwrap_or(0);
1482                let emitted_batch_size_warning = slf.emitted_batch_size_warning.clone();
1483                let task = tokio::spawn(async move {
1484                    let next_task = next_task?;
1485                    next_task.into_batch(emitted_batch_size_warning)
1486                });
1487                (task, num_rows)
1488            });
1489            next_task.map(|(task, num_rows)| {
1490                let task = task.map(|join_wrapper| join_wrapper.unwrap()).boxed();
1491                // This should be true since batch size is u32
1492                debug_assert!(num_rows <= u32::MAX as u64);
1493                let next_task = ReadBatchTask {
1494                    task,
1495                    num_rows: num_rows as u32,
1496                };
1497                (next_task, slf)
1498            })
1499        });
1500        stream.boxed()
1501    }
1502}
1503
1504// Utility types to smooth out the differences between the 2.0 and 2.1 decoders so that
1505// we can have a single implementation of the batch decode iterator
1506enum RootDecoderMessage {
1507    LoadedPage(LoadedPage),
1508    LegacyPage(DecoderReady),
1509}
1510trait RootDecoderType {
1511    fn accept_message(&mut self, message: RootDecoderMessage) -> Result<()>;
1512    fn drain_batch(&mut self, num_rows: u64) -> Result<NextDecodeTask>;
1513    fn wait(&mut self, loaded_need: u64, runtime: &tokio::runtime::Runtime) -> Result<()>;
1514}
1515impl RootDecoderType for StructuralStructDecoder {
1516    fn accept_message(&mut self, message: RootDecoderMessage) -> Result<()> {
1517        let RootDecoderMessage::LoadedPage(loaded_page) = message else {
1518            unreachable!()
1519        };
1520        self.accept_page(loaded_page)
1521    }
1522    fn drain_batch(&mut self, num_rows: u64) -> Result<NextDecodeTask> {
1523        self.drain_batch_task(num_rows)
1524    }
1525    fn wait(&mut self, _: u64, _: &tokio::runtime::Runtime) -> Result<()> {
1526        // Waiting happens elsewhere (not as part of the decoder)
1527        Ok(())
1528    }
1529}
1530impl RootDecoderType for SimpleStructDecoder {
1531    fn accept_message(&mut self, message: RootDecoderMessage) -> Result<()> {
1532        let RootDecoderMessage::LegacyPage(legacy_page) = message else {
1533            unreachable!()
1534        };
1535        self.accept_child(legacy_page)
1536    }
1537    fn drain_batch(&mut self, num_rows: u64) -> Result<NextDecodeTask> {
1538        self.drain(num_rows)
1539    }
1540    fn wait(&mut self, loaded_need: u64, runtime: &tokio::runtime::Runtime) -> Result<()> {
1541        runtime.block_on(self.wait_for_loaded(loaded_need))
1542    }
1543}
1544
1545/// A blocking batch decoder that performs synchronous decoding
1546struct BatchDecodeIterator<T: RootDecoderType> {
1547    messages: VecDeque<Result<DecoderMessage>>,
1548    root_decoder: T,
1549    rows_remaining: u64,
1550    rows_per_batch: u32,
1551    rows_scheduled: u64,
1552    rows_drained: u64,
1553    emitted_batch_size_warning: Arc<Once>,
1554    // Note: this is not the runtime on which I/O happens.
1555    // That's always in the scheduler.  This is just a runtime we use to
1556    // sleep the current thread if I/O is unready
1557    wait_for_io_runtime: tokio::runtime::Runtime,
1558    schema: Arc<ArrowSchema>,
1559}
1560
1561impl<T: RootDecoderType> BatchDecodeIterator<T> {
1562    /// Create a new instance of a batch decode iterator
1563    pub fn new(
1564        messages: VecDeque<Result<DecoderMessage>>,
1565        rows_per_batch: u32,
1566        num_rows: u64,
1567        root_decoder: T,
1568        schema: Arc<ArrowSchema>,
1569    ) -> Self {
1570        Self {
1571            messages,
1572            root_decoder,
1573            rows_remaining: num_rows,
1574            rows_per_batch,
1575            rows_scheduled: 0,
1576            rows_drained: 0,
1577            wait_for_io_runtime: tokio::runtime::Builder::new_current_thread()
1578                .build()
1579                .unwrap(),
1580            emitted_batch_size_warning: Arc::new(Once::new()),
1581            schema,
1582        }
1583    }
1584
1585    /// Wait for a single page of data to finish loading
1586    ///
1587    /// If the data is not available this will perform a *blocking* wait (put
1588    /// the current thread to sleep)
1589    fn wait_for_page(&self, unloaded_page: UnloadedPage) -> Result<LoadedPage> {
1590        match maybe_done(unloaded_page.0) {
1591            // Fast path, avoid all runtime shenanigans if the data is ready
1592            MaybeDone::Done(loaded_page) => loaded_page,
1593            // Slow path, we need to wait on I/O, enter the runtime
1594            MaybeDone::Future(fut) => self.wait_for_io_runtime.block_on(fut),
1595            MaybeDone::Gone => unreachable!(),
1596        }
1597    }
1598
1599    /// Waits for I/O until `scheduled_need` rows have been loaded
1600    ///
1601    /// Note that `scheduled_need` is cumulative.  E.g. this method
1602    /// should be called with 5, 10, 15 and not 5, 5, 5
1603    #[instrument(skip_all)]
1604    fn wait_for_io(&mut self, scheduled_need: u64) -> Result<u64> {
1605        while self.rows_scheduled < scheduled_need && !self.messages.is_empty() {
1606            let message = self.messages.pop_front().unwrap()?;
1607            self.rows_scheduled = message.scheduled_so_far;
1608            for decoder_message in message.decoders {
1609                match decoder_message {
1610                    MessageType::UnloadedPage(unloaded_page) => {
1611                        let loaded_page = self.wait_for_page(unloaded_page)?;
1612                        self.root_decoder
1613                            .accept_message(RootDecoderMessage::LoadedPage(loaded_page))?;
1614                    }
1615                    MessageType::DecoderReady(decoder_ready) => {
1616                        // The root decoder we can ignore
1617                        if !decoder_ready.path.is_empty() {
1618                            self.root_decoder
1619                                .accept_message(RootDecoderMessage::LegacyPage(decoder_ready))?;
1620                        }
1621                    }
1622                }
1623            }
1624        }
1625
1626        let loaded_need = self.rows_drained + self.rows_per_batch as u64 - 1;
1627
1628        self.root_decoder
1629            .wait(loaded_need, &self.wait_for_io_runtime)?;
1630        Ok(self.rows_scheduled)
1631    }
1632
1633    #[instrument(level = "debug", skip_all)]
1634    fn next_batch_task(&mut self) -> Result<Option<RecordBatch>> {
1635        trace!(
1636            "Draining batch task (rows_remaining={} rows_drained={} rows_scheduled={})",
1637            self.rows_remaining,
1638            self.rows_drained,
1639            self.rows_scheduled,
1640        );
1641        if self.rows_remaining == 0 {
1642            return Ok(None);
1643        }
1644
1645        let mut to_take = self.rows_remaining.min(self.rows_per_batch as u64);
1646        self.rows_remaining -= to_take;
1647
1648        let scheduled_need = (self.rows_drained + to_take).saturating_sub(self.rows_scheduled);
1649        trace!("scheduled_need = {} because rows_drained = {} and to_take = {} and rows_scheduled = {}", scheduled_need, self.rows_drained, to_take, self.rows_scheduled);
1650        if scheduled_need > 0 {
1651            let desired_scheduled = scheduled_need + self.rows_scheduled;
1652            trace!(
1653                "Draining from scheduler (desire at least {} scheduled rows)",
1654                desired_scheduled
1655            );
1656            let actually_scheduled = self.wait_for_io(desired_scheduled)?;
1657            if actually_scheduled < desired_scheduled {
1658                let under_scheduled = desired_scheduled - actually_scheduled;
1659                to_take -= under_scheduled;
1660            }
1661        }
1662
1663        if to_take == 0 {
1664            return Ok(None);
1665        }
1666
1667        let next_task = self.root_decoder.drain_batch(to_take)?;
1668
1669        self.rows_drained += to_take;
1670
1671        let batch = next_task.into_batch(self.emitted_batch_size_warning.clone())?;
1672
1673        Ok(Some(batch))
1674    }
1675}
1676
1677impl<T: RootDecoderType> Iterator for BatchDecodeIterator<T> {
1678    type Item = ArrowResult<RecordBatch>;
1679
1680    fn next(&mut self) -> Option<Self::Item> {
1681        self.next_batch_task()
1682            .transpose()
1683            .map(|r| r.map_err(ArrowError::from))
1684    }
1685}
1686
1687impl<T: RootDecoderType> RecordBatchReader for BatchDecodeIterator<T> {
1688    fn schema(&self) -> Arc<ArrowSchema> {
1689        self.schema.clone()
1690    }
1691}
1692
1693/// A stream that takes scheduled jobs and generates decode tasks from them.
1694pub struct StructuralBatchDecodeStream {
1695    context: DecoderContext,
1696    root_decoder: StructuralStructDecoder,
1697    rows_remaining: u64,
1698    rows_per_batch: u32,
1699    rows_scheduled: u64,
1700    rows_drained: u64,
1701    scheduler_exhausted: bool,
1702    emitted_batch_size_warning: Arc<Once>,
1703}
1704
1705impl StructuralBatchDecodeStream {
1706    /// Create a new instance of a batch decode stream
1707    ///
1708    /// # Arguments
1709    ///
1710    /// * `scheduled` - an incoming stream of decode tasks from a
1711    ///   [`crate::decode::DecodeBatchScheduler`]
1712    /// * `schema` - the schema of the data to create
1713    /// * `rows_per_batch` the number of rows to create before making a batch
1714    /// * `num_rows` the total number of rows scheduled
1715    /// * `num_columns` the total number of columns in the file
1716    pub fn new(
1717        scheduled: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
1718        rows_per_batch: u32,
1719        num_rows: u64,
1720        root_decoder: StructuralStructDecoder,
1721    ) -> Self {
1722        Self {
1723            context: DecoderContext::new(scheduled),
1724            root_decoder,
1725            rows_remaining: num_rows,
1726            rows_per_batch,
1727            rows_scheduled: 0,
1728            rows_drained: 0,
1729            scheduler_exhausted: false,
1730            emitted_batch_size_warning: Arc::new(Once::new()),
1731        }
1732    }
1733
1734    async fn wait_for_scheduled(&mut self, scheduled_need: u64) -> Result<u64> {
1735        if self.scheduler_exhausted {
1736            return Ok(self.rows_scheduled);
1737        }
1738        while self.rows_scheduled < scheduled_need {
1739            let next_message = self.context.source.recv().await;
1740            match next_message {
1741                Some(scan_line) => {
1742                    let scan_line = scan_line?;
1743                    self.rows_scheduled = scan_line.scheduled_so_far;
1744                    for message in scan_line.decoders {
1745                        let unloaded_page = message.into_structural();
1746                        let loaded_page = unloaded_page.0.await?;
1747                        self.root_decoder.accept_page(loaded_page)?;
1748                    }
1749                }
1750                None => {
1751                    // Schedule ended before we got all the data we expected.  This probably
1752                    // means some kind of pushdown filter was applied and we didn't load as
1753                    // much data as we thought we would.
1754                    self.scheduler_exhausted = true;
1755                    return Ok(self.rows_scheduled);
1756                }
1757            }
1758        }
1759        Ok(scheduled_need)
1760    }
1761
1762    #[instrument(level = "debug", skip_all)]
1763    async fn next_batch_task(&mut self) -> Result<Option<NextDecodeTask>> {
1764        trace!(
1765            "Draining batch task (rows_remaining={} rows_drained={} rows_scheduled={})",
1766            self.rows_remaining,
1767            self.rows_drained,
1768            self.rows_scheduled,
1769        );
1770        if self.rows_remaining == 0 {
1771            return Ok(None);
1772        }
1773
1774        let mut to_take = self.rows_remaining.min(self.rows_per_batch as u64);
1775        self.rows_remaining -= to_take;
1776
1777        let scheduled_need = (self.rows_drained + to_take).saturating_sub(self.rows_scheduled);
1778        trace!("scheduled_need = {} because rows_drained = {} and to_take = {} and rows_scheduled = {}", scheduled_need, self.rows_drained, to_take, self.rows_scheduled);
1779        if scheduled_need > 0 {
1780            let desired_scheduled = scheduled_need + self.rows_scheduled;
1781            trace!(
1782                "Draining from scheduler (desire at least {} scheduled rows)",
1783                desired_scheduled
1784            );
1785            let actually_scheduled = self.wait_for_scheduled(desired_scheduled).await?;
1786            if actually_scheduled < desired_scheduled {
1787                let under_scheduled = desired_scheduled - actually_scheduled;
1788                to_take -= under_scheduled;
1789            }
1790        }
1791
1792        if to_take == 0 {
1793            return Ok(None);
1794        }
1795
1796        let next_task = self.root_decoder.drain_batch_task(to_take)?;
1797        self.rows_drained += to_take;
1798        Ok(Some(next_task))
1799    }
1800
1801    pub fn into_stream(self) -> BoxStream<'static, ReadBatchTask> {
1802        let stream = futures::stream::unfold(self, |mut slf| async move {
1803            let next_task = slf.next_batch_task().await;
1804            let next_task = next_task.transpose().map(|next_task| {
1805                let num_rows = next_task.as_ref().map(|t| t.num_rows).unwrap_or(0);
1806                let emitted_batch_size_warning = slf.emitted_batch_size_warning.clone();
1807                let task = tokio::spawn(async move {
1808                    let next_task = next_task?;
1809                    next_task.into_batch(emitted_batch_size_warning)
1810                });
1811                (task, num_rows)
1812            });
1813            next_task.map(|(task, num_rows)| {
1814                let task = task.map(|join_wrapper| join_wrapper.unwrap()).boxed();
1815                // This should be true since batch size is u32
1816                debug_assert!(num_rows <= u32::MAX as u64);
1817                let next_task = ReadBatchTask {
1818                    task,
1819                    num_rows: num_rows as u32,
1820                };
1821                (next_task, slf)
1822            })
1823        });
1824        stream.boxed()
1825    }
1826}
1827
1828#[derive(Debug)]
1829pub enum RequestedRows {
1830    Ranges(Vec<Range<u64>>),
1831    Indices(Vec<u64>),
1832}
1833
1834impl RequestedRows {
1835    pub fn num_rows(&self) -> u64 {
1836        match self {
1837            Self::Ranges(ranges) => ranges.iter().map(|r| r.end - r.start).sum(),
1838            Self::Indices(indices) => indices.len() as u64,
1839        }
1840    }
1841}
1842
1843#[derive(Debug, Clone)]
1844pub struct SchedulerDecoderConfig {
1845    pub decoder_plugins: Arc<DecoderPlugins>,
1846    pub batch_size: u32,
1847    pub io: Arc<dyn EncodingsIo>,
1848    pub cache: Arc<FileMetadataCache>,
1849    pub should_validate: bool,
1850}
1851
1852fn check_scheduler_on_drop(
1853    stream: BoxStream<'static, ReadBatchTask>,
1854    scheduler_handle: tokio::task::JoinHandle<()>,
1855) -> BoxStream<'static, ReadBatchTask> {
1856    // This is a bit weird but we create an "empty stream" that unwraps the scheduler handle (which
1857    // will panic if the scheduler panicked).  This let's us check if the scheduler panicked
1858    // when the stream finishes.
1859    let mut scheduler_handle = Some(scheduler_handle);
1860    let check_scheduler = stream::unfold((), move |_| {
1861        let handle = scheduler_handle.take();
1862        async move {
1863            if let Some(handle) = handle {
1864                handle.await.unwrap();
1865            }
1866            None
1867        }
1868    });
1869    stream.chain(check_scheduler).boxed()
1870}
1871
1872pub fn create_decode_stream(
1873    schema: &Schema,
1874    num_rows: u64,
1875    batch_size: u32,
1876    is_structural: bool,
1877    should_validate: bool,
1878    rx: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
1879) -> BoxStream<'static, ReadBatchTask> {
1880    if is_structural {
1881        let arrow_schema = ArrowSchema::from(schema);
1882        let structural_decoder = StructuralStructDecoder::new(
1883            arrow_schema.fields,
1884            should_validate,
1885            /*is_root=*/ true,
1886        );
1887        StructuralBatchDecodeStream::new(rx, batch_size, num_rows, structural_decoder).into_stream()
1888    } else {
1889        let arrow_schema = ArrowSchema::from(schema);
1890        let root_fields = arrow_schema.fields;
1891
1892        let simple_struct_decoder = SimpleStructDecoder::new(root_fields, num_rows);
1893        BatchDecodeStream::new(rx, batch_size, num_rows, simple_struct_decoder).into_stream()
1894    }
1895}
1896
1897/// Creates a iterator that decodes a set of messages in a blocking fashion
1898///
1899/// See [`schedule_and_decode_blocking`] for more information.
1900pub fn create_decode_iterator(
1901    schema: &Schema,
1902    num_rows: u64,
1903    batch_size: u32,
1904    should_validate: bool,
1905    is_structural: bool,
1906    messages: VecDeque<Result<DecoderMessage>>,
1907) -> Box<dyn RecordBatchReader> {
1908    let arrow_schema = Arc::new(ArrowSchema::from(schema));
1909    let root_fields = arrow_schema.fields.clone();
1910    if is_structural {
1911        let simple_struct_decoder =
1912            StructuralStructDecoder::new(root_fields, should_validate, /*is_root=*/ true);
1913        Box::new(BatchDecodeIterator::new(
1914            messages,
1915            batch_size,
1916            num_rows,
1917            simple_struct_decoder,
1918            arrow_schema,
1919        ))
1920    } else {
1921        let root_decoder = SimpleStructDecoder::new(root_fields, num_rows);
1922        Box::new(BatchDecodeIterator::new(
1923            messages,
1924            batch_size,
1925            num_rows,
1926            root_decoder,
1927            arrow_schema,
1928        ))
1929    }
1930}
1931
1932fn create_scheduler_decoder(
1933    column_infos: Vec<Arc<ColumnInfo>>,
1934    requested_rows: RequestedRows,
1935    filter: FilterExpression,
1936    column_indices: Vec<u32>,
1937    target_schema: Arc<Schema>,
1938    config: SchedulerDecoderConfig,
1939) -> Result<BoxStream<'static, ReadBatchTask>> {
1940    let num_rows = requested_rows.num_rows();
1941
1942    let is_structural = column_infos[0].is_structural();
1943
1944    let (tx, rx) = mpsc::unbounded_channel();
1945
1946    let decode_stream = create_decode_stream(
1947        &target_schema,
1948        num_rows,
1949        config.batch_size,
1950        is_structural,
1951        config.should_validate,
1952        rx,
1953    );
1954
1955    let scheduler_handle = tokio::task::spawn(async move {
1956        let mut decode_scheduler = match DecodeBatchScheduler::try_new(
1957            target_schema.as_ref(),
1958            &column_indices,
1959            &column_infos,
1960            &vec![],
1961            num_rows,
1962            config.decoder_plugins,
1963            config.io.clone(),
1964            config.cache,
1965            &filter,
1966        )
1967        .await
1968        {
1969            Ok(scheduler) => scheduler,
1970            Err(e) => {
1971                let _ = tx.send(Err(e));
1972                return;
1973            }
1974        };
1975
1976        match requested_rows {
1977            RequestedRows::Ranges(ranges) => {
1978                decode_scheduler.schedule_ranges(&ranges, &filter, tx, config.io)
1979            }
1980            RequestedRows::Indices(indices) => {
1981                decode_scheduler.schedule_take(&indices, &filter, tx, config.io)
1982            }
1983        }
1984    });
1985
1986    Ok(check_scheduler_on_drop(decode_stream, scheduler_handle))
1987}
1988
1989/// Launches a scheduler on a dedicated (spawned) task and creates a decoder to
1990/// decode the scheduled data and returns the decoder as a stream of record batches.
1991///
1992/// This is a convenience function that creates both the scheduler and the decoder
1993/// which can be a little tricky to get right.
1994pub fn schedule_and_decode(
1995    column_infos: Vec<Arc<ColumnInfo>>,
1996    requested_rows: RequestedRows,
1997    filter: FilterExpression,
1998    column_indices: Vec<u32>,
1999    target_schema: Arc<Schema>,
2000    config: SchedulerDecoderConfig,
2001) -> BoxStream<'static, ReadBatchTask> {
2002    if requested_rows.num_rows() == 0 {
2003        return stream::empty().boxed();
2004    }
2005    // For convenience we really want this method to be a snchronous method where all
2006    // errors happen on the stream.  There is some async initialization that must happen
2007    // when creating a scheduler.  We wrap that all up in the very first task.
2008    match create_scheduler_decoder(
2009        column_infos,
2010        requested_rows,
2011        filter,
2012        column_indices,
2013        target_schema,
2014        config,
2015    ) {
2016        // If the initialization failed make it look like a failed task
2017        Ok(stream) => stream,
2018        Err(e) => stream::once(std::future::ready(ReadBatchTask {
2019            num_rows: 0,
2020            task: std::future::ready(Err(e)).boxed(),
2021        }))
2022        .boxed(),
2023    }
2024}
2025
2026lazy_static::lazy_static! {
2027    pub static ref WAITER_RT: tokio::runtime::Runtime = tokio::runtime::Builder::new_current_thread()
2028        .build()
2029        .unwrap();
2030}
2031
2032/// Schedules and decodes the requested data in a blocking fashion
2033///
2034/// This function is a blocking version of [`schedule_and_decode`]. It schedules the requested data
2035/// and decodes it in the current thread.
2036///
2037/// This can be useful when the disk is fast (or the data is in memory) and the amount
2038/// of data is relatively small.  For example, when doing a take against NVMe or in-memory data.
2039///
2040/// This should NOT be used for full scans.  Even if the data is in memory this function will
2041/// not parallelize the decode and will be slower than the async version.  Full scans typically
2042/// make relatively few IOPs and so the asynchronous overhead is much smaller.
2043///
2044/// This method will first completely run the scheduling process.  Then it will run the
2045/// decode process.
2046pub fn schedule_and_decode_blocking(
2047    column_infos: Vec<Arc<ColumnInfo>>,
2048    requested_rows: RequestedRows,
2049    filter: FilterExpression,
2050    column_indices: Vec<u32>,
2051    target_schema: Arc<Schema>,
2052    config: SchedulerDecoderConfig,
2053) -> Result<Box<dyn RecordBatchReader>> {
2054    if requested_rows.num_rows() == 0 {
2055        let arrow_schema = Arc::new(ArrowSchema::from(target_schema.as_ref()));
2056        return Ok(Box::new(RecordBatchIterator::new(vec![], arrow_schema)));
2057    }
2058
2059    let num_rows = requested_rows.num_rows();
2060    let is_structural = column_infos[0].is_structural();
2061
2062    let (tx, mut rx) = mpsc::unbounded_channel();
2063
2064    // Initialize the scheduler.  This is still "asynchronous" but we run it with a current-thread
2065    // runtime.
2066    let mut decode_scheduler = WAITER_RT.block_on(DecodeBatchScheduler::try_new(
2067        target_schema.as_ref(),
2068        &column_indices,
2069        &column_infos,
2070        &vec![],
2071        num_rows,
2072        config.decoder_plugins,
2073        config.io.clone(),
2074        config.cache,
2075        &filter,
2076    ))?;
2077
2078    // Schedule the requested rows
2079    match requested_rows {
2080        RequestedRows::Ranges(ranges) => {
2081            decode_scheduler.schedule_ranges(&ranges, &filter, tx, config.io)
2082        }
2083        RequestedRows::Indices(indices) => {
2084            decode_scheduler.schedule_take(&indices, &filter, tx, config.io)
2085        }
2086    }
2087
2088    // Drain the scheduler queue into a vec of decode messages
2089    let mut messages = Vec::new();
2090    while rx
2091        .recv_many(&mut messages, usize::MAX)
2092        .now_or_never()
2093        .unwrap()
2094        != 0
2095    {}
2096
2097    // Create a decoder to decode the messages
2098    let decode_iterator = create_decode_iterator(
2099        &target_schema,
2100        num_rows,
2101        config.batch_size,
2102        config.should_validate,
2103        is_structural,
2104        messages.into(),
2105    );
2106
2107    Ok(decode_iterator)
2108}
2109
2110/// A decoder for single-column encodings of primitive data (this includes fixed size
2111/// lists of primitive data)
2112///
2113/// Physical decoders are able to decode into existing buffers for zero-copy operation.
2114///
2115/// Instances should be stateless and `Send` / `Sync`.  This is because multiple decode
2116/// tasks could reference the same page.  For example, imagine a page covers rows 0-2000
2117/// and the decoder stream has a batch size of 1024.  The decoder will be needed by both
2118/// the decode task for batch 0 and the decode task for batch 1.
2119///
2120/// See [`crate::decoder`] for more information
2121pub trait PrimitivePageDecoder: Send + Sync {
2122    /// Decode data into buffers
2123    ///
2124    /// This may be a simple zero-copy from a disk buffer or could involve complex decoding
2125    /// such as decompressing from some compressed representation.
2126    ///
2127    /// Capacity is stored as a tuple of (num_bytes: u64, is_needed: bool).  The `is_needed`
2128    /// portion only needs to be updated if the encoding has some concept of an "optional"
2129    /// buffer.
2130    ///
2131    /// Encodings can have any number of input or output buffers.  For example, a dictionary
2132    /// decoding will convert two buffers (indices + dictionary) into a single buffer
2133    ///
2134    /// Binary decodings have two output buffers (one for values, one for offsets)
2135    ///
2136    /// Other decodings could even expand the # of output buffers.  For example, we could decode
2137    /// fixed size strings into variable length strings going from one input buffer to multiple output
2138    /// buffers.
2139    ///
2140    /// Each Arrow data type typically has a fixed structure of buffers and the encoding chain will
2141    /// generally end at one of these structures.  However, intermediate structures may exist which
2142    /// do not correspond to any Arrow type at all.  For example, a bitpacking encoding will deal
2143    /// with buffers that have bits-per-value that is not a multiple of 8.
2144    ///
2145    /// The `primitive_array_from_buffers` method has an expected buffer layout for each arrow
2146    /// type (order matters) and encodings that aim to decode into arrow types should respect
2147    /// this layout.
2148    /// # Arguments
2149    ///
2150    /// * `rows_to_skip` - how many rows to skip (within the page) before decoding
2151    /// * `num_rows` - how many rows to decode
2152    /// * `all_null` - A mutable bool, set to true if a decoder determines all values are null
2153    fn decode(&self, rows_to_skip: u64, num_rows: u64) -> Result<DataBlock>;
2154}
2155
2156/// A scheduler for single-column encodings of primitive data
2157///
2158/// The scheduler is responsible for calculating what I/O is needed for the requested rows
2159///
2160/// Instances should be stateless and `Send` and `Sync`.  This is because instances can
2161/// be shared in follow-up I/O tasks.
2162///
2163/// See [`crate::decoder`] for more information
2164pub trait PageScheduler: Send + Sync + std::fmt::Debug {
2165    /// Schedules a batch of I/O to load the data needed for the requested ranges
2166    ///
2167    /// Returns a future that will yield a decoder once the data has been loaded
2168    ///
2169    /// # Arguments
2170    ///
2171    /// * `range` - the range of row offsets (relative to start of page) requested
2172    ///             these must be ordered and must not overlap
2173    /// * `scheduler` - a scheduler to submit the I/O request to
2174    /// * `top_level_row` - the row offset of the top level field currently being
2175    ///   scheduled.  This can be used to assign priority to I/O requests
2176    fn schedule_ranges(
2177        &self,
2178        ranges: &[Range<u64>],
2179        scheduler: &Arc<dyn EncodingsIo>,
2180        top_level_row: u64,
2181    ) -> BoxFuture<'static, Result<Box<dyn PrimitivePageDecoder>>>;
2182}
2183
2184/// A trait to control the priority of I/O
2185pub trait PriorityRange: std::fmt::Debug + Send + Sync {
2186    fn advance(&mut self, num_rows: u64);
2187    fn current_priority(&self) -> u64;
2188    fn box_clone(&self) -> Box<dyn PriorityRange>;
2189}
2190
2191/// A simple priority scheme for top-level fields with no parent
2192/// repetition
2193#[derive(Debug)]
2194pub struct SimplePriorityRange {
2195    priority: u64,
2196}
2197
2198impl SimplePriorityRange {
2199    fn new(priority: u64) -> Self {
2200        Self { priority }
2201    }
2202}
2203
2204impl PriorityRange for SimplePriorityRange {
2205    fn advance(&mut self, num_rows: u64) {
2206        self.priority += num_rows;
2207    }
2208
2209    fn current_priority(&self) -> u64 {
2210        self.priority
2211    }
2212
2213    fn box_clone(&self) -> Box<dyn PriorityRange> {
2214        Box::new(Self {
2215            priority: self.priority,
2216        })
2217    }
2218}
2219
2220/// Determining the priority of a list request is tricky.  We want
2221/// the priority to be the top-level row.  So if we have a
2222/// list<list<int>> and each outer list has 10 rows and each inner
2223/// list has 5 rows then the priority of the 100th item is 1 because
2224/// it is the 5th item in the 10th item of the *second* row.
2225///
2226/// This structure allows us to keep track of this complicated priority
2227/// relationship.
2228///
2229/// There's a fair amount of bookkeeping involved here.
2230///
2231/// A better approach (using repetition levels) is coming in the future.
2232pub struct ListPriorityRange {
2233    base: Box<dyn PriorityRange>,
2234    offsets: Arc<[u64]>,
2235    cur_index_into_offsets: usize,
2236    cur_position: u64,
2237}
2238
2239impl ListPriorityRange {
2240    pub(crate) fn new(base: Box<dyn PriorityRange>, offsets: Arc<[u64]>) -> Self {
2241        Self {
2242            base,
2243            offsets,
2244            cur_index_into_offsets: 0,
2245            cur_position: 0,
2246        }
2247    }
2248}
2249
2250impl std::fmt::Debug for ListPriorityRange {
2251    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
2252        f.debug_struct("ListPriorityRange")
2253            .field("base", &self.base)
2254            .field("offsets.len()", &self.offsets.len())
2255            .field("cur_index_into_offsets", &self.cur_index_into_offsets)
2256            .field("cur_position", &self.cur_position)
2257            .finish()
2258    }
2259}
2260
2261impl PriorityRange for ListPriorityRange {
2262    fn advance(&mut self, num_rows: u64) {
2263        // We've scheduled X items.  Now walk through the offsets to
2264        // determine how many rows we've scheduled.
2265        self.cur_position += num_rows;
2266        let mut idx_into_offsets = self.cur_index_into_offsets;
2267        while idx_into_offsets + 1 < self.offsets.len()
2268            && self.offsets[idx_into_offsets + 1] <= self.cur_position
2269        {
2270            idx_into_offsets += 1;
2271        }
2272        let base_rows_advanced = idx_into_offsets - self.cur_index_into_offsets;
2273        self.cur_index_into_offsets = idx_into_offsets;
2274        self.base.advance(base_rows_advanced as u64);
2275    }
2276
2277    fn current_priority(&self) -> u64 {
2278        self.base.current_priority()
2279    }
2280
2281    fn box_clone(&self) -> Box<dyn PriorityRange> {
2282        Box::new(Self {
2283            base: self.base.box_clone(),
2284            offsets: self.offsets.clone(),
2285            cur_index_into_offsets: self.cur_index_into_offsets,
2286            cur_position: self.cur_position,
2287        })
2288    }
2289}
2290
2291/// Contains the context for a scheduler
2292pub struct SchedulerContext {
2293    recv: Option<mpsc::UnboundedReceiver<DecoderMessage>>,
2294    io: Arc<dyn EncodingsIo>,
2295    cache: Arc<FileMetadataCache>,
2296    name: String,
2297    path: Vec<u32>,
2298    path_names: Vec<String>,
2299}
2300
2301pub struct ScopedSchedulerContext<'a> {
2302    pub context: &'a mut SchedulerContext,
2303}
2304
2305impl<'a> ScopedSchedulerContext<'a> {
2306    pub fn pop(self) -> &'a mut SchedulerContext {
2307        self.context.pop();
2308        self.context
2309    }
2310}
2311
2312impl SchedulerContext {
2313    pub fn new(io: Arc<dyn EncodingsIo>, cache: Arc<FileMetadataCache>) -> Self {
2314        Self {
2315            io,
2316            cache,
2317            recv: None,
2318            name: "".to_string(),
2319            path: Vec::new(),
2320            path_names: Vec::new(),
2321        }
2322    }
2323
2324    pub fn io(&self) -> &Arc<dyn EncodingsIo> {
2325        &self.io
2326    }
2327
2328    pub fn cache(&self) -> &Arc<FileMetadataCache> {
2329        &self.cache
2330    }
2331
2332    pub fn push(&mut self, name: &str, index: u32) -> ScopedSchedulerContext {
2333        self.path.push(index);
2334        self.path_names.push(name.to_string());
2335        ScopedSchedulerContext { context: self }
2336    }
2337
2338    pub fn pop(&mut self) {
2339        self.path.pop();
2340        self.path_names.pop();
2341    }
2342
2343    pub fn path_name(&self) -> String {
2344        let path = self.path_names.join("/");
2345        if self.recv.is_some() {
2346            format!("TEMP({}){}", self.name, path)
2347        } else {
2348            format!("ROOT{}", path)
2349        }
2350    }
2351
2352    pub fn current_path(&self) -> VecDeque<u32> {
2353        VecDeque::from_iter(self.path.iter().copied())
2354    }
2355
2356    pub fn locate_decoder(&mut self, decoder: Box<dyn LogicalPageDecoder>) -> DecoderReady {
2357        trace!(
2358            "Scheduling decoder of type {:?} for {:?}",
2359            decoder.data_type(),
2360            self.path,
2361        );
2362        DecoderReady {
2363            decoder,
2364            path: self.current_path(),
2365        }
2366    }
2367}
2368
2369pub struct UnloadedPage(pub BoxFuture<'static, Result<LoadedPage>>);
2370
2371impl std::fmt::Debug for UnloadedPage {
2372    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
2373        f.debug_struct("UnloadedPage").finish()
2374    }
2375}
2376
2377#[derive(Debug)]
2378pub struct ScheduledScanLine {
2379    pub rows_scheduled: u64,
2380    pub decoders: Vec<MessageType>,
2381}
2382
2383pub trait SchedulingJob: std::fmt::Debug {
2384    fn schedule_next(
2385        &mut self,
2386        context: &mut SchedulerContext,
2387        priority: &dyn PriorityRange,
2388    ) -> Result<ScheduledScanLine>;
2389
2390    fn num_rows(&self) -> u64;
2391}
2392
2393pub trait StructuralSchedulingJob: std::fmt::Debug {
2394    fn schedule_next(
2395        &mut self,
2396        context: &mut SchedulerContext,
2397    ) -> Result<Option<ScheduledScanLine>>;
2398}
2399
2400/// A filter expression to apply to the data
2401///
2402/// The core decoders do not currently take advantage of filtering in
2403/// any way.  In order to maintain the abstraction we represent filters
2404/// as an arbitrary byte sequence.
2405///
2406/// We recommend that encodings use Substrait for filters.
2407pub struct FilterExpression(pub Bytes);
2408
2409impl FilterExpression {
2410    /// Create a filter expression that does not filter any data
2411    ///
2412    /// This is currently represented by an empty byte array.  Encoders
2413    /// that are "filter aware" should make sure they handle this case.
2414    pub fn no_filter() -> Self {
2415        Self(Bytes::new())
2416    }
2417
2418    /// Returns true if the filter is the same as the [`Self::no_filter`] filter
2419    pub fn is_noop(&self) -> bool {
2420        self.0.is_empty()
2421    }
2422}
2423
2424/// A scheduler for a field's worth of data
2425///
2426/// Each field in a reader's output schema maps to one field scheduler.  This scheduler may
2427/// map to more than one column.  For example, one field of struct data may
2428/// cover many columns of child data.  In fact, the entire file is treated as one
2429/// top-level struct field.
2430///
2431/// The scheduler is responsible for calculating the necessary I/O.  One schedule_range
2432/// request could trigger multiple batches of I/O across multiple columns.  The scheduler
2433/// should emit decoders into the sink as quickly as possible.
2434///
2435/// As soon as the scheduler encounters a batch of data that can decoded then the scheduler
2436/// should emit a decoder in the "unloaded" state.  The decode stream will pull the decoder
2437/// and start decoding.
2438///
2439/// The order in which decoders are emitted is important.  Pages should be emitted in
2440/// row-major order allowing decode of complete rows as quickly as possible.
2441///
2442/// The `FieldScheduler` should be stateless and `Send` and `Sync`.  This is
2443/// because it might need to be shared.  For example, a list page has a reference to
2444/// the field schedulers for its items column.  This is shared with the follow-up I/O
2445/// task created when the offsets are loaded.
2446///
2447/// See [`crate::decoder`] for more information
2448pub trait FieldScheduler: Send + Sync + std::fmt::Debug {
2449    /// Called at the beginning of scheduling to initialize the scheduler
2450    fn initialize<'a>(
2451        &'a self,
2452        filter: &'a FilterExpression,
2453        context: &'a SchedulerContext,
2454    ) -> BoxFuture<'a, Result<()>>;
2455    /// Schedules I/O for the requested portions of the field.
2456    ///
2457    /// Note: `ranges` must be ordered and non-overlapping
2458    /// TODO: Support unordered or overlapping ranges in file scheduler
2459    fn schedule_ranges<'a>(
2460        &'a self,
2461        ranges: &[Range<u64>],
2462        filter: &FilterExpression,
2463    ) -> Result<Box<dyn SchedulingJob + 'a>>;
2464    /// The number of rows in this field
2465    fn num_rows(&self) -> u64;
2466}
2467
2468pub trait StructuralFieldScheduler: Send + std::fmt::Debug {
2469    fn initialize<'a>(
2470        &'a mut self,
2471        filter: &'a FilterExpression,
2472        context: &'a SchedulerContext,
2473    ) -> BoxFuture<'a, Result<()>>;
2474    fn schedule_ranges<'a>(
2475        &'a self,
2476        ranges: &[Range<u64>],
2477        filter: &FilterExpression,
2478    ) -> Result<Box<dyn StructuralSchedulingJob + 'a>>;
2479}
2480
2481/// A trait for tasks that decode data into an Arrow array
2482pub trait DecodeArrayTask: Send {
2483    /// Decodes the data into an Arrow array
2484    fn decode(self: Box<Self>) -> Result<ArrayRef>;
2485}
2486
2487impl DecodeArrayTask for Box<dyn StructuralDecodeArrayTask> {
2488    fn decode(self: Box<Self>) -> Result<ArrayRef> {
2489        StructuralDecodeArrayTask::decode(*self).map(|decoded_array| decoded_array.array)
2490    }
2491}
2492
2493/// A task to decode data into an Arrow record batch
2494///
2495/// It has a child `task` which decodes a struct array with no nulls.
2496/// This is then converted into a record batch.
2497pub struct NextDecodeTask {
2498    /// The decode task itself
2499    pub task: Box<dyn DecodeArrayTask>,
2500    /// The number of rows that will be created
2501    pub num_rows: u64,
2502}
2503
2504impl NextDecodeTask {
2505    // Run the task and produce a record batch
2506    //
2507    // If the batch is very large this function will log a warning message
2508    // suggesting the user try a smaller batch size.
2509    #[instrument(name = "task_to_batch", level = "debug", skip_all)]
2510    fn into_batch(self, emitted_batch_size_warning: Arc<Once>) -> Result<RecordBatch> {
2511        let struct_arr = self.task.decode();
2512        match struct_arr {
2513            Ok(struct_arr) => {
2514                let batch = RecordBatch::from(struct_arr.as_struct());
2515                let size_bytes = batch.get_array_memory_size() as u64;
2516                if size_bytes > BATCH_SIZE_BYTES_WARNING {
2517                    emitted_batch_size_warning.call_once(|| {
2518                        let size_mb = size_bytes / 1024 / 1024;
2519                        debug!("Lance read in a single batch that contained more than {}MiB of data.  You may want to consider reducing the batch size.", size_mb);
2520                    });
2521                }
2522                Ok(batch)
2523            }
2524            Err(e) => {
2525                let e = Error::Internal {
2526                    message: format!("Error decoding batch: {}", e),
2527                    location: location!(),
2528                };
2529                Err(e)
2530            }
2531        }
2532    }
2533}
2534
2535#[derive(Debug)]
2536pub struct DecoderReady {
2537    // The decoder that is ready to be decoded
2538    pub decoder: Box<dyn LogicalPageDecoder>,
2539    // The path to the decoder, the first value is the column index
2540    // following values, if present, are nested child indices
2541    //
2542    // For example, a path of [1, 1, 0] would mean to grab the second
2543    // column, then the second child, and then the first child.
2544    //
2545    // It could represent x in the following schema:
2546    //
2547    // score: float64
2548    // points: struct
2549    //   color: string
2550    //   location: struct
2551    //     x: float64
2552    //
2553    // Currently, only struct decoders have "children" although other
2554    // decoders may at some point as well.  List children are only
2555    // handled through indirect I/O at the moment and so they don't
2556    // need to be represented (yet)
2557    pub path: VecDeque<u32>,
2558}
2559
2560// An envelope to wrap both 2.0 style messages and 2.1 style messages so we can
2561// share some code paths between the two.  Decoders can safely unwrap into whatever
2562// style they expect since a file will be either all-2.0 or all-2.1
2563#[derive(Debug)]
2564pub enum MessageType {
2565    // The older v2.0 scheduler/decoder used a scheme where the message was the
2566    // decoder itself.  The messages were not sent in priority order and the decoder
2567    // had to wait for I/O, figuring out the correct priority.  This was a lot of
2568    // complexity.
2569    DecoderReady(DecoderReady),
2570    // Starting in 2.1 we use a simpler scheme where the scheduling happens in priority
2571    // order and the message is an unloaded decoder.  These can be awaited, in order, and
2572    // the decoder does not have to worry about waiting for I/O.
2573    UnloadedPage(UnloadedPage),
2574}
2575
2576impl MessageType {
2577    pub fn into_legacy(self) -> DecoderReady {
2578        match self {
2579            Self::DecoderReady(decoder) => decoder,
2580            Self::UnloadedPage(_) => {
2581                panic!("Expected DecoderReady but got UnloadedPage")
2582            }
2583        }
2584    }
2585
2586    pub fn into_structural(self) -> UnloadedPage {
2587        match self {
2588            Self::UnloadedPage(unloaded) => unloaded,
2589            Self::DecoderReady(_) => {
2590                panic!("Expected UnloadedPage but got DecoderReady")
2591            }
2592        }
2593    }
2594}
2595
2596pub struct DecoderMessage {
2597    pub scheduled_so_far: u64,
2598    pub decoders: Vec<MessageType>,
2599}
2600
2601pub struct DecoderContext {
2602    source: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
2603}
2604
2605impl DecoderContext {
2606    pub fn new(source: mpsc::UnboundedReceiver<Result<DecoderMessage>>) -> Self {
2607        Self { source }
2608    }
2609}
2610
2611/// A decoder for a field's worth of data
2612///
2613/// The decoder is initially "unloaded" (doesn't have all its data).  The [`Self::wait`]
2614/// method should be called to wait for the needed I/O data before attempting to decode
2615/// any further.
2616///
2617/// Unlike the other decoder types it is assumed that `LogicalPageDecoder` is stateful
2618/// and only `Send`.  This is why we don't need a `rows_to_skip` argument in [`Self::drain`]
2619pub trait LogicalPageDecoder: std::fmt::Debug + Send {
2620    /// Add a newly scheduled child decoder
2621    ///
2622    /// The default implementation does not expect children and returns
2623    /// an error.
2624    fn accept_child(&mut self, _child: DecoderReady) -> Result<()> {
2625        Err(Error::Internal {
2626            message: format!(
2627                "The decoder {:?} does not expect children but received a child",
2628                self
2629            ),
2630            location: location!(),
2631        })
2632    }
2633    /// Waits until at least `num_rows` have been loaded
2634    fn wait_for_loaded(&mut self, loaded_need: u64) -> BoxFuture<Result<()>>;
2635    /// The number of rows loaded so far
2636    fn rows_loaded(&self) -> u64;
2637    /// The number of rows that still need loading
2638    fn rows_unloaded(&self) -> u64 {
2639        self.num_rows() - self.rows_loaded()
2640    }
2641    /// The total number of rows in the field
2642    fn num_rows(&self) -> u64;
2643    /// The number of rows that have been drained so far
2644    fn rows_drained(&self) -> u64;
2645    /// The number of rows that are still available to drain
2646    fn rows_left(&self) -> u64 {
2647        self.num_rows() - self.rows_drained()
2648    }
2649    /// Creates a task to decode `num_rows` of data into an array
2650    fn drain(&mut self, num_rows: u64) -> Result<NextDecodeTask>;
2651    /// The data type of the decoded data
2652    fn data_type(&self) -> &DataType;
2653}
2654
2655pub struct DecodedPage {
2656    pub data: DataBlock,
2657    pub repdef: RepDefUnraveler,
2658}
2659
2660pub trait DecodePageTask: Send + std::fmt::Debug {
2661    /// Decodes the data into an Arrow array
2662    fn decode(self: Box<Self>) -> Result<DecodedPage>;
2663}
2664
2665pub trait StructuralPageDecoder: std::fmt::Debug + Send {
2666    fn drain(&mut self, num_rows: u64) -> Result<Box<dyn DecodePageTask>>;
2667    fn num_rows(&self) -> u64;
2668}
2669
2670#[derive(Debug)]
2671pub struct LoadedPage {
2672    // The decoder that is ready to be decoded
2673    pub decoder: Box<dyn StructuralPageDecoder>,
2674    // The path to the decoder, the first value is the column index
2675    // following values, if present, are nested child indices
2676    //
2677    // For example, a path of [1, 1, 0] would mean to grab the second
2678    // column, then the second child, and then the first child.
2679    //
2680    // It could represent x in the following schema:
2681    //
2682    // score: float64
2683    // points: struct
2684    //   color: string
2685    //   location: struct
2686    //     x: float64
2687    //
2688    // Currently, only struct decoders have "children" although other
2689    // decoders may at some point as well.  List children are only
2690    // handled through indirect I/O at the moment and so they don't
2691    // need to be represented (yet)
2692    pub path: VecDeque<u32>,
2693    pub page_index: usize,
2694}
2695
2696pub struct DecodedArray {
2697    pub array: ArrayRef,
2698    pub repdef: CompositeRepDefUnraveler,
2699}
2700
2701pub trait StructuralDecodeArrayTask: std::fmt::Debug + Send {
2702    fn decode(self: Box<Self>) -> Result<DecodedArray>;
2703}
2704
2705pub trait StructuralFieldDecoder: std::fmt::Debug + Send {
2706    /// Add a newly scheduled child decoder
2707    ///
2708    /// The default implementation does not expect children and returns
2709    /// an error.
2710    fn accept_page(&mut self, _child: LoadedPage) -> Result<()>;
2711    /// Creates a task to decode `num_rows` of data into an array
2712    fn drain(&mut self, num_rows: u64) -> Result<Box<dyn StructuralDecodeArrayTask>>;
2713    /// The data type of the decoded data
2714    fn data_type(&self) -> &DataType;
2715}
2716
2717#[derive(Debug, Default)]
2718pub struct DecoderPlugins {}
2719
2720/// Decodes a batch of data from an in-memory structure created by [`crate::encoder::encode_batch`]
2721pub async fn decode_batch(
2722    batch: &EncodedBatch,
2723    filter: &FilterExpression,
2724    decoder_plugins: Arc<DecoderPlugins>,
2725    should_validate: bool,
2726    version: LanceFileVersion,
2727    cache: Option<Arc<FileMetadataCache>>,
2728) -> Result<RecordBatch> {
2729    // The io is synchronous so it shouldn't be possible for any async stuff to still be in progress
2730    // Still, if we just use now_or_never we hit misfires because some futures (channels) need to be
2731    // polled twice.
2732
2733    let io_scheduler = Arc::new(BufferScheduler::new(batch.data.clone())) as Arc<dyn EncodingsIo>;
2734    let cache = cache.unwrap_or_else(|| {
2735        Arc::new(FileMetadataCache::with_capacity(
2736            128 * 1024 * 1024,
2737            CapacityMode::Bytes,
2738        ))
2739    });
2740    let mut decode_scheduler = DecodeBatchScheduler::try_new(
2741        batch.schema.as_ref(),
2742        &batch.top_level_columns,
2743        &batch.page_table,
2744        &vec![],
2745        batch.num_rows,
2746        decoder_plugins,
2747        io_scheduler.clone(),
2748        cache,
2749        filter,
2750    )
2751    .await?;
2752    let (tx, rx) = unbounded_channel();
2753    decode_scheduler.schedule_range(0..batch.num_rows, filter, tx, io_scheduler);
2754    let is_structural = version >= LanceFileVersion::V2_1;
2755    let mut decode_stream = create_decode_stream(
2756        &batch.schema,
2757        batch.num_rows,
2758        batch.num_rows as u32,
2759        is_structural,
2760        should_validate,
2761        rx,
2762    );
2763    decode_stream.next().await.unwrap().task.await
2764}