You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "tustvold (via GitHub)" <gi...@apache.org> on 2023/06/07 17:33:33 UTC

[GitHub] [arrow-rs] tustvold opened a new pull request, #4381: Documentation Improvements

tustvold opened a new pull request, #4381:
URL: https://github.com/apache/arrow-rs/pull/4381

   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Part of #4071
   
   # Rationale for this change
    
   <!--
   Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
   Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.
   -->
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   # Are there any user-facing changes?
   
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!---
   If there are any breaking changes to public APIs, please add the `breaking change` label.
   -->
   


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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1221954684


##########
arrow/src/lib.rs:
##########
@@ -317,6 +285,36 @@
 //! assert_eq!(string.value(1), "foo");
 //! ```
 //!
+//! # Crate Topology

Review Comment:
   This is moved lower-down so that the first thing people see is how to use an array



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1221955030


##########
arrow/src/lib.rs:
##########
@@ -109,7 +77,7 @@
 //! assert_eq!(min(&StringArray::from(vec!["b", "a", "c"])), Some("a"));
 //! ```
 //!
-//! For more examples, and details consult the [arrow_array] docs.
+//! **For more examples, and details consult the [arrow_array] docs.**

Review Comment:
   This is very easy to miss, as it is sandwiched between a code block and heading. Making it bold helps avoid this



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


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4381: Documentation Improvements

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1222980792


##########
arrow-array/src/array/boolean_array.rs:
##########
@@ -27,51 +27,47 @@ use std::sync::Arc;
 
 /// An array of [boolean values](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout)
 ///
-/// # Examples
+/// # Example: From a Vec
 ///
-/// Construction
+/// ```
+/// # use arrow_array::{Array, BooleanArray};
+/// let arr: BooleanArray = vec![true, true, false].into();
+/// let values: Vec<_> = arr.iter().collect();
+/// assert_eq!(&values, &[Some(true), Some(true), Some(false)])

Review Comment:
   
   
   I don't think this assert adds anything to the doc example: I think it is already clear from the docs what the array contains. 



##########
arrow-array/src/array/boolean_array.rs:
##########
@@ -27,51 +27,47 @@ use std::sync::Arc;
 
 /// An array of [boolean values](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout)
 ///
-/// # Examples
+/// # Example: From a Vec
 ///
-/// Construction
+/// ```
+/// # use arrow_array::{Array, BooleanArray};
+/// let arr: BooleanArray = vec![true, true, false].into();
+/// let values: Vec<_> = arr.iter().collect();
+/// assert_eq!(&values, &[Some(true), Some(true), Some(false)])
+/// ```
+///
+/// # Example: From an optional Vec
 ///
 /// ```
-///#     use arrow_array::{Array, BooleanArray};
-/// // Create from Vec<Option<bool>>
-/// let arr = BooleanArray::from(vec![Some(false), Some(true), None, Some(true)]);
-/// // Create from Vec<bool>
-/// let arr = BooleanArray::from(vec![false, true, true]);
-/// // Create from iter/collect
-/// let arr: BooleanArray = std::iter::repeat(Some(true)).take(10).collect();
+/// # use arrow_array::{Array, BooleanArray};
+/// let arr: BooleanArray = vec![Some(true), None, Some(false)].into();
+/// let values: Vec<_> = arr.iter().collect();
+/// assert_eq!(&values, &[Some(true), None, Some(false)])
 /// ```
 ///
-/// Construction and Access
+/// # Example: From an iterator
 ///
 /// ```
-/// use arrow_array::{Array, BooleanArray};
-/// let arr = BooleanArray::from(vec![Some(false), Some(true), None, Some(true)]);
-/// assert_eq!(4, arr.len());
-/// assert_eq!(1, arr.null_count());
-/// assert!(arr.is_valid(0));
-/// assert!(!arr.is_null(0));
-/// assert_eq!(false, arr.value(0));
-/// assert!(!arr.is_valid(2));
-/// assert!(arr.is_null(2));
+/// # use arrow_array::{Array, BooleanArray};
+/// let arr: BooleanArray = (0..5).map(|x| (x % 2 == 0).then(|| x % 3 == 0)).collect();
+/// let values: Vec<_> = arr.iter().collect();
+/// assert_eq!(&values, &[Some(true), None, Some(false), None, Some(false)])
 /// ```
 ///
-/// Using `collect`
+/// # Example: Using Builder
+///
 /// ```
-/// use arrow_array::{Array, BooleanArray};
-/// let v = vec![Some(false), Some(true), Some(false), Some(true)];
-/// let arr = v.into_iter().collect::<BooleanArray>();
-/// assert_eq!(4, arr.len());
-/// assert_eq!(0, arr.offset());
-/// assert_eq!(0, arr.null_count());
-/// assert!(arr.is_valid(0));
-/// assert_eq!(false, arr.value(0));
-/// assert!(arr.is_valid(1));
-/// assert_eq!(true, arr.value(1));
-/// assert!(arr.is_valid(2));
-/// assert_eq!(false, arr.value(2));
-/// assert!(arr.is_valid(3));
-/// assert_eq!(true, arr.value(3));
+/// # use arrow_array::Array;
+/// # use arrow_array::builder::BooleanBuilder;
+/// let mut builder = BooleanBuilder::new();
+/// builder.append_value(true);
+/// builder.append_null();
+/// builder.append_value(false);
+/// let array = builder.finish();
+/// let values: Vec<_> = array.iter().collect();
+/// assert_eq!(&values, &[Some(true), None, Some(false)])

Review Comment:
   this assert adds value as it shows the equivalent form that is different than the builder's construction'



##########
arrow-array/src/array/boolean_array.rs:
##########
@@ -27,51 +27,47 @@ use std::sync::Arc;
 
 /// An array of [boolean values](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout)
 ///
-/// # Examples
+/// # Example: From a Vec
 ///
-/// Construction
+/// ```
+/// # use arrow_array::{Array, BooleanArray};
+/// let arr: BooleanArray = vec![true, true, false].into();
+/// let values: Vec<_> = arr.iter().collect();
+/// assert_eq!(&values, &[Some(true), Some(true), Some(false)])
+/// ```
+///
+/// # Example: From an optional Vec
 ///
 /// ```
-///#     use arrow_array::{Array, BooleanArray};
-/// // Create from Vec<Option<bool>>
-/// let arr = BooleanArray::from(vec![Some(false), Some(true), None, Some(true)]);
-/// // Create from Vec<bool>
-/// let arr = BooleanArray::from(vec![false, true, true]);
-/// // Create from iter/collect
-/// let arr: BooleanArray = std::iter::repeat(Some(true)).take(10).collect();
+/// # use arrow_array::{Array, BooleanArray};
+/// let arr: BooleanArray = vec![Some(true), None, Some(false)].into();
+/// let values: Vec<_> = arr.iter().collect();
+/// assert_eq!(&values, &[Some(true), None, Some(false)])
 /// ```
 ///
-/// Construction and Access
+/// # Example: From an iterator

Review Comment:
   this is a good example



##########
arrow-array/src/array/boolean_array.rs:
##########
@@ -27,51 +27,47 @@ use std::sync::Arc;
 
 /// An array of [boolean values](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout)
 ///
-/// # Examples
+/// # Example: From a Vec
 ///
-/// Construction
+/// ```
+/// # use arrow_array::{Array, BooleanArray};
+/// let arr: BooleanArray = vec![true, true, false].into();
+/// let values: Vec<_> = arr.iter().collect();

Review Comment:
   This example seems unnecessarily complicated to me - why would we encourage this pattern instead of what was there
   
   ```rust
   let arr =  BooleanArray::from(vec![true, true, false])
   ```



##########
arrow-array/src/array/dictionary_array.rs:
##########
@@ -30,116 +30,28 @@ use arrow_schema::{ArrowError, DataType};
 use std::any::Any;
 use std::sync::Arc;
 
-/// A dictionary array indexed by `i8`

Review Comment:
   As I explained above, I think the duplication and specific example is very important to make this library to use for new users



##########
arrow-array/src/array/byte_array.rs:
##########
@@ -34,6 +34,52 @@ use std::sync::Arc;
 ///
 /// See [`BinaryArray`] and [`LargeBinaryArray`] for storing arbitrary bytes
 ///
+/// # Example: From a Vec
+///
+/// ```
+/// # use arrow_array::{Array, GenericByteArray, types::Utf8Type};
+/// let arr: GenericByteArray<Utf8Type> = vec!["hello", "world", ""].into();

Review Comment:
   I think this is not what a beginner user would use to create a `StringArray` -- while I realize doing it like this makes the code generic I think it makes figuring out how to make a `StringArray` significantly more complicated for a beginner



##########
arrow-array/src/array/primitive_array.rs:
##########
@@ -34,202 +34,40 @@ use half::f16;
 use std::any::Any;
 use std::sync::Arc;
 
-/// An array of `i8`

Review Comment:
   I think removing the specific examples is a significant reduction in UX as explained above



##########
arrow/src/lib.rs:
##########
@@ -317,6 +285,36 @@
 //! assert_eq!(string.value(1), "foo");
 //! ```
 //!
+//! # Crate Topology

Review Comment:
   I think this is a good change



##########
arrow-array/src/array/list_array.rs:
##########
@@ -57,6 +57,8 @@ impl OffsetSizeTrait for i64 {
 /// An array of [variable length arrays](https://arrow.apache.org/docs/format/Columnar.html#variable-size-list-layout)
 ///
 /// See [`ListArray`] and [`LargeListArray`]`
+///
+/// See [`GenericListBuilder`](crate::builder::GenericListBuilder) for how to construct a [`GenericListArray`]

Review Comment:
   I think this makes sense 👍 



##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   I think the duplication is worth the improved usability in this case and here is why:
   
   Imagine you are a new users who wants to use something like `DataFusion` to process your data and you figure out "ok, I need a `BinaryArray`"
   
   Today, if you go to the  docs https://docs.rs/arrow/latest/arrow/array/type.BinaryArray.html you'll get an example you can copy/paste/modify and you are on your way. 
   
   If instead the documentation simply says 
   
   ```
   /// A [`GenericBinaryArray`] of `[u8]` using `i32` offsets
   ```
   
   
   Now you need to click into `GenericByteArray` which will have examples of how to create a `StringArray` (after this PR) https://github.com/apache/arrow-rs/pull/4381/files#diff-6d2cfda31352113d2d964b2e4a3bfa99831739f316f15a92bd77657ca4496c9bR37
   
   Now I have to draw the connection that I can replace `&str` with `u8` and everything is fine. If I know / have experience with arrow this is fine. If I am a new user my initial experience has become  significantly more complicated (though perhaps I have learned about how arrow represents binary arrays and string arrays)



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1222994776


##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   Perhaps I could add a mix of StringArray and BinaryArray to the GenericBinaryArray examples would that aleviate your concern?
   
   The major thing I'm trying to avoid is people look at the docs for `StringArray`, which are always going to be because rust docs doesn't show the functions implemented for `GenericByteArray`, and then get stuck
   
   The fact this drastically cuts down on maintenance overheads is largely a happy accident



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1223068169


##########
arrow-array/src/array/byte_array.rs:
##########
@@ -37,8 +37,8 @@ use std::sync::Arc;
 /// # Example: From a Vec
 ///
 /// ```
-/// # use arrow_array::{Array, GenericByteArray, types::Utf8Type};
-/// let arr: GenericByteArray<Utf8Type> = vec!["hello", "world", ""].into();
+/// # use arrow_array::{Array, StringArray};

Review Comment:
   I think we can safely assume some modicum of Rust competency, but we can discuss synchronously. If the user can only copy and paste code snippets without even a surface level comprehension of them, they're going to very quickly hit a wall



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1222899561


##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   Rather than storing a potentially incomplete snapshot of the examples on the typedef, we instead link through to the underlying type which then contains documentation examples.
   
   This not only avoids a huge amount of duplication, but makes it clearer where people should look for the examples
   
   FYI @alamb as you added many of these in #4379



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


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4381: Documentation Improvements

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1223404073


##########
arrow-array/src/array/dictionary_array.rs:
##########
@@ -175,39 +191,54 @@ pub type UInt64DictionaryArray = DictionaryArray<UInt64Type>;
 ///              length = 6
 /// ```
 ///
-/// Example **with nullable** data:
+/// # Example: From Nullable Data
 ///
 /// ```
-/// use arrow_array::{DictionaryArray, Int8Array, types::Int8Type};
+/// # use arrow_array::{DictionaryArray, Int8Array, types::Int8Type};
 /// let test = vec!["a", "a", "b", "c"];
 /// let array : DictionaryArray<Int8Type> = test.iter().map(|&x| if x == "b" {None} else {Some(x)}).collect();
 /// assert_eq!(array.keys(), &Int8Array::from(vec![Some(0), Some(0), None, Some(1)]));
 /// ```
 ///
-/// Example **without nullable** data:
+/// # Example: From Non-Nullable Data
 ///
 /// ```
-/// use arrow_array::{DictionaryArray, Int8Array, types::Int8Type};
+/// # use arrow_array::{DictionaryArray, Int8Array, types::Int8Type};
 /// let test = vec!["a", "a", "b", "c"];
 /// let array : DictionaryArray<Int8Type> = test.into_iter().collect();
 /// assert_eq!(array.keys(), &Int8Array::from(vec![0, 0, 1, 2]));
 /// ```
 ///
-/// Example from existing arrays:
+/// # Example: From Existing Arrays
 ///
 /// ```
-/// use std::sync::Arc;
-/// use arrow_array::{DictionaryArray, Int8Array, StringArray, types::Int8Type};
+/// # use std::sync::Arc;
+/// # use arrow_array::{DictionaryArray, Int8Array, StringArray, types::Int8Type};
 /// // You can form your own DictionaryArray by providing the
 /// // values (dictionary) and keys (indexes into the dictionary):
 /// let values = StringArray::from_iter_values(["a", "b", "c"]);
 /// let keys = Int8Array::from_iter_values([0, 0, 1, 2]);
 /// let array = DictionaryArray::<Int8Type>::try_new(keys, Arc::new(values)).unwrap();
-/// let expected: DictionaryArray::<Int8Type> = vec!["a", "a", "b", "c"]
-///    .into_iter()
-///    .collect();
+/// let expected: DictionaryArray::<Int8Type> = vec!["a", "a", "b", "c"].into_iter().collect();
 /// assert_eq!(&array, &expected);
 /// ```
+///

Review Comment:
   👍 



##########
arrow-array/src/array/primitive_array.rs:
##########
@@ -384,23 +407,70 @@ pub type Decimal128Array = PrimitiveArray<Decimal128Type>;
 /// // Create iter/collect
 /// let arr: Decimal256Array = std::iter::repeat(i256::from(42)).take(10).collect();
 /// ```
+///
+/// See [`PrimitiveArray`] for more information and examples
 pub type Decimal256Array = PrimitiveArray<Decimal256Type>;
 
 pub use crate::types::ArrowPrimitiveType;
 
 /// An array of [primitive values](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout)
 ///
+/// # Example: From a Vec
+///
+/// ```
+/// # use arrow_array::{Array, PrimitiveArray, types::Int32Type};
+/// let arr: PrimitiveArray<Int32Type> = vec![1, 2, 3, 4].into();
+/// assert_eq!(4, arr.len());
+/// assert_eq!(0, arr.null_count());
+/// assert_eq!(arr.values(), &[1, 2, 3, 4])
+/// ```
+///
+/// # Example: From an optional Vec
+///
+/// ```
+/// # use arrow_array::{Array, PrimitiveArray, types::Int32Type};
+/// let arr: PrimitiveArray<Int32Type> = vec![Some(1), None, Some(3), None].into();
+/// assert_eq!(4, arr.len());
+/// assert_eq!(2, arr.null_count());
+/// assert_eq!(arr.values(), &[1, 0, 3, 0])

Review Comment:
   is it worth pointing out that the `0` (for the `null` locations) are arbitrary values?



##########
arrow-array/src/array/list_array.rs:
##########
@@ -471,58 +473,10 @@ impl<OffsetSize: OffsetSizeTrait> std::fmt::Debug for GenericListArray<OffsetSiz
     }
 }
 
-/// An array of variable size lists, storing offsets as `i32`.
-///
-/// # Example
-///
-/// ```
-/// # use arrow_array::{Array, ListArray, Int32Array, types::Int32Type};
-/// # use arrow_schema::DataType;
-/// let data = vec![
-///    Some(vec![]),
-///    None,
-///    Some(vec![Some(3), None, Some(5), Some(19)]),
-///    Some(vec![Some(6), Some(7)]),
-/// ];
-/// let list_array = ListArray::from_iter_primitive::<Int32Type, _, _>(data);
-///
-/// assert_eq!(false, list_array.is_valid(1));
-///
-/// let list0 = list_array.value(0);
-/// let list2 = list_array.value(2);
-/// let list3 = list_array.value(3);
-///
-/// assert_eq!(&[] as &[i32], list0.as_any().downcast_ref::<Int32Array>().unwrap().values());
-/// assert_eq!(false, list2.as_any().downcast_ref::<Int32Array>().unwrap().is_valid(1));
-/// assert_eq!(&[6, 7], list3.as_any().downcast_ref::<Int32Array>().unwrap().values());
-/// ```
+/// A [`GenericListArray`] of variable size lists, storing offsets as `i32`.

Review Comment:
   Can we also add a link here to the builder too?
   
   ```suggestion
   /// A [`GenericListArray`] of variable size lists, storing offsets as `i32`.
   ///
   /// See [`GenericListBuilder`](crate::builder::GenericListBuilder) for how to construct a [`GenericListArray`]
   ```
   



##########
arrow-array/src/array/list_array.rs:
##########
@@ -471,58 +473,10 @@ impl<OffsetSize: OffsetSizeTrait> std::fmt::Debug for GenericListArray<OffsetSiz
     }
 }
 
-/// An array of variable size lists, storing offsets as `i32`.
-///
-/// # Example
-///
-/// ```
-/// # use arrow_array::{Array, ListArray, Int32Array, types::Int32Type};
-/// # use arrow_schema::DataType;
-/// let data = vec![
-///    Some(vec![]),
-///    None,
-///    Some(vec![Some(3), None, Some(5), Some(19)]),
-///    Some(vec![Some(6), Some(7)]),
-/// ];
-/// let list_array = ListArray::from_iter_primitive::<Int32Type, _, _>(data);
-///
-/// assert_eq!(false, list_array.is_valid(1));
-///
-/// let list0 = list_array.value(0);
-/// let list2 = list_array.value(2);
-/// let list3 = list_array.value(3);
-///
-/// assert_eq!(&[] as &[i32], list0.as_any().downcast_ref::<Int32Array>().unwrap().values());
-/// assert_eq!(false, list2.as_any().downcast_ref::<Int32Array>().unwrap().is_valid(1));
-/// assert_eq!(&[6, 7], list3.as_any().downcast_ref::<Int32Array>().unwrap().values());
-/// ```
+/// A [`GenericListArray`] of variable size lists, storing offsets as `i32`.
 pub type ListArray = GenericListArray<i32>;
 
-/// An array of variable size lists, storing offsets as `i64`.
-///
-/// # Example
-///
-/// ```
-/// # use arrow_array::{Array, LargeListArray, Int32Array, types::Int32Type};
-/// # use arrow_schema::DataType;
-/// let data = vec![
-///    Some(vec![]),
-///    None,
-///    Some(vec![Some(3), None, Some(5), Some(19)]),
-///    Some(vec![Some(6), Some(7)]),
-/// ];
-/// let list_array = LargeListArray::from_iter_primitive::<Int32Type, _, _>(data);
-///
-/// assert_eq!(false, list_array.is_valid(1));
-///
-/// let list0 = list_array.value(0);
-/// let list2 = list_array.value(2);
-/// let list3 = list_array.value(3);
-///
-/// assert_eq!(&[] as &[i32], list0.as_any().downcast_ref::<Int32Array>().unwrap().values());
-/// assert_eq!(false, list2.as_any().downcast_ref::<Int32Array>().unwrap().is_valid(1));
-/// assert_eq!(&[6, 7], list3.as_any().downcast_ref::<Int32Array>().unwrap().values());
-/// ```
+/// A [`GenericListArray`] of variable size lists, storing offsets as `i64`.

Review Comment:
   ```suggestion
   /// A [`GenericListArray`] of variable size lists, storing offsets as `i64`.
   ///
   /// See [`GenericListBuilder`](crate::builder::GenericListBuilder) for how to construct a [`GenericListArray`]
   ```



##########
arrow-array/src/array/primitive_array.rs:
##########
@@ -384,23 +407,70 @@ pub type Decimal128Array = PrimitiveArray<Decimal128Type>;
 /// // Create iter/collect
 /// let arr: Decimal256Array = std::iter::repeat(i256::from(42)).take(10).collect();
 /// ```
+///
+/// See [`PrimitiveArray`] for more information and examples
 pub type Decimal256Array = PrimitiveArray<Decimal256Type>;
 
 pub use crate::types::ArrowPrimitiveType;
 
 /// An array of [primitive values](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout)
 ///
+/// # Example: From a Vec

Review Comment:
   I think these are nice examples 👍 



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1222903745


##########
arrow-array/src/array/list_array.rs:
##########
@@ -57,6 +57,8 @@ impl OffsetSizeTrait for i64 {
 /// An array of [variable length arrays](https://arrow.apache.org/docs/format/Columnar.html#variable-size-list-layout)
 ///
 /// See [`ListArray`] and [`LargeListArray`]`
+///
+/// See [`GenericListBuilder`](crate::builder::GenericListBuilder) for how to construct a [`GenericListArray`]

Review Comment:
   For the more complex types I opted to just point people to the builders, whilst the arrays can be constructed in other ways these necessarily require knowledge of how these types are structured, or are limited to a specific subset. I think it is better to just encourage people to use the builders as a first port of call



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1222994776


##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   Perhaps I could add a mix of StringArray and BinaryArray to the GenericBinaryArray examples would that aleviate your concern?
   
   The major thing I'm trying to avoid is people look at the docs for `StringArray`, which are incomplete and then get stuck because rust docs doesn't show all the member functions implemented on `GenericByteArray`



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1222994776


##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   Perhaps I could add a mix of StringArray and BinaryArray to the GenericBinaryArray examples would that aleviate your concern?
   
   The major thing I'm trying to avoid is people look at the docs for `StringArray`, which are always going to be incomplete because rust docs doesn't show the functions implemented for `GenericByteArray`, and then get stuck
   
   The fact this drastically cuts down on maintenance overheads is largely a happy accident



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1223296496


##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   I've restored the examples, and added a link to say where to find further information and examples



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


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4381: Documentation Improvements

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1223042952


##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   > The major thing I'm trying to avoid is people look at the docs for StringArray, which are always going to be incomplete because rust docs doesn't show the functions implemented for GenericByteArray, and then get stuck
   
   Maybe we can add a note at the top of each typedef saying something like
   
   "Note this is a typedef, see the [`GenericBinaryArray`] for available methods



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1223068169


##########
arrow-array/src/array/byte_array.rs:
##########
@@ -37,8 +37,8 @@ use std::sync::Arc;
 /// # Example: From a Vec
 ///
 /// ```
-/// # use arrow_array::{Array, GenericByteArray, types::Utf8Type};
-/// let arr: GenericByteArray<Utf8Type> = vec!["hello", "world", ""].into();
+/// # use arrow_array::{Array, StringArray};

Review Comment:
   I think we can safely assume some modicum of Rust competency, but we can discuss synchronously



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1222994776


##########
arrow-array/src/array/binary_array.rs:
##########
@@ -217,84 +217,10 @@ where
     }
 }
 
-/// An array of `[u8]` using `i32` offsets
-///
-/// The byte length of each element is represented by an i32.

Review Comment:
   Perhaps I could add a mix of StringArray and BinaryArray to the GenericBinaryArray examples would that aleviate your concern?
   
   The major thing I'm trying to avoid is people look at the docs for `StringArray`, which are incomplete and then get stuck because rust docs doesn't show all the member functions implemented on `GenericByteArray`.
   
   The fact this drastically cuts down on maintenance overheads is largely a happy accident



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


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4381: Documentation Improvements

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1223044257


##########
arrow-array/src/array/byte_array.rs:
##########
@@ -37,8 +37,8 @@ use std::sync::Arc;
 /// # Example: From a Vec
 ///
 /// ```
-/// # use arrow_array::{Array, GenericByteArray, types::Utf8Type};
-/// let arr: GenericByteArray<Utf8Type> = vec!["hello", "world", ""].into();
+/// # use arrow_array::{Array, StringArray};

Review Comment:
   I think this is now even more confusing at it doesn't show creating a `StringAray` from `Vec<Option<&str>>`



##########
arrow-array/src/array/primitive_array.rs:
##########
@@ -200,8 +200,8 @@ pub use crate::types::ArrowPrimitiveType;
 /// # Example: From a Vec
 ///
 /// ```
-/// # use arrow_array::{Array, PrimitiveArray, types::Int32Type};
-/// let arr: PrimitiveArray<Int32Type> = vec![1, 2, 3, 4].into();
+/// # use arrow_array::{Array, Int32Array};

Review Comment:
   similarly the use of different typedefs here I think will be very confusing



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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on code in PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381#discussion_r1223068169


##########
arrow-array/src/array/byte_array.rs:
##########
@@ -37,8 +37,8 @@ use std::sync::Arc;
 /// # Example: From a Vec
 ///
 /// ```
-/// # use arrow_array::{Array, GenericByteArray, types::Utf8Type};
-/// let arr: GenericByteArray<Utf8Type> = vec!["hello", "world", ""].into();
+/// # use arrow_array::{Array, StringArray};

Review Comment:
   I think we can safely assume some modicum of Rust competency, but we can discuss synchronously. If the user can only copy and paste code snippets, they're going to very quickly hit a wall



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


[GitHub] [arrow-rs] tustvold merged pull request #4381: Documentation Improvements

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold merged PR #4381:
URL: https://github.com/apache/arrow-rs/pull/4381


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