1use std::{ops::Range, sync::Arc};
11
12use arrow_array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait, StructArray, cast::AsArray};
13use arrow_buffer::{BooleanBufferBuilder, NullBuffer, OffsetBuffer, ScalarBuffer};
14use arrow_schema::DataType;
15use futures::future::BoxFuture;
16use lance_arrow::deepcopy::deep_copy_nulls;
17use lance_core::{Error, Result};
18
19use crate::{
20 decoder::{
21 DecodedArray, FilterExpression, ScheduledScanLine, SchedulerContext,
22 StructuralDecodeArrayTask, StructuralFieldDecoder, StructuralFieldScheduler,
23 StructuralSchedulingJob,
24 },
25 encoder::{EncodeTask, FieldEncoder, OutOfLineBuffers},
26 repdef::RepDefBuilder,
27};
28
29pub struct FixedSizeListStructuralEncoder {
34 keep_original_array: bool,
35 child: Box<dyn FieldEncoder>,
36}
37
38impl FixedSizeListStructuralEncoder {
39 pub fn new(keep_original_array: bool, child: Box<dyn FieldEncoder>) -> Self {
40 Self {
41 keep_original_array,
42 child,
43 }
44 }
45}
46
47impl FieldEncoder for FixedSizeListStructuralEncoder {
48 fn maybe_encode(
49 &mut self,
50 array: ArrayRef,
51 external_buffers: &mut OutOfLineBuffers,
52 mut repdef: RepDefBuilder,
53 row_number: u64,
54 num_rows: u64,
55 ) -> Result<Vec<EncodeTask>> {
56 let fsl_arr = array.as_fixed_size_list_opt().ok_or_else(|| {
57 Error::internal("FixedSizeList encoder used for non-fixed-size-list data".to_string())
58 })?;
59
60 let dimension = fsl_arr.value_length() as usize;
61 let values = fsl_arr.values().clone();
62
63 let validity = if self.keep_original_array {
64 array.nulls().cloned()
65 } else {
66 deep_copy_nulls(array.nulls())
67 };
68 repdef.add_fsl(validity.clone(), dimension, num_rows as usize);
69
70 let values = if let Some(ref fsl_validity) = validity {
73 if needs_garbage_filtering(values.data_type()) {
74 let is_garbage =
75 expand_garbage_mask(&fsl_validity_to_garbage_mask(fsl_validity), dimension);
76 filter_fsl_child_garbage(values, &is_garbage)
77 } else {
78 values
79 }
80 } else {
81 values
82 };
83
84 self.child.maybe_encode(
85 values,
86 external_buffers,
87 repdef,
88 row_number,
89 num_rows * dimension as u64,
90 )
91 }
92
93 fn flush(&mut self, external_buffers: &mut OutOfLineBuffers) -> Result<Vec<EncodeTask>> {
94 self.child.flush(external_buffers)
95 }
96
97 fn num_columns(&self) -> u32 {
98 self.child.num_columns()
99 }
100
101 fn finish(
102 &mut self,
103 external_buffers: &mut OutOfLineBuffers,
104 ) -> BoxFuture<'_, Result<Vec<crate::encoder::EncodedColumn>>> {
105 self.child.finish(external_buffers)
106 }
107}
108
109#[derive(Debug)]
114pub struct StructuralFixedSizeListScheduler {
115 child: Box<dyn StructuralFieldScheduler>,
116 dimension: u64,
117}
118
119impl StructuralFixedSizeListScheduler {
120 pub fn new(child: Box<dyn StructuralFieldScheduler>, dimension: i32) -> Self {
121 Self {
122 child,
123 dimension: dimension as u64,
124 }
125 }
126}
127
128impl StructuralFieldScheduler for StructuralFixedSizeListScheduler {
129 fn schedule_ranges<'a>(
130 &'a self,
131 ranges: &[Range<u64>],
132 filter: &FilterExpression,
133 ) -> Result<Box<dyn StructuralSchedulingJob + 'a>> {
134 let child_ranges: Vec<Range<u64>> = ranges
136 .iter()
137 .map(|r| (r.start * self.dimension)..(r.end * self.dimension))
138 .collect();
139 let child = self.child.schedule_ranges(&child_ranges, filter)?;
140 Ok(Box::new(StructuralFixedSizeListSchedulingJob::new(
141 child,
142 self.dimension,
143 )))
144 }
145
146 fn initialize<'a>(
147 &'a mut self,
148 filter: &'a FilterExpression,
149 context: &'a SchedulerContext,
150 ) -> BoxFuture<'a, Result<()>> {
151 self.child.initialize(filter, context)
152 }
153}
154
155#[derive(Debug)]
156struct StructuralFixedSizeListSchedulingJob<'a> {
157 child: Box<dyn StructuralSchedulingJob + 'a>,
158 dimension: u64,
159}
160
161impl<'a> StructuralFixedSizeListSchedulingJob<'a> {
162 fn new(child: Box<dyn StructuralSchedulingJob + 'a>, dimension: u64) -> Self {
163 Self { child, dimension }
164 }
165}
166
167impl StructuralSchedulingJob for StructuralFixedSizeListSchedulingJob<'_> {
168 fn schedule_next(&mut self, context: &mut SchedulerContext) -> Result<Vec<ScheduledScanLine>> {
169 let child_scan_lines = self.child.schedule_next(context)?;
171
172 Ok(child_scan_lines
174 .into_iter()
175 .map(|scan_line| ScheduledScanLine {
176 decoders: scan_line.decoders,
177 rows_scheduled: scan_line.rows_scheduled / self.dimension,
178 })
179 .collect())
180 }
181}
182
183#[derive(Debug)]
188pub struct StructuralFixedSizeListDecoder {
189 child: Box<dyn StructuralFieldDecoder>,
190 data_type: DataType,
191}
192
193impl StructuralFixedSizeListDecoder {
194 pub fn new(child: Box<dyn StructuralFieldDecoder>, data_type: DataType) -> Self {
195 Self { child, data_type }
196 }
197}
198
199impl StructuralFieldDecoder for StructuralFixedSizeListDecoder {
200 fn accept_page(&mut self, child: crate::decoder::LoadedPageShard) -> Result<()> {
201 self.child.accept_page(child)
202 }
203
204 fn drain(&mut self, num_rows: u64) -> Result<Box<dyn StructuralDecodeArrayTask>> {
205 let dimension = match &self.data_type {
207 DataType::FixedSizeList(_, d) => *d as u64,
208 _ => {
209 return Err(Error::internal(
210 "FixedSizeListDecoder has non-FSL data type".to_string(),
211 ));
212 }
213 };
214 let child_task = self.child.drain(num_rows * dimension)?;
215 Ok(Box::new(StructuralFixedSizeListDecodeTask::new(
216 child_task,
217 self.data_type.clone(),
218 num_rows,
219 )))
220 }
221
222 fn data_type(&self) -> &DataType {
223 &self.data_type
224 }
225}
226
227#[derive(Debug)]
228struct StructuralFixedSizeListDecodeTask {
229 child_task: Box<dyn StructuralDecodeArrayTask>,
230 data_type: DataType,
231 num_rows: u64,
232}
233
234impl StructuralFixedSizeListDecodeTask {
235 fn new(
236 child_task: Box<dyn StructuralDecodeArrayTask>,
237 data_type: DataType,
238 num_rows: u64,
239 ) -> Self {
240 Self {
241 child_task,
242 data_type,
243 num_rows,
244 }
245 }
246}
247
248impl StructuralDecodeArrayTask for StructuralFixedSizeListDecodeTask {
249 fn decode(self: Box<Self>) -> Result<DecodedArray> {
250 let DecodedArray {
251 array,
252 mut repdef,
253 data_size,
254 } = self.child_task.decode()?;
255 match &self.data_type {
256 DataType::FixedSizeList(child_field, dimension) => {
257 let num_rows = self.num_rows as usize;
258 let validity = repdef.unravel_fsl_validity(num_rows, *dimension as usize);
259 let fsl_array = arrow_array::FixedSizeListArray::try_new(
260 child_field.clone(),
261 *dimension,
262 array,
263 validity,
264 )?;
265 Ok(DecodedArray {
266 array: Arc::new(fsl_array),
267 repdef,
268 data_size,
269 })
270 }
271 _ => Err(Error::internal(
272 "FixedSizeList decoder did not have a fixed-size list field".to_string(),
273 )),
274 }
275 }
276}
277
278fn needs_garbage_filtering(data_type: &DataType) -> bool {
285 match data_type {
286 DataType::List(_)
287 | DataType::LargeList(_)
288 | DataType::ListView(_)
289 | DataType::LargeListView(_)
290 | DataType::Map(_, _) => true,
291 DataType::Struct(fields) => fields
292 .iter()
293 .any(|f| needs_garbage_filtering(f.data_type())),
294 DataType::FixedSizeList(field, _) => needs_garbage_filtering(field.data_type()),
295 _ => false,
296 }
297}
298
299fn filter_fsl_child_garbage(array: ArrayRef, is_garbage: &[bool]) -> ArrayRef {
306 debug_assert_eq!(array.len(), is_garbage.len());
307
308 match array.data_type() {
309 DataType::List(_) => filter_list_garbage(array.as_list::<i32>(), is_garbage),
310 DataType::LargeList(_) => filter_list_garbage(array.as_list::<i64>(), is_garbage),
311 DataType::ListView(_) | DataType::LargeListView(_) => {
312 unimplemented!("ListView inside complex FSL is not yet supported")
313 }
314 DataType::Map(_, _) => filter_map_garbage(array.as_map(), is_garbage),
315 DataType::FixedSizeList(_, dim) => {
316 filter_nested_fsl_garbage(array.as_fixed_size_list(), is_garbage, *dim as usize)
317 }
318 DataType::Struct(_) => filter_struct_garbage(array.as_struct(), is_garbage),
319 _ => array,
320 }
321}
322
323fn filter_struct_garbage(struct_arr: &StructArray, is_garbage: &[bool]) -> ArrayRef {
324 let needs_filtering = struct_arr
325 .fields()
326 .iter()
327 .any(|f| needs_garbage_filtering(f.data_type()));
328
329 if !needs_filtering {
330 return Arc::new(struct_arr.clone());
331 }
332
333 let new_columns: Vec<ArrayRef> = struct_arr
334 .columns()
335 .iter()
336 .zip(struct_arr.fields().iter())
337 .map(|(col, field)| {
338 if needs_garbage_filtering(field.data_type()) {
339 filter_fsl_child_garbage(col.clone(), is_garbage)
340 } else {
341 col.clone()
342 }
343 })
344 .collect();
345
346 Arc::new(StructArray::new(
347 struct_arr.fields().clone(),
348 new_columns,
349 struct_arr.nulls().cloned(),
350 ))
351}
352
353fn expand_garbage_mask(is_garbage: &[bool], dimension: usize) -> Vec<bool> {
354 let mut expanded = Vec::with_capacity(is_garbage.len() * dimension);
355 for &garbage in is_garbage {
356 for _ in 0..dimension {
357 expanded.push(garbage);
358 }
359 }
360 expanded
361}
362
363fn fsl_validity_to_garbage_mask(fsl_validity: &NullBuffer) -> Vec<bool> {
364 fsl_validity.iter().map(|valid| !valid).collect()
365}
366
367fn filter_list_garbage<O: OffsetSizeTrait>(
368 list_arr: &GenericListArray<O>,
369 is_garbage: &[bool],
370) -> ArrayRef {
371 debug_assert_eq!(
372 list_arr.len(),
373 is_garbage.len(),
374 "list length must match garbage mask length"
375 );
376
377 let old_offsets = list_arr.offsets();
378 let value_field = match list_arr.data_type() {
379 DataType::List(f) | DataType::LargeList(f) => f.clone(),
380 _ => unreachable!(),
381 };
382
383 let mut new_offsets: Vec<O> = Vec::with_capacity(list_arr.len() + 1);
384 let mut values_to_keep: Vec<usize> = Vec::new();
385 let mut validity_builder = BooleanBufferBuilder::new(list_arr.len());
386 let mut current_offset = O::usize_as(0);
387 new_offsets.push(current_offset);
388 let old_validity = list_arr.nulls();
389
390 for (i, &garbage) in is_garbage.iter().enumerate() {
391 if garbage {
392 new_offsets.push(current_offset);
393 validity_builder.append(false);
394 } else {
395 let start = old_offsets[i].as_usize();
396 let end = old_offsets[i + 1].as_usize();
397 values_to_keep.extend(start..end);
398 current_offset += O::usize_as(end - start);
399 new_offsets.push(current_offset);
400 validity_builder.append(old_validity.map(|v| v.is_valid(i)).unwrap_or(true));
401 }
402 }
403
404 let new_values = if values_to_keep.is_empty() {
405 list_arr.values().slice(0, 0)
406 } else {
407 let indices =
408 arrow_array::UInt64Array::from_iter_values(values_to_keep.iter().map(|&i| i as u64));
409 arrow_select::take::take(list_arr.values().as_ref(), &indices, None)
410 .expect("take should succeed")
411 };
412
413 let new_values = if needs_garbage_filtering(value_field.data_type()) && !new_values.is_empty() {
414 let len = new_values.len();
415 filter_fsl_child_garbage(new_values, &vec![false; len])
416 } else {
417 new_values
418 };
419
420 let new_validity = NullBuffer::new(validity_builder.finish());
421 Arc::new(GenericListArray::new(
422 value_field,
423 OffsetBuffer::new(ScalarBuffer::from(new_offsets)),
424 new_values,
425 Some(new_validity),
426 ))
427}
428
429fn filter_map_garbage(map_arr: &arrow_array::MapArray, is_garbage: &[bool]) -> ArrayRef {
430 debug_assert_eq!(map_arr.len(), is_garbage.len());
431
432 let old_offsets = map_arr.offsets();
433 let entries_field = match map_arr.data_type() {
434 DataType::Map(field, _) => field.clone(),
435 _ => unreachable!(),
436 };
437
438 let mut new_offsets: Vec<i32> = Vec::with_capacity(map_arr.len() + 1);
439 let mut values_to_keep: Vec<usize> = Vec::new();
440 let mut validity_builder = BooleanBufferBuilder::new(map_arr.len());
441 let mut current_offset: i32 = 0;
442 new_offsets.push(current_offset);
443 let old_validity = map_arr.nulls();
444
445 for (i, &garbage) in is_garbage.iter().enumerate() {
446 if garbage {
447 new_offsets.push(current_offset);
448 validity_builder.append(false);
449 } else {
450 let start = old_offsets[i] as usize;
451 let end = old_offsets[i + 1] as usize;
452 values_to_keep.extend(start..end);
453 current_offset += (end - start) as i32;
454 new_offsets.push(current_offset);
455 validity_builder.append(old_validity.map(|v| v.is_valid(i)).unwrap_or(true));
456 }
457 }
458
459 let new_entries: ArrayRef = if values_to_keep.is_empty() {
460 Arc::new(map_arr.entries().slice(0, 0))
461 } else {
462 let indices =
463 arrow_array::UInt64Array::from_iter_values(values_to_keep.iter().map(|&i| i as u64));
464 arrow_select::take::take(map_arr.entries(), &indices, None).expect("take should succeed")
465 };
466
467 let new_entries =
468 if needs_garbage_filtering(entries_field.data_type()) && !new_entries.is_empty() {
469 let len = new_entries.len();
470 filter_fsl_child_garbage(new_entries, &vec![false; len])
471 } else {
472 new_entries
473 };
474
475 let new_validity = NullBuffer::new(validity_builder.finish());
476 let keys_sorted = matches!(map_arr.data_type(), DataType::Map(_, true));
477
478 Arc::new(
479 arrow_array::MapArray::try_new(
480 entries_field,
481 OffsetBuffer::new(ScalarBuffer::from(new_offsets)),
482 new_entries.as_struct().clone(),
483 Some(new_validity),
484 keys_sorted,
485 )
486 .expect("MapArray construction should succeed"),
487 )
488}
489
490fn filter_nested_fsl_garbage(
492 fsl_arr: &arrow_array::FixedSizeListArray,
493 is_garbage: &[bool],
494 dimension: usize,
495) -> ArrayRef {
496 debug_assert_eq!(fsl_arr.len(), is_garbage.len());
497
498 let child_field = match fsl_arr.data_type() {
499 DataType::FixedSizeList(field, _) => field.clone(),
500 _ => unreachable!(),
501 };
502
503 if !needs_garbage_filtering(child_field.data_type()) {
504 return Arc::new(fsl_arr.clone());
505 }
506
507 let child_garbage = expand_garbage_mask(is_garbage, dimension);
508 let new_values = filter_fsl_child_garbage(fsl_arr.values().clone(), &child_garbage);
509
510 Arc::new(arrow_array::FixedSizeListArray::new(
511 child_field,
512 dimension as i32,
513 new_values,
514 fsl_arr.nulls().cloned(),
515 ))
516}
517
518#[cfg(test)]
519mod tests {
520 use std::{collections::HashMap, sync::Arc};
521
522 use arrow_array::{
523 Array, FixedSizeListArray,
524 builder::{Int32Builder, ListBuilder},
525 cast::AsArray,
526 };
527 use arrow_schema::{DataType, Field, Fields};
528 use rstest::rstest;
529
530 use super::filter_nested_fsl_garbage;
531 use crate::{
532 constants::{
533 STRUCTURAL_ENCODING_FULLZIP, STRUCTURAL_ENCODING_META_KEY,
534 STRUCTURAL_ENCODING_MINIBLOCK,
535 },
536 testing::{TestCases, check_specific_random},
537 version::LanceFileVersion,
538 };
539
540 fn make_fsl_struct_type(struct_fields: Fields, dimension: i32) -> DataType {
541 DataType::FixedSizeList(
542 Arc::new(Field::new("item", DataType::Struct(struct_fields), true)),
543 dimension,
544 )
545 }
546
547 fn simple_struct_fields() -> Fields {
548 Fields::from(vec![
549 Field::new("x", DataType::Float64, false),
550 Field::new("y", DataType::Float64, false),
551 ])
552 }
553
554 fn nested_struct_fields() -> Fields {
555 let inner = Fields::from(vec![
556 Field::new("a", DataType::Int32, false),
557 Field::new("b", DataType::Int32, false),
558 ]);
559 Fields::from(vec![
560 Field::new("outer_val", DataType::Float64, false),
561 Field::new("inner", DataType::Struct(inner), true),
562 ])
563 }
564
565 fn nested_struct_with_list_fields() -> Fields {
566 let inner = Fields::from(vec![Field::new(
567 "values",
568 DataType::List(Arc::new(Field::new("item", DataType::Int32, true))),
569 true,
570 )]);
571 Fields::from(vec![
572 Field::new("id", DataType::Int32, false),
573 Field::new("inner", DataType::Struct(inner), true),
574 ])
575 }
576
577 fn struct_with_list_fields() -> Fields {
578 Fields::from(vec![
579 Field::new("id", DataType::Int32, false),
580 Field::new(
581 "values",
582 DataType::List(Arc::new(Field::new("item", DataType::Int32, true))),
583 true,
584 ),
585 ])
586 }
587
588 fn struct_with_large_list_fields() -> Fields {
589 Fields::from(vec![
590 Field::new("id", DataType::Int32, false),
591 Field::new(
592 "values",
593 DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))),
594 true,
595 ),
596 ])
597 }
598
599 fn struct_with_nested_fsl_fields() -> Fields {
600 Fields::from(vec![
601 Field::new("id", DataType::Int32, false),
602 Field::new(
603 "vectors",
604 DataType::FixedSizeList(Arc::new(Field::new("item", DataType::Float32, true)), 4),
605 true,
606 ),
607 ])
608 }
609
610 fn struct_with_map_fields() -> Fields {
611 let entries_field = Arc::new(Field::new(
612 "entries",
613 DataType::Struct(Fields::from(vec![
614 Field::new("keys", DataType::Utf8, false),
615 Field::new("values", DataType::Int32, true),
616 ])),
617 false,
618 ));
619 Fields::from(vec![
620 Field::new("id", DataType::Int32, false),
621 Field::new("props", DataType::Map(entries_field, false), true),
622 ])
623 }
624
625 fn make_fsl_of_list() -> DataType {
626 DataType::FixedSizeList(
627 Arc::new(Field::new(
628 "item",
629 DataType::List(Arc::new(Field::new("item", DataType::Int32, true))),
630 true,
631 )),
632 2,
633 )
634 }
635
636 fn make_fsl_of_large_list() -> DataType {
637 DataType::FixedSizeList(
638 Arc::new(Field::new(
639 "item",
640 DataType::LargeList(Arc::new(Field::new("item", DataType::Int32, true))),
641 true,
642 )),
643 2,
644 )
645 }
646
647 fn make_fsl_of_map() -> DataType {
648 DataType::FixedSizeList(
649 Arc::new(Field::new(
650 "item",
651 DataType::Map(
652 Arc::new(Field::new(
653 "entries",
654 DataType::Struct(Fields::from(vec![
655 Field::new("key", DataType::Utf8, false),
656 Field::new("value", DataType::Int32, true),
657 ])),
658 false,
659 )),
660 false,
661 ),
662 true,
663 )),
664 2,
665 )
666 }
667
668 fn make_fsl_of_nested_fsl_struct() -> DataType {
669 DataType::FixedSizeList(
670 Arc::new(Field::new(
671 "item",
672 DataType::FixedSizeList(
673 Arc::new(Field::new(
674 "item",
675 DataType::Struct(Fields::from(vec![Field::new(
676 "x",
677 DataType::Int32,
678 true,
679 )])),
680 true,
681 )),
682 4,
683 ),
684 true,
685 )),
686 2,
687 )
688 }
689
690 #[rstest]
691 #[case::simple(simple_struct_fields(), 2, LanceFileVersion::V2_2)]
692 #[case::nested_struct(nested_struct_fields(), 2, LanceFileVersion::V2_2)]
693 #[case::struct_with_list(struct_with_list_fields(), 2, LanceFileVersion::V2_2)]
694 #[case::struct_with_large_list(struct_with_large_list_fields(), 2, LanceFileVersion::V2_2)]
695 #[case::nested_struct_with_list(nested_struct_with_list_fields(), 2, LanceFileVersion::V2_2)]
696 #[case::struct_with_nested_fsl(struct_with_nested_fsl_fields(), 2, LanceFileVersion::V2_2)]
697 #[case::struct_with_map(struct_with_map_fields(), 2, LanceFileVersion::V2_2)]
698 #[test_log::test(tokio::test)]
699 async fn test_fsl_struct_random(
700 #[case] struct_fields: Fields,
701 #[case] dimension: i32,
702 #[case] min_version: LanceFileVersion,
703 #[values(STRUCTURAL_ENCODING_MINIBLOCK, STRUCTURAL_ENCODING_FULLZIP)]
704 structural_encoding: &str,
705 ) {
706 let data_type = make_fsl_struct_type(struct_fields, dimension);
707 let mut field_metadata = HashMap::new();
708 field_metadata.insert(
709 STRUCTURAL_ENCODING_META_KEY.to_string(),
710 structural_encoding.into(),
711 );
712 let field = Field::new("", data_type, true).with_metadata(field_metadata);
713 let test_cases = TestCases::basic().with_min_file_version(min_version);
714 check_specific_random(field, test_cases).await;
715 }
716
717 #[rstest]
718 #[case::list(make_fsl_of_list())]
719 #[case::large_list(make_fsl_of_large_list())]
720 #[case::map(make_fsl_of_map())]
721 #[case::nested_fsl_struct(make_fsl_of_nested_fsl_struct())]
722 fn test_unsupported_fsl_child_types_return_error(#[case] data_type: DataType) {
723 let arrow_field = Field::new("test", data_type, true);
724 let err = lance_core::datatypes::Field::try_from(&arrow_field).unwrap_err();
725 assert!(err.to_string().contains("Unsupported data type"));
726 }
727
728 #[test]
729 fn test_filter_nested_fsl_garbage() {
730 let mut list_builder = ListBuilder::new(Int32Builder::new());
732 for i in 1..=6 {
733 list_builder.values().append_value(i);
734 list_builder.append(true);
735 }
736 let list_arr = list_builder.finish();
737
738 let fsl_field = Arc::new(Field::new(
739 "item",
740 DataType::List(Arc::new(Field::new("item", DataType::Int32, true))),
741 true,
742 ));
743 let fsl = FixedSizeListArray::new(fsl_field, 2, Arc::new(list_arr), None);
744
745 let result = filter_nested_fsl_garbage(&fsl, &[false, true, false], 2);
747 let result = result.as_fixed_size_list();
748
749 let child_list = result.values().as_list::<i32>();
751 assert_eq!(
752 (0..6).map(|i| child_list.is_valid(i)).collect::<Vec<_>>(),
753 vec![true, true, false, false, true, true]
754 );
755 }
756
757 #[test]
758 fn test_filter_nested_fsl_no_list_child() {
759 let fsl_field = Arc::new(Field::new("item", DataType::Int32, true));
761 let values = arrow_array::Int32Array::from(vec![1, 2, 3, 4, 5, 6]);
762 let fsl = FixedSizeListArray::new(fsl_field, 2, Arc::new(values), None);
763
764 let result = filter_nested_fsl_garbage(&fsl, &[false, true, false], 2);
765 assert_eq!(result.len(), 3);
767 }
768}