You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/05/06 18:34:23 UTC

[GitHub] [arrow-rs] alamb commented on a diff in pull request #1588: Add support for nested list arrays from parquet to arrow arrays (#993)

alamb commented on code in PR #1588:
URL: https://github.com/apache/arrow-rs/pull/1588#discussion_r867065942


##########
parquet/src/arrow/array_reader/list_array.rs:
##########
@@ -35,12 +35,12 @@ pub struct ListArrayReader<OffsetSize: OffsetSizeTrait> {
     item_reader: Box<dyn ArrayReader>,
     data_type: ArrowType,
     item_type: ArrowType,
-    list_def_level: i16,
-    list_rep_level: i16,
-    list_empty_def_level: i16,
-    list_null_def_level: i16,
-    def_level_buffer: Option<Buffer>,
-    rep_level_buffer: Option<Buffer>,
+    // The definition level at which this list is not null
+    def_level: i16,
+    // The repetition level that corresponds to a new value in this array

Review Comment:
   ```suggestion
       /// The repetition level that corresponds to a new value in this array
   ```



##########
parquet/src/arrow/array_reader/list_array.rs:
##########
@@ -35,12 +35,12 @@ pub struct ListArrayReader<OffsetSize: OffsetSizeTrait> {
     item_reader: Box<dyn ArrayReader>,
     data_type: ArrowType,
     item_type: ArrowType,
-    list_def_level: i16,
-    list_rep_level: i16,
-    list_empty_def_level: i16,
-    list_null_def_level: i16,
-    def_level_buffer: Option<Buffer>,
-    rep_level_buffer: Option<Buffer>,
+    // The definition level at which this list is not null
+    def_level: i16,
+    // The repetition level that corresponds to a new value in this array
+    rep_level: i16,
+    // If this list is nullable

Review Comment:
   ```suggestion
       /// If this list is nullable
   ```



##########
parquet/src/arrow/array_reader/list_array.rs:
##########
@@ -35,12 +35,12 @@ pub struct ListArrayReader<OffsetSize: OffsetSizeTrait> {
     item_reader: Box<dyn ArrayReader>,
     data_type: ArrowType,
     item_type: ArrowType,
-    list_def_level: i16,
-    list_rep_level: i16,
-    list_empty_def_level: i16,
-    list_null_def_level: i16,
-    def_level_buffer: Option<Buffer>,
-    rep_level_buffer: Option<Buffer>,
+    // The definition level at which this list is not null

Review Comment:
   ```suggestion
       /// The definition level at which this list is not null
   ```



##########
parquet/src/schema/visitor.rs:
##########
@@ -27,17 +27,30 @@ pub trait TypeVisitor<R, C> {
 
     /// Default implementation when visiting a list.
     ///
-    /// It checks list type definition and calls `visit_list_with_item` with extracted
+    /// It checks list type definition and calls [`Self::visit_list_with_item`] with extracted
     /// item type.
     ///
     /// To fully understand this algorithm, please refer to
     /// [parquet doc](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md).
+    ///
+    /// For example, a standard list type looks like:

Review Comment:
   ❤️ 



##########
parquet/src/arrow/array_reader/list_array.rs:
##########
@@ -193,122 +245,267 @@ mod tests {
     use crate::file::reader::{FileReader, SerializedFileReader};
     use crate::schema::parser::parse_message_type;
     use crate::schema::types::SchemaDescriptor;
-    use arrow::array::{Array, LargeListArray, ListArray, PrimitiveArray};
-    use arrow::datatypes::{Field, Int32Type as ArrowInt32};
+    use arrow::array::{Array, ArrayDataBuilder, PrimitiveArray};
+    use arrow::datatypes::{Field, Int32Type as ArrowInt32, Int32Type};
     use std::sync::Arc;
 
-    #[test]
-    fn test_list_array_reader() {
-        // [[1, null, 2], null, [3, 4]]
+    fn list_type<OffsetSize: OffsetSizeTrait>(
+        data_type: ArrowType,
+        item_nullable: bool,
+    ) -> ArrowType {
+        let field = Box::new(Field::new("item", data_type, item_nullable));
+        match OffsetSize::is_large() {
+            true => ArrowType::LargeList(field),
+            false => ArrowType::List(field),
+        }
+    }
+
+    fn downcast<OffsetSize: OffsetSizeTrait>(
+        array: &ArrayRef,
+    ) -> &'_ GenericListArray<OffsetSize> {
+        array
+            .as_any()
+            .downcast_ref::<GenericListArray<OffsetSize>>()
+            .unwrap()
+    }
+
+    fn to_offsets<OffsetSize: OffsetSizeTrait>(values: Vec<usize>) -> Buffer {
+        Buffer::from_iter(
+            values
+                .into_iter()
+                .map(|x| OffsetSize::from_usize(x).unwrap()),
+        )
+    }
+
+    fn test_nested_list<OffsetSize: OffsetSizeTrait>() {
+        // 3 lists, with first and third nullable
+        // [
+        //     [
+        //         [[1, null], null, [4], []],
+        //         [],
+        //         [[7]],
+        //         [[]]
+        //     ],
+        //     null,
+        //     [],
+        //     [[[11]]]
+        // ]
+
+        let l3_item_type = ArrowType::Int32;
+        let l3_type = list_type::<OffsetSize>(l3_item_type.clone(), true);
+
+        let l2_item_type = l3_type.clone();
+        let l2_type = list_type::<OffsetSize>(l2_item_type.clone(), true);
+
+        let l1_item_type = l2_type.clone();
+        let l1_type = list_type::<OffsetSize>(l1_item_type.clone(), false);
+
+        let leaf = PrimitiveArray::<Int32Type>::from_iter(vec![
+            Some(1),
+            None,
+            Some(4),
+            Some(7),
+            Some(11),
+        ]);
+
+        // [[1, null], null, [4], [], [7], [], [11]]
+        let offsets = to_offsets::<OffsetSize>(vec![0, 2, 2, 3, 3, 4, 4, 5]);
+        let l3 = ArrayDataBuilder::new(l3_type.clone())
+            .len(7)
+            .add_buffer(offsets)
+            .add_child_data(leaf.data().clone())
+            .null_bit_buffer(Buffer::from([0b01111101]))
+            .build()
+            .unwrap();
+
+        // [[[1, null], null, [4], []],[], [[7]], [[]], [[11]]]
+        let offsets = to_offsets::<OffsetSize>(vec![0, 4, 4, 5, 6, 7]);
+        let l2 = ArrayDataBuilder::new(l2_type.clone())
+            .len(5)
+            .add_buffer(offsets)
+            .add_child_data(l3)
+            .build()
+            .unwrap();
+
+        let offsets = to_offsets::<OffsetSize>(vec![0, 4, 4, 4, 5]);

Review Comment:
   Would it be valuable to have a sublist that has more than 1 item? 
   
   It appears that this test only has `null`, `[]` or a single element `[x]` list. I may be misunderstanding the intent of the test or what is possible in nested lists



##########
parquet/src/arrow/array_reader/list_array.rs:
##########
@@ -193,122 +245,267 @@ mod tests {
     use crate::file::reader::{FileReader, SerializedFileReader};
     use crate::schema::parser::parse_message_type;
     use crate::schema::types::SchemaDescriptor;
-    use arrow::array::{Array, LargeListArray, ListArray, PrimitiveArray};
-    use arrow::datatypes::{Field, Int32Type as ArrowInt32};
+    use arrow::array::{Array, ArrayDataBuilder, PrimitiveArray};
+    use arrow::datatypes::{Field, Int32Type as ArrowInt32, Int32Type};
     use std::sync::Arc;
 
-    #[test]
-    fn test_list_array_reader() {
-        // [[1, null, 2], null, [3, 4]]
+    fn list_type<OffsetSize: OffsetSizeTrait>(
+        data_type: ArrowType,
+        item_nullable: bool,
+    ) -> ArrowType {
+        let field = Box::new(Field::new("item", data_type, item_nullable));
+        match OffsetSize::is_large() {
+            true => ArrowType::LargeList(field),
+            false => ArrowType::List(field),
+        }
+    }
+
+    fn downcast<OffsetSize: OffsetSizeTrait>(
+        array: &ArrayRef,
+    ) -> &'_ GenericListArray<OffsetSize> {
+        array
+            .as_any()
+            .downcast_ref::<GenericListArray<OffsetSize>>()
+            .unwrap()
+    }
+
+    fn to_offsets<OffsetSize: OffsetSizeTrait>(values: Vec<usize>) -> Buffer {
+        Buffer::from_iter(
+            values
+                .into_iter()
+                .map(|x| OffsetSize::from_usize(x).unwrap()),
+        )
+    }
+
+    fn test_nested_list<OffsetSize: OffsetSizeTrait>() {
+        // 3 lists, with first and third nullable
+        // [
+        //     [
+        //         [[1, null], null, [4], []],
+        //         [],
+        //         [[7]],
+        //         [[]]
+        //     ],
+        //     null,
+        //     [],
+        //     [[[11]]]
+        // ]
+
+        let l3_item_type = ArrowType::Int32;
+        let l3_type = list_type::<OffsetSize>(l3_item_type.clone(), true);
+
+        let l2_item_type = l3_type.clone();
+        let l2_type = list_type::<OffsetSize>(l2_item_type.clone(), true);
+
+        let l1_item_type = l2_type.clone();
+        let l1_type = list_type::<OffsetSize>(l1_item_type.clone(), false);
+
+        let leaf = PrimitiveArray::<Int32Type>::from_iter(vec![
+            Some(1),
+            None,
+            Some(4),
+            Some(7),
+            Some(11),
+        ]);
+
+        // [[1, null], null, [4], [], [7], [], [11]]
+        let offsets = to_offsets::<OffsetSize>(vec![0, 2, 2, 3, 3, 4, 4, 5]);
+        let l3 = ArrayDataBuilder::new(l3_type.clone())
+            .len(7)
+            .add_buffer(offsets)
+            .add_child_data(leaf.data().clone())
+            .null_bit_buffer(Buffer::from([0b01111101]))
+            .build()
+            .unwrap();
+
+        // [[[1, null], null, [4], []],[], [[7]], [[]], [[11]]]
+        let offsets = to_offsets::<OffsetSize>(vec![0, 4, 4, 5, 6, 7]);
+        let l2 = ArrayDataBuilder::new(l2_type.clone())
+            .len(5)
+            .add_buffer(offsets)
+            .add_child_data(l3)
+            .build()
+            .unwrap();
+
+        let offsets = to_offsets::<OffsetSize>(vec![0, 4, 4, 4, 5]);
+        let l1 = ArrayDataBuilder::new(l1_type.clone())
+            .len(4)
+            .add_buffer(offsets)
+            .add_child_data(l2)
+            .null_bit_buffer(Buffer::from([0b00001101]))
+            .build()
+            .unwrap();
+
+        let expected = GenericListArray::<OffsetSize>::from(l1);
+
+        let values = Arc::new(PrimitiveArray::<Int32Type>::from(vec![
+            Some(1),
+            None,
+            None,
+            Some(4),
+            None,
+            None,
+            Some(7),
+            None,
+            None,
+            None,
+            Some(11),
+        ]));
+
+        let item_array_reader = InMemoryArrayReader::new(
+            ArrowType::Int32,
+            values,
+            Some(vec![6, 5, 3, 6, 4, 2, 6, 4, 0, 1, 6]),
+            Some(vec![0, 3, 2, 2, 2, 1, 1, 1, 0, 0, 0]),
+        );
+
+        let l3 = ListArrayReader::<OffsetSize>::new(
+            Box::new(item_array_reader),
+            l3_type,
+            l3_item_type,
+            5,
+            3,
+            true,
+        );
+
+        let l2 = ListArrayReader::<OffsetSize>::new(
+            Box::new(l3),
+            l2_type,
+            l2_item_type,
+            3,
+            2,
+            false,
+        );
+
+        let mut l1 = ListArrayReader::<OffsetSize>::new(
+            Box::new(l2),
+            l1_type,
+            l1_item_type,
+            2,
+            1,
+            true,
+        );
+
+        let actual = l1.next_batch(1024).unwrap();

Review Comment:
   Would it also make sense to test with `l1.next_batch(2)` (aka something that doesn't decode the entire array in one go?)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org