1use 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::InlineBitpacking;
255use crate::encodings::physical::block_compress::CompressedBufferEncoder;
256use crate::encodings::physical::fsst::{FsstMiniBlockDecompressor, FsstPerValueDecompressor};
257use crate::encodings::physical::struct_encoding::PackedStructFixedWidthMiniBlockDecompressor;
258use crate::encodings::physical::value::{ConstantDecompressor, ValueDecompressor};
259use crate::encodings::physical::{ColumnBuffers, FileBuffers};
260use crate::format::pb::{self, column_encoding};
261use crate::repdef::{CompositeRepDefUnraveler, RepDefUnraveler};
262use crate::version::LanceFileVersion;
263use crate::{BufferScheduler, EncodingsIo};
264
265const BATCH_SIZE_BYTES_WARNING: u64 = 10 * 1024 * 1024;
267
268#[derive(Debug)]
275pub enum PageEncoding {
276 Legacy(pb::ArrayEncoding),
277 Structural(pb::PageLayout),
278}
279
280impl PageEncoding {
281 pub fn as_legacy(&self) -> &pb::ArrayEncoding {
282 match self {
283 Self::Legacy(enc) => enc,
284 Self::Structural(_) => panic!("Expected a legacy encoding"),
285 }
286 }
287
288 pub fn as_structural(&self) -> &pb::PageLayout {
289 match self {
290 Self::Structural(enc) => enc,
291 Self::Legacy(_) => panic!("Expected a structural encoding"),
292 }
293 }
294
295 pub fn is_structural(&self) -> bool {
296 matches!(self, Self::Structural(_))
297 }
298}
299
300#[derive(Debug)]
304pub struct PageInfo {
305 pub num_rows: u64,
307 pub priority: u64,
311 pub encoding: PageEncoding,
313 pub buffer_offsets_and_sizes: Arc<[(u64, u64)]>,
315}
316
317#[derive(Debug, Clone)]
321pub struct ColumnInfo {
322 pub index: u32,
324 pub page_infos: Arc<[PageInfo]>,
326 pub buffer_offsets_and_sizes: Arc<[(u64, u64)]>,
328 pub encoding: pb::ColumnEncoding,
329}
330
331impl ColumnInfo {
332 pub fn new(
334 index: u32,
335 page_infos: Arc<[PageInfo]>,
336 buffer_offsets_and_sizes: Vec<(u64, u64)>,
337 encoding: pb::ColumnEncoding,
338 ) -> Self {
339 Self {
340 index,
341 page_infos,
342 buffer_offsets_and_sizes: buffer_offsets_and_sizes.into_boxed_slice().into(),
343 encoding,
344 }
345 }
346
347 pub fn is_structural(&self) -> bool {
348 self.page_infos
349 .first()
351 .map(|page| page.encoding.is_structural())
352 .unwrap_or(false)
353 }
354}
355
356enum RootScheduler {
357 Structural(Box<dyn StructuralFieldScheduler>),
358 Legacy(Arc<dyn FieldScheduler>),
359}
360
361impl RootScheduler {
362 fn as_legacy(&self) -> &Arc<dyn FieldScheduler> {
363 match self {
364 Self::Structural(_) => panic!("Expected a legacy scheduler"),
365 Self::Legacy(s) => s,
366 }
367 }
368
369 fn as_structural(&self) -> &dyn StructuralFieldScheduler {
370 match self {
371 Self::Structural(s) => s.as_ref(),
372 Self::Legacy(_) => panic!("Expected a structural scheduler"),
373 }
374 }
375}
376
377pub struct DecodeBatchScheduler {
399 root_scheduler: RootScheduler,
400 pub root_fields: Fields,
401 cache: Arc<FileMetadataCache>,
402}
403
404pub struct ColumnInfoIter<'a> {
405 column_infos: Vec<Arc<ColumnInfo>>,
406 column_indices: &'a [u32],
407 column_info_pos: usize,
408 column_indices_pos: usize,
409}
410
411impl<'a> ColumnInfoIter<'a> {
412 pub fn new(column_infos: Vec<Arc<ColumnInfo>>, column_indices: &'a [u32]) -> Self {
413 let initial_pos = column_indices[0] as usize;
414 Self {
415 column_infos,
416 column_indices,
417 column_info_pos: initial_pos,
418 column_indices_pos: 0,
419 }
420 }
421
422 pub fn peek(&self) -> &Arc<ColumnInfo> {
423 &self.column_infos[self.column_info_pos]
424 }
425
426 pub fn peek_transform(&mut self, transform: impl FnOnce(Arc<ColumnInfo>) -> Arc<ColumnInfo>) {
427 let column_info = self.column_infos[self.column_info_pos].clone();
428 let transformed = transform(column_info);
429 self.column_infos[self.column_info_pos] = transformed;
430 }
431
432 pub fn expect_next(&mut self) -> Result<&Arc<ColumnInfo>> {
433 self.next().ok_or_else(|| {
434 Error::invalid_input(
435 "there were more fields in the schema than provided column indices / infos",
436 location!(),
437 )
438 })
439 }
440
441 fn next(&mut self) -> Option<&Arc<ColumnInfo>> {
442 if self.column_info_pos < self.column_infos.len() {
443 let info = &self.column_infos[self.column_info_pos];
444 self.column_info_pos += 1;
445 Some(info)
446 } else {
447 None
448 }
449 }
450
451 pub(crate) fn next_top_level(&mut self) {
452 self.column_indices_pos += 1;
453 if self.column_indices_pos < self.column_indices.len() {
454 self.column_info_pos = self.column_indices[self.column_indices_pos] as usize;
455 } else {
456 self.column_info_pos = self.column_infos.len();
457 }
458 }
459}
460
461pub trait MiniBlockDecompressor: std::fmt::Debug + Send + Sync {
462 fn decompress(&self, data: Vec<LanceBuffer>, num_values: u64) -> Result<DataBlock>;
463}
464
465pub trait FixedPerValueDecompressor: std::fmt::Debug + Send + Sync {
466 fn decompress(&self, data: FixedWidthDataBlock, num_values: u64) -> Result<DataBlock>;
468 fn bits_per_value(&self) -> u64;
472}
473
474pub trait VariablePerValueDecompressor: std::fmt::Debug + Send + Sync {
475 fn decompress(&self, data: VariableWidthBlock) -> Result<DataBlock>;
477}
478
479pub trait BlockDecompressor: std::fmt::Debug + Send + Sync {
480 fn decompress(&self, data: LanceBuffer, num_values: u64) -> Result<DataBlock>;
481}
482
483pub trait DecompressorStrategy: std::fmt::Debug + Send + Sync {
484 fn create_miniblock_decompressor(
485 &self,
486 description: &pb::ArrayEncoding,
487 ) -> Result<Box<dyn MiniBlockDecompressor>>;
488
489 fn create_fixed_per_value_decompressor(
490 &self,
491 description: &pb::ArrayEncoding,
492 ) -> Result<Box<dyn FixedPerValueDecompressor>>;
493
494 fn create_variable_per_value_decompressor(
495 &self,
496 description: &pb::ArrayEncoding,
497 ) -> Result<Box<dyn VariablePerValueDecompressor>>;
498
499 fn create_block_decompressor(
500 &self,
501 description: &pb::ArrayEncoding,
502 ) -> Result<Box<dyn BlockDecompressor>>;
503}
504
505#[derive(Debug, Default)]
506pub struct CoreDecompressorStrategy {}
507
508impl DecompressorStrategy for CoreDecompressorStrategy {
509 fn create_miniblock_decompressor(
510 &self,
511 description: &pb::ArrayEncoding,
512 ) -> Result<Box<dyn MiniBlockDecompressor>> {
513 match description.array_encoding.as_ref().unwrap() {
514 pb::array_encoding::ArrayEncoding::Flat(flat) => {
515 Ok(Box::new(ValueDecompressor::from_flat(flat)))
516 }
517 pb::array_encoding::ArrayEncoding::InlineBitpacking(description) => {
518 Ok(Box::new(InlineBitpacking::from_description(description)))
519 }
520 pb::array_encoding::ArrayEncoding::Variable(_) => {
521 Ok(Box::new(BinaryMiniBlockDecompressor::default()))
522 }
523 pb::array_encoding::ArrayEncoding::Fsst(description) => {
524 Ok(Box::new(FsstMiniBlockDecompressor::new(description)))
525 }
526 pb::array_encoding::ArrayEncoding::PackedStructFixedWidthMiniBlock(description) => {
527 Ok(Box::new(PackedStructFixedWidthMiniBlockDecompressor::new(
528 description,
529 )))
530 }
531 pb::array_encoding::ArrayEncoding::FixedSizeList(fsl) => {
532 Ok(Box::new(ValueDecompressor::from_fsl(fsl)))
535 }
536 _ => todo!(),
537 }
538 }
539
540 fn create_fixed_per_value_decompressor(
541 &self,
542 description: &pb::ArrayEncoding,
543 ) -> Result<Box<dyn FixedPerValueDecompressor>> {
544 match description.array_encoding.as_ref().unwrap() {
545 pb::array_encoding::ArrayEncoding::Flat(flat) => {
546 Ok(Box::new(ValueDecompressor::from_flat(flat)))
547 }
548 pb::array_encoding::ArrayEncoding::FixedSizeList(fsl) => {
549 Ok(Box::new(ValueDecompressor::from_fsl(fsl)))
550 }
551 _ => todo!("fixed-per-value decompressor for {:?}", description),
552 }
553 }
554
555 fn create_variable_per_value_decompressor(
556 &self,
557 description: &pb::ArrayEncoding,
558 ) -> Result<Box<dyn VariablePerValueDecompressor>> {
559 match *description.array_encoding.as_ref().unwrap() {
560 pb::array_encoding::ArrayEncoding::Variable(variable) => {
561 assert!(variable.bits_per_offset < u8::MAX as u32);
562 Ok(Box::new(VariableDecoder::default()))
563 }
564 pb::array_encoding::ArrayEncoding::Fsst(ref fsst) => {
565 Ok(Box::new(FsstPerValueDecompressor::new(
566 LanceBuffer::from_bytes(fsst.symbol_table.clone(), 1),
567 Box::new(VariableDecoder::default()),
568 )))
569 }
570 pb::array_encoding::ArrayEncoding::Block(ref block) => Ok(Box::new(
571 CompressedBufferEncoder::from_scheme(&block.scheme)?,
572 )),
573 _ => todo!("variable-per-value decompressor for {:?}", description),
574 }
575 }
576
577 fn create_block_decompressor(
578 &self,
579 description: &pb::ArrayEncoding,
580 ) -> Result<Box<dyn BlockDecompressor>> {
581 match description.array_encoding.as_ref().unwrap() {
582 pb::array_encoding::ArrayEncoding::Flat(flat) => {
583 Ok(Box::new(ValueDecompressor::from_flat(flat)))
584 }
585 pb::array_encoding::ArrayEncoding::Constant(constant) => {
586 let scalar = LanceBuffer::from_bytes(constant.value.clone(), 1);
587 Ok(Box::new(ConstantDecompressor::new(scalar)))
588 }
589 pb::array_encoding::ArrayEncoding::Variable(_) => {
590 Ok(Box::new(BinaryBlockDecompressor::default()))
591 }
592 _ => todo!(),
593 }
594 }
595}
596
597#[derive(Debug)]
599pub struct CoreFieldDecoderStrategy {
600 pub validate_data: bool,
601 pub decompressor_strategy: Arc<dyn DecompressorStrategy>,
602}
603
604impl Default for CoreFieldDecoderStrategy {
605 fn default() -> Self {
606 Self {
607 validate_data: false,
608 decompressor_strategy: Arc::new(CoreDecompressorStrategy {}),
609 }
610 }
611}
612
613impl CoreFieldDecoderStrategy {
614 fn ensure_values_encoded(column_info: &ColumnInfo, field_name: &str) -> Result<()> {
617 let column_encoding = column_info
618 .encoding
619 .column_encoding
620 .as_ref()
621 .ok_or_else(|| {
622 Error::invalid_input(
623 format!(
624 "the column at index {} was missing a ColumnEncoding",
625 column_info.index
626 ),
627 location!(),
628 )
629 })?;
630 if matches!(
631 column_encoding,
632 pb::column_encoding::ColumnEncoding::Values(_)
633 ) {
634 Ok(())
635 } else {
636 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!()))
637 }
638 }
639
640 fn is_primitive(data_type: &DataType) -> bool {
641 if data_type.is_primitive() {
642 true
643 } else {
644 match data_type {
645 DataType::Boolean | DataType::Null | DataType::FixedSizeBinary(_) => true,
647 DataType::FixedSizeList(inner, _) => Self::is_primitive(inner.data_type()),
648 _ => false,
649 }
650 }
651 }
652
653 fn create_primitive_scheduler(
654 &self,
655 field: &Field,
656 column: &ColumnInfo,
657 buffers: FileBuffers,
658 ) -> Result<Box<dyn FieldScheduler>> {
659 Self::ensure_values_encoded(column, &field.name)?;
660 let column_buffers = ColumnBuffers {
662 file_buffers: buffers,
663 positions_and_sizes: &column.buffer_offsets_and_sizes,
664 };
665 Ok(Box::new(PrimitiveFieldScheduler::new(
666 column.index,
667 field.data_type(),
668 column.page_infos.clone(),
669 column_buffers,
670 self.validate_data,
671 )))
672 }
673
674 fn check_simple_struct(column_info: &ColumnInfo, field_name: &str) -> Result<()> {
676 Self::ensure_values_encoded(column_info, field_name)?;
677 if column_info.page_infos.len() != 1 {
678 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!() });
679 }
680 let encoding = &column_info.page_infos[0].encoding;
681 match encoding.as_legacy().array_encoding.as_ref().unwrap() {
682 pb::array_encoding::ArrayEncoding::Struct(_) => Ok(()),
683 _ => 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!() }),
684 }
685 }
686
687 fn check_packed_struct(column_info: &ColumnInfo) -> bool {
688 let encoding = &column_info.page_infos[0].encoding;
689 matches!(
690 encoding.as_legacy().array_encoding.as_ref().unwrap(),
691 pb::array_encoding::ArrayEncoding::PackedStruct(_)
692 )
693 }
694
695 fn create_list_scheduler(
696 &self,
697 list_field: &Field,
698 column_infos: &mut ColumnInfoIter,
699 buffers: FileBuffers,
700 offsets_column: &ColumnInfo,
701 ) -> Result<Box<dyn FieldScheduler>> {
702 Self::ensure_values_encoded(offsets_column, &list_field.name)?;
703 let offsets_column_buffers = ColumnBuffers {
704 file_buffers: buffers,
705 positions_and_sizes: &offsets_column.buffer_offsets_and_sizes,
706 };
707 let items_scheduler =
708 self.create_legacy_field_scheduler(&list_field.children[0], column_infos, buffers)?;
709
710 let (inner_infos, null_offset_adjustments): (Vec<_>, Vec<_>) = offsets_column
711 .page_infos
712 .iter()
713 .filter(|offsets_page| offsets_page.num_rows > 0)
714 .map(|offsets_page| {
715 if let Some(pb::array_encoding::ArrayEncoding::List(list_encoding)) =
716 &offsets_page.encoding.as_legacy().array_encoding
717 {
718 let inner = PageInfo {
719 buffer_offsets_and_sizes: offsets_page.buffer_offsets_and_sizes.clone(),
720 encoding: PageEncoding::Legacy(
721 list_encoding.offsets.as_ref().unwrap().as_ref().clone(),
722 ),
723 num_rows: offsets_page.num_rows,
724 priority: 0,
725 };
726 (
727 inner,
728 OffsetPageInfo {
729 offsets_in_page: offsets_page.num_rows,
730 null_offset_adjustment: list_encoding.null_offset_adjustment,
731 num_items_referenced_by_page: list_encoding.num_items,
732 },
733 )
734 } else {
735 panic!("Expected a list column");
737 }
738 })
739 .unzip();
740 let inner = Arc::new(PrimitiveFieldScheduler::new(
741 offsets_column.index,
742 DataType::UInt64,
743 Arc::from(inner_infos.into_boxed_slice()),
744 offsets_column_buffers,
745 self.validate_data,
746 )) as Arc<dyn FieldScheduler>;
747 let items_field = match list_field.data_type() {
748 DataType::List(inner) => inner,
749 DataType::LargeList(inner) => inner,
750 _ => unreachable!(),
751 };
752 let offset_type = if matches!(list_field.data_type(), DataType::List(_)) {
753 DataType::Int32
754 } else {
755 DataType::Int64
756 };
757 Ok(Box::new(ListFieldScheduler::new(
758 inner,
759 items_scheduler.into(),
760 items_field,
761 offset_type,
762 null_offset_adjustments,
763 )))
764 }
765
766 fn unwrap_blob(column_info: &ColumnInfo) -> Option<ColumnInfo> {
767 if let column_encoding::ColumnEncoding::Blob(blob) =
768 column_info.encoding.column_encoding.as_ref().unwrap()
769 {
770 let mut column_info = column_info.clone();
771 column_info.encoding = blob.inner.as_ref().unwrap().as_ref().clone();
772 Some(column_info)
773 } else {
774 None
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 scheduler = Box::new(StructuralPrimitiveFieldScheduler::try_new(
787 column_info.as_ref(),
788 self.decompressor_strategy.as_ref(),
789 )?);
790
791 column_infos.next_top_level();
793
794 return Ok(scheduler);
795 }
796 match &data_type {
797 DataType::Struct(fields) => {
798 if field.is_packed_struct() {
799 let column_info = column_infos.expect_next()?;
800 let scheduler = Box::new(StructuralPrimitiveFieldScheduler::try_new(
801 column_info.as_ref(),
802 self.decompressor_strategy.as_ref(),
803 )?);
804
805 column_infos.next_top_level();
807
808 return Ok(scheduler);
809 }
810 let mut child_schedulers = Vec::with_capacity(field.children.len());
811 for field in field.children.iter() {
812 let field_scheduler =
813 self.create_structural_field_scheduler(field, column_infos)?;
814 child_schedulers.push(field_scheduler);
815 }
816
817 let fields = fields.clone();
818 Ok(
819 Box::new(StructuralStructScheduler::new(child_schedulers, fields))
820 as Box<dyn StructuralFieldScheduler>,
821 )
822 }
823 DataType::Binary | DataType::Utf8 => {
824 let column_info = column_infos.expect_next()?;
825 let scheduler = Box::new(StructuralPrimitiveFieldScheduler::try_new(
826 column_info.as_ref(),
827 self.decompressor_strategy.as_ref(),
828 )?);
829 column_infos.next_top_level();
830 Ok(scheduler)
831 }
832 DataType::List(_) | DataType::LargeList(_) => {
833 let child = field
834 .children
835 .first()
836 .expect("List field must have a child");
837 let child_scheduler =
838 self.create_structural_field_scheduler(child, column_infos)?;
839 Ok(Box::new(StructuralListScheduler::new(child_scheduler))
840 as Box<dyn StructuralFieldScheduler>)
841 }
842 _ => todo!(),
843 }
844 }
845
846 fn create_legacy_field_scheduler(
847 &self,
848 field: &Field,
849 column_infos: &mut ColumnInfoIter,
850 buffers: FileBuffers,
851 ) -> Result<Box<dyn FieldScheduler>> {
852 let data_type = field.data_type();
853 if Self::is_primitive(&data_type) {
854 let column_info = column_infos.expect_next()?;
855 let scheduler = self.create_primitive_scheduler(field, column_info, buffers)?;
856 return Ok(scheduler);
857 } else if data_type.is_binary_like() {
858 let column_info = column_infos.next().unwrap().clone();
859 if let Some(blob_col) = Self::unwrap_blob(column_info.as_ref()) {
861 let desc_scheduler =
862 self.create_primitive_scheduler(&BLOB_DESC_LANCE_FIELD, &blob_col, buffers)?;
863 let blob_scheduler = Box::new(BlobFieldScheduler::new(desc_scheduler.into()));
864 return Ok(blob_scheduler);
865 }
866 if let Some(page_info) = column_info.page_infos.first() {
867 if matches!(
868 page_info.encoding.as_legacy(),
869 pb::ArrayEncoding {
870 array_encoding: Some(pb::array_encoding::ArrayEncoding::List(..))
871 }
872 ) {
873 let list_type = if matches!(data_type, DataType::Utf8 | DataType::Binary) {
874 DataType::List(Arc::new(ArrowField::new("item", DataType::UInt8, false)))
875 } else {
876 DataType::LargeList(Arc::new(ArrowField::new(
877 "item",
878 DataType::UInt8,
879 false,
880 )))
881 };
882 let list_field = Field::try_from(ArrowField::new(
883 field.name.clone(),
884 list_type,
885 field.nullable,
886 ))
887 .unwrap();
888 let list_scheduler = self.create_list_scheduler(
889 &list_field,
890 column_infos,
891 buffers,
892 &column_info,
893 )?;
894 let binary_scheduler = Box::new(BinaryFieldScheduler::new(
895 list_scheduler.into(),
896 field.data_type(),
897 ));
898 return Ok(binary_scheduler);
899 } else {
900 let scheduler =
901 self.create_primitive_scheduler(field, &column_info, buffers)?;
902 return Ok(scheduler);
903 }
904 } else {
905 return self.create_primitive_scheduler(field, &column_info, buffers);
906 }
907 }
908 match &data_type {
909 DataType::FixedSizeList(inner, _dimension) => {
910 if Self::is_primitive(inner.data_type()) {
913 let primitive_col = column_infos.expect_next()?;
914 let scheduler =
915 self.create_primitive_scheduler(field, primitive_col, buffers)?;
916 Ok(scheduler)
917 } else {
918 todo!()
919 }
920 }
921 DataType::Dictionary(_key_type, value_type) => {
922 if Self::is_primitive(value_type) || value_type.is_binary_like() {
923 let primitive_col = column_infos.expect_next()?;
924 let scheduler =
925 self.create_primitive_scheduler(field, primitive_col, buffers)?;
926 Ok(scheduler)
927 } else {
928 Err(Error::NotSupported {
929 source: format!(
930 "No way to decode into a dictionary field of type {}",
931 value_type
932 )
933 .into(),
934 location: location!(),
935 })
936 }
937 }
938 DataType::List(_) | DataType::LargeList(_) => {
939 let offsets_column = column_infos.expect_next()?.clone();
940 column_infos.next_top_level();
941 self.create_list_scheduler(field, column_infos, buffers, &offsets_column)
942 }
943 DataType::Struct(fields) => {
944 let column_info = column_infos.expect_next()?;
945
946 if let Some(blob_col) = Self::unwrap_blob(column_info.as_ref()) {
948 return self.create_primitive_scheduler(field, &blob_col, buffers);
950 }
951
952 if Self::check_packed_struct(column_info) {
953 self.create_primitive_scheduler(field, column_info, buffers)
955 } else {
956 Self::check_simple_struct(column_info, &field.name).unwrap();
958 let num_rows = column_info
959 .page_infos
960 .iter()
961 .map(|page| page.num_rows)
962 .sum();
963 let mut child_schedulers = Vec::with_capacity(field.children.len());
964 for field in &field.children {
965 column_infos.next_top_level();
966 let field_scheduler =
967 self.create_legacy_field_scheduler(field, column_infos, buffers)?;
968 child_schedulers.push(Arc::from(field_scheduler));
969 }
970
971 let fields = fields.clone();
972 Ok(Box::new(SimpleStructScheduler::new(
973 child_schedulers,
974 fields,
975 num_rows,
976 )))
977 }
978 }
979 _ => todo!(),
981 }
982 }
983}
984
985fn root_column(num_rows: u64) -> ColumnInfo {
987 let num_root_pages = num_rows.div_ceil(u32::MAX as u64);
988 let final_page_num_rows = num_rows % (u32::MAX as u64);
989 let root_pages = (0..num_root_pages)
990 .map(|i| PageInfo {
991 num_rows: if i == num_root_pages - 1 {
992 final_page_num_rows
993 } else {
994 u64::MAX
995 },
996 encoding: PageEncoding::Legacy(pb::ArrayEncoding {
997 array_encoding: Some(pb::array_encoding::ArrayEncoding::Struct(
998 pb::SimpleStruct {},
999 )),
1000 }),
1001 priority: 0, buffer_offsets_and_sizes: Arc::new([]),
1003 })
1004 .collect::<Vec<_>>();
1005 ColumnInfo {
1006 buffer_offsets_and_sizes: Arc::new([]),
1007 encoding: values_column_encoding(),
1008 index: u32::MAX,
1009 page_infos: Arc::from(root_pages),
1010 }
1011}
1012
1013pub enum RootDecoder {
1014 Structural(StructuralStructDecoder),
1015 Legacy(SimpleStructDecoder),
1016}
1017
1018impl RootDecoder {
1019 pub fn into_structural(self) -> StructuralStructDecoder {
1020 match self {
1021 Self::Structural(decoder) => decoder,
1022 Self::Legacy(_) => panic!("Expected a structural decoder"),
1023 }
1024 }
1025
1026 pub fn into_legacy(self) -> SimpleStructDecoder {
1027 match self {
1028 Self::Legacy(decoder) => decoder,
1029 Self::Structural(_) => panic!("Expected a legacy decoder"),
1030 }
1031 }
1032}
1033
1034impl DecodeBatchScheduler {
1035 #[allow(clippy::too_many_arguments)]
1038 pub async fn try_new<'a>(
1039 schema: &'a Schema,
1040 column_indices: &[u32],
1041 column_infos: &[Arc<ColumnInfo>],
1042 file_buffer_positions_and_sizes: &'a Vec<(u64, u64)>,
1043 num_rows: u64,
1044 _decoder_plugins: Arc<DecoderPlugins>,
1045 io: Arc<dyn EncodingsIo>,
1046 cache: Arc<FileMetadataCache>,
1047 filter: &FilterExpression,
1048 ) -> Result<Self> {
1049 assert!(num_rows > 0);
1050 let buffers = FileBuffers {
1051 positions_and_sizes: file_buffer_positions_and_sizes,
1052 };
1053 let arrow_schema = ArrowSchema::from(schema);
1054 let root_fields = arrow_schema.fields().clone();
1055 let root_type = DataType::Struct(root_fields.clone());
1056 let mut root_field = Field::try_from(&ArrowField::new("root", root_type, false))?;
1057 root_field.children.clone_from(&schema.fields);
1061 root_field
1062 .metadata
1063 .insert("__lance_decoder_root".to_string(), "true".to_string());
1064
1065 if column_infos[0].is_structural() {
1066 let mut column_iter = ColumnInfoIter::new(column_infos.to_vec(), column_indices);
1067
1068 let mut root_scheduler = CoreFieldDecoderStrategy::default()
1069 .create_structural_field_scheduler(&root_field, &mut column_iter)?;
1070
1071 let context = SchedulerContext::new(io, cache.clone());
1072 root_scheduler.initialize(filter, &context).await?;
1073
1074 Ok(Self {
1075 root_scheduler: RootScheduler::Structural(root_scheduler),
1076 root_fields,
1077 cache,
1078 })
1079 } else {
1080 let mut columns = Vec::with_capacity(column_infos.len() + 1);
1083 columns.push(Arc::new(root_column(num_rows)));
1084 columns.extend(column_infos.iter().cloned());
1085
1086 let adjusted_column_indices = [0_u32]
1087 .into_iter()
1088 .chain(column_indices.iter().map(|i| i.saturating_add(1)))
1089 .collect::<Vec<_>>();
1090 let mut column_iter = ColumnInfoIter::new(columns, &adjusted_column_indices);
1091 let root_scheduler = CoreFieldDecoderStrategy::default()
1092 .create_legacy_field_scheduler(&root_field, &mut column_iter, buffers)?;
1093
1094 let context = SchedulerContext::new(io, cache.clone());
1095 root_scheduler.initialize(filter, &context).await?;
1096
1097 Ok(Self {
1098 root_scheduler: RootScheduler::Legacy(root_scheduler.into()),
1099 root_fields,
1100 cache,
1101 })
1102 }
1103 }
1104
1105 pub fn from_scheduler(
1106 root_scheduler: Arc<dyn FieldScheduler>,
1107 root_fields: Fields,
1108 cache: Arc<FileMetadataCache>,
1109 ) -> Self {
1110 Self {
1111 root_scheduler: RootScheduler::Legacy(root_scheduler),
1112 root_fields,
1113 cache,
1114 }
1115 }
1116
1117 fn do_schedule_ranges_structural(
1118 &mut self,
1119 ranges: &[Range<u64>],
1120 filter: &FilterExpression,
1121 io: Arc<dyn EncodingsIo>,
1122 mut schedule_action: impl FnMut(Result<DecoderMessage>) -> bool,
1123 ) {
1124 let root_scheduler = self.root_scheduler.as_structural();
1125 let mut context = SchedulerContext::new(io, self.cache.clone());
1126 let maybe_root_job = root_scheduler.schedule_ranges(ranges, filter);
1127 if let Err(schedule_ranges_err) = maybe_root_job {
1128 schedule_action(Err(schedule_ranges_err));
1129 return;
1130 }
1131 let mut root_job = maybe_root_job.unwrap();
1132 let mut num_rows_scheduled = 0;
1133 loop {
1134 let maybe_next_scan_line = root_job.schedule_next(&mut context);
1135 if let Err(err) = maybe_next_scan_line {
1136 schedule_action(Err(err));
1137 return;
1138 }
1139 let next_scan_line = maybe_next_scan_line.unwrap();
1140 match next_scan_line {
1141 Some(next_scan_line) => {
1142 trace!(
1143 "Scheduled scan line of {} rows and {} decoders",
1144 next_scan_line.rows_scheduled,
1145 next_scan_line.decoders.len()
1146 );
1147 num_rows_scheduled += next_scan_line.rows_scheduled;
1148 if !schedule_action(Ok(DecoderMessage {
1149 scheduled_so_far: num_rows_scheduled,
1150 decoders: next_scan_line.decoders,
1151 })) {
1152 return;
1154 }
1155 }
1156 None => return,
1157 }
1158 }
1159 }
1160
1161 fn do_schedule_ranges_legacy(
1162 &mut self,
1163 ranges: &[Range<u64>],
1164 filter: &FilterExpression,
1165 io: Arc<dyn EncodingsIo>,
1166 mut schedule_action: impl FnMut(Result<DecoderMessage>) -> bool,
1167 priority: Option<Box<dyn PriorityRange>>,
1171 ) {
1172 let root_scheduler = self.root_scheduler.as_legacy();
1173 let rows_requested = ranges.iter().map(|r| r.end - r.start).sum::<u64>();
1174 trace!(
1175 "Scheduling {} ranges across {}..{} ({} rows){}",
1176 ranges.len(),
1177 ranges.first().unwrap().start,
1178 ranges.last().unwrap().end,
1179 rows_requested,
1180 priority
1181 .as_ref()
1182 .map(|p| format!(" (priority={:?})", p))
1183 .unwrap_or_default()
1184 );
1185
1186 let mut context = SchedulerContext::new(io, self.cache.clone());
1187 let maybe_root_job = root_scheduler.schedule_ranges(ranges, filter);
1188 if let Err(schedule_ranges_err) = maybe_root_job {
1189 schedule_action(Err(schedule_ranges_err));
1190 return;
1191 }
1192 let mut root_job = maybe_root_job.unwrap();
1193 let mut num_rows_scheduled = 0;
1194 let mut rows_to_schedule = root_job.num_rows();
1195 let mut priority = priority.unwrap_or(Box::new(SimplePriorityRange::new(0)));
1196 trace!("Scheduled ranges refined to {} rows", rows_to_schedule);
1197 while rows_to_schedule > 0 {
1198 let maybe_next_scan_line = root_job.schedule_next(&mut context, priority.as_ref());
1199 if let Err(schedule_next_err) = maybe_next_scan_line {
1200 schedule_action(Err(schedule_next_err));
1201 return;
1202 }
1203 let next_scan_line = maybe_next_scan_line.unwrap();
1204 priority.advance(next_scan_line.rows_scheduled);
1205 num_rows_scheduled += next_scan_line.rows_scheduled;
1206 rows_to_schedule -= next_scan_line.rows_scheduled;
1207 trace!(
1208 "Scheduled scan line of {} rows and {} decoders",
1209 next_scan_line.rows_scheduled,
1210 next_scan_line.decoders.len()
1211 );
1212 if !schedule_action(Ok(DecoderMessage {
1213 scheduled_so_far: num_rows_scheduled,
1214 decoders: next_scan_line.decoders,
1215 })) {
1216 return;
1218 }
1219
1220 trace!("Finished scheduling {} ranges", ranges.len());
1221 }
1222 }
1223
1224 fn do_schedule_ranges(
1225 &mut self,
1226 ranges: &[Range<u64>],
1227 filter: &FilterExpression,
1228 io: Arc<dyn EncodingsIo>,
1229 schedule_action: impl FnMut(Result<DecoderMessage>) -> bool,
1230 priority: Option<Box<dyn PriorityRange>>,
1234 ) {
1235 match &self.root_scheduler {
1236 RootScheduler::Legacy(_) => {
1237 self.do_schedule_ranges_legacy(ranges, filter, io, schedule_action, priority)
1238 }
1239 RootScheduler::Structural(_) => {
1240 self.do_schedule_ranges_structural(ranges, filter, io, schedule_action)
1241 }
1242 }
1243 }
1244
1245 pub fn schedule_ranges_to_vec(
1248 &mut self,
1249 ranges: &[Range<u64>],
1250 filter: &FilterExpression,
1251 io: Arc<dyn EncodingsIo>,
1252 priority: Option<Box<dyn PriorityRange>>,
1253 ) -> Result<Vec<DecoderMessage>> {
1254 let mut decode_messages = Vec::new();
1255 self.do_schedule_ranges(
1256 ranges,
1257 filter,
1258 io,
1259 |msg| {
1260 decode_messages.push(msg);
1261 true
1262 },
1263 priority,
1264 );
1265 decode_messages.into_iter().collect::<Result<Vec<_>>>()
1266 }
1267
1268 #[instrument(skip_all)]
1278 pub fn schedule_ranges(
1279 &mut self,
1280 ranges: &[Range<u64>],
1281 filter: &FilterExpression,
1282 sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
1283 scheduler: Arc<dyn EncodingsIo>,
1284 ) {
1285 self.do_schedule_ranges(
1286 ranges,
1287 filter,
1288 scheduler,
1289 |msg| {
1290 match sink.send(msg) {
1291 Ok(_) => true,
1292 Err(SendError { .. }) => {
1293 debug!(
1296 "schedule_ranges aborting early since decoder appears to have been dropped"
1297 );
1298 false
1299 }
1300 }
1301 },
1302 None,
1303 )
1304 }
1305
1306 #[instrument(skip_all)]
1314 pub fn schedule_range(
1315 &mut self,
1316 range: Range<u64>,
1317 filter: &FilterExpression,
1318 sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
1319 scheduler: Arc<dyn EncodingsIo>,
1320 ) {
1321 self.schedule_ranges(&[range], filter, sink, scheduler)
1322 }
1323
1324 pub fn schedule_take(
1332 &mut self,
1333 indices: &[u64],
1334 filter: &FilterExpression,
1335 sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
1336 scheduler: Arc<dyn EncodingsIo>,
1337 ) {
1338 debug_assert!(indices.windows(2).all(|w| w[0] <= w[1]));
1339 if indices.is_empty() {
1340 return;
1341 }
1342 trace!("Scheduling take of {} rows", indices.len());
1343 let ranges = Self::indices_to_ranges(indices);
1344 self.schedule_ranges(&ranges, filter, sink, scheduler)
1345 }
1346
1347 fn indices_to_ranges(indices: &[u64]) -> Vec<Range<u64>> {
1349 let mut ranges = Vec::new();
1350 let mut start = indices[0];
1351
1352 for window in indices.windows(2) {
1353 if window[1] != window[0] + 1 {
1354 ranges.push(start..window[0] + 1);
1355 start = window[1];
1356 }
1357 }
1358
1359 ranges.push(start..*indices.last().unwrap() + 1);
1360 ranges
1361 }
1362}
1363
1364pub struct ReadBatchTask {
1365 pub task: BoxFuture<'static, Result<RecordBatch>>,
1366 pub num_rows: u32,
1367}
1368
1369pub struct BatchDecodeStream {
1371 context: DecoderContext,
1372 root_decoder: SimpleStructDecoder,
1373 rows_remaining: u64,
1374 rows_per_batch: u32,
1375 rows_scheduled: u64,
1376 rows_drained: u64,
1377 scheduler_exhausted: bool,
1378 emitted_batch_size_warning: Arc<Once>,
1379}
1380
1381impl BatchDecodeStream {
1382 pub fn new(
1393 scheduled: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
1394 rows_per_batch: u32,
1395 num_rows: u64,
1396 root_decoder: SimpleStructDecoder,
1397 ) -> Self {
1398 Self {
1399 context: DecoderContext::new(scheduled),
1400 root_decoder,
1401 rows_remaining: num_rows,
1402 rows_per_batch,
1403 rows_scheduled: 0,
1404 rows_drained: 0,
1405 scheduler_exhausted: false,
1406 emitted_batch_size_warning: Arc::new(Once::new()),
1407 }
1408 }
1409
1410 fn accept_decoder(&mut self, decoder: DecoderReady) -> Result<()> {
1411 if decoder.path.is_empty() {
1412 Ok(())
1414 } else {
1415 self.root_decoder.accept_child(decoder)
1416 }
1417 }
1418
1419 async fn wait_for_scheduled(&mut self, scheduled_need: u64) -> Result<u64> {
1420 if self.scheduler_exhausted {
1421 return Ok(self.rows_scheduled);
1422 }
1423 while self.rows_scheduled < scheduled_need {
1424 let next_message = self.context.source.recv().await;
1425 match next_message {
1426 Some(scan_line) => {
1427 let scan_line = scan_line?;
1428 self.rows_scheduled = scan_line.scheduled_so_far;
1429 for message in scan_line.decoders {
1430 self.accept_decoder(message.into_legacy())?;
1431 }
1432 }
1433 None => {
1434 self.scheduler_exhausted = true;
1438 return Ok(self.rows_scheduled);
1439 }
1440 }
1441 }
1442 Ok(scheduled_need)
1443 }
1444
1445 #[instrument(level = "debug", skip_all)]
1446 async fn next_batch_task(&mut self) -> Result<Option<NextDecodeTask>> {
1447 trace!(
1448 "Draining batch task (rows_remaining={} rows_drained={} rows_scheduled={})",
1449 self.rows_remaining,
1450 self.rows_drained,
1451 self.rows_scheduled,
1452 );
1453 if self.rows_remaining == 0 {
1454 return Ok(None);
1455 }
1456
1457 let mut to_take = self.rows_remaining.min(self.rows_per_batch as u64);
1458 self.rows_remaining -= to_take;
1459
1460 let scheduled_need = (self.rows_drained + to_take).saturating_sub(self.rows_scheduled);
1461 trace!("scheduled_need = {} because rows_drained = {} and to_take = {} and rows_scheduled = {}", scheduled_need, self.rows_drained, to_take, self.rows_scheduled);
1462 if scheduled_need > 0 {
1463 let desired_scheduled = scheduled_need + self.rows_scheduled;
1464 trace!(
1465 "Draining from scheduler (desire at least {} scheduled rows)",
1466 desired_scheduled
1467 );
1468 let actually_scheduled = self.wait_for_scheduled(desired_scheduled).await?;
1469 if actually_scheduled < desired_scheduled {
1470 let under_scheduled = desired_scheduled - actually_scheduled;
1471 to_take -= under_scheduled;
1472 }
1473 }
1474
1475 if to_take == 0 {
1476 return Ok(None);
1477 }
1478
1479 let loaded_need = self.rows_drained + to_take - 1;
1481 trace!(
1482 "Waiting for I/O (desire at least {} fully loaded rows)",
1483 loaded_need
1484 );
1485 self.root_decoder.wait_for_loaded(loaded_need).await?;
1486
1487 let next_task = self.root_decoder.drain(to_take)?;
1488 self.rows_drained += to_take;
1489 Ok(Some(next_task))
1490 }
1491
1492 pub fn into_stream(self) -> BoxStream<'static, ReadBatchTask> {
1493 let stream = futures::stream::unfold(self, |mut slf| async move {
1494 let next_task = slf.next_batch_task().await;
1495 let next_task = next_task.transpose().map(|next_task| {
1496 let num_rows = next_task.as_ref().map(|t| t.num_rows).unwrap_or(0);
1497 let emitted_batch_size_warning = slf.emitted_batch_size_warning.clone();
1498 let task = tokio::spawn(async move {
1499 let next_task = next_task?;
1500 next_task.into_batch(emitted_batch_size_warning)
1501 });
1502 (task, num_rows)
1503 });
1504 next_task.map(|(task, num_rows)| {
1505 let task = task.map(|join_wrapper| join_wrapper.unwrap()).boxed();
1506 debug_assert!(num_rows <= u32::MAX as u64);
1508 let next_task = ReadBatchTask {
1509 task,
1510 num_rows: num_rows as u32,
1511 };
1512 (next_task, slf)
1513 })
1514 });
1515 stream.boxed()
1516 }
1517}
1518
1519enum RootDecoderMessage {
1522 LoadedPage(LoadedPage),
1523 LegacyPage(DecoderReady),
1524}
1525trait RootDecoderType {
1526 fn accept_message(&mut self, message: RootDecoderMessage) -> Result<()>;
1527 fn drain_batch(&mut self, num_rows: u64) -> Result<NextDecodeTask>;
1528 fn wait(&mut self, loaded_need: u64, runtime: &tokio::runtime::Runtime) -> Result<()>;
1529}
1530impl RootDecoderType for StructuralStructDecoder {
1531 fn accept_message(&mut self, message: RootDecoderMessage) -> Result<()> {
1532 let RootDecoderMessage::LoadedPage(loaded_page) = message else {
1533 unreachable!()
1534 };
1535 self.accept_page(loaded_page)
1536 }
1537 fn drain_batch(&mut self, num_rows: u64) -> Result<NextDecodeTask> {
1538 self.drain_batch_task(num_rows)
1539 }
1540 fn wait(&mut self, _: u64, _: &tokio::runtime::Runtime) -> Result<()> {
1541 Ok(())
1543 }
1544}
1545impl RootDecoderType for SimpleStructDecoder {
1546 fn accept_message(&mut self, message: RootDecoderMessage) -> Result<()> {
1547 let RootDecoderMessage::LegacyPage(legacy_page) = message else {
1548 unreachable!()
1549 };
1550 self.accept_child(legacy_page)
1551 }
1552 fn drain_batch(&mut self, num_rows: u64) -> Result<NextDecodeTask> {
1553 self.drain(num_rows)
1554 }
1555 fn wait(&mut self, loaded_need: u64, runtime: &tokio::runtime::Runtime) -> Result<()> {
1556 runtime.block_on(self.wait_for_loaded(loaded_need))
1557 }
1558}
1559
1560struct BatchDecodeIterator<T: RootDecoderType> {
1562 messages: VecDeque<Result<DecoderMessage>>,
1563 root_decoder: T,
1564 rows_remaining: u64,
1565 rows_per_batch: u32,
1566 rows_scheduled: u64,
1567 rows_drained: u64,
1568 emitted_batch_size_warning: Arc<Once>,
1569 wait_for_io_runtime: tokio::runtime::Runtime,
1573 schema: Arc<ArrowSchema>,
1574}
1575
1576impl<T: RootDecoderType> BatchDecodeIterator<T> {
1577 pub fn new(
1579 messages: VecDeque<Result<DecoderMessage>>,
1580 rows_per_batch: u32,
1581 num_rows: u64,
1582 root_decoder: T,
1583 schema: Arc<ArrowSchema>,
1584 ) -> Self {
1585 Self {
1586 messages,
1587 root_decoder,
1588 rows_remaining: num_rows,
1589 rows_per_batch,
1590 rows_scheduled: 0,
1591 rows_drained: 0,
1592 wait_for_io_runtime: tokio::runtime::Builder::new_current_thread()
1593 .build()
1594 .unwrap(),
1595 emitted_batch_size_warning: Arc::new(Once::new()),
1596 schema,
1597 }
1598 }
1599
1600 fn wait_for_page(&self, unloaded_page: UnloadedPage) -> Result<LoadedPage> {
1605 match maybe_done(unloaded_page.0) {
1606 MaybeDone::Done(loaded_page) => loaded_page,
1608 MaybeDone::Future(fut) => self.wait_for_io_runtime.block_on(fut),
1610 MaybeDone::Gone => unreachable!(),
1611 }
1612 }
1613
1614 #[instrument(skip_all)]
1619 fn wait_for_io(&mut self, scheduled_need: u64) -> Result<u64> {
1620 while self.rows_scheduled < scheduled_need && !self.messages.is_empty() {
1621 let message = self.messages.pop_front().unwrap()?;
1622 self.rows_scheduled = message.scheduled_so_far;
1623 for decoder_message in message.decoders {
1624 match decoder_message {
1625 MessageType::UnloadedPage(unloaded_page) => {
1626 let loaded_page = self.wait_for_page(unloaded_page)?;
1627 self.root_decoder
1628 .accept_message(RootDecoderMessage::LoadedPage(loaded_page))?;
1629 }
1630 MessageType::DecoderReady(decoder_ready) => {
1631 if !decoder_ready.path.is_empty() {
1633 self.root_decoder
1634 .accept_message(RootDecoderMessage::LegacyPage(decoder_ready))?;
1635 }
1636 }
1637 }
1638 }
1639 }
1640
1641 let loaded_need = self.rows_drained + self.rows_per_batch as u64 - 1;
1642
1643 self.root_decoder
1644 .wait(loaded_need, &self.wait_for_io_runtime)?;
1645 Ok(self.rows_scheduled)
1646 }
1647
1648 #[instrument(level = "debug", skip_all)]
1649 fn next_batch_task(&mut self) -> Result<Option<RecordBatch>> {
1650 trace!(
1651 "Draining batch task (rows_remaining={} rows_drained={} rows_scheduled={})",
1652 self.rows_remaining,
1653 self.rows_drained,
1654 self.rows_scheduled,
1655 );
1656 if self.rows_remaining == 0 {
1657 return Ok(None);
1658 }
1659
1660 let mut to_take = self.rows_remaining.min(self.rows_per_batch as u64);
1661 self.rows_remaining -= to_take;
1662
1663 let scheduled_need = (self.rows_drained + to_take).saturating_sub(self.rows_scheduled);
1664 trace!("scheduled_need = {} because rows_drained = {} and to_take = {} and rows_scheduled = {}", scheduled_need, self.rows_drained, to_take, self.rows_scheduled);
1665 if scheduled_need > 0 {
1666 let desired_scheduled = scheduled_need + self.rows_scheduled;
1667 trace!(
1668 "Draining from scheduler (desire at least {} scheduled rows)",
1669 desired_scheduled
1670 );
1671 let actually_scheduled = self.wait_for_io(desired_scheduled)?;
1672 if actually_scheduled < desired_scheduled {
1673 let under_scheduled = desired_scheduled - actually_scheduled;
1674 to_take -= under_scheduled;
1675 }
1676 }
1677
1678 if to_take == 0 {
1679 return Ok(None);
1680 }
1681
1682 let next_task = self.root_decoder.drain_batch(to_take)?;
1683
1684 self.rows_drained += to_take;
1685
1686 let batch = next_task.into_batch(self.emitted_batch_size_warning.clone())?;
1687
1688 Ok(Some(batch))
1689 }
1690}
1691
1692impl<T: RootDecoderType> Iterator for BatchDecodeIterator<T> {
1693 type Item = ArrowResult<RecordBatch>;
1694
1695 fn next(&mut self) -> Option<Self::Item> {
1696 self.next_batch_task()
1697 .transpose()
1698 .map(|r| r.map_err(ArrowError::from))
1699 }
1700}
1701
1702impl<T: RootDecoderType> RecordBatchReader for BatchDecodeIterator<T> {
1703 fn schema(&self) -> Arc<ArrowSchema> {
1704 self.schema.clone()
1705 }
1706}
1707
1708pub struct StructuralBatchDecodeStream {
1710 context: DecoderContext,
1711 root_decoder: StructuralStructDecoder,
1712 rows_remaining: u64,
1713 rows_per_batch: u32,
1714 rows_scheduled: u64,
1715 rows_drained: u64,
1716 scheduler_exhausted: bool,
1717 emitted_batch_size_warning: Arc<Once>,
1718}
1719
1720impl StructuralBatchDecodeStream {
1721 pub fn new(
1732 scheduled: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
1733 rows_per_batch: u32,
1734 num_rows: u64,
1735 root_decoder: StructuralStructDecoder,
1736 ) -> Self {
1737 Self {
1738 context: DecoderContext::new(scheduled),
1739 root_decoder,
1740 rows_remaining: num_rows,
1741 rows_per_batch,
1742 rows_scheduled: 0,
1743 rows_drained: 0,
1744 scheduler_exhausted: false,
1745 emitted_batch_size_warning: Arc::new(Once::new()),
1746 }
1747 }
1748
1749 async fn wait_for_scheduled(&mut self, scheduled_need: u64) -> Result<u64> {
1750 if self.scheduler_exhausted {
1751 return Ok(self.rows_scheduled);
1752 }
1753 while self.rows_scheduled < scheduled_need {
1754 let next_message = self.context.source.recv().await;
1755 match next_message {
1756 Some(scan_line) => {
1757 let scan_line = scan_line?;
1758 self.rows_scheduled = scan_line.scheduled_so_far;
1759 for message in scan_line.decoders {
1760 let unloaded_page = message.into_structural();
1761 let loaded_page = unloaded_page.0.await?;
1762 self.root_decoder.accept_page(loaded_page)?;
1763 }
1764 }
1765 None => {
1766 self.scheduler_exhausted = true;
1770 return Ok(self.rows_scheduled);
1771 }
1772 }
1773 }
1774 Ok(scheduled_need)
1775 }
1776
1777 #[instrument(level = "debug", skip_all)]
1778 async fn next_batch_task(&mut self) -> Result<Option<NextDecodeTask>> {
1779 trace!(
1780 "Draining batch task (rows_remaining={} rows_drained={} rows_scheduled={})",
1781 self.rows_remaining,
1782 self.rows_drained,
1783 self.rows_scheduled,
1784 );
1785 if self.rows_remaining == 0 {
1786 return Ok(None);
1787 }
1788
1789 let mut to_take = self.rows_remaining.min(self.rows_per_batch as u64);
1790 self.rows_remaining -= to_take;
1791
1792 let scheduled_need = (self.rows_drained + to_take).saturating_sub(self.rows_scheduled);
1793 trace!("scheduled_need = {} because rows_drained = {} and to_take = {} and rows_scheduled = {}", scheduled_need, self.rows_drained, to_take, self.rows_scheduled);
1794 if scheduled_need > 0 {
1795 let desired_scheduled = scheduled_need + self.rows_scheduled;
1796 trace!(
1797 "Draining from scheduler (desire at least {} scheduled rows)",
1798 desired_scheduled
1799 );
1800 let actually_scheduled = self.wait_for_scheduled(desired_scheduled).await?;
1801 if actually_scheduled < desired_scheduled {
1802 let under_scheduled = desired_scheduled - actually_scheduled;
1803 to_take -= under_scheduled;
1804 }
1805 }
1806
1807 if to_take == 0 {
1808 return Ok(None);
1809 }
1810
1811 let next_task = self.root_decoder.drain_batch_task(to_take)?;
1812 self.rows_drained += to_take;
1813 Ok(Some(next_task))
1814 }
1815
1816 pub fn into_stream(self) -> BoxStream<'static, ReadBatchTask> {
1817 let stream = futures::stream::unfold(self, |mut slf| async move {
1818 let next_task = slf.next_batch_task().await;
1819 let next_task = next_task.transpose().map(|next_task| {
1820 let num_rows = next_task.as_ref().map(|t| t.num_rows).unwrap_or(0);
1821 let emitted_batch_size_warning = slf.emitted_batch_size_warning.clone();
1822 let task = tokio::spawn(async move {
1823 let next_task = next_task?;
1824 next_task.into_batch(emitted_batch_size_warning)
1825 });
1826 (task, num_rows)
1827 });
1828 next_task.map(|(task, num_rows)| {
1829 let task = task.map(|join_wrapper| join_wrapper.unwrap()).boxed();
1830 debug_assert!(num_rows <= u32::MAX as u64);
1832 let next_task = ReadBatchTask {
1833 task,
1834 num_rows: num_rows as u32,
1835 };
1836 (next_task, slf)
1837 })
1838 });
1839 stream.boxed()
1840 }
1841}
1842
1843#[derive(Debug)]
1844pub enum RequestedRows {
1845 Ranges(Vec<Range<u64>>),
1846 Indices(Vec<u64>),
1847}
1848
1849impl RequestedRows {
1850 pub fn num_rows(&self) -> u64 {
1851 match self {
1852 Self::Ranges(ranges) => ranges.iter().map(|r| r.end - r.start).sum(),
1853 Self::Indices(indices) => indices.len() as u64,
1854 }
1855 }
1856}
1857
1858#[derive(Debug, Clone)]
1859pub struct SchedulerDecoderConfig {
1860 pub decoder_plugins: Arc<DecoderPlugins>,
1861 pub batch_size: u32,
1862 pub io: Arc<dyn EncodingsIo>,
1863 pub cache: Arc<FileMetadataCache>,
1864 pub should_validate: bool,
1865}
1866
1867fn check_scheduler_on_drop(
1868 stream: BoxStream<'static, ReadBatchTask>,
1869 scheduler_handle: tokio::task::JoinHandle<()>,
1870) -> BoxStream<'static, ReadBatchTask> {
1871 let mut scheduler_handle = Some(scheduler_handle);
1875 let check_scheduler = stream::unfold((), move |_| {
1876 let handle = scheduler_handle.take();
1877 async move {
1878 if let Some(handle) = handle {
1879 handle.await.unwrap();
1880 }
1881 None
1882 }
1883 });
1884 stream.chain(check_scheduler).boxed()
1885}
1886
1887pub fn create_decode_stream(
1888 schema: &Schema,
1889 num_rows: u64,
1890 batch_size: u32,
1891 is_structural: bool,
1892 should_validate: bool,
1893 rx: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
1894) -> BoxStream<'static, ReadBatchTask> {
1895 if is_structural {
1896 let arrow_schema = ArrowSchema::from(schema);
1897 let structural_decoder = StructuralStructDecoder::new(
1898 arrow_schema.fields,
1899 should_validate,
1900 true,
1901 );
1902 StructuralBatchDecodeStream::new(rx, batch_size, num_rows, structural_decoder).into_stream()
1903 } else {
1904 let arrow_schema = ArrowSchema::from(schema);
1905 let root_fields = arrow_schema.fields;
1906
1907 let simple_struct_decoder = SimpleStructDecoder::new(root_fields, num_rows);
1908 BatchDecodeStream::new(rx, batch_size, num_rows, simple_struct_decoder).into_stream()
1909 }
1910}
1911
1912pub fn create_decode_iterator(
1916 schema: &Schema,
1917 num_rows: u64,
1918 batch_size: u32,
1919 should_validate: bool,
1920 is_structural: bool,
1921 messages: VecDeque<Result<DecoderMessage>>,
1922) -> Box<dyn RecordBatchReader + Send + 'static> {
1923 let arrow_schema = Arc::new(ArrowSchema::from(schema));
1924 let root_fields = arrow_schema.fields.clone();
1925 if is_structural {
1926 let simple_struct_decoder =
1927 StructuralStructDecoder::new(root_fields, should_validate, true);
1928 Box::new(BatchDecodeIterator::new(
1929 messages,
1930 batch_size,
1931 num_rows,
1932 simple_struct_decoder,
1933 arrow_schema,
1934 ))
1935 } else {
1936 let root_decoder = SimpleStructDecoder::new(root_fields, num_rows);
1937 Box::new(BatchDecodeIterator::new(
1938 messages,
1939 batch_size,
1940 num_rows,
1941 root_decoder,
1942 arrow_schema,
1943 ))
1944 }
1945}
1946
1947fn create_scheduler_decoder(
1948 column_infos: Vec<Arc<ColumnInfo>>,
1949 requested_rows: RequestedRows,
1950 filter: FilterExpression,
1951 column_indices: Vec<u32>,
1952 target_schema: Arc<Schema>,
1953 config: SchedulerDecoderConfig,
1954) -> Result<BoxStream<'static, ReadBatchTask>> {
1955 let num_rows = requested_rows.num_rows();
1956
1957 let is_structural = column_infos[0].is_structural();
1958
1959 let (tx, rx) = mpsc::unbounded_channel();
1960
1961 let decode_stream = create_decode_stream(
1962 &target_schema,
1963 num_rows,
1964 config.batch_size,
1965 is_structural,
1966 config.should_validate,
1967 rx,
1968 );
1969
1970 let scheduler_handle = tokio::task::spawn(async move {
1971 let mut decode_scheduler = match DecodeBatchScheduler::try_new(
1972 target_schema.as_ref(),
1973 &column_indices,
1974 &column_infos,
1975 &vec![],
1976 num_rows,
1977 config.decoder_plugins,
1978 config.io.clone(),
1979 config.cache,
1980 &filter,
1981 )
1982 .await
1983 {
1984 Ok(scheduler) => scheduler,
1985 Err(e) => {
1986 let _ = tx.send(Err(e));
1987 return;
1988 }
1989 };
1990
1991 match requested_rows {
1992 RequestedRows::Ranges(ranges) => {
1993 decode_scheduler.schedule_ranges(&ranges, &filter, tx, config.io)
1994 }
1995 RequestedRows::Indices(indices) => {
1996 decode_scheduler.schedule_take(&indices, &filter, tx, config.io)
1997 }
1998 }
1999 });
2000
2001 Ok(check_scheduler_on_drop(decode_stream, scheduler_handle))
2002}
2003
2004pub fn schedule_and_decode(
2010 column_infos: Vec<Arc<ColumnInfo>>,
2011 requested_rows: RequestedRows,
2012 filter: FilterExpression,
2013 column_indices: Vec<u32>,
2014 target_schema: Arc<Schema>,
2015 config: SchedulerDecoderConfig,
2016) -> BoxStream<'static, ReadBatchTask> {
2017 if requested_rows.num_rows() == 0 {
2018 return stream::empty().boxed();
2019 }
2020 match create_scheduler_decoder(
2024 column_infos,
2025 requested_rows,
2026 filter,
2027 column_indices,
2028 target_schema,
2029 config,
2030 ) {
2031 Ok(stream) => stream,
2033 Err(e) => stream::once(std::future::ready(ReadBatchTask {
2034 num_rows: 0,
2035 task: std::future::ready(Err(e)).boxed(),
2036 }))
2037 .boxed(),
2038 }
2039}
2040
2041lazy_static::lazy_static! {
2042 pub static ref WAITER_RT: tokio::runtime::Runtime = tokio::runtime::Builder::new_current_thread()
2043 .build()
2044 .unwrap();
2045}
2046
2047pub fn schedule_and_decode_blocking(
2062 column_infos: Vec<Arc<ColumnInfo>>,
2063 requested_rows: RequestedRows,
2064 filter: FilterExpression,
2065 column_indices: Vec<u32>,
2066 target_schema: Arc<Schema>,
2067 config: SchedulerDecoderConfig,
2068) -> Result<Box<dyn RecordBatchReader + Send + 'static>> {
2069 if requested_rows.num_rows() == 0 {
2070 let arrow_schema = Arc::new(ArrowSchema::from(target_schema.as_ref()));
2071 return Ok(Box::new(RecordBatchIterator::new(vec![], arrow_schema)));
2072 }
2073
2074 let num_rows = requested_rows.num_rows();
2075 let is_structural = column_infos[0].is_structural();
2076
2077 let (tx, mut rx) = mpsc::unbounded_channel();
2078
2079 let mut decode_scheduler = WAITER_RT.block_on(DecodeBatchScheduler::try_new(
2082 target_schema.as_ref(),
2083 &column_indices,
2084 &column_infos,
2085 &vec![],
2086 num_rows,
2087 config.decoder_plugins,
2088 config.io.clone(),
2089 config.cache,
2090 &filter,
2091 ))?;
2092
2093 match requested_rows {
2095 RequestedRows::Ranges(ranges) => {
2096 decode_scheduler.schedule_ranges(&ranges, &filter, tx, config.io)
2097 }
2098 RequestedRows::Indices(indices) => {
2099 decode_scheduler.schedule_take(&indices, &filter, tx, config.io)
2100 }
2101 }
2102
2103 let mut messages = Vec::new();
2105 while rx
2106 .recv_many(&mut messages, usize::MAX)
2107 .now_or_never()
2108 .unwrap()
2109 != 0
2110 {}
2111
2112 let decode_iterator = create_decode_iterator(
2114 &target_schema,
2115 num_rows,
2116 config.batch_size,
2117 config.should_validate,
2118 is_structural,
2119 messages.into(),
2120 );
2121
2122 Ok(decode_iterator)
2123}
2124
2125pub trait PrimitivePageDecoder: Send + Sync {
2137 fn decode(&self, rows_to_skip: u64, num_rows: u64) -> Result<DataBlock>;
2169}
2170
2171pub trait PageScheduler: Send + Sync + std::fmt::Debug {
2180 fn schedule_ranges(
2192 &self,
2193 ranges: &[Range<u64>],
2194 scheduler: &Arc<dyn EncodingsIo>,
2195 top_level_row: u64,
2196 ) -> BoxFuture<'static, Result<Box<dyn PrimitivePageDecoder>>>;
2197}
2198
2199pub trait PriorityRange: std::fmt::Debug + Send + Sync {
2201 fn advance(&mut self, num_rows: u64);
2202 fn current_priority(&self) -> u64;
2203 fn box_clone(&self) -> Box<dyn PriorityRange>;
2204}
2205
2206#[derive(Debug)]
2209pub struct SimplePriorityRange {
2210 priority: u64,
2211}
2212
2213impl SimplePriorityRange {
2214 fn new(priority: u64) -> Self {
2215 Self { priority }
2216 }
2217}
2218
2219impl PriorityRange for SimplePriorityRange {
2220 fn advance(&mut self, num_rows: u64) {
2221 self.priority += num_rows;
2222 }
2223
2224 fn current_priority(&self) -> u64 {
2225 self.priority
2226 }
2227
2228 fn box_clone(&self) -> Box<dyn PriorityRange> {
2229 Box::new(Self {
2230 priority: self.priority,
2231 })
2232 }
2233}
2234
2235pub struct ListPriorityRange {
2248 base: Box<dyn PriorityRange>,
2249 offsets: Arc<[u64]>,
2250 cur_index_into_offsets: usize,
2251 cur_position: u64,
2252}
2253
2254impl ListPriorityRange {
2255 pub(crate) fn new(base: Box<dyn PriorityRange>, offsets: Arc<[u64]>) -> Self {
2256 Self {
2257 base,
2258 offsets,
2259 cur_index_into_offsets: 0,
2260 cur_position: 0,
2261 }
2262 }
2263}
2264
2265impl std::fmt::Debug for ListPriorityRange {
2266 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
2267 f.debug_struct("ListPriorityRange")
2268 .field("base", &self.base)
2269 .field("offsets.len()", &self.offsets.len())
2270 .field("cur_index_into_offsets", &self.cur_index_into_offsets)
2271 .field("cur_position", &self.cur_position)
2272 .finish()
2273 }
2274}
2275
2276impl PriorityRange for ListPriorityRange {
2277 fn advance(&mut self, num_rows: u64) {
2278 self.cur_position += num_rows;
2281 let mut idx_into_offsets = self.cur_index_into_offsets;
2282 while idx_into_offsets + 1 < self.offsets.len()
2283 && self.offsets[idx_into_offsets + 1] <= self.cur_position
2284 {
2285 idx_into_offsets += 1;
2286 }
2287 let base_rows_advanced = idx_into_offsets - self.cur_index_into_offsets;
2288 self.cur_index_into_offsets = idx_into_offsets;
2289 self.base.advance(base_rows_advanced as u64);
2290 }
2291
2292 fn current_priority(&self) -> u64 {
2293 self.base.current_priority()
2294 }
2295
2296 fn box_clone(&self) -> Box<dyn PriorityRange> {
2297 Box::new(Self {
2298 base: self.base.box_clone(),
2299 offsets: self.offsets.clone(),
2300 cur_index_into_offsets: self.cur_index_into_offsets,
2301 cur_position: self.cur_position,
2302 })
2303 }
2304}
2305
2306pub struct SchedulerContext {
2308 recv: Option<mpsc::UnboundedReceiver<DecoderMessage>>,
2309 io: Arc<dyn EncodingsIo>,
2310 cache: Arc<FileMetadataCache>,
2311 name: String,
2312 path: Vec<u32>,
2313 path_names: Vec<String>,
2314}
2315
2316pub struct ScopedSchedulerContext<'a> {
2317 pub context: &'a mut SchedulerContext,
2318}
2319
2320impl<'a> ScopedSchedulerContext<'a> {
2321 pub fn pop(self) -> &'a mut SchedulerContext {
2322 self.context.pop();
2323 self.context
2324 }
2325}
2326
2327impl SchedulerContext {
2328 pub fn new(io: Arc<dyn EncodingsIo>, cache: Arc<FileMetadataCache>) -> Self {
2329 Self {
2330 io,
2331 cache,
2332 recv: None,
2333 name: "".to_string(),
2334 path: Vec::new(),
2335 path_names: Vec::new(),
2336 }
2337 }
2338
2339 pub fn io(&self) -> &Arc<dyn EncodingsIo> {
2340 &self.io
2341 }
2342
2343 pub fn cache(&self) -> &Arc<FileMetadataCache> {
2344 &self.cache
2345 }
2346
2347 pub fn push(&mut self, name: &str, index: u32) -> ScopedSchedulerContext {
2348 self.path.push(index);
2349 self.path_names.push(name.to_string());
2350 ScopedSchedulerContext { context: self }
2351 }
2352
2353 pub fn pop(&mut self) {
2354 self.path.pop();
2355 self.path_names.pop();
2356 }
2357
2358 pub fn path_name(&self) -> String {
2359 let path = self.path_names.join("/");
2360 if self.recv.is_some() {
2361 format!("TEMP({}){}", self.name, path)
2362 } else {
2363 format!("ROOT{}", path)
2364 }
2365 }
2366
2367 pub fn current_path(&self) -> VecDeque<u32> {
2368 VecDeque::from_iter(self.path.iter().copied())
2369 }
2370
2371 pub fn locate_decoder(&mut self, decoder: Box<dyn LogicalPageDecoder>) -> DecoderReady {
2372 trace!(
2373 "Scheduling decoder of type {:?} for {:?}",
2374 decoder.data_type(),
2375 self.path,
2376 );
2377 DecoderReady {
2378 decoder,
2379 path: self.current_path(),
2380 }
2381 }
2382}
2383
2384pub struct UnloadedPage(pub BoxFuture<'static, Result<LoadedPage>>);
2385
2386impl std::fmt::Debug for UnloadedPage {
2387 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
2388 f.debug_struct("UnloadedPage").finish()
2389 }
2390}
2391
2392#[derive(Debug)]
2393pub struct ScheduledScanLine {
2394 pub rows_scheduled: u64,
2395 pub decoders: Vec<MessageType>,
2396}
2397
2398pub trait SchedulingJob: std::fmt::Debug {
2399 fn schedule_next(
2400 &mut self,
2401 context: &mut SchedulerContext,
2402 priority: &dyn PriorityRange,
2403 ) -> Result<ScheduledScanLine>;
2404
2405 fn num_rows(&self) -> u64;
2406}
2407
2408pub trait StructuralSchedulingJob: std::fmt::Debug {
2409 fn schedule_next(
2410 &mut self,
2411 context: &mut SchedulerContext,
2412 ) -> Result<Option<ScheduledScanLine>>;
2413}
2414
2415pub struct FilterExpression(pub Bytes);
2423
2424impl FilterExpression {
2425 pub fn no_filter() -> Self {
2430 Self(Bytes::new())
2431 }
2432
2433 pub fn is_noop(&self) -> bool {
2435 self.0.is_empty()
2436 }
2437}
2438
2439pub trait FieldScheduler: Send + Sync + std::fmt::Debug {
2464 fn initialize<'a>(
2466 &'a self,
2467 filter: &'a FilterExpression,
2468 context: &'a SchedulerContext,
2469 ) -> BoxFuture<'a, Result<()>>;
2470 fn schedule_ranges<'a>(
2475 &'a self,
2476 ranges: &[Range<u64>],
2477 filter: &FilterExpression,
2478 ) -> Result<Box<dyn SchedulingJob + 'a>>;
2479 fn num_rows(&self) -> u64;
2481}
2482
2483pub trait StructuralFieldScheduler: Send + std::fmt::Debug {
2484 fn initialize<'a>(
2485 &'a mut self,
2486 filter: &'a FilterExpression,
2487 context: &'a SchedulerContext,
2488 ) -> BoxFuture<'a, Result<()>>;
2489 fn schedule_ranges<'a>(
2490 &'a self,
2491 ranges: &[Range<u64>],
2492 filter: &FilterExpression,
2493 ) -> Result<Box<dyn StructuralSchedulingJob + 'a>>;
2494}
2495
2496pub trait DecodeArrayTask: Send {
2498 fn decode(self: Box<Self>) -> Result<ArrayRef>;
2500}
2501
2502impl DecodeArrayTask for Box<dyn StructuralDecodeArrayTask> {
2503 fn decode(self: Box<Self>) -> Result<ArrayRef> {
2504 StructuralDecodeArrayTask::decode(*self).map(|decoded_array| decoded_array.array)
2505 }
2506}
2507
2508pub struct NextDecodeTask {
2513 pub task: Box<dyn DecodeArrayTask>,
2515 pub num_rows: u64,
2517}
2518
2519impl NextDecodeTask {
2520 #[instrument(name = "task_to_batch", level = "debug", skip_all)]
2525 fn into_batch(self, emitted_batch_size_warning: Arc<Once>) -> Result<RecordBatch> {
2526 let struct_arr = self.task.decode();
2527 match struct_arr {
2528 Ok(struct_arr) => {
2529 let batch = RecordBatch::from(struct_arr.as_struct());
2530 let size_bytes = batch.get_array_memory_size() as u64;
2531 if size_bytes > BATCH_SIZE_BYTES_WARNING {
2532 emitted_batch_size_warning.call_once(|| {
2533 let size_mb = size_bytes / 1024 / 1024;
2534 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);
2535 });
2536 }
2537 Ok(batch)
2538 }
2539 Err(e) => {
2540 let e = Error::Internal {
2541 message: format!("Error decoding batch: {}", e),
2542 location: location!(),
2543 };
2544 Err(e)
2545 }
2546 }
2547 }
2548}
2549
2550#[derive(Debug)]
2551pub struct DecoderReady {
2552 pub decoder: Box<dyn LogicalPageDecoder>,
2554 pub path: VecDeque<u32>,
2573}
2574
2575#[derive(Debug)]
2579pub enum MessageType {
2580 DecoderReady(DecoderReady),
2585 UnloadedPage(UnloadedPage),
2589}
2590
2591impl MessageType {
2592 pub fn into_legacy(self) -> DecoderReady {
2593 match self {
2594 Self::DecoderReady(decoder) => decoder,
2595 Self::UnloadedPage(_) => {
2596 panic!("Expected DecoderReady but got UnloadedPage")
2597 }
2598 }
2599 }
2600
2601 pub fn into_structural(self) -> UnloadedPage {
2602 match self {
2603 Self::UnloadedPage(unloaded) => unloaded,
2604 Self::DecoderReady(_) => {
2605 panic!("Expected UnloadedPage but got DecoderReady")
2606 }
2607 }
2608 }
2609}
2610
2611pub struct DecoderMessage {
2612 pub scheduled_so_far: u64,
2613 pub decoders: Vec<MessageType>,
2614}
2615
2616pub struct DecoderContext {
2617 source: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
2618}
2619
2620impl DecoderContext {
2621 pub fn new(source: mpsc::UnboundedReceiver<Result<DecoderMessage>>) -> Self {
2622 Self { source }
2623 }
2624}
2625
2626pub trait LogicalPageDecoder: std::fmt::Debug + Send {
2635 fn accept_child(&mut self, _child: DecoderReady) -> Result<()> {
2640 Err(Error::Internal {
2641 message: format!(
2642 "The decoder {:?} does not expect children but received a child",
2643 self
2644 ),
2645 location: location!(),
2646 })
2647 }
2648 fn wait_for_loaded(&mut self, loaded_need: u64) -> BoxFuture<Result<()>>;
2650 fn rows_loaded(&self) -> u64;
2652 fn rows_unloaded(&self) -> u64 {
2654 self.num_rows() - self.rows_loaded()
2655 }
2656 fn num_rows(&self) -> u64;
2658 fn rows_drained(&self) -> u64;
2660 fn rows_left(&self) -> u64 {
2662 self.num_rows() - self.rows_drained()
2663 }
2664 fn drain(&mut self, num_rows: u64) -> Result<NextDecodeTask>;
2666 fn data_type(&self) -> &DataType;
2668}
2669
2670pub struct DecodedPage {
2671 pub data: DataBlock,
2672 pub repdef: RepDefUnraveler,
2673}
2674
2675pub trait DecodePageTask: Send + std::fmt::Debug {
2676 fn decode(self: Box<Self>) -> Result<DecodedPage>;
2678}
2679
2680pub trait StructuralPageDecoder: std::fmt::Debug + Send {
2681 fn drain(&mut self, num_rows: u64) -> Result<Box<dyn DecodePageTask>>;
2682 fn num_rows(&self) -> u64;
2683}
2684
2685#[derive(Debug)]
2686pub struct LoadedPage {
2687 pub decoder: Box<dyn StructuralPageDecoder>,
2689 pub path: VecDeque<u32>,
2708 pub page_index: usize,
2709}
2710
2711pub struct DecodedArray {
2712 pub array: ArrayRef,
2713 pub repdef: CompositeRepDefUnraveler,
2714}
2715
2716pub trait StructuralDecodeArrayTask: std::fmt::Debug + Send {
2717 fn decode(self: Box<Self>) -> Result<DecodedArray>;
2718}
2719
2720pub trait StructuralFieldDecoder: std::fmt::Debug + Send {
2721 fn accept_page(&mut self, _child: LoadedPage) -> Result<()>;
2726 fn drain(&mut self, num_rows: u64) -> Result<Box<dyn StructuralDecodeArrayTask>>;
2728 fn data_type(&self) -> &DataType;
2730}
2731
2732#[derive(Debug, Default)]
2733pub struct DecoderPlugins {}
2734
2735pub async fn decode_batch(
2737 batch: &EncodedBatch,
2738 filter: &FilterExpression,
2739 decoder_plugins: Arc<DecoderPlugins>,
2740 should_validate: bool,
2741 version: LanceFileVersion,
2742 cache: Option<Arc<FileMetadataCache>>,
2743) -> Result<RecordBatch> {
2744 let io_scheduler = Arc::new(BufferScheduler::new(batch.data.clone())) as Arc<dyn EncodingsIo>;
2749 let cache = cache.unwrap_or_else(|| {
2750 Arc::new(FileMetadataCache::with_capacity(
2751 128 * 1024 * 1024,
2752 CapacityMode::Bytes,
2753 ))
2754 });
2755 let mut decode_scheduler = DecodeBatchScheduler::try_new(
2756 batch.schema.as_ref(),
2757 &batch.top_level_columns,
2758 &batch.page_table,
2759 &vec![],
2760 batch.num_rows,
2761 decoder_plugins,
2762 io_scheduler.clone(),
2763 cache,
2764 filter,
2765 )
2766 .await?;
2767 let (tx, rx) = unbounded_channel();
2768 decode_scheduler.schedule_range(0..batch.num_rows, filter, tx, io_scheduler);
2769 let is_structural = version >= LanceFileVersion::V2_1;
2770 let mut decode_stream = create_decode_stream(
2771 &batch.schema,
2772 batch.num_rows,
2773 batch.num_rows as u32,
2774 is_structural,
2775 should_validate,
2776 rx,
2777 );
2778 decode_stream.next().await.unwrap().task.await
2779}
2780
2781#[cfg(test)]
2782mod tests {
2784 use super::*;
2785
2786 #[test]
2787 fn test_coalesce_indices_to_ranges_with_single_index() {
2788 let indices = vec![1];
2789 let ranges = DecodeBatchScheduler::indices_to_ranges(&indices);
2790 assert_eq!(ranges, vec![1..2]);
2791 }
2792
2793 #[test]
2794 fn test_coalesce_indices_to_ranges() {
2795 let indices = vec![1, 2, 3, 4, 5, 6, 7, 8, 9];
2796 let ranges = DecodeBatchScheduler::indices_to_ranges(&indices);
2797 assert_eq!(ranges, vec![1..10]);
2798 }
2799
2800 #[test]
2801 fn test_coalesce_indices_to_ranges_with_gaps() {
2802 let indices = vec![1, 2, 3, 5, 6, 7, 9];
2803 let ranges = DecodeBatchScheduler::indices_to_ranges(&indices);
2804 assert_eq!(ranges, vec![1..4, 5..8, 9..10]);
2805 }
2806}