lance_encoding/
encoder.rs

1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright The Lance Authors
3
4//! The top-level encoding module for Lance files.
5//!
6//! Lance files are encoded using a [`FieldEncodingStrategy`] which choose
7//! what encoder to use for each field.
8//!
9//! The current strategy is the [`StructuralEncodingStrategy`] which uses "structural"
10//! encoding.  A tree of encoders is built up for each field.  The struct & list encoders
11//! simply pull off the validity and offsets and collect them.  Then, in the primitive leaf
12//! encoder the validity, offsets, and values are accumulated in an accumulation buffer.  Once
13//! enough data has been collected the primitive encoder will either use a miniblock encoding
14//! or a full zip encoding to create a page of data from the accumulation buffer.
15
16use std::{collections::HashMap, sync::Arc};
17
18use arrow_array::{Array, ArrayRef, RecordBatch};
19use arrow_schema::DataType;
20use bytes::{Bytes, BytesMut};
21use futures::future::BoxFuture;
22use lance_core::datatypes::{Field, Schema};
23use lance_core::utils::bit::{is_pwr_two, pad_bytes_to};
24use lance_core::{Error, Result};
25use snafu::location;
26
27use crate::buffer::LanceBuffer;
28use crate::compression::{CompressionStrategy, DefaultCompressionStrategy};
29use crate::compression_config::CompressionParams;
30use crate::decoder::PageEncoding;
31use crate::encodings::logical::blob::BlobStructuralEncoder;
32use crate::encodings::logical::list::ListStructuralEncoder;
33use crate::encodings::logical::primitive::PrimitiveStructuralEncoder;
34use crate::encodings::logical::r#struct::StructStructuralEncoder;
35use crate::repdef::RepDefBuilder;
36use crate::version::LanceFileVersion;
37use crate::{
38    decoder::{ColumnInfo, PageInfo},
39    format::pb,
40};
41
42/// The minimum alignment for a page buffer.  Writers must respect this.
43pub const MIN_PAGE_BUFFER_ALIGNMENT: u64 = 8;
44
45/// An encoded page of data
46///
47/// Maps to a top-level array
48///
49/// For example, FixedSizeList<Int32> will have two EncodedArray instances and one EncodedPage
50#[derive(Debug)]
51pub struct EncodedPage {
52    // The encoded page buffers
53    pub data: Vec<LanceBuffer>,
54    // A description of the encoding used to encode the page
55    pub description: PageEncoding,
56    /// The number of rows in the encoded page
57    pub num_rows: u64,
58    /// The top-level row number of the first row in the page
59    ///
60    /// Generally the number of "top-level" rows and the number of rows are the same.  However,
61    /// when there is repetition (list/fixed-size-list) there will be more or less items than rows.
62    ///
63    /// A top-level row can never be split across a page boundary.
64    pub row_number: u64,
65    /// The index of the column
66    pub column_idx: u32,
67}
68
69pub struct EncodedColumn {
70    pub column_buffers: Vec<LanceBuffer>,
71    pub encoding: pb::ColumnEncoding,
72    pub final_pages: Vec<EncodedPage>,
73}
74
75impl Default for EncodedColumn {
76    fn default() -> Self {
77        Self {
78            column_buffers: Default::default(),
79            encoding: pb::ColumnEncoding {
80                column_encoding: Some(pb::column_encoding::ColumnEncoding::Values(())),
81            },
82            final_pages: Default::default(),
83        }
84    }
85}
86
87/// A tool to reserve space for buffers that are not in-line with the data
88///
89/// In most cases, buffers are stored in the page and referred to in the encoding
90/// metadata by their index in the page.  This keeps all buffers within a page together.
91/// As a result, most encoders should not need to use this structure.
92///
93/// In some cases (currently only the large binary encoding) there is a need to access
94/// buffers that are not in the page (because storing the position / offset of every page
95/// in the page metadata would be too expensive).
96///
97/// To do this you can add a buffer with `add_buffer` and then use the returned position
98/// in some way (in the large binary encoding the returned position is stored in the page
99/// data as a position / size array).
100pub struct OutOfLineBuffers {
101    position: u64,
102    buffer_alignment: u64,
103    buffers: Vec<LanceBuffer>,
104}
105
106impl OutOfLineBuffers {
107    pub fn new(base_position: u64, buffer_alignment: u64) -> Self {
108        Self {
109            position: base_position,
110            buffer_alignment,
111            buffers: Vec::new(),
112        }
113    }
114
115    pub fn add_buffer(&mut self, buffer: LanceBuffer) -> u64 {
116        let position = self.position;
117        self.position += buffer.len() as u64;
118        self.position += pad_bytes_to(buffer.len(), self.buffer_alignment as usize) as u64;
119        self.buffers.push(buffer);
120        position
121    }
122
123    pub fn take_buffers(self) -> Vec<LanceBuffer> {
124        self.buffers
125    }
126
127    pub fn reset_position(&mut self, position: u64) {
128        self.position = position;
129    }
130}
131
132/// A task to create a page of data
133pub type EncodeTask = BoxFuture<'static, Result<EncodedPage>>;
134
135/// Top level encoding trait to code any Arrow array type into one or more pages.
136///
137/// The field encoder implements buffering and encoding of a single input column
138/// but it may map to multiple output columns.  For example, a list array or struct
139/// array will be encoded into multiple columns.
140///
141/// Also, fields may be encoded at different speeds.  For example, given a struct
142/// column with three fields (a boolean field, an int32 field, and a 4096-dimension
143/// tensor field) the tensor field is likely to emit encoded pages much more frequently
144/// than the boolean field.
145pub trait FieldEncoder: Send {
146    /// Buffer the data and, if there is enough data in the buffer to form a page, return
147    /// an encoding task to encode the data.
148    ///
149    /// This may return more than one task because a single column may be mapped to multiple
150    /// output columns.  For example, if encoding a struct column with three children then
151    /// up to three tasks may be returned from each call to maybe_encode.
152    ///
153    /// It may also return multiple tasks for a single column if the input array is larger
154    /// than a single disk page.
155    ///
156    /// It could also return an empty Vec if there is not enough data yet to encode any pages.
157    ///
158    /// The `row_number` must be passed which is the top-level row number currently being encoded
159    /// This is stored in any pages produced by this call so that we can know the priority of the
160    /// page.
161    ///
162    /// The `num_rows` is the number of top level rows.  It is initially the same as `array.len()`
163    /// however it is passed seprately because array will become flattened over time (if there is
164    /// repetition) and we need to know the original number of rows for various purposes.
165    fn maybe_encode(
166        &mut self,
167        array: ArrayRef,
168        external_buffers: &mut OutOfLineBuffers,
169        repdef: RepDefBuilder,
170        row_number: u64,
171        num_rows: u64,
172    ) -> Result<Vec<EncodeTask>>;
173    /// Flush any remaining data from the buffers into encoding tasks
174    ///
175    /// Each encode task produces a single page.  The order of these pages will be maintained
176    /// in the file (we do not worry about order between columns but all pages in the same
177    /// column should maintain order)
178    ///
179    /// This may be called intermittently throughout encoding but will always be called
180    /// once at the end of encoding just before calling finish
181    fn flush(&mut self, external_buffers: &mut OutOfLineBuffers) -> Result<Vec<EncodeTask>>;
182    /// Finish encoding and return column metadata
183    ///
184    /// This is called only once, after all encode tasks have completed
185    ///
186    /// This returns a Vec because a single field may have created multiple columns
187    fn finish(
188        &mut self,
189        external_buffers: &mut OutOfLineBuffers,
190    ) -> BoxFuture<'_, Result<Vec<EncodedColumn>>>;
191
192    /// The number of output columns this encoding will create
193    fn num_columns(&self) -> u32;
194}
195
196/// Keeps track of the current column index and makes a mapping
197/// from field id to column index
198#[derive(Debug, Default)]
199pub struct ColumnIndexSequence {
200    current_index: u32,
201    mapping: Vec<(u32, u32)>,
202}
203
204impl ColumnIndexSequence {
205    pub fn next_column_index(&mut self, field_id: u32) -> u32 {
206        let idx = self.current_index;
207        self.current_index += 1;
208        self.mapping.push((field_id, idx));
209        idx
210    }
211
212    pub fn skip(&mut self) {
213        self.current_index += 1;
214    }
215}
216
217/// Options that control the encoding process
218pub struct EncodingOptions {
219    /// How much data (in bytes) to cache in-memory before writing a page
220    ///
221    /// This cache is applied on a per-column basis
222    pub cache_bytes_per_column: u64,
223    /// The maximum size of a page in bytes, if a single array would create
224    /// a page larger than this then it will be split into multiple pages
225    pub max_page_bytes: u64,
226    /// If false (the default) then arrays will be copied (deeply) before
227    /// being cached.  This ensures any data kept alive by the array can
228    /// be discarded safely and helps avoid writer accumulation.  However,
229    /// there is an associated cost.
230    pub keep_original_array: bool,
231    /// The alignment that the writer is applying to buffers
232    ///
233    /// The encoder needs to know this so it figures the position of out-of-line
234    /// buffers correctly
235    pub buffer_alignment: u64,
236}
237
238impl Default for EncodingOptions {
239    fn default() -> Self {
240        Self {
241            cache_bytes_per_column: 8 * 1024 * 1024,
242            max_page_bytes: 32 * 1024 * 1024,
243            keep_original_array: true,
244            buffer_alignment: 64,
245        }
246    }
247}
248
249/// A trait to pick which kind of field encoding to use for a field
250///
251/// Unlike the ArrayEncodingStrategy, the field encoding strategy is
252/// chosen before any data is generated and the same field encoder is
253/// used for all data in the field.
254pub trait FieldEncodingStrategy: Send + Sync + std::fmt::Debug {
255    /// Choose and create an appropriate field encoder for the given
256    /// field.
257    ///
258    /// The field encoder can be chosen on the data type as well as
259    /// any metadata that is attached to the field.
260    ///
261    /// The `encoding_strategy_root` is the encoder that should be
262    /// used to encode any inner data in struct / list / etc. fields.
263    ///
264    /// Initially it is the same as `self` and generally should be
265    /// forwarded to any inner encoding strategy.
266    fn create_field_encoder(
267        &self,
268        encoding_strategy_root: &dyn FieldEncodingStrategy,
269        field: &Field,
270        column_index: &mut ColumnIndexSequence,
271        options: &EncodingOptions,
272    ) -> Result<Box<dyn FieldEncoder>>;
273}
274
275pub fn default_encoding_strategy(version: LanceFileVersion) -> Box<dyn FieldEncodingStrategy> {
276    match version.resolve() {
277        LanceFileVersion::Legacy => panic!(),
278        LanceFileVersion::V2_0 => Box::new(
279            crate::previous::encoder::CoreFieldEncodingStrategy::new(version),
280        ),
281        _ => Box::new(StructuralEncodingStrategy::default()),
282    }
283}
284
285/// Create an encoding strategy with user-configured compression parameters
286pub fn default_encoding_strategy_with_params(
287    version: LanceFileVersion,
288    params: CompressionParams,
289) -> Result<Box<dyn FieldEncodingStrategy>> {
290    match version.resolve() {
291        LanceFileVersion::Legacy | LanceFileVersion::V2_0 => Err(Error::invalid_input(
292            "Compression parameters are only supported in Lance file version 2.1 and later",
293            location!(),
294        )),
295        _ => {
296            let compression_strategy = Arc::new(DefaultCompressionStrategy::with_params(params));
297            Ok(Box::new(StructuralEncodingStrategy {
298                compression_strategy,
299                version,
300            }))
301        }
302    }
303}
304
305/// An encoding strategy used for 2.1+ files
306#[derive(Debug)]
307pub struct StructuralEncodingStrategy {
308    pub compression_strategy: Arc<dyn CompressionStrategy>,
309    pub version: LanceFileVersion,
310}
311
312// For some reason, clippy thinks we can add Default to the above derive but
313// rustc doesn't agree (no default for Arc<dyn Trait>)
314#[allow(clippy::derivable_impls)]
315impl Default for StructuralEncodingStrategy {
316    fn default() -> Self {
317        Self {
318            compression_strategy: Arc::new(DefaultCompressionStrategy::new()),
319            version: LanceFileVersion::default(),
320        }
321    }
322}
323
324impl StructuralEncodingStrategy {
325    fn is_primitive_type(data_type: &DataType) -> bool {
326        matches!(
327            data_type,
328            DataType::Boolean
329                | DataType::Date32
330                | DataType::Date64
331                | DataType::Decimal128(_, _)
332                | DataType::Decimal256(_, _)
333                | DataType::Duration(_)
334                | DataType::Float16
335                | DataType::Float32
336                | DataType::Float64
337                | DataType::Int16
338                | DataType::Int32
339                | DataType::Int64
340                | DataType::Int8
341                | DataType::Interval(_)
342                | DataType::Null
343                | DataType::Time32(_)
344                | DataType::Time64(_)
345                | DataType::Timestamp(_, _)
346                | DataType::UInt16
347                | DataType::UInt32
348                | DataType::UInt64
349                | DataType::UInt8
350                | DataType::FixedSizeBinary(_)
351                | DataType::FixedSizeList(_, _)
352                | DataType::Binary
353                | DataType::LargeBinary
354                | DataType::Utf8
355                | DataType::LargeUtf8,
356        )
357    }
358
359    fn do_create_field_encoder(
360        &self,
361        _encoding_strategy_root: &dyn FieldEncodingStrategy,
362        field: &Field,
363        column_index: &mut ColumnIndexSequence,
364        options: &EncodingOptions,
365        root_field_metadata: &HashMap<String, String>,
366    ) -> Result<Box<dyn FieldEncoder>> {
367        let data_type = field.data_type();
368
369        // Check if field is marked as blob
370        if field.is_blob() {
371            match data_type {
372                DataType::Binary | DataType::LargeBinary => {
373                    return Ok(Box::new(BlobStructuralEncoder::new(
374                        field,
375                        column_index.next_column_index(field.id as u32),
376                        options,
377                        self.compression_strategy.clone(),
378                    )?));
379                }
380                _ => {
381                    return Err(Error::InvalidInput {
382                        source: format!(
383                            "Blob encoding only supports Binary/LargeBinary, got {}",
384                            data_type
385                        )
386                        .into(),
387                        location: location!(),
388                    });
389                }
390            }
391        }
392
393        if Self::is_primitive_type(&data_type) {
394            Ok(Box::new(PrimitiveStructuralEncoder::try_new(
395                options,
396                self.compression_strategy.clone(),
397                column_index.next_column_index(field.id as u32),
398                field.clone(),
399                Arc::new(root_field_metadata.clone()),
400            )?))
401        } else {
402            match data_type {
403                DataType::List(_) | DataType::LargeList(_) => {
404                    let child = field.children.first().expect("List should have a child");
405                    let child_encoder = self.do_create_field_encoder(
406                        _encoding_strategy_root,
407                        child,
408                        column_index,
409                        options,
410                        root_field_metadata,
411                    )?;
412                    Ok(Box::new(ListStructuralEncoder::new(
413                        options.keep_original_array,
414                        child_encoder,
415                    )))
416                }
417                DataType::Struct(_) => {
418                    if field.is_packed_struct() {
419                        Ok(Box::new(PrimitiveStructuralEncoder::try_new(
420                            options,
421                            self.compression_strategy.clone(),
422                            column_index.next_column_index(field.id as u32),
423                            field.clone(),
424                            Arc::new(root_field_metadata.clone()),
425                        )?))
426                    } else {
427                        let children_encoders = field
428                            .children
429                            .iter()
430                            .map(|field| {
431                                self.do_create_field_encoder(
432                                    _encoding_strategy_root,
433                                    field,
434                                    column_index,
435                                    options,
436                                    root_field_metadata,
437                                )
438                            })
439                            .collect::<Result<Vec<_>>>()?;
440                        Ok(Box::new(StructStructuralEncoder::new(
441                            options.keep_original_array,
442                            children_encoders,
443                        )))
444                    }
445                }
446                DataType::Dictionary(_, value_type) => {
447                    // A dictionary of primitive is, itself, primitive
448                    if Self::is_primitive_type(&value_type) {
449                        Ok(Box::new(PrimitiveStructuralEncoder::try_new(
450                            options,
451                            self.compression_strategy.clone(),
452                            column_index.next_column_index(field.id as u32),
453                            field.clone(),
454                            Arc::new(root_field_metadata.clone()),
455                        )?))
456                    } else {
457                        // A dictionary of logical is, itself, logical and we don't support that today
458                        // It could be possible (e.g. store indices in one column and values in remaining columns)
459                        // but would be a significant amount of work
460                        //
461                        // An easier fallback implementation would be to decode-on-write and encode-on-read
462                        Err(Error::NotSupported { source: format!("cannot encode a dictionary column whose value type is a logical type ({})", value_type).into(), location: location!() })
463                    }
464                }
465                _ => todo!("Implement encoding for field {}", field),
466            }
467        }
468    }
469}
470
471impl FieldEncodingStrategy for StructuralEncodingStrategy {
472    fn create_field_encoder(
473        &self,
474        encoding_strategy_root: &dyn FieldEncodingStrategy,
475        field: &Field,
476        column_index: &mut ColumnIndexSequence,
477        options: &EncodingOptions,
478    ) -> Result<Box<dyn FieldEncoder>> {
479        self.do_create_field_encoder(
480            encoding_strategy_root,
481            field,
482            column_index,
483            options,
484            &field.metadata,
485        )
486    }
487}
488
489/// A batch encoder that encodes RecordBatch objects by delegating
490/// to field encoders for each top-level field in the batch.
491pub struct BatchEncoder {
492    pub field_encoders: Vec<Box<dyn FieldEncoder>>,
493    pub field_id_to_column_index: Vec<(u32, u32)>,
494}
495
496impl BatchEncoder {
497    pub fn try_new(
498        schema: &Schema,
499        strategy: &dyn FieldEncodingStrategy,
500        options: &EncodingOptions,
501    ) -> Result<Self> {
502        let mut col_idx = 0;
503        let mut col_idx_sequence = ColumnIndexSequence::default();
504        let field_encoders = schema
505            .fields
506            .iter()
507            .map(|field| {
508                let encoder = strategy.create_field_encoder(
509                    strategy,
510                    field,
511                    &mut col_idx_sequence,
512                    options,
513                )?;
514                col_idx += encoder.as_ref().num_columns();
515                Ok(encoder)
516            })
517            .collect::<Result<Vec<_>>>()?;
518        Ok(Self {
519            field_encoders,
520            field_id_to_column_index: col_idx_sequence.mapping,
521        })
522    }
523
524    pub fn num_columns(&self) -> u32 {
525        self.field_encoders
526            .iter()
527            .map(|field_encoder| field_encoder.num_columns())
528            .sum::<u32>()
529    }
530}
531
532/// An encoded batch of data and a page table describing it
533///
534/// This is returned by [`crate::encoder::encode_batch`]
535#[derive(Debug)]
536pub struct EncodedBatch {
537    pub data: Bytes,
538    pub page_table: Vec<Arc<ColumnInfo>>,
539    pub schema: Arc<Schema>,
540    pub top_level_columns: Vec<u32>,
541    pub num_rows: u64,
542}
543
544fn write_page_to_data_buffer(page: EncodedPage, data_buffer: &mut BytesMut) -> PageInfo {
545    let buffers = page.data;
546    let mut buffer_offsets_and_sizes = Vec::with_capacity(buffers.len());
547    for buffer in buffers {
548        let buffer_offset = data_buffer.len() as u64;
549        data_buffer.extend_from_slice(&buffer);
550        let size = data_buffer.len() as u64 - buffer_offset;
551        buffer_offsets_and_sizes.push((buffer_offset, size));
552    }
553
554    PageInfo {
555        buffer_offsets_and_sizes: Arc::from(buffer_offsets_and_sizes),
556        encoding: page.description,
557        num_rows: page.num_rows,
558        priority: page.row_number,
559    }
560}
561
562/// Helper method to encode a batch of data into memory
563///
564/// This is primarily for testing and benchmarking but could be useful in other
565/// niche situations like IPC.
566pub async fn encode_batch(
567    batch: &RecordBatch,
568    schema: Arc<Schema>,
569    encoding_strategy: &dyn FieldEncodingStrategy,
570    options: &EncodingOptions,
571) -> Result<EncodedBatch> {
572    if !is_pwr_two(options.buffer_alignment) || options.buffer_alignment < MIN_PAGE_BUFFER_ALIGNMENT
573    {
574        return Err(Error::InvalidInput {
575            source: format!(
576                "buffer_alignment must be a power of two and at least {}",
577                MIN_PAGE_BUFFER_ALIGNMENT
578            )
579            .into(),
580            location: location!(),
581        });
582    }
583
584    let mut data_buffer = BytesMut::new();
585    let lance_schema = Schema::try_from(batch.schema().as_ref())?;
586    let options = EncodingOptions {
587        keep_original_array: true,
588        ..*options
589    };
590    let batch_encoder = BatchEncoder::try_new(&lance_schema, encoding_strategy, &options)?;
591    let mut page_table = Vec::new();
592    let mut col_idx_offset = 0;
593    for (arr, mut encoder) in batch.columns().iter().zip(batch_encoder.field_encoders) {
594        let mut external_buffers =
595            OutOfLineBuffers::new(data_buffer.len() as u64, options.buffer_alignment);
596        let repdef = RepDefBuilder::default();
597        let encoder = encoder.as_mut();
598        let num_rows = arr.len() as u64;
599        let mut tasks =
600            encoder.maybe_encode(arr.clone(), &mut external_buffers, repdef, 0, num_rows)?;
601        tasks.extend(encoder.flush(&mut external_buffers)?);
602        for buffer in external_buffers.take_buffers() {
603            data_buffer.extend_from_slice(&buffer);
604        }
605        let mut pages = HashMap::<u32, Vec<PageInfo>>::new();
606        for task in tasks {
607            let encoded_page = task.await?;
608            // Write external buffers first
609            pages
610                .entry(encoded_page.column_idx)
611                .or_default()
612                .push(write_page_to_data_buffer(encoded_page, &mut data_buffer));
613        }
614        let mut external_buffers =
615            OutOfLineBuffers::new(data_buffer.len() as u64, options.buffer_alignment);
616        let encoded_columns = encoder.finish(&mut external_buffers).await?;
617        for buffer in external_buffers.take_buffers() {
618            data_buffer.extend_from_slice(&buffer);
619        }
620        let num_columns = encoded_columns.len();
621        for (col_idx, encoded_column) in encoded_columns.into_iter().enumerate() {
622            let col_idx = col_idx + col_idx_offset;
623            let mut col_buffer_offsets_and_sizes = Vec::new();
624            for buffer in encoded_column.column_buffers {
625                let buffer_offset = data_buffer.len() as u64;
626                data_buffer.extend_from_slice(&buffer);
627                let size = data_buffer.len() as u64 - buffer_offset;
628                col_buffer_offsets_and_sizes.push((buffer_offset, size));
629            }
630            for page in encoded_column.final_pages {
631                pages
632                    .entry(page.column_idx)
633                    .or_default()
634                    .push(write_page_to_data_buffer(page, &mut data_buffer));
635            }
636            let col_pages = std::mem::take(pages.entry(col_idx as u32).or_default());
637            page_table.push(Arc::new(ColumnInfo {
638                index: col_idx as u32,
639                buffer_offsets_and_sizes: Arc::from(
640                    col_buffer_offsets_and_sizes.into_boxed_slice(),
641                ),
642                page_infos: Arc::from(col_pages.into_boxed_slice()),
643                encoding: encoded_column.encoding,
644            }))
645        }
646        col_idx_offset += num_columns;
647    }
648    let top_level_columns = batch_encoder
649        .field_id_to_column_index
650        .iter()
651        .map(|(_, idx)| *idx)
652        .collect();
653    Ok(EncodedBatch {
654        data: data_buffer.freeze(),
655        top_level_columns,
656        page_table,
657        schema,
658        num_rows: batch.num_rows() as u64,
659    })
660}
661
662#[cfg(test)]
663mod tests {
664    use super::*;
665    use crate::compression_config::{CompressionFieldParams, CompressionParams};
666
667    #[test]
668    fn test_configured_encoding_strategy() {
669        // Create test parameters
670        let mut params = CompressionParams::new();
671        params.columns.insert(
672            "*_id".to_string(),
673            CompressionFieldParams {
674                rle_threshold: Some(0.5),
675                compression: Some("lz4".to_string()),
676                compression_level: None,
677                bss: None,
678            },
679        );
680
681        // Test with V2.1 - should succeed
682        let strategy =
683            default_encoding_strategy_with_params(LanceFileVersion::V2_1, params.clone())
684                .expect("Should succeed for V2.1");
685
686        // Verify it's a StructuralEncodingStrategy
687        assert!(format!("{:?}", strategy).contains("StructuralEncodingStrategy"));
688        assert!(format!("{:?}", strategy).contains("DefaultCompressionStrategy"));
689
690        // Test with V2.0 - should fail
691        let err = default_encoding_strategy_with_params(LanceFileVersion::V2_0, params.clone())
692            .expect_err("Should fail for V2.0");
693        assert!(err
694            .to_string()
695            .contains("only supported in Lance file version 2.1"));
696
697        // Test with Legacy - should fail
698        let err = default_encoding_strategy_with_params(LanceFileVersion::Legacy, params)
699            .expect_err("Should fail for Legacy");
700        assert!(err
701            .to_string()
702            .contains("only supported in Lance file version 2.1"));
703    }
704}