You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by tu...@apache.org on 2023/06/05 09:52:30 UTC

[arrow-rs] branch master updated: Add support for FixedSizeList in array_to_json_array (#4361)

This is an automated email from the ASF dual-hosted git repository.

tustvold pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git


The following commit(s) were added to refs/heads/master by this push:
     new ac9c6fa13 Add support for FixedSizeList in array_to_json_array (#4361)
ac9c6fa13 is described below

commit ac9c6fa134280581c7e19750910b6c74153a75d4
Author: dadepo <da...@gmail.com>
AuthorDate: Mon Jun 5 13:52:24 2023 +0400

    Add support for FixedSizeList in array_to_json_array (#4361)
    
    * Add support for FixedSizeList in array_to_json_array
    
    * Properly named the test function
    
    * Added iter implementation FixedSizeListArray. Updated array_to_json_array to use it
---
 arrow-array/src/array/fixed_size_list_array.rs | 26 ++++++++++++++++---
 arrow-array/src/cast.rs                        |  9 +++++++
 arrow-array/src/iterator.rs                    |  3 +++
 arrow-json/src/writer.rs                       | 36 ++++++++++++++++++++++++--
 4 files changed, 69 insertions(+), 5 deletions(-)

diff --git a/arrow-array/src/array/fixed_size_list_array.rs b/arrow-array/src/array/fixed_size_list_array.rs
index 3df108ced..6c1598ce9 100644
--- a/arrow-array/src/array/fixed_size_list_array.rs
+++ b/arrow-array/src/array/fixed_size_list_array.rs
@@ -17,6 +17,7 @@
 
 use crate::array::print_long_array;
 use crate::builder::{FixedSizeListBuilder, PrimitiveBuilder};
+use crate::iterator::FixedSizeListIter;
 use crate::{make_array, Array, ArrayAccessor, ArrayRef, ArrowPrimitiveType};
 use arrow_buffer::buffer::NullBuffer;
 use arrow_buffer::ArrowNativeType;
@@ -277,6 +278,11 @@ impl FixedSizeListArray {
         }
         builder.finish()
     }
+
+    /// constructs a new iterator
+    pub fn iter(&self) -> FixedSizeListIter<'_> {
+        FixedSizeListIter::new(self)
+    }
 }
 
 impl From<ArrayData> for FixedSizeListArray {
@@ -389,14 +395,28 @@ impl std::fmt::Debug for FixedSizeListArray {
     }
 }
 
+impl<'a> ArrayAccessor for &'a FixedSizeListArray {
+    type Item = ArrayRef;
+
+    fn value(&self, index: usize) -> Self::Item {
+        FixedSizeListArray::value(self, index)
+    }
+
+    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
+        FixedSizeListArray::value(self, index)
+    }
+}
+
 #[cfg(test)]
 mod tests {
-    use super::*;
+    use arrow_buffer::{bit_util, BooleanBuffer, Buffer};
+    use arrow_schema::Field;
+
     use crate::cast::AsArray;
     use crate::types::Int32Type;
     use crate::Int32Array;
-    use arrow_buffer::{bit_util, BooleanBuffer, Buffer};
-    use arrow_schema::Field;
+
+    use super::*;
 
     #[test]
     fn test_fixed_size_list_array() {
diff --git a/arrow-array/src/cast.rs b/arrow-array/src/cast.rs
index af7e7d606..1f8bb6587 100644
--- a/arrow-array/src/cast.rs
+++ b/arrow-array/src/cast.rs
@@ -593,6 +593,15 @@ pub fn as_list_array(arr: &dyn Array) -> &ListArray {
     as_generic_list_array::<i32>(arr)
 }
 
+/// Force downcast of an [`Array`], such as an [`ArrayRef`] to
+/// [`FixedSizeListArray`], panic'ing on failure.
+#[inline]
+pub fn as_fixed_size_list_array(arr: &dyn Array) -> &FixedSizeListArray {
+    arr.as_any()
+        .downcast_ref::<FixedSizeListArray>()
+        .expect("Unable to downcast to fixed size list array")
+}
+
 /// Force downcast of an [`Array`], such as an [`ArrayRef`] to
 /// [`LargeListArray`], panic'ing on failure.
 #[inline]
diff --git a/arrow-array/src/iterator.rs b/arrow-array/src/iterator.rs
index ff9923312..fa76e09b2 100644
--- a/arrow-array/src/iterator.rs
+++ b/arrow-array/src/iterator.rs
@@ -21,6 +21,7 @@ use crate::array::{
     ArrayAccessor, BooleanArray, FixedSizeBinaryArray, GenericBinaryArray,
     GenericListArray, GenericStringArray, PrimitiveArray,
 };
+use crate::FixedSizeListArray;
 
 /// An iterator that returns Some(T) or None, that can be used on any [`ArrayAccessor`]
 ///
@@ -124,6 +125,8 @@ pub type GenericStringIter<'a, T> = ArrayIter<&'a GenericStringArray<T>>;
 pub type GenericBinaryIter<'a, T> = ArrayIter<&'a GenericBinaryArray<T>>;
 /// an iterator that returns Some(T) or None, that can be used on any FixedSizeBinaryArray
 pub type FixedSizeBinaryIter<'a> = ArrayIter<&'a FixedSizeBinaryArray>;
+/// an iterator that returns Some(T) or None, that can be used on any FixedSizeListArray
+pub type FixedSizeListIter<'a> = ArrayIter<&'a FixedSizeListArray>;
 /// an iterator that returns Some(T) or None, that can be used on any ListArray
 pub type GenericListArrayIter<'a, O> = ArrayIter<&'a GenericListArray<O>>;
 
diff --git a/arrow-json/src/writer.rs b/arrow-json/src/writer.rs
index d00662a72..d2365118a 100644
--- a/arrow-json/src/writer.rs
+++ b/arrow-json/src/writer.rs
@@ -191,6 +191,13 @@ pub fn array_to_json_array(array: &ArrayRef) -> Result<Vec<Value>, ArrowError> {
                 None => Ok(Value::Null),
             })
             .collect(),
+        DataType::FixedSizeList(_, _) => as_fixed_size_list_array(array)
+            .iter()
+            .map(|maybe_value| match maybe_value {
+                Some(v) => Ok(Value::Array(array_to_json_array(&v)?)),
+                None => Ok(Value::Null),
+            })
+            .collect(),
         DataType::Struct(_) => {
             let jsonmaps = struct_array_to_jsonmap_array(array.as_struct())?;
             Ok(jsonmaps.into_iter().map(Value::Object).collect())
@@ -610,10 +617,12 @@ mod tests {
     use std::io::{BufReader, Seek};
     use std::sync::Arc;
 
-    use crate::reader::*;
+    use serde_json::json;
+
     use arrow_buffer::{Buffer, ToByteSlice};
     use arrow_data::ArrayData;
-    use serde_json::json;
+
+    use crate::reader::*;
 
     use super::*;
 
@@ -1488,4 +1497,27 @@ mod tests {
             assert_eq!(serde_json::from_str::<Value>(r).unwrap(), expected_json,);
         }
     }
+
+    #[test]
+    fn test_array_to_json_array_for_fixed_size_list_array() {
+        let expected_json = vec![
+            json!([0, 1, 2]),
+            json!(null),
+            json!([3, null, 5]),
+            json!([6, 7, 45]),
+        ];
+
+        let data = vec![
+            Some(vec![Some(0), Some(1), Some(2)]),
+            None,
+            Some(vec![Some(3), None, Some(5)]),
+            Some(vec![Some(6), Some(7), Some(45)]),
+        ];
+
+        let list_array =
+            FixedSizeListArray::from_iter_primitive::<Int32Type, _, _>(data, 3);
+        let list_array = Arc::new(list_array) as ArrayRef;
+
+        assert_eq!(array_to_json_array(&list_array).unwrap(), expected_json);
+    }
 }