You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by tu...@apache.org on 2022/10/28 03:04:30 UTC

[arrow-rs] branch master updated: Add GenericByteArray (#2946) (#2947)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new b6f08a87e Add GenericByteArray (#2946) (#2947)
b6f08a87e is described below

commit b6f08a87e02144277bb0a7aa3708e42f6faf7a26
Author: Raphael Taylor-Davies <17...@users.noreply.github.com>
AuthorDate: Fri Oct 28 16:04:24 2022 +1300

    Add GenericByteArray (#2946) (#2947)
    
    * Add GenericByteArray (#2946)
    
    * Lint
    
    * Review feedback
    
    * Review feedback
---
 arrow-array/src/array/binary_array.rs | 191 +------------------------------
 arrow-array/src/array/byte_array.rs   | 208 ++++++++++++++++++++++++++++++++++
 arrow-array/src/array/mod.rs          |   3 +
 arrow-array/src/array/string_array.rs | 178 +----------------------------
 arrow-array/src/types.rs              |  90 ++++++++++++++-
 5 files changed, 309 insertions(+), 361 deletions(-)

diff --git a/arrow-array/src/array/binary_array.rs b/arrow-array/src/array/binary_array.rs
index c8407b252..259d949d4 100644
--- a/arrow-array/src/array/binary_array.rs
+++ b/arrow-array/src/array/binary_array.rs
@@ -15,118 +15,23 @@
 // specific language governing permissions and limitations
 // under the License.
 
-use crate::iterator::GenericBinaryIter;
-use crate::raw_pointer::RawPtrBox;
-use crate::{
-    empty_offsets, print_long_array, Array, ArrayAccessor, GenericListArray,
-    OffsetSizeTrait,
-};
+use crate::types::GenericBinaryType;
+use crate::{Array, GenericByteArray, GenericListArray, OffsetSizeTrait};
 use arrow_buffer::{bit_util, Buffer, MutableBuffer};
 use arrow_data::ArrayData;
 use arrow_schema::DataType;
-use std::any::Any;
 
 /// See [`BinaryArray`] and [`LargeBinaryArray`] for storing
 /// binary data.
-pub struct GenericBinaryArray<OffsetSize: OffsetSizeTrait> {
-    data: ArrayData,
-    value_offsets: RawPtrBox<OffsetSize>,
-    value_data: RawPtrBox<u8>,
-}
+pub type GenericBinaryArray<OffsetSize> = GenericByteArray<GenericBinaryType<OffsetSize>>;
 
 impl<OffsetSize: OffsetSizeTrait> GenericBinaryArray<OffsetSize> {
-    /// Data type of the array.
-    pub const DATA_TYPE: DataType = if OffsetSize::IS_LARGE {
-        DataType::LargeBinary
-    } else {
-        DataType::Binary
-    };
-
     /// Get the data type of the array.
     #[deprecated(note = "please use `Self::DATA_TYPE` instead")]
     pub const fn get_data_type() -> DataType {
         Self::DATA_TYPE
     }
 
-    /// Returns the length for value at index `i`.
-    #[inline]
-    pub fn value_length(&self, i: usize) -> OffsetSize {
-        let offsets = self.value_offsets();
-        offsets[i + 1] - offsets[i]
-    }
-
-    /// Returns a clone of the value data buffer
-    pub fn value_data(&self) -> Buffer {
-        self.data.buffers()[1].clone()
-    }
-
-    /// Returns the offset values in the offsets buffer
-    #[inline]
-    pub fn value_offsets(&self) -> &[OffsetSize] {
-        // Soundness
-        //     pointer alignment & location is ensured by RawPtrBox
-        //     buffer bounds/offset is ensured by the ArrayData instance.
-        unsafe {
-            std::slice::from_raw_parts(
-                self.value_offsets.as_ptr().add(self.data.offset()),
-                self.len() + 1,
-            )
-        }
-    }
-
-    /// Returns the element at index `i` as bytes slice
-    /// # Safety
-    /// Caller is responsible for ensuring that the index is within the bounds of the array
-    pub unsafe fn value_unchecked(&self, i: usize) -> &[u8] {
-        let end = *self.value_offsets().get_unchecked(i + 1);
-        let start = *self.value_offsets().get_unchecked(i);
-
-        // Soundness
-        // pointer alignment & location is ensured by RawPtrBox
-        // buffer bounds/offset is ensured by the value_offset invariants
-
-        // Safety of `to_isize().unwrap()`
-        // `start` and `end` are &OffsetSize, which is a generic type that implements the
-        // OffsetSizeTrait. Currently, only i32 and i64 implement OffsetSizeTrait,
-        // both of which should cleanly cast to isize on an architecture that supports
-        // 32/64-bit offsets
-        std::slice::from_raw_parts(
-            self.value_data.as_ptr().offset(start.to_isize().unwrap()),
-            (end - start).to_usize().unwrap(),
-        )
-    }
-
-    /// Returns the element at index `i` as bytes slice
-    /// # Panics
-    /// Panics if index `i` is out of bounds.
-    pub fn value(&self, i: usize) -> &[u8] {
-        assert!(
-            i < self.data.len(),
-            "Trying to access an element at index {} from a BinaryArray of length {}",
-            i,
-            self.len()
-        );
-        //Soundness: length checked above, offset buffer length is 1 larger than logical array length
-        let end = unsafe { self.value_offsets().get_unchecked(i + 1) };
-        let start = unsafe { self.value_offsets().get_unchecked(i) };
-
-        // Soundness
-        // pointer alignment & location is ensured by RawPtrBox
-        // buffer bounds/offset is ensured by the value_offset invariants
-
-        // Safety of `to_isize().unwrap()`
-        // `start` and `end` are &OffsetSize, which is a generic type that implements the
-        // OffsetSizeTrait. Currently, only i32 and i64 implement OffsetSizeTrait,
-        // both of which should cleanly cast to isize on an architecture that supports
-        // 32/64-bit offsets
-        unsafe {
-            std::slice::from_raw_parts(
-                self.value_data.as_ptr().offset(start.to_isize().unwrap()),
-                (*end - *start).to_usize().unwrap(),
-            )
-        }
-    }
-
     /// Creates a [GenericBinaryArray] from a vector of byte slices
     ///
     /// See also [`Self::from_iter_values`]
@@ -230,85 +135,6 @@ impl<OffsetSize: OffsetSizeTrait> GenericBinaryArray<OffsetSize> {
     ) -> impl Iterator<Item = Option<&[u8]>> + 'a {
         indexes.map(|opt_index| opt_index.map(|index| self.value_unchecked(index)))
     }
-
-    /// constructs a new iterator
-    pub fn iter(&self) -> GenericBinaryIter<'_, OffsetSize> {
-        GenericBinaryIter::<'_, OffsetSize>::new(self)
-    }
-}
-
-impl<OffsetSize: OffsetSizeTrait> std::fmt::Debug for GenericBinaryArray<OffsetSize> {
-    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
-        let prefix = OffsetSize::PREFIX;
-
-        write!(f, "{}BinaryArray\n[\n", prefix)?;
-        print_long_array(self, f, |array, index, f| {
-            std::fmt::Debug::fmt(&array.value(index), f)
-        })?;
-        write!(f, "]")
-    }
-}
-
-impl<OffsetSize: OffsetSizeTrait> Array for GenericBinaryArray<OffsetSize> {
-    fn as_any(&self) -> &dyn Any {
-        self
-    }
-
-    fn data(&self) -> &ArrayData {
-        &self.data
-    }
-
-    fn into_data(self) -> ArrayData {
-        self.into()
-    }
-}
-
-impl<'a, OffsetSize: OffsetSizeTrait> ArrayAccessor
-    for &'a GenericBinaryArray<OffsetSize>
-{
-    type Item = &'a [u8];
-
-    fn value(&self, index: usize) -> Self::Item {
-        GenericBinaryArray::value(self, index)
-    }
-
-    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
-        GenericBinaryArray::value_unchecked(self, index)
-    }
-}
-
-impl<OffsetSize: OffsetSizeTrait> From<ArrayData> for GenericBinaryArray<OffsetSize> {
-    fn from(data: ArrayData) -> Self {
-        assert_eq!(
-            data.data_type(),
-            &Self::DATA_TYPE,
-            "[Large]BinaryArray expects Datatype::[Large]Binary"
-        );
-        assert_eq!(
-            data.buffers().len(),
-            2,
-            "BinaryArray data should contain 2 buffers only (offsets and values)"
-        );
-        // Handle case of empty offsets
-        let offsets = match data.is_empty() && data.buffers()[0].is_empty() {
-            true => empty_offsets::<OffsetSize>().as_ptr() as *const _,
-            false => data.buffers()[0].as_ptr(),
-        };
-        let values = data.buffers()[1].as_ptr();
-        Self {
-            data,
-            // SAFETY:
-            // ArrayData must be valid, and validated data type above
-            value_offsets: unsafe { RawPtrBox::new(offsets) },
-            value_data: unsafe { RawPtrBox::new(values) },
-        }
-    }
-}
-
-impl<OffsetSize: OffsetSizeTrait> From<GenericBinaryArray<OffsetSize>> for ArrayData {
-    fn from(array: GenericBinaryArray<OffsetSize>) -> Self {
-        array.data
-    }
 }
 
 impl<OffsetSize: OffsetSizeTrait> From<Vec<Option<&[u8]>>>
@@ -374,15 +200,6 @@ where
     }
 }
 
-impl<'a, T: OffsetSizeTrait> IntoIterator for &'a GenericBinaryArray<T> {
-    type Item = Option<&'a [u8]>;
-    type IntoIter = GenericBinaryIter<'a, T>;
-
-    fn into_iter(self) -> Self::IntoIter {
-        GenericBinaryIter::<'a, T>::new(self)
-    }
-}
-
 /// An array where each element contains 0 or more bytes.
 /// The byte length of each element is represented by an i32.
 ///
@@ -836,7 +653,7 @@ mod tests {
     }
 
     #[test]
-    #[should_panic(expected = "[Large]BinaryArray expects Datatype::[Large]Binary")]
+    #[should_panic(expected = "LargeBinaryArray expects DataType::LargeBinary")]
     fn test_binary_array_validation() {
         let array = BinaryArray::from_iter_values(&[&[1, 2]]);
         let _ = LargeBinaryArray::from(array.into_data());
diff --git a/arrow-array/src/array/byte_array.rs b/arrow-array/src/array/byte_array.rs
new file mode 100644
index 000000000..8dd206bd2
--- /dev/null
+++ b/arrow-array/src/array/byte_array.rs
@@ -0,0 +1,208 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::array::{empty_offsets, print_long_array};
+use crate::iterator::ArrayIter;
+use crate::raw_pointer::RawPtrBox;
+use crate::types::bytes::ByteArrayNativeType;
+use crate::types::ByteArrayType;
+use crate::{Array, ArrayAccessor, OffsetSizeTrait};
+use arrow_buffer::{ArrowNativeType, Buffer};
+use arrow_data::ArrayData;
+use arrow_schema::DataType;
+use std::any::Any;
+
+/// Generic struct for variable-size byte arrays
+///
+/// See [`StringArray`] and [`LargeStringArray`] for storing utf8 encoded string data
+///
+/// See [`BinaryArray`] and [`LargeBinaryArray`] for storing arbitrary bytes
+///
+/// [`StringArray`]: crate::StringArray
+/// [`LargeStringArray`]: crate::LargeStringArray
+/// [`BinaryArray`]: crate::BinaryArray
+/// [`LargeBinaryArray`]: crate::LargeBinaryArray
+pub struct GenericByteArray<T: ByteArrayType> {
+    data: ArrayData,
+    value_offsets: RawPtrBox<T::Offset>,
+    value_data: RawPtrBox<u8>,
+}
+
+impl<T: ByteArrayType> GenericByteArray<T> {
+    /// Data type of the array.
+    pub const DATA_TYPE: DataType = T::DATA_TYPE;
+
+    /// Returns the length for value at index `i`.
+    /// # Panics
+    /// Panics if index `i` is out of bounds.
+    #[inline]
+    pub fn value_length(&self, i: usize) -> T::Offset {
+        let offsets = self.value_offsets();
+        offsets[i + 1] - offsets[i]
+    }
+
+    /// Returns a clone of the value data buffer
+    pub fn value_data(&self) -> Buffer {
+        self.data.buffers()[1].clone()
+    }
+
+    /// Returns the offset values in the offsets buffer
+    #[inline]
+    pub fn value_offsets(&self) -> &[T::Offset] {
+        // Soundness
+        //     pointer alignment & location is ensured by RawPtrBox
+        //     buffer bounds/offset is ensured by the ArrayData instance.
+        unsafe {
+            std::slice::from_raw_parts(
+                self.value_offsets.as_ptr().add(self.data.offset()),
+                self.len() + 1,
+            )
+        }
+    }
+
+    /// Returns the element at index `i`
+    /// # Safety
+    /// Caller is responsible for ensuring that the index is within the bounds of the array
+    pub unsafe fn value_unchecked(&self, i: usize) -> &T::Native {
+        let end = *self.value_offsets().get_unchecked(i + 1);
+        let start = *self.value_offsets().get_unchecked(i);
+
+        // Soundness
+        // pointer alignment & location is ensured by RawPtrBox
+        // buffer bounds/offset is ensured by the value_offset invariants
+
+        // Safety of `to_isize().unwrap()`
+        // `start` and `end` are &OffsetSize, which is a generic type that implements the
+        // OffsetSizeTrait. Currently, only i32 and i64 implement OffsetSizeTrait,
+        // both of which should cleanly cast to isize on an architecture that supports
+        // 32/64-bit offsets
+        let b = std::slice::from_raw_parts(
+            self.value_data.as_ptr().offset(start.to_isize().unwrap()),
+            (end - start).to_usize().unwrap(),
+        );
+
+        // SAFETY:
+        // ArrayData is valid
+        T::Native::from_bytes_unchecked(b)
+    }
+
+    /// Returns the element at index `i`
+    /// # Panics
+    /// Panics if index `i` is out of bounds.
+    pub fn value(&self, i: usize) -> &T::Native {
+        assert!(
+            i < self.data.len(),
+            "Trying to access an element at index {} from a {}{}Array of length {}",
+            i,
+            T::Offset::PREFIX,
+            T::PREFIX,
+            self.len()
+        );
+        // SAFETY:
+        // Verified length above
+        unsafe { self.value_unchecked(i) }
+    }
+
+    /// constructs a new iterator
+    pub fn iter(&self) -> ArrayIter<&Self> {
+        ArrayIter::new(self)
+    }
+}
+
+impl<T: ByteArrayType> std::fmt::Debug for GenericByteArray<T> {
+    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
+        write!(f, "{}{}Array\n[\n", T::Offset::PREFIX, T::PREFIX)?;
+        print_long_array(self, f, |array, index, f| {
+            std::fmt::Debug::fmt(&array.value(index), f)
+        })?;
+        write!(f, "]")
+    }
+}
+
+impl<T: ByteArrayType> Array for GenericByteArray<T> {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn data(&self) -> &ArrayData {
+        &self.data
+    }
+
+    fn into_data(self) -> ArrayData {
+        self.into()
+    }
+}
+
+impl<'a, T: ByteArrayType> ArrayAccessor for &'a GenericByteArray<T> {
+    type Item = &'a T::Native;
+
+    fn value(&self, index: usize) -> Self::Item {
+        GenericByteArray::value(self, index)
+    }
+
+    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
+        GenericByteArray::value_unchecked(self, index)
+    }
+}
+
+impl<T: ByteArrayType> From<ArrayData> for GenericByteArray<T> {
+    fn from(data: ArrayData) -> Self {
+        assert_eq!(
+            data.data_type(),
+            &Self::DATA_TYPE,
+            "{}{}Array expects DataType::{}",
+            T::Offset::PREFIX,
+            T::PREFIX,
+            Self::DATA_TYPE
+        );
+        assert_eq!(
+            data.buffers().len(),
+            2,
+            "{}{}Array data should contain 2 buffers only (offsets and values)",
+            T::Offset::PREFIX,
+            T::PREFIX,
+        );
+        // Handle case of empty offsets
+        let offsets = match data.is_empty() && data.buffers()[0].is_empty() {
+            true => empty_offsets::<T::Offset>().as_ptr() as *const _,
+            false => data.buffers()[0].as_ptr(),
+        };
+        let values = data.buffers()[1].as_ptr();
+        Self {
+            data,
+            // SAFETY:
+            // ArrayData must be valid, and validated data type above
+            value_offsets: unsafe { RawPtrBox::new(offsets) },
+            value_data: unsafe { RawPtrBox::new(values) },
+        }
+    }
+}
+
+impl<T: ByteArrayType> From<GenericByteArray<T>> for ArrayData {
+    fn from(array: GenericByteArray<T>) -> Self {
+        array.data
+    }
+}
+
+impl<'a, T: ByteArrayType> IntoIterator for &'a GenericByteArray<T> {
+    type Item = Option<&'a T::Native>;
+    type IntoIter = ArrayIter<Self>;
+
+    fn into_iter(self) -> Self::IntoIter {
+        ArrayIter::new(self)
+    }
+}
diff --git a/arrow-array/src/array/mod.rs b/arrow-array/src/array/mod.rs
index 1613e4a69..41aa438c9 100644
--- a/arrow-array/src/array/mod.rs
+++ b/arrow-array/src/array/mod.rs
@@ -31,6 +31,9 @@ pub use binary_array::*;
 mod boolean_array;
 pub use boolean_array::*;
 
+mod byte_array;
+pub use byte_array::*;
+
 mod dictionary_array;
 pub use dictionary_array::*;
 
diff --git a/arrow-array/src/array/string_array.rs b/arrow-array/src/array/string_array.rs
index 0cf45a448..94fcbae02 100644
--- a/arrow-array/src/array/string_array.rs
+++ b/arrow-array/src/array/string_array.rs
@@ -15,67 +15,27 @@
 // specific language governing permissions and limitations
 // under the License.
 
-use crate::iterator::GenericStringIter;
-use crate::raw_pointer::RawPtrBox;
+use crate::types::GenericStringType;
 use crate::{
-    empty_offsets, print_long_array, Array, ArrayAccessor, GenericBinaryArray,
-    GenericListArray, OffsetSizeTrait,
+    Array, GenericBinaryArray, GenericByteArray, GenericListArray, OffsetSizeTrait,
 };
-use arrow_buffer::{bit_util, Buffer, MutableBuffer};
+use arrow_buffer::{bit_util, MutableBuffer};
 use arrow_data::ArrayData;
 use arrow_schema::DataType;
-use std::any::Any;
 
 /// Generic struct for \[Large\]StringArray
 ///
 /// See [`StringArray`] and [`LargeStringArray`] for storing
 /// specific string data.
-pub struct GenericStringArray<OffsetSize: OffsetSizeTrait> {
-    data: ArrayData,
-    value_offsets: RawPtrBox<OffsetSize>,
-    value_data: RawPtrBox<u8>,
-}
+pub type GenericStringArray<OffsetSize> = GenericByteArray<GenericStringType<OffsetSize>>;
 
 impl<OffsetSize: OffsetSizeTrait> GenericStringArray<OffsetSize> {
-    /// Data type of the array.
-    pub const DATA_TYPE: DataType = if OffsetSize::IS_LARGE {
-        DataType::LargeUtf8
-    } else {
-        DataType::Utf8
-    };
-
     /// Get the data type of the array.
     #[deprecated(note = "please use `Self::DATA_TYPE` instead")]
     pub const fn get_data_type() -> DataType {
         Self::DATA_TYPE
     }
 
-    /// Returns the length for the element at index `i`.
-    #[inline]
-    pub fn value_length(&self, i: usize) -> OffsetSize {
-        let offsets = self.value_offsets();
-        offsets[i + 1] - offsets[i]
-    }
-
-    /// Returns the offset values in the offsets buffer
-    #[inline]
-    pub fn value_offsets(&self) -> &[OffsetSize] {
-        // Soundness
-        //     pointer alignment & location is ensured by RawPtrBox
-        //     buffer bounds/offset is ensured by the ArrayData instance.
-        unsafe {
-            std::slice::from_raw_parts(
-                self.value_offsets.as_ptr().add(self.data.offset()),
-                self.len() + 1,
-            )
-        }
-    }
-
-    /// Returns a clone of the value data buffer
-    pub fn value_data(&self) -> Buffer {
-        self.data.buffers()[1].clone()
-    }
-
     /// Returns the number of `Unicode Scalar Value` in the string at index `i`.
     /// # Performance
     /// This function has `O(n)` time complexity where `n` is the string length.
@@ -85,45 +45,6 @@ impl<OffsetSize: OffsetSizeTrait> GenericStringArray<OffsetSize> {
         self.value(i).chars().count()
     }
 
-    /// Returns the element at index
-    /// # Safety
-    /// caller is responsible for ensuring that index is within the array bounds
-    #[inline]
-    pub unsafe fn value_unchecked(&self, i: usize) -> &str {
-        let end = self.value_offsets().get_unchecked(i + 1).as_usize();
-        let start = self.value_offsets().get_unchecked(i).as_usize();
-
-        // Soundness
-        // pointer alignment & location is ensured by RawPtrBox
-        // buffer bounds/offset is ensured by the value_offset invariants
-        // ISSUE: utf-8 well formedness is not checked
-
-        // Safety of `to_isize().unwrap()`
-        // `start` and `end` are &OffsetSize, which is a generic type that implements the
-        // OffsetSizeTrait. Currently, only i32 and i64 implement OffsetSizeTrait,
-        // both of which should cleanly cast to isize on an architecture that supports
-        // 32/64-bit offsets
-        let slice =
-            std::slice::from_raw_parts(self.value_data.as_ptr().add(start), end - start);
-        std::str::from_utf8_unchecked(slice)
-    }
-
-    /// Returns the element at index `i` as &str
-    /// # Panics
-    /// Panics if index `i` is out of bounds.
-    #[inline]
-    pub fn value(&self, i: usize) -> &str {
-        assert!(
-            i < self.data.len(),
-            "Trying to access an element at index {} from a StringArray of length {}",
-            i,
-            self.len()
-        );
-        // Safety:
-        // `i < self.data.len()
-        unsafe { self.value_unchecked(i) }
-    }
-
     /// Convert a list array to a string array.
     ///
     /// Note: this performs potentially expensive UTF-8 validation, consider using
@@ -283,62 +204,6 @@ where
     }
 }
 
-impl<'a, T: OffsetSizeTrait> IntoIterator for &'a GenericStringArray<T> {
-    type Item = Option<&'a str>;
-    type IntoIter = GenericStringIter<'a, T>;
-
-    fn into_iter(self) -> Self::IntoIter {
-        GenericStringIter::<'a, T>::new(self)
-    }
-}
-
-impl<'a, T: OffsetSizeTrait> GenericStringArray<T> {
-    /// constructs a new iterator
-    pub fn iter(&'a self) -> GenericStringIter<'a, T> {
-        GenericStringIter::<'a, T>::new(self)
-    }
-}
-
-impl<OffsetSize: OffsetSizeTrait> std::fmt::Debug for GenericStringArray<OffsetSize> {
-    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
-        let prefix = OffsetSize::PREFIX;
-
-        write!(f, "{}StringArray\n[\n", prefix)?;
-        print_long_array(self, f, |array, index, f| {
-            std::fmt::Debug::fmt(&array.value(index), f)
-        })?;
-        write!(f, "]")
-    }
-}
-
-impl<OffsetSize: OffsetSizeTrait> Array for GenericStringArray<OffsetSize> {
-    fn as_any(&self) -> &dyn Any {
-        self
-    }
-
-    fn data(&self) -> &ArrayData {
-        &self.data
-    }
-
-    fn into_data(self) -> ArrayData {
-        self.into()
-    }
-}
-
-impl<'a, OffsetSize: OffsetSizeTrait> ArrayAccessor
-    for &'a GenericStringArray<OffsetSize>
-{
-    type Item = &'a str;
-
-    fn value(&self, index: usize) -> Self::Item {
-        GenericStringArray::value(self, index)
-    }
-
-    unsafe fn value_unchecked(&self, index: usize) -> Self::Item {
-        GenericStringArray::value_unchecked(self, index)
-    }
-}
-
 impl<OffsetSize: OffsetSizeTrait> From<GenericListArray<OffsetSize>>
     for GenericStringArray<OffsetSize>
 {
@@ -356,32 +221,6 @@ impl<OffsetSize: OffsetSizeTrait> From<GenericBinaryArray<OffsetSize>>
     }
 }
 
-impl<OffsetSize: OffsetSizeTrait> From<ArrayData> for GenericStringArray<OffsetSize> {
-    fn from(data: ArrayData) -> Self {
-        assert_eq!(
-            data.data_type(),
-            &Self::DATA_TYPE,
-            "[Large]StringArray expects Datatype::[Large]Utf8"
-        );
-        assert_eq!(
-            data.buffers().len(),
-            2,
-            "StringArray data should contain 2 buffers only (offsets and values)"
-        );
-        // Handle case of empty offsets
-        let offsets = match data.is_empty() && data.buffers()[0].is_empty() {
-            true => empty_offsets::<OffsetSize>().as_ptr() as *const _,
-            false => data.buffers()[0].as_ptr(),
-        };
-        let values = data.buffers()[1].as_ptr();
-        Self {
-            data,
-            value_offsets: unsafe { RawPtrBox::new(offsets) },
-            value_data: unsafe { RawPtrBox::new(values) },
-        }
-    }
-}
-
 impl<OffsetSize: OffsetSizeTrait> From<Vec<Option<&str>>>
     for GenericStringArray<OffsetSize>
 {
@@ -402,12 +241,6 @@ impl<OffsetSize: OffsetSizeTrait> From<Vec<String>> for GenericStringArray<Offse
     }
 }
 
-impl<OffsetSize: OffsetSizeTrait> From<GenericStringArray<OffsetSize>> for ArrayData {
-    fn from(array: GenericStringArray<OffsetSize>) -> Self {
-        array.data
-    }
-}
-
 /// An array where each element is a variable-sized sequence of bytes representing a string
 /// whose maximum length (in bytes) is represented by a i32.
 ///
@@ -436,6 +269,7 @@ pub type LargeStringArray = GenericStringArray<i64>;
 mod tests {
     use super::*;
     use crate::builder::{ListBuilder, StringBuilder};
+    use arrow_buffer::Buffer;
     use arrow_schema::Field;
 
     #[test]
@@ -464,7 +298,7 @@ mod tests {
     }
 
     #[test]
-    #[should_panic(expected = "[Large]StringArray expects Datatype::[Large]Utf8")]
+    #[should_panic(expected = "StringArray expects DataType::Utf8")]
     fn test_string_array_from_int() {
         let array = LargeStringArray::from(vec!["a", "b"]);
         drop(StringArray::from(array.into_data()));
diff --git a/arrow-array/src/types.rs b/arrow-array/src/types.rs
index edf6d40f3..e6197eed1 100644
--- a/arrow-array/src/types.rs
+++ b/arrow-array/src/types.rs
@@ -19,6 +19,7 @@
 
 use crate::array::ArrowPrimitiveType;
 use crate::delta::shift_months;
+use crate::OffsetSizeTrait;
 use arrow_buffer::i256;
 use arrow_data::decimal::{
     validate_decimal256_precision_with_lt_bytes, validate_decimal_precision,
@@ -28,6 +29,7 @@ use arrow_data::decimal::{
 use arrow_schema::{ArrowError, DataType, IntervalUnit, TimeUnit};
 use chrono::{Duration, NaiveDate};
 use half::f16;
+use std::marker::PhantomData;
 use std::ops::{Add, Sub};
 
 // BooleanType is special: its bit-width is not the size of the primitive type, and its `index`
@@ -464,7 +466,10 @@ impl Date64Type {
     }
 }
 
-mod private {
+/// Crate private types for Decimal Arrays
+///
+/// Not intended to be used outside this crate
+mod decimal {
     use super::*;
 
     pub trait DecimalTypeSealed {}
@@ -482,7 +487,7 @@ mod private {
 /// [`Decimal128Array`]: [crate::array::Decimal128Array]
 /// [`Decimal256Array`]: [crate::array::Decimal256Array]
 pub trait DecimalType:
-    'static + Send + Sync + ArrowPrimitiveType + private::DecimalTypeSealed
+    'static + Send + Sync + ArrowPrimitiveType + decimal::DecimalTypeSealed
 {
     const BYTE_LENGTH: usize;
     const MAX_PRECISION: u8;
@@ -574,6 +579,87 @@ fn format_decimal_str(value_str: &str, precision: usize, scale: usize) -> String
     }
 }
 
+/// Crate private types for Byte Arrays
+///
+/// Not intended to be used outside this crate
+pub(crate) mod bytes {
+    use super::*;
+
+    pub trait ByteArrayTypeSealed {}
+    impl<O: OffsetSizeTrait> ByteArrayTypeSealed for GenericStringType<O> {}
+    impl<O: OffsetSizeTrait> ByteArrayTypeSealed for GenericBinaryType<O> {}
+
+    pub trait ByteArrayNativeType: std::fmt::Debug + Send + Sync {
+        /// # Safety
+        ///
+        /// `b` must be a valid byte sequence for `Self`
+        unsafe fn from_bytes_unchecked(b: &[u8]) -> &Self;
+    }
+
+    impl ByteArrayNativeType for [u8] {
+        unsafe fn from_bytes_unchecked(b: &[u8]) -> &Self {
+            b
+        }
+    }
+
+    impl ByteArrayNativeType for str {
+        unsafe fn from_bytes_unchecked(b: &[u8]) -> &Self {
+            std::str::from_utf8_unchecked(b)
+        }
+    }
+}
+
+/// A trait over the variable-size byte array types
+///
+/// See [Variable Size Binary Layout](https://arrow.apache.org/docs/format/Columnar.html#variable-size-binary-layout)
+pub trait ByteArrayType: 'static + Send + Sync + bytes::ByteArrayTypeSealed {
+    type Offset: OffsetSizeTrait;
+    type Native: bytes::ByteArrayNativeType + AsRef<[u8]> + ?Sized;
+    /// "Binary" or "String", for use in error messages
+    const PREFIX: &'static str;
+    const DATA_TYPE: DataType;
+}
+
+/// [`ByteArrayType`] for string arrays
+pub struct GenericStringType<O: OffsetSizeTrait> {
+    phantom: PhantomData<O>,
+}
+
+impl<O: OffsetSizeTrait> ByteArrayType for GenericStringType<O> {
+    type Offset = O;
+    type Native = str;
+    const PREFIX: &'static str = "String";
+
+    const DATA_TYPE: DataType = if O::IS_LARGE {
+        DataType::LargeUtf8
+    } else {
+        DataType::Utf8
+    };
+}
+
+pub type Utf8Type = GenericStringType<i32>;
+pub type LargeUtf8Type = GenericStringType<i64>;
+
+/// [`ByteArrayType`] for binary arrays
+pub struct GenericBinaryType<O: OffsetSizeTrait> {
+    phantom: PhantomData<O>,
+}
+
+impl<O: OffsetSizeTrait> ByteArrayType for GenericBinaryType<O> {
+    type Offset = O;
+    type Native = [u8];
+    const PREFIX: &'static str = "Binary";
+
+    const DATA_TYPE: DataType = if O::IS_LARGE {
+        DataType::LargeBinary
+    } else {
+        DataType::Binary
+    };
+}
+
+pub type BinaryType = GenericBinaryType<i32>;
+pub type LargeBinaryType = GenericBinaryType<i64>;
+
 #[cfg(test)]
 mod tests {
     use super::*;