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 19:30:40 UTC

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

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


##########
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:
   There is a `[1, null]`, can probably add more though



-- 
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