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 2021/01/06 17:34:13 UTC

[GitHub] [arrow] ovr opened a new pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

ovr opened a new pull request #9114:
URL: https://github.com/apache/arrow/pull/9114


   Hello!
   
   Thanks


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

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



[GitHub] [arrow] ovr commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r560497147



##########
File path: rust/arrow/src/array/array_list.rs
##########
@@ -295,6 +299,264 @@ impl fmt::Debug for FixedSizeListArray {
     }
 }
 
+macro_rules! build_empty_list_array_with_primitive_items {
+    ($item_type:ident, $offset_type:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, PrimitiveBuilder<$item_type>>::new(
+                values_builder,
+            );
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_list_array_with_non_primitive_items {
+    ($type_builder:ident, $offset_type:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, $type_builder>::new(values_builder);

Review comment:
       Arrow API is a Complex, I still dont know how to do it pretty simple as you suggested with Buffer. I think it's not a big performance impact to use builder in this place.




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

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



[GitHub] [arrow] ovr commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r552842524



##########
File path: rust/arrow/src/array/array_list.rs
##########
@@ -295,6 +298,185 @@ impl fmt::Debug for FixedSizeListArray {
     }
 }
 
+macro_rules! build_empty_list_array_with_primitive_items {
+    ($item_type:ident, $list_builder:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder = $list_builder::new(values_builder);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_list_array_with_non_primitive_items {
+    ($type_builder:ident, $list_builder:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder = $list_builder::new(values_builder);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! make_empty_list_fn {
+    ($name:ident, $list_builder:ident) => {
+        pub fn $name(item_type: DataType) -> Result<ArrayRef> {
+            match item_type {
+                DataType::UInt8 => {
+                    build_empty_list_array_with_primitive_items!(UInt8Type, $list_builder)
+                }
+                DataType::UInt16 => {
+                    build_empty_list_array_with_primitive_items!(
+                        UInt16Type,
+                        $list_builder
+                    )
+                }
+                DataType::UInt32 => {
+                    build_empty_list_array_with_primitive_items!(
+                        UInt32Type,
+                        $list_builder
+                    )
+                }
+                DataType::UInt64 => {
+                    build_empty_list_array_with_primitive_items!(
+                        UInt64Type,
+                        $list_builder
+                    )
+                }
+                DataType::Int8 => {
+                    build_empty_list_array_with_primitive_items!(Int8Type, $list_builder)
+                }
+                DataType::Int16 => {
+                    build_empty_list_array_with_primitive_items!(Int16Type, $list_builder)
+                }
+                DataType::Int32 => {
+                    build_empty_list_array_with_primitive_items!(Int32Type, $list_builder)
+                }
+                DataType::Int64 => {
+                    build_empty_list_array_with_primitive_items!(Int64Type, $list_builder)
+                }
+                DataType::Float32 => {
+                    build_empty_list_array_with_primitive_items!(
+                        Float32Type,
+                        $list_builder
+                    )
+                }
+                DataType::Float64 => {
+                    build_empty_list_array_with_primitive_items!(
+                        Float64Type,
+                        $list_builder
+                    )
+                }
+                DataType::Boolean => {
+                    //build_empty_list_array_with_primitive_items!(ArrowBooleanType)
+                    todo!()
+                }
+                DataType::Date32(_) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Date32Type,
+                        $list_builder
+                    )
+                }
+                DataType::Date64(_) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Date64Type,
+                        $list_builder
+                    )
+                }
+                DataType::Time32(TimeUnit::Second) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time32SecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Time32(TimeUnit::Millisecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time32MillisecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Time64(TimeUnit::Microsecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time64MicrosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Time64(TimeUnit::Nanosecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time64NanosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Second) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationSecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Millisecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationMillisecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Microsecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationMicrosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Nanosecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationNanosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Second, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampSecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Millisecond, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampMillisecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Microsecond, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampMicrosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Nanosecond, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampNanosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Utf8 => {
+                    build_empty_list_array_with_non_primitive_items!(
+                        StringBuilder,
+                        $list_builder
+                    )
+                }
+                DataType::Binary => {
+                    build_empty_list_array_with_non_primitive_items!(
+                        BinaryBuilder,
+                        $list_builder
+                    )
+                }
+                _ => Err(ArrowError::Unsupported(format!(

Review comment:
       What `ArrowError` should I throw on unimplemented? todo!? Or let's declare this one?
   
   Thanks




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

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



[GitHub] [arrow] jorgecarleitao commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-755931020


   I believe that we can make this with generics, now that we have `GenericListBuilder::<Offset>()`. It would make the code simpler to read, and a consistent function signature.


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

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



[GitHub] [arrow] ovr commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-758273338


   @andygrove Can you take a look? Thanks


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

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



[GitHub] [arrow] alamb commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r560935453



##########
File path: rust/arrow/src/error.rs
##########
@@ -24,6 +24,9 @@ use std::error::Error;
 /// Many different operations in the `arrow` crate return this error type.
 #[derive(Debug)]
 pub enum ArrowError {
+    /// "Not yet implemented" Arrow error.
+    /// Returned when functionality is not yet available.
+    NYI(String),

Review comment:
       I agree that `NotYetImplemented` is a better name




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

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



[GitHub] [arrow] jorgecarleitao commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r557868676



##########
File path: rust/arrow/src/array/array_list.rs
##########
@@ -295,6 +299,264 @@ impl fmt::Debug for FixedSizeListArray {
     }
 }
 
+macro_rules! build_empty_list_array_with_primitive_items {
+    ($item_type:ident, $offset_type:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, PrimitiveBuilder<$item_type>>::new(
+                values_builder,
+            );
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_list_array_with_non_primitive_items {
+    ($type_builder:ident, $offset_type:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, $type_builder>::new(values_builder);

Review comment:
       Note that for an empty list, we know that the offset buffer will be a single entry, `0`, and the values buffer will be an empty buffer (len = `0`). Therefore, this code could be simplified by just passing the buffers directly instead of using builders.
   

##########
File path: rust/arrow/src/error.rs
##########
@@ -90,6 +91,9 @@ impl From<serde_json::Error> for ArrowError {
 impl Display for ArrowError {
     fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
         match self {
+            ArrowError::Unsupported(source) => {

Review comment:
       What do you think of `Unimplemented` instead of `Unsupported`? Just to be consistent with the `unimplemented!` macro that rust already offers.

##########
File path: rust/arrow/src/array/array_list.rs
##########
@@ -295,6 +299,264 @@ impl fmt::Debug for FixedSizeListArray {
     }
 }
 
+macro_rules! build_empty_list_array_with_primitive_items {
+    ($item_type:ident, $offset_type:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, PrimitiveBuilder<$item_type>>::new(
+                values_builder,
+            );
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_list_array_with_non_primitive_items {
+    ($type_builder:ident, $offset_type:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, $type_builder>::new(values_builder);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+pub fn build_empty_list_array<OffsetSize: OffsetSizeTrait>(
+    item_type: DataType,
+) -> Result<ArrayRef> {
+    match item_type {
+        DataType::UInt8 => {
+            build_empty_list_array_with_primitive_items!(UInt8Type, OffsetSize)
+        }
+        DataType::UInt16 => {
+            build_empty_list_array_with_primitive_items!(UInt16Type, OffsetSize)
+        }
+        DataType::UInt32 => {
+            build_empty_list_array_with_primitive_items!(UInt32Type, OffsetSize)
+        }
+        DataType::UInt64 => {
+            build_empty_list_array_with_primitive_items!(UInt64Type, OffsetSize)
+        }
+        DataType::Int8 => {
+            build_empty_list_array_with_primitive_items!(Int8Type, OffsetSize)
+        }
+        DataType::Int16 => {
+            build_empty_list_array_with_primitive_items!(Int16Type, OffsetSize)
+        }
+        DataType::Int32 => {
+            build_empty_list_array_with_primitive_items!(Int32Type, OffsetSize)
+        }
+        DataType::Int64 => {
+            build_empty_list_array_with_primitive_items!(Int64Type, OffsetSize)
+        }
+        DataType::Float32 => {
+            build_empty_list_array_with_primitive_items!(Float32Type, OffsetSize)
+        }
+        DataType::Float64 => {
+            build_empty_list_array_with_primitive_items!(Float64Type, OffsetSize)
+        }
+        DataType::Boolean => {
+            build_empty_list_array_with_non_primitive_items!(BooleanBuilder, OffsetSize)
+        }
+        DataType::Date32(_) => {
+            build_empty_list_array_with_primitive_items!(Date32Type, OffsetSize)
+        }
+        DataType::Date64(_) => {
+            build_empty_list_array_with_primitive_items!(Date64Type, OffsetSize)
+        }
+        DataType::Time32(TimeUnit::Second) => {
+            build_empty_list_array_with_primitive_items!(Time32SecondType, OffsetSize)
+        }
+        DataType::Time32(TimeUnit::Millisecond) => {
+            build_empty_list_array_with_primitive_items!(
+                Time32MillisecondType,
+                OffsetSize
+            )
+        }
+        DataType::Time64(TimeUnit::Microsecond) => {
+            build_empty_list_array_with_primitive_items!(
+                Time64MicrosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Time64(TimeUnit::Nanosecond) => {
+            build_empty_list_array_with_primitive_items!(Time64NanosecondType, OffsetSize)
+        }
+        DataType::Duration(TimeUnit::Second) => {
+            build_empty_list_array_with_primitive_items!(DurationSecondType, OffsetSize)
+        }
+        DataType::Duration(TimeUnit::Millisecond) => {
+            build_empty_list_array_with_primitive_items!(
+                DurationMillisecondType,
+                OffsetSize
+            )
+        }
+        DataType::Duration(TimeUnit::Microsecond) => {
+            build_empty_list_array_with_primitive_items!(
+                DurationMicrosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Duration(TimeUnit::Nanosecond) => {
+            build_empty_list_array_with_primitive_items!(
+                DurationNanosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Timestamp(TimeUnit::Second, _) => {
+            build_empty_list_array_with_primitive_items!(TimestampSecondType, OffsetSize)
+        }
+        DataType::Timestamp(TimeUnit::Millisecond, _) => {
+            build_empty_list_array_with_primitive_items!(
+                TimestampMillisecondType,
+                OffsetSize
+            )
+        }
+        DataType::Timestamp(TimeUnit::Microsecond, _) => {
+            build_empty_list_array_with_primitive_items!(
+                TimestampMicrosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Timestamp(TimeUnit::Nanosecond, _) => {
+            build_empty_list_array_with_primitive_items!(
+                TimestampNanosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Utf8 => {
+            build_empty_list_array_with_non_primitive_items!(StringBuilder, OffsetSize)
+        }
+        DataType::Binary => {
+            build_empty_list_array_with_non_primitive_items!(BinaryBuilder, OffsetSize)
+        }
+        _ => Err(ArrowError::Unsupported(format!(
+            "GenericListBuilder of type List({:?}) is not supported",
+            item_type
+        ))),
+    }
+}
+
+macro_rules! build_empty_fixed_size_list_array_with_primitive_items {
+    ($item_type:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder = FixedSizeListBuilder::new(values_builder, 0);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_fixed_size_list_array_with_non_primitive_items {
+    ($type_builder:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder = FixedSizeListBuilder::new(values_builder, 0);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+pub fn build_empty_fixed_size_list_array(item_type: DataType) -> Result<ArrayRef> {
+    match item_type {
+        DataType::UInt8 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt8Type)
+        }
+        DataType::UInt16 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt16Type)
+        }
+        DataType::UInt32 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt32Type)
+        }
+        DataType::UInt64 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt64Type)
+        }
+        DataType::Int8 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int8Type)
+        }
+        DataType::Int16 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int16Type)
+        }
+        DataType::Int32 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int32Type)
+        }
+        DataType::Int64 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int64Type)
+        }
+        DataType::Float32 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Float32Type)
+        }
+        DataType::Float64 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Float64Type)
+        }
+        DataType::Boolean => {
+            build_empty_fixed_size_list_array_with_non_primitive_items!(BooleanBuilder)
+        }
+        DataType::Date32(_) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Date32Type)
+        }
+        DataType::Date64(_) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Date64Type)
+        }
+        DataType::Time32(TimeUnit::Second) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time32SecondType)
+        }
+        DataType::Time32(TimeUnit::Millisecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time32MillisecondType)
+        }
+        DataType::Time64(TimeUnit::Microsecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time64MicrosecondType)
+        }
+        DataType::Time64(TimeUnit::Nanosecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time64NanosecondType)
+        }
+        DataType::Duration(TimeUnit::Second) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(DurationSecondType)
+        }
+        DataType::Duration(TimeUnit::Millisecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                DurationMillisecondType
+            )
+        }
+        DataType::Duration(TimeUnit::Microsecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                DurationMicrosecondType
+            )
+        }
+        DataType::Duration(TimeUnit::Nanosecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                DurationNanosecondType
+            )
+        }
+        DataType::Timestamp(TimeUnit::Second, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(TimestampSecondType)
+        }
+        DataType::Timestamp(TimeUnit::Millisecond, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                TimestampMillisecondType
+            )
+        }
+        DataType::Timestamp(TimeUnit::Microsecond, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                TimestampMicrosecondType
+            )
+        }
+        DataType::Timestamp(TimeUnit::Nanosecond, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                TimestampNanosecondType
+            )
+        }
+        DataType::Utf8 => {
+            build_empty_fixed_size_list_array_with_non_primitive_items!(StringBuilder)
+        }
+        DataType::Binary => {
+            build_empty_fixed_size_list_array_with_non_primitive_items!(BinaryBuilder)
+        }
+        _ => Err(ArrowError::Unsupported(format!(
+            "FixedSizeListBuilder of type FixedSizeList({:?}) is not supported",
+            item_type
+        ))),
+    }
+}
+
 #[cfg(test)]
 mod tests {
     use crate::{

Review comment:
       Could you add a test just to verify?




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

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



[GitHub] [arrow] ovr commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r552842524



##########
File path: rust/arrow/src/array/array_list.rs
##########
@@ -295,6 +298,185 @@ impl fmt::Debug for FixedSizeListArray {
     }
 }
 
+macro_rules! build_empty_list_array_with_primitive_items {
+    ($item_type:ident, $list_builder:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder = $list_builder::new(values_builder);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_list_array_with_non_primitive_items {
+    ($type_builder:ident, $list_builder:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder = $list_builder::new(values_builder);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! make_empty_list_fn {
+    ($name:ident, $list_builder:ident) => {
+        pub fn $name(item_type: DataType) -> Result<ArrayRef> {
+            match item_type {
+                DataType::UInt8 => {
+                    build_empty_list_array_with_primitive_items!(UInt8Type, $list_builder)
+                }
+                DataType::UInt16 => {
+                    build_empty_list_array_with_primitive_items!(
+                        UInt16Type,
+                        $list_builder
+                    )
+                }
+                DataType::UInt32 => {
+                    build_empty_list_array_with_primitive_items!(
+                        UInt32Type,
+                        $list_builder
+                    )
+                }
+                DataType::UInt64 => {
+                    build_empty_list_array_with_primitive_items!(
+                        UInt64Type,
+                        $list_builder
+                    )
+                }
+                DataType::Int8 => {
+                    build_empty_list_array_with_primitive_items!(Int8Type, $list_builder)
+                }
+                DataType::Int16 => {
+                    build_empty_list_array_with_primitive_items!(Int16Type, $list_builder)
+                }
+                DataType::Int32 => {
+                    build_empty_list_array_with_primitive_items!(Int32Type, $list_builder)
+                }
+                DataType::Int64 => {
+                    build_empty_list_array_with_primitive_items!(Int64Type, $list_builder)
+                }
+                DataType::Float32 => {
+                    build_empty_list_array_with_primitive_items!(
+                        Float32Type,
+                        $list_builder
+                    )
+                }
+                DataType::Float64 => {
+                    build_empty_list_array_with_primitive_items!(
+                        Float64Type,
+                        $list_builder
+                    )
+                }
+                DataType::Boolean => {
+                    //build_empty_list_array_with_primitive_items!(ArrowBooleanType)
+                    todo!()
+                }
+                DataType::Date32(_) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Date32Type,
+                        $list_builder
+                    )
+                }
+                DataType::Date64(_) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Date64Type,
+                        $list_builder
+                    )
+                }
+                DataType::Time32(TimeUnit::Second) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time32SecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Time32(TimeUnit::Millisecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time32MillisecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Time64(TimeUnit::Microsecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time64MicrosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Time64(TimeUnit::Nanosecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        Time64NanosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Second) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationSecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Millisecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationMillisecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Microsecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationMicrosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Duration(TimeUnit::Nanosecond) => {
+                    build_empty_list_array_with_primitive_items!(
+                        DurationNanosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Second, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampSecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Millisecond, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampMillisecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Microsecond, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampMicrosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Timestamp(TimeUnit::Nanosecond, _) => {
+                    build_empty_list_array_with_primitive_items!(
+                        TimestampNanosecondType,
+                        $list_builder
+                    )
+                }
+                DataType::Utf8 => {
+                    build_empty_list_array_with_non_primitive_items!(
+                        StringBuilder,
+                        $list_builder
+                    )
+                }
+                DataType::Binary => {
+                    build_empty_list_array_with_non_primitive_items!(
+                        BinaryBuilder,
+                        $list_builder
+                    )
+                }
+                _ => Err(ArrowError::Unsupported(format!(

Review comment:
       What `ArrowError` should I throw on implemented? todo!? Or let's declare this one?
   
   Thanks




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

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



[GitHub] [arrow] jorgecarleitao commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r560369235



##########
File path: rust/arrow/src/error.rs
##########
@@ -24,6 +24,9 @@ use std::error::Error;
 /// Many different operations in the `arrow` crate return this error type.
 #[derive(Debug)]
 pub enum ArrowError {
+    /// "Not yet implemented" Arrow error.
+    /// Returned when functionality is not yet available.
+    NYI(String),

Review comment:
       I think that the error name should be explicit, like all others. I can't understand what `NYI` means without having to go to the docs, and in an error message, the person often has no access to the docs (at least not in a 1 click thing). 
   
   `NotYetImplemented`?




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

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



[GitHub] [arrow] ovr commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r558895099



##########
File path: rust/arrow/src/error.rs
##########
@@ -90,6 +91,9 @@ impl From<serde_json::Error> for ArrowError {
 impl Display for ArrowError {
     fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
         match self {
+            ArrowError::Unsupported(source) => {

Review comment:
       I found a better name from reading parquet sources, It's called `NYI` (Not yet implemented), I think it's better to use similar names accross packages.
   
   Renamed.
   
   Thanks




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

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



[GitHub] [arrow] alamb edited a comment on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
alamb edited a comment on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-763838115


   The travis CI run is backed up -- https://github.com/apache/arrow/pull/9114/checks?check_run_id=1735587739 hasn't finished -- and this PR has no non-rust changes. I think it is good to 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.

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



[GitHub] [arrow] alamb commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-763838115


   The travis CI run is backed up -- and this PR has no non-rust changes. I think it is good to 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.

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



[GitHub] [arrow] codecov-io commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-761570918


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/9114?src=pr&el=h1) Report
   > Merging [#9114](https://codecov.io/gh/apache/arrow/pull/9114?src=pr&el=desc) (849d8d3) into [master](https://codecov.io/gh/apache/arrow/commit/1393188e1aa1b3d59993ce7d4ade7f7ac8570959?el=desc) (1393188) will **decrease** coverage by `0.01%`.
   > The diff coverage is `39.79%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/9114/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/9114?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #9114      +/-   ##
   ==========================================
   - Coverage   81.61%   81.59%   -0.02%     
   ==========================================
     Files         215      215              
     Lines       51867    51928      +61     
   ==========================================
   + Hits        42329    42370      +41     
   - Misses       9538     9558      +20     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/9114?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/datafusion/src/physical\_plan/common.rs](https://codecov.io/gh/apache/arrow/pull/9114/diff?src=pr&el=tree#diff-cnVzdC9kYXRhZnVzaW9uL3NyYy9waHlzaWNhbF9wbGFuL2NvbW1vbi5ycw==) | `74.28% <0.00%> (-4.51%)` | :arrow_down: |
   | [rust/parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow/pull/9114/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9hcnJvdy9hcnJheV9yZWFkZXIucnM=) | `73.65% <0.00%> (+2.25%)` | :arrow_up: |
   | [rust/arrow/src/array/array\_list.rs](https://codecov.io/gh/apache/arrow/pull/9114/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvYXJyYXlfbGlzdC5ycw==) | `83.33% <43.67%> (-9.78%)` | :arrow_down: |
   | [rust/arrow/src/error.rs](https://codecov.io/gh/apache/arrow/pull/9114/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvZXJyb3IucnM=) | `9.52% <50.00%> (-0.48%)` | :arrow_down: |
   | [rust/parquet/src/encodings/encoding.rs](https://codecov.io/gh/apache/arrow/pull/9114/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9lbmNvZGluZ3MvZW5jb2RpbmcucnM=) | `94.86% <0.00%> (-0.20%)` | :arrow_down: |
   | [rust/arrow/src/array/builder.rs](https://codecov.io/gh/apache/arrow/pull/9114/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYXJyYXkvYnVpbGRlci5ycw==) | `86.30% <0.00%> (+0.40%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/9114?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/9114?src=pr&el=footer). Last update [eaa7b7a...c898107](https://codecov.io/gh/apache/arrow/pull/9114?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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

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



[GitHub] [arrow] ovr commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-756253421


   @jorgecarleitao and @nevi-me 👍 
   
   I've moved build_empty_list_array to function `build_empty_list_array<OffsetSize: OffsetSizeTrait>`, and introduce `build_empty_fixed_size_list_array` by different implementation as suggested above.
   
   Thanks


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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-755643977


   https://issues.apache.org/jira/browse/ARROW-11149


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

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



[GitHub] [arrow] ovr commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r558895099



##########
File path: rust/arrow/src/error.rs
##########
@@ -90,6 +91,9 @@ impl From<serde_json::Error> for ArrowError {
 impl Display for ArrowError {
     fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
         match self {
+            ArrowError::Unsupported(source) => {

Review comment:
       I found a better name from reading parquet sources, It's called `NYI` (Not yet implemented), I think it's better to use similar names accross packages.
   
   Thanks




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

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



[GitHub] [arrow] nevi-me commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
nevi-me commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-756213425


   Yes, better to implement it differently


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

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



[GitHub] [arrow] jorgecarleitao commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-756212761


   Last time I tried something similar, I had to implement the `FixedSizeList` differently. A fixed sized will always need to receive an extra parameter.


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

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



[GitHub] [arrow] alamb closed pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
alamb closed pull request #9114:
URL: https://github.com/apache/arrow/pull/9114


   


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

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



[GitHub] [arrow] ovr commented on a change in pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on a change in pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#discussion_r558894537



##########
File path: rust/arrow/src/array/array_list.rs
##########
@@ -295,6 +299,264 @@ impl fmt::Debug for FixedSizeListArray {
     }
 }
 
+macro_rules! build_empty_list_array_with_primitive_items {
+    ($item_type:ident, $offset_type:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, PrimitiveBuilder<$item_type>>::new(
+                values_builder,
+            );
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_list_array_with_non_primitive_items {
+    ($type_builder:ident, $offset_type:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder =
+            GenericListBuilder::<$offset_type, $type_builder>::new(values_builder);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+pub fn build_empty_list_array<OffsetSize: OffsetSizeTrait>(
+    item_type: DataType,
+) -> Result<ArrayRef> {
+    match item_type {
+        DataType::UInt8 => {
+            build_empty_list_array_with_primitive_items!(UInt8Type, OffsetSize)
+        }
+        DataType::UInt16 => {
+            build_empty_list_array_with_primitive_items!(UInt16Type, OffsetSize)
+        }
+        DataType::UInt32 => {
+            build_empty_list_array_with_primitive_items!(UInt32Type, OffsetSize)
+        }
+        DataType::UInt64 => {
+            build_empty_list_array_with_primitive_items!(UInt64Type, OffsetSize)
+        }
+        DataType::Int8 => {
+            build_empty_list_array_with_primitive_items!(Int8Type, OffsetSize)
+        }
+        DataType::Int16 => {
+            build_empty_list_array_with_primitive_items!(Int16Type, OffsetSize)
+        }
+        DataType::Int32 => {
+            build_empty_list_array_with_primitive_items!(Int32Type, OffsetSize)
+        }
+        DataType::Int64 => {
+            build_empty_list_array_with_primitive_items!(Int64Type, OffsetSize)
+        }
+        DataType::Float32 => {
+            build_empty_list_array_with_primitive_items!(Float32Type, OffsetSize)
+        }
+        DataType::Float64 => {
+            build_empty_list_array_with_primitive_items!(Float64Type, OffsetSize)
+        }
+        DataType::Boolean => {
+            build_empty_list_array_with_non_primitive_items!(BooleanBuilder, OffsetSize)
+        }
+        DataType::Date32(_) => {
+            build_empty_list_array_with_primitive_items!(Date32Type, OffsetSize)
+        }
+        DataType::Date64(_) => {
+            build_empty_list_array_with_primitive_items!(Date64Type, OffsetSize)
+        }
+        DataType::Time32(TimeUnit::Second) => {
+            build_empty_list_array_with_primitive_items!(Time32SecondType, OffsetSize)
+        }
+        DataType::Time32(TimeUnit::Millisecond) => {
+            build_empty_list_array_with_primitive_items!(
+                Time32MillisecondType,
+                OffsetSize
+            )
+        }
+        DataType::Time64(TimeUnit::Microsecond) => {
+            build_empty_list_array_with_primitive_items!(
+                Time64MicrosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Time64(TimeUnit::Nanosecond) => {
+            build_empty_list_array_with_primitive_items!(Time64NanosecondType, OffsetSize)
+        }
+        DataType::Duration(TimeUnit::Second) => {
+            build_empty_list_array_with_primitive_items!(DurationSecondType, OffsetSize)
+        }
+        DataType::Duration(TimeUnit::Millisecond) => {
+            build_empty_list_array_with_primitive_items!(
+                DurationMillisecondType,
+                OffsetSize
+            )
+        }
+        DataType::Duration(TimeUnit::Microsecond) => {
+            build_empty_list_array_with_primitive_items!(
+                DurationMicrosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Duration(TimeUnit::Nanosecond) => {
+            build_empty_list_array_with_primitive_items!(
+                DurationNanosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Timestamp(TimeUnit::Second, _) => {
+            build_empty_list_array_with_primitive_items!(TimestampSecondType, OffsetSize)
+        }
+        DataType::Timestamp(TimeUnit::Millisecond, _) => {
+            build_empty_list_array_with_primitive_items!(
+                TimestampMillisecondType,
+                OffsetSize
+            )
+        }
+        DataType::Timestamp(TimeUnit::Microsecond, _) => {
+            build_empty_list_array_with_primitive_items!(
+                TimestampMicrosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Timestamp(TimeUnit::Nanosecond, _) => {
+            build_empty_list_array_with_primitive_items!(
+                TimestampNanosecondType,
+                OffsetSize
+            )
+        }
+        DataType::Utf8 => {
+            build_empty_list_array_with_non_primitive_items!(StringBuilder, OffsetSize)
+        }
+        DataType::Binary => {
+            build_empty_list_array_with_non_primitive_items!(BinaryBuilder, OffsetSize)
+        }
+        _ => Err(ArrowError::Unsupported(format!(
+            "GenericListBuilder of type List({:?}) is not supported",
+            item_type
+        ))),
+    }
+}
+
+macro_rules! build_empty_fixed_size_list_array_with_primitive_items {
+    ($item_type:ident) => {{
+        let values_builder = PrimitiveBuilder::<$item_type>::new(0);
+        let mut builder = FixedSizeListBuilder::new(values_builder, 0);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+macro_rules! build_empty_fixed_size_list_array_with_non_primitive_items {
+    ($type_builder:ident) => {{
+        let values_builder = $type_builder::new(0);
+        let mut builder = FixedSizeListBuilder::new(values_builder, 0);
+        let empty_list_array = builder.finish();
+        Ok(Arc::new(empty_list_array))
+    }};
+}
+
+pub fn build_empty_fixed_size_list_array(item_type: DataType) -> Result<ArrayRef> {
+    match item_type {
+        DataType::UInt8 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt8Type)
+        }
+        DataType::UInt16 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt16Type)
+        }
+        DataType::UInt32 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt32Type)
+        }
+        DataType::UInt64 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(UInt64Type)
+        }
+        DataType::Int8 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int8Type)
+        }
+        DataType::Int16 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int16Type)
+        }
+        DataType::Int32 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int32Type)
+        }
+        DataType::Int64 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Int64Type)
+        }
+        DataType::Float32 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Float32Type)
+        }
+        DataType::Float64 => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Float64Type)
+        }
+        DataType::Boolean => {
+            build_empty_fixed_size_list_array_with_non_primitive_items!(BooleanBuilder)
+        }
+        DataType::Date32(_) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Date32Type)
+        }
+        DataType::Date64(_) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Date64Type)
+        }
+        DataType::Time32(TimeUnit::Second) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time32SecondType)
+        }
+        DataType::Time32(TimeUnit::Millisecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time32MillisecondType)
+        }
+        DataType::Time64(TimeUnit::Microsecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time64MicrosecondType)
+        }
+        DataType::Time64(TimeUnit::Nanosecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(Time64NanosecondType)
+        }
+        DataType::Duration(TimeUnit::Second) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(DurationSecondType)
+        }
+        DataType::Duration(TimeUnit::Millisecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                DurationMillisecondType
+            )
+        }
+        DataType::Duration(TimeUnit::Microsecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                DurationMicrosecondType
+            )
+        }
+        DataType::Duration(TimeUnit::Nanosecond) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                DurationNanosecondType
+            )
+        }
+        DataType::Timestamp(TimeUnit::Second, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(TimestampSecondType)
+        }
+        DataType::Timestamp(TimeUnit::Millisecond, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                TimestampMillisecondType
+            )
+        }
+        DataType::Timestamp(TimeUnit::Microsecond, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                TimestampMicrosecondType
+            )
+        }
+        DataType::Timestamp(TimeUnit::Nanosecond, _) => {
+            build_empty_fixed_size_list_array_with_primitive_items!(
+                TimestampNanosecondType
+            )
+        }
+        DataType::Utf8 => {
+            build_empty_fixed_size_list_array_with_non_primitive_items!(StringBuilder)
+        }
+        DataType::Binary => {
+            build_empty_fixed_size_list_array_with_non_primitive_items!(BinaryBuilder)
+        }
+        _ => Err(ArrowError::Unsupported(format!(
+            "FixedSizeListBuilder of type FixedSizeList({:?}) is not supported",
+            item_type
+        ))),
+    }
+}
+
 #[cfg(test)]
 mod tests {
     use crate::{

Review comment:
       Thanks, added.




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

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



[GitHub] [arrow] alamb commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-763837298


   Awesome -- thanks @ovr !


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

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



[GitHub] [arrow] ovr commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList/FixedSizeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-763672988


   @alamb Rebased + renamed NYI -> `NotYetImplemented` in Arrow. Thanks


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

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



[GitHub] [arrow] ovr commented on pull request #9114: ARROW-11149: [Rust] DF Support List/LargeList in create_batch_empty

Posted by GitBox <gi...@apache.org>.
ovr commented on pull request #9114:
URL: https://github.com/apache/arrow/pull/9114#issuecomment-756192484


   @jorgecarleitao You are right, but I was thinking about modifying `make_empty_list_fn` macros to support  `FixedSizeListBuilder`, which is not `GenericListBuilder`.
   
   Something like:
   
   `make_empty_list_fn!(build_empty_fixed_size_list_array, FixedSizeListBuilder);`
   
   But I am having a problem with constructing it, `FixedSizeListBuilder` requires 2 arguments in `new` method instead of one like `GenericListBuilder`.
   
   `pub fn new(values_builder: T, length: i32) -> Self {`
   
   Do you have any solutions how is it possible to do by Marcos? And is it a good idea?
   
   Thanks!
   


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

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