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::with_version(version)),
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 =
297                Arc::new(DefaultCompressionStrategy::with_params(params).with_version(version));
298            Ok(Box::new(StructuralEncodingStrategy {
299                compression_strategy,
300                version,
301            }))
302        }
303    }
304}
305
306/// An encoding strategy used for 2.1+ files
307#[derive(Debug)]
308pub struct StructuralEncodingStrategy {
309    pub compression_strategy: Arc<dyn CompressionStrategy>,
310    pub version: LanceFileVersion,
311}
312
313// For some reason, clippy thinks we can add Default to the above derive but
314// rustc doesn't agree (no default for Arc<dyn Trait>)
315#[allow(clippy::derivable_impls)]
316impl Default for StructuralEncodingStrategy {
317    fn default() -> Self {
318        Self {
319            compression_strategy: Arc::new(DefaultCompressionStrategy::new()),
320            version: LanceFileVersion::default(),
321        }
322    }
323}
324
325impl StructuralEncodingStrategy {
326    pub fn with_version(version: LanceFileVersion) -> Self {
327        Self {
328            compression_strategy: Arc::new(DefaultCompressionStrategy::new().with_version(version)),
329            version,
330        }
331    }
332
333    fn is_primitive_type(data_type: &DataType) -> bool {
334        matches!(
335            data_type,
336            DataType::Boolean
337                | DataType::Date32
338                | DataType::Date64
339                | DataType::Decimal128(_, _)
340                | DataType::Decimal256(_, _)
341                | DataType::Duration(_)
342                | DataType::Float16
343                | DataType::Float32
344                | DataType::Float64
345                | DataType::Int16
346                | DataType::Int32
347                | DataType::Int64
348                | DataType::Int8
349                | DataType::Interval(_)
350                | DataType::Null
351                | DataType::Time32(_)
352                | DataType::Time64(_)
353                | DataType::Timestamp(_, _)
354                | DataType::UInt16
355                | DataType::UInt32
356                | DataType::UInt64
357                | DataType::UInt8
358                | DataType::FixedSizeBinary(_)
359                | DataType::FixedSizeList(_, _)
360                | DataType::Binary
361                | DataType::LargeBinary
362                | DataType::Utf8
363                | DataType::LargeUtf8,
364        )
365    }
366
367    fn do_create_field_encoder(
368        &self,
369        _encoding_strategy_root: &dyn FieldEncodingStrategy,
370        field: &Field,
371        column_index: &mut ColumnIndexSequence,
372        options: &EncodingOptions,
373        root_field_metadata: &HashMap<String, String>,
374    ) -> Result<Box<dyn FieldEncoder>> {
375        let data_type = field.data_type();
376
377        // Check if field is marked as blob
378        if field.is_blob() {
379            match data_type {
380                DataType::Binary | DataType::LargeBinary => {
381                    return Ok(Box::new(BlobStructuralEncoder::new(
382                        field,
383                        column_index.next_column_index(field.id as u32),
384                        options,
385                        self.compression_strategy.clone(),
386                    )?));
387                }
388                _ => {
389                    return Err(Error::InvalidInput {
390                        source: format!(
391                            "Blob encoding only supports Binary/LargeBinary, got {}",
392                            data_type
393                        )
394                        .into(),
395                        location: location!(),
396                    });
397                }
398            }
399        }
400
401        if Self::is_primitive_type(&data_type) {
402            Ok(Box::new(PrimitiveStructuralEncoder::try_new(
403                options,
404                self.compression_strategy.clone(),
405                column_index.next_column_index(field.id as u32),
406                field.clone(),
407                Arc::new(root_field_metadata.clone()),
408            )?))
409        } else {
410            match data_type {
411                DataType::List(_) | DataType::LargeList(_) => {
412                    let child = field.children.first().expect("List should have a child");
413                    let child_encoder = self.do_create_field_encoder(
414                        _encoding_strategy_root,
415                        child,
416                        column_index,
417                        options,
418                        root_field_metadata,
419                    )?;
420                    Ok(Box::new(ListStructuralEncoder::new(
421                        options.keep_original_array,
422                        child_encoder,
423                    )))
424                }
425                DataType::Struct(fields) => {
426                    if field.is_packed_struct() || fields.is_empty() {
427                        // Both packed structs and empty structs are encoded as primitive
428                        Ok(Box::new(PrimitiveStructuralEncoder::try_new(
429                            options,
430                            self.compression_strategy.clone(),
431                            column_index.next_column_index(field.id as u32),
432                            field.clone(),
433                            Arc::new(root_field_metadata.clone()),
434                        )?))
435                    } else {
436                        let children_encoders = field
437                            .children
438                            .iter()
439                            .map(|field| {
440                                self.do_create_field_encoder(
441                                    _encoding_strategy_root,
442                                    field,
443                                    column_index,
444                                    options,
445                                    root_field_metadata,
446                                )
447                            })
448                            .collect::<Result<Vec<_>>>()?;
449                        Ok(Box::new(StructStructuralEncoder::new(
450                            options.keep_original_array,
451                            children_encoders,
452                        )))
453                    }
454                }
455                DataType::Dictionary(_, value_type) => {
456                    // A dictionary of primitive is, itself, primitive
457                    if Self::is_primitive_type(&value_type) {
458                        Ok(Box::new(PrimitiveStructuralEncoder::try_new(
459                            options,
460                            self.compression_strategy.clone(),
461                            column_index.next_column_index(field.id as u32),
462                            field.clone(),
463                            Arc::new(root_field_metadata.clone()),
464                        )?))
465                    } else {
466                        // A dictionary of logical is, itself, logical and we don't support that today
467                        // It could be possible (e.g. store indices in one column and values in remaining columns)
468                        // but would be a significant amount of work
469                        //
470                        // An easier fallback implementation would be to decode-on-write and encode-on-read
471                        Err(Error::NotSupported { source: format!("cannot encode a dictionary column whose value type is a logical type ({})", value_type).into(), location: location!() })
472                    }
473                }
474                _ => todo!("Implement encoding for field {}", field),
475            }
476        }
477    }
478}
479
480impl FieldEncodingStrategy for StructuralEncodingStrategy {
481    fn create_field_encoder(
482        &self,
483        encoding_strategy_root: &dyn FieldEncodingStrategy,
484        field: &Field,
485        column_index: &mut ColumnIndexSequence,
486        options: &EncodingOptions,
487    ) -> Result<Box<dyn FieldEncoder>> {
488        self.do_create_field_encoder(
489            encoding_strategy_root,
490            field,
491            column_index,
492            options,
493            &field.metadata,
494        )
495    }
496}
497
498/// A batch encoder that encodes RecordBatch objects by delegating
499/// to field encoders for each top-level field in the batch.
500pub struct BatchEncoder {
501    pub field_encoders: Vec<Box<dyn FieldEncoder>>,
502    pub field_id_to_column_index: Vec<(u32, u32)>,
503}
504
505impl BatchEncoder {
506    pub fn try_new(
507        schema: &Schema,
508        strategy: &dyn FieldEncodingStrategy,
509        options: &EncodingOptions,
510    ) -> Result<Self> {
511        let mut col_idx = 0;
512        let mut col_idx_sequence = ColumnIndexSequence::default();
513        let field_encoders = schema
514            .fields
515            .iter()
516            .map(|field| {
517                let encoder = strategy.create_field_encoder(
518                    strategy,
519                    field,
520                    &mut col_idx_sequence,
521                    options,
522                )?;
523                col_idx += encoder.as_ref().num_columns();
524                Ok(encoder)
525            })
526            .collect::<Result<Vec<_>>>()?;
527        Ok(Self {
528            field_encoders,
529            field_id_to_column_index: col_idx_sequence.mapping,
530        })
531    }
532
533    pub fn num_columns(&self) -> u32 {
534        self.field_encoders
535            .iter()
536            .map(|field_encoder| field_encoder.num_columns())
537            .sum::<u32>()
538    }
539}
540
541/// An encoded batch of data and a page table describing it
542///
543/// This is returned by [`crate::encoder::encode_batch`]
544#[derive(Debug)]
545pub struct EncodedBatch {
546    pub data: Bytes,
547    pub page_table: Vec<Arc<ColumnInfo>>,
548    pub schema: Arc<Schema>,
549    pub top_level_columns: Vec<u32>,
550    pub num_rows: u64,
551}
552
553fn write_page_to_data_buffer(page: EncodedPage, data_buffer: &mut BytesMut) -> PageInfo {
554    let buffers = page.data;
555    let mut buffer_offsets_and_sizes = Vec::with_capacity(buffers.len());
556    for buffer in buffers {
557        let buffer_offset = data_buffer.len() as u64;
558        data_buffer.extend_from_slice(&buffer);
559        let size = data_buffer.len() as u64 - buffer_offset;
560        buffer_offsets_and_sizes.push((buffer_offset, size));
561    }
562
563    PageInfo {
564        buffer_offsets_and_sizes: Arc::from(buffer_offsets_and_sizes),
565        encoding: page.description,
566        num_rows: page.num_rows,
567        priority: page.row_number,
568    }
569}
570
571/// Helper method to encode a batch of data into memory
572///
573/// This is primarily for testing and benchmarking but could be useful in other
574/// niche situations like IPC.
575pub async fn encode_batch(
576    batch: &RecordBatch,
577    schema: Arc<Schema>,
578    encoding_strategy: &dyn FieldEncodingStrategy,
579    options: &EncodingOptions,
580) -> Result<EncodedBatch> {
581    if !is_pwr_two(options.buffer_alignment) || options.buffer_alignment < MIN_PAGE_BUFFER_ALIGNMENT
582    {
583        return Err(Error::InvalidInput {
584            source: format!(
585                "buffer_alignment must be a power of two and at least {}",
586                MIN_PAGE_BUFFER_ALIGNMENT
587            )
588            .into(),
589            location: location!(),
590        });
591    }
592
593    let mut data_buffer = BytesMut::new();
594    let lance_schema = Schema::try_from(batch.schema().as_ref())?;
595    let options = EncodingOptions {
596        keep_original_array: true,
597        ..*options
598    };
599    let batch_encoder = BatchEncoder::try_new(&lance_schema, encoding_strategy, &options)?;
600    let mut page_table = Vec::new();
601    let mut col_idx_offset = 0;
602    for (arr, mut encoder) in batch.columns().iter().zip(batch_encoder.field_encoders) {
603        let mut external_buffers =
604            OutOfLineBuffers::new(data_buffer.len() as u64, options.buffer_alignment);
605        let repdef = RepDefBuilder::default();
606        let encoder = encoder.as_mut();
607        let num_rows = arr.len() as u64;
608        let mut tasks =
609            encoder.maybe_encode(arr.clone(), &mut external_buffers, repdef, 0, num_rows)?;
610        tasks.extend(encoder.flush(&mut external_buffers)?);
611        for buffer in external_buffers.take_buffers() {
612            data_buffer.extend_from_slice(&buffer);
613        }
614        let mut pages = HashMap::<u32, Vec<PageInfo>>::new();
615        for task in tasks {
616            let encoded_page = task.await?;
617            // Write external buffers first
618            pages
619                .entry(encoded_page.column_idx)
620                .or_default()
621                .push(write_page_to_data_buffer(encoded_page, &mut data_buffer));
622        }
623        let mut external_buffers =
624            OutOfLineBuffers::new(data_buffer.len() as u64, options.buffer_alignment);
625        let encoded_columns = encoder.finish(&mut external_buffers).await?;
626        for buffer in external_buffers.take_buffers() {
627            data_buffer.extend_from_slice(&buffer);
628        }
629        let num_columns = encoded_columns.len();
630        for (col_idx, encoded_column) in encoded_columns.into_iter().enumerate() {
631            let col_idx = col_idx + col_idx_offset;
632            let mut col_buffer_offsets_and_sizes = Vec::new();
633            for buffer in encoded_column.column_buffers {
634                let buffer_offset = data_buffer.len() as u64;
635                data_buffer.extend_from_slice(&buffer);
636                let size = data_buffer.len() as u64 - buffer_offset;
637                col_buffer_offsets_and_sizes.push((buffer_offset, size));
638            }
639            for page in encoded_column.final_pages {
640                pages
641                    .entry(page.column_idx)
642                    .or_default()
643                    .push(write_page_to_data_buffer(page, &mut data_buffer));
644            }
645            let col_pages = std::mem::take(pages.entry(col_idx as u32).or_default());
646            page_table.push(Arc::new(ColumnInfo {
647                index: col_idx as u32,
648                buffer_offsets_and_sizes: Arc::from(
649                    col_buffer_offsets_and_sizes.into_boxed_slice(),
650                ),
651                page_infos: Arc::from(col_pages.into_boxed_slice()),
652                encoding: encoded_column.encoding,
653            }))
654        }
655        col_idx_offset += num_columns;
656    }
657    let top_level_columns = batch_encoder
658        .field_id_to_column_index
659        .iter()
660        .map(|(_, idx)| *idx)
661        .collect();
662    Ok(EncodedBatch {
663        data: data_buffer.freeze(),
664        top_level_columns,
665        page_table,
666        schema,
667        num_rows: batch.num_rows() as u64,
668    })
669}
670
671#[cfg(test)]
672mod tests {
673    use super::*;
674    use crate::compression_config::{CompressionFieldParams, CompressionParams};
675
676    #[test]
677    fn test_configured_encoding_strategy() {
678        // Create test parameters
679        let mut params = CompressionParams::new();
680        params.columns.insert(
681            "*_id".to_string(),
682            CompressionFieldParams {
683                rle_threshold: Some(0.5),
684                compression: Some("lz4".to_string()),
685                compression_level: None,
686                bss: None,
687            },
688        );
689
690        // Test with V2.1 - should succeed
691        let strategy =
692            default_encoding_strategy_with_params(LanceFileVersion::V2_1, params.clone())
693                .expect("Should succeed for V2.1");
694
695        // Verify it's a StructuralEncodingStrategy
696        assert!(format!("{:?}", strategy).contains("StructuralEncodingStrategy"));
697        assert!(format!("{:?}", strategy).contains("DefaultCompressionStrategy"));
698
699        // Test with V2.0 - should fail
700        let err = default_encoding_strategy_with_params(LanceFileVersion::V2_0, params.clone())
701            .expect_err("Should fail for V2.0");
702        assert!(err
703            .to_string()
704            .contains("only supported in Lance file version 2.1"));
705
706        // Test with Legacy - should fail
707        let err = default_encoding_strategy_with_params(LanceFileVersion::Legacy, params)
708            .expect_err("Should fail for Legacy");
709        assert!(err
710            .to_string()
711            .contains("only supported in Lance file version 2.1"));
712    }
713}