You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2021/12/04 11:54:30 UTC

[arrow-rs] 01/01: Add full data validation for ArrayData::try_new() (#921)

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

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

commit 2f43802621fcd0daeb6120d821bda22a7b5c12f2
Author: Andrew Lamb <an...@nerdnetworks.org>
AuthorDate: Sat Dec 4 06:43:45 2021 -0500

    Add full data validation for ArrayData::try_new() (#921)
    
    * Add full data validation for ArrayData::try_new()
    
    * Only look at offset+len indexes
    
    Co-authored-by: Jörn Horstmann <gi...@jhorstmann.net>
    
    * fix test
    
    * fmt
    
    * test for array indexes
    
    Co-authored-by: Jörn Horstmann <gi...@jhorstmann.net>
---
 arrow/src/array/array_binary.rs     |  30 ++
 arrow/src/array/array_dictionary.rs |  10 +
 arrow/src/array/array_string.rs     |  21 +-
 arrow/src/array/data.rs             | 644 +++++++++++++++++++++++++++++++++++-
 4 files changed, 687 insertions(+), 18 deletions(-)

diff --git a/arrow/src/array/array_binary.rs b/arrow/src/array/array_binary.rs
index ecf1850..f5b25ac 100644
--- a/arrow/src/array/array_binary.rs
+++ b/arrow/src/array/array_binary.rs
@@ -1375,4 +1375,34 @@ mod tests {
         assert_eq!(2, arr.value_length());
         assert_eq!(5, arr.len())
     }
+
+    #[test]
+    fn test_binary_array_all_null() {
+        let data = vec![None];
+        let array = BinaryArray::from(data);
+        array
+            .data()
+            .validate_full()
+            .expect("All null array has valid array data");
+    }
+
+    #[test]
+    fn test_large_binary_array_all_null() {
+        let data = vec![None];
+        let array = LargeBinaryArray::from(data);
+        array
+            .data()
+            .validate_full()
+            .expect("All null array has valid array data");
+    }
+
+    #[test]
+    fn fixed_size_binary_array_all_null() {
+        let data = vec![None] as Vec<Option<String>>;
+        let array = FixedSizeBinaryArray::try_from_sparse_iter(data.into_iter()).unwrap();
+        array
+            .data()
+            .validate_full()
+            .expect("All null array has valid array data");
+    }
 }
diff --git a/arrow/src/array/array_dictionary.rs b/arrow/src/array/array_dictionary.rs
index c684c25..75d4f3a 100644
--- a/arrow/src/array/array_dictionary.rs
+++ b/arrow/src/array/array_dictionary.rs
@@ -412,4 +412,14 @@ mod tests {
         assert_eq!(1, keys.value(2));
         assert_eq!(0, keys.value(5));
     }
+
+    #[test]
+    fn test_dictionary_all_nulls() {
+        let test = vec![None, None, None];
+        let array: DictionaryArray<Int32Type> = test.into_iter().collect();
+        array
+            .data()
+            .validate_full()
+            .expect("All null array has valid array data");
+    }
 }
diff --git a/arrow/src/array/array_string.rs b/arrow/src/array/array_string.rs
index 0d8dcf1..82c9f16 100644
--- a/arrow/src/array/array_string.rs
+++ b/arrow/src/array/array_string.rs
@@ -554,13 +554,22 @@ mod tests {
     }
 
     #[test]
-    fn test_string_array_from_string_vec() {
-        let data = vec!["Foo".to_owned(), "Bar".to_owned(), "Baz".to_owned()];
+    fn test_string_array_all_null() {
+        let data = vec![None];
         let array = StringArray::from(data);
+        array
+            .data()
+            .validate_full()
+            .expect("All null array has valid array data");
+    }
 
-        assert_eq!(array.len(), 3);
-        assert_eq!(array.value(0), "Foo");
-        assert_eq!(array.value(1), "Bar");
-        assert_eq!(array.value(2), "Baz");
+    #[test]
+    fn test_large_string_array_all_null() {
+        let data = vec![None];
+        let array = LargeStringArray::from(data);
+        array
+            .data()
+            .validate_full()
+            .expect("All null array has valid array data");
     }
 }
diff --git a/arrow/src/array/data.rs b/arrow/src/array/data.rs
index 40a8bee..7284837 100644
--- a/arrow/src/array/data.rs
+++ b/arrow/src/array/data.rs
@@ -20,6 +20,7 @@
 
 use std::convert::TryInto;
 use std::mem;
+use std::ops::Range;
 use std::sync::Arc;
 
 use crate::datatypes::{DataType, IntervalUnit};
@@ -319,7 +320,7 @@ impl ArrayData {
         buffers: Vec<Buffer>,
         child_data: Vec<ArrayData>,
     ) -> Result<Self> {
-        // Safetly justification: `validate` is (will be) called below
+        // Safety justification: `validate_full` is called below
         let new_self = unsafe {
             Self::new_unchecked(
                 data_type,
@@ -332,7 +333,8 @@ impl ArrayData {
             )
         };
 
-        new_self.validate()?;
+        // As the data is not trusted, do a full validation of its contents
+        new_self.validate_full()?;
         Ok(new_self)
     }
 
@@ -579,7 +581,8 @@ impl ArrayData {
     /// contents of the buffers (e.g. that all offsets for UTF8 arrays
     /// are within the bounds of the values buffer).
     ///
-    /// TODO: add a validate_full that validates the offsets and valid utf8 data
+    /// See [`validate_full`] to validate fully the offset content
+    /// and the validitiy of utf8 data
     pub fn validate(&self) -> Result<()> {
         // Need at least this mich space in each buffer
         let len_plus_offset = self.len + self.offset;
@@ -686,19 +689,20 @@ impl ArrayData {
         Ok(())
     }
 
-    /// Does a cheap sanity check that the `self.len` values in `buffer` are valid
-    /// offsets (of type T> into some other buffer of `values_length` bytes long
-    fn validate_offsets<T: ArrowNativeType + num::Num + std::fmt::Display>(
-        &self,
-        buffer: &Buffer,
-        values_length: usize,
-    ) -> Result<()> {
+    /// Returns a reference to the data in `buffer` as a typed slice
+    /// (typically `&[i32]` or `&[i64]`) after validating. The
+    /// returned slice is guaranteed to have at least `self.len + 1`
+    /// entries
+    fn typed_offsets<'a, T: ArrowNativeType + num::Num + std::fmt::Display>(
+        &'a self,
+        buffer: &'a Buffer,
+    ) -> Result<&'a [T]> {
         // Validate that there are the correct number of offsets for this array's length
         let required_offsets = self.len + self.offset + 1;
 
         // An empty list-like array can have 0 offsets
         if buffer.is_empty() {
-            return Ok(());
+            return Ok(&[]);
         }
 
         if (buffer.len() / std::mem::size_of::<T>()) < required_offsets {
@@ -709,7 +713,23 @@ impl ArrayData {
         }
 
         // Justification: buffer size was validated above
-        let offsets = unsafe { &(buffer.typed_data::<T>()[self.offset..]) };
+        Ok(unsafe {
+            &(buffer.typed_data::<T>()[self.offset..self.offset + self.len + 1])
+        })
+    }
+
+    /// Does a cheap sanity check that the `self.len` values in `buffer` are valid
+    /// offsets (of type T) into some other buffer of `values_length` bytes long
+    fn validate_offsets<T: ArrowNativeType + num::Num + std::fmt::Display>(
+        &self,
+        buffer: &Buffer,
+        values_length: usize,
+    ) -> Result<()> {
+        // Justification: buffer size was validated above
+        let offsets = self.typed_offsets::<T>(buffer)?;
+        if offsets.is_empty() {
+            return Ok(());
+        }
 
         let first_offset = offsets[0].to_usize().ok_or_else(|| {
             ArrowError::InvalidArgumentError(format!(
@@ -875,6 +895,251 @@ impl ArrayData {
         values_data.validate()?;
         Ok(values_data)
     }
+
+    /// "expensive" validation that ensures:
+    ///
+    /// 1. Null count is correct
+    /// 2. All offsets are valid
+    /// 3. All String data is  valid UTF-8
+    /// 3. All dictionary offsets are valid
+    ///
+    /// Does not (yet) check
+    /// 1. Union type_ids are valid (see https://github.com/apache/arrow-rs/issues/85)
+    /// Note calls `validate()` internally
+    pub fn validate_full(&self) -> Result<()> {
+        // Check all buffer sizes prior to looking at them more deeply in this function
+        self.validate()?;
+
+        let null_bitmap_buffer = self
+            .null_bitmap
+            .as_ref()
+            .map(|null_bitmap| null_bitmap.buffer_ref());
+
+        let actual_null_count = count_nulls(null_bitmap_buffer, self.offset, self.len);
+        if actual_null_count != self.null_count {
+            return Err(ArrowError::InvalidArgumentError(format!(
+                "null_count value ({}) doesn't match actual number of nulls in array ({})",
+                self.null_count, actual_null_count
+            )));
+        }
+
+        match &self.data_type {
+            DataType::Utf8 => {
+                self.validate_utf8::<i32>()?;
+            }
+            DataType::LargeUtf8 => {
+                self.validate_utf8::<i64>()?;
+            }
+            DataType::Binary => {
+                self.validate_offsets_full::<i32>(self.buffers[1].len())?;
+            }
+            DataType::LargeBinary => {
+                self.validate_offsets_full::<i64>(self.buffers[1].len())?;
+            }
+            DataType::List(_) | DataType::Map(_, _) => {
+                let child = &self.child_data[0];
+                self.validate_offsets_full::<i32>(child.len + child.offset)?;
+            }
+            DataType::LargeList(_) => {
+                let child = &self.child_data[0];
+                self.validate_offsets_full::<i64>(child.len + child.offset)?;
+            }
+            DataType::Union(_) => {
+                // Validate Union Array as part of implementing new Union semantics
+                // See comments in `ArrayData::validate()`
+                // https://github.com/apache/arrow-rs/issues/85
+            }
+            DataType::Dictionary(key_type, _value_type) => {
+                let dictionary_length: i64 = self.child_data[0].len.try_into().unwrap();
+                let max_value = dictionary_length - 1;
+                match key_type.as_ref() {
+                    DataType::UInt8 => self.check_bounds::<u8>(max_value)?,
+                    DataType::UInt16 => self.check_bounds::<u16>(max_value)?,
+                    DataType::UInt32 => self.check_bounds::<u32>(max_value)?,
+                    DataType::UInt64 => self.check_bounds::<u64>(max_value)?,
+                    DataType::Int8 => self.check_bounds::<i8>(max_value)?,
+                    DataType::Int16 => self.check_bounds::<i16>(max_value)?,
+                    DataType::Int32 => self.check_bounds::<i32>(max_value)?,
+                    DataType::Int64 => self.check_bounds::<i64>(max_value)?,
+                    _ => unreachable!(),
+                }
+            }
+            _ => {
+                // No extra validation check required for other types
+            }
+        };
+
+        // validate all children recursively
+        self.child_data
+            .iter()
+            .enumerate()
+            .try_for_each(|(i, child_data)| {
+                child_data.validate_full().map_err(|e| {
+                    ArrowError::InvalidArgumentError(format!(
+                        "{} child #{} invalid: {}",
+                        self.data_type, i, e
+                    ))
+                })
+            })?;
+
+        Ok(())
+    }
+
+    /// Calls the `validate(item_index, range)` function for each of
+    /// the ranges specified in the arrow offset buffer of type
+    /// `T`. Also validates that each offset is smaller than
+    /// `max_offset`
+    ///
+    /// For example, the offset buffer contained `[1, 2, 4]`, this
+    /// function would call `validate([1,2])`, and `validate([2,4])`
+    fn validate_each_offset<T, V>(
+        &self,
+        offset_buffer: &Buffer,
+        offset_limit: usize,
+        validate: V,
+    ) -> Result<()>
+    where
+        T: ArrowNativeType + std::convert::TryInto<usize> + num::Num + std::fmt::Display,
+        V: Fn(usize, Range<usize>) -> Result<()>,
+    {
+        // An empty binary-like array can have 0 offsets
+        if self.len == 0 && offset_buffer.is_empty() {
+            return Ok(());
+        }
+
+        let offsets = self.typed_offsets::<T>(offset_buffer)?;
+
+        offsets
+            .iter()
+            .zip(offsets.iter().skip(1))
+            .enumerate()
+            .map(|(i, (&start_offset, &end_offset))| {
+                let start_offset: usize = start_offset
+                    .try_into()
+                    .map_err(|_| {
+                        ArrowError::InvalidArgumentError(format!(
+                            "Offset invariant failure: could not convert start_offset {} to usize in slot {}",
+                            start_offset, i))
+                    })?;
+                let end_offset: usize = end_offset
+                    .try_into()
+                    .map_err(|_| {
+                        ArrowError::InvalidArgumentError(format!(
+                            "Offset invariant failure: Could not convert end_offset {} to usize in slot {}",
+                            end_offset, i+1))
+                    })?;
+
+                if start_offset > offset_limit {
+                    return Err(ArrowError::InvalidArgumentError(format!(
+                        "Offset invariant failure: offset for slot {} out of bounds: {} > {}",
+                        i, start_offset, offset_limit))
+                    );
+                }
+
+                if end_offset > offset_limit {
+                    return Err(ArrowError::InvalidArgumentError(format!(
+                        "Offset invariant failure: offset for slot {} out of bounds: {} > {}",
+                        i, end_offset, offset_limit))
+                    );
+                }
+
+                // check range actually is low -> high
+                if start_offset > end_offset {
+                    return Err(ArrowError::InvalidArgumentError(format!(
+                        "Offset invariant failure: non-monotonic offset at slot {}: {} > {}",
+                        i, start_offset, end_offset))
+                    );
+                }
+
+                Ok((i, start_offset..end_offset))
+            })
+            .try_for_each(|res: Result<(usize, Range<usize>)>| {
+                let (item_index, range) = res?;
+                validate(item_index, range)
+            })
+    }
+
+    /// Ensures that all strings formed by the offsets in buffers[0]
+    /// into buffers[1] are valid utf8 sequences
+    fn validate_utf8<T>(&self) -> Result<()>
+    where
+        T: ArrowNativeType + std::convert::TryInto<usize> + num::Num + std::fmt::Display,
+    {
+        let offset_buffer = &self.buffers[0];
+        let values_buffer = &self.buffers[1].as_slice();
+
+        self.validate_each_offset::<T, _>(
+            offset_buffer,
+            values_buffer.len(),
+            |string_index, range| {
+                std::str::from_utf8(&values_buffer[range.clone()]).map_err(|e| {
+                    ArrowError::InvalidArgumentError(format!(
+                        "Invalid UTF8 sequence at string index {} ({:?}): {}",
+                        string_index, range, e
+                    ))
+                })?;
+                Ok(())
+            },
+        )
+    }
+
+    /// Ensures that all offsets in buffers[0] into buffers[1] are
+    /// between `0` and `offset_limit`
+    fn validate_offsets_full<T>(&self, offset_limit: usize) -> Result<()>
+    where
+        T: ArrowNativeType + std::convert::TryInto<usize> + num::Num + std::fmt::Display,
+    {
+        let offset_buffer = &self.buffers[0];
+
+        self.validate_each_offset::<T, _>(
+            offset_buffer,
+            offset_limit,
+            |_string_index, _range| {
+                // No validation applied to each value, but the iteration
+                // itself applies bounds checking to each range
+                Ok(())
+            },
+        )
+    }
+
+    /// Validates that each value in self.buffers (typed as T)
+    /// is within the range [0, max_value], inclusive
+    fn check_bounds<T>(&self, max_value: i64) -> Result<()>
+    where
+        T: ArrowNativeType + std::convert::TryInto<i64> + num::Num + std::fmt::Display,
+    {
+        let required_len = self.len + self.offset;
+        let buffer = &self.buffers[0];
+
+        // This should have been checked as part of `validate()` prior
+        // to calling `validate_full()` but double check to be sure
+        assert!(buffer.len() / std::mem::size_of::<T>() >= required_len);
+
+        // Justification: buffer size was validated above
+        let indexes: &[T] =
+            unsafe { &(buffer.typed_data::<T>()[self.offset..self.offset + self.len]) };
+
+        indexes.iter().enumerate().try_for_each(|(i, &dict_index)| {
+            // Do not check the value is null (value can be arbitrary)
+            if self.is_null(i) {
+                return Ok(());
+            }
+            let dict_index: i64 = dict_index.try_into().map_err(|_| {
+                ArrowError::InvalidArgumentError(format!(
+                    "Value at position {} out of bounds: {} (can not convert to i64)",
+                    i, dict_index
+                ))
+            })?;
+
+            if dict_index < 0 || dict_index > max_value {
+                return Err(ArrowError::InvalidArgumentError(format!(
+                    "Value at position {} out of bounds: {} (should be in [0, {}])",
+                    i, dict_index, max_value
+                )));
+            }
+            Ok(())
+        })
+    }
 }
 
 /// Return the expected [`DataTypeLayout`] Arrays of this data
@@ -1115,6 +1380,7 @@ mod tests {
 
     use crate::array::{
         Array, BooleanBuilder, Int32Array, Int32Builder, StringArray, StructBuilder,
+        UInt64Array,
     };
     use crate::buffer::Buffer;
     use crate::datatypes::Field;
@@ -1632,6 +1898,360 @@ mod tests {
         .unwrap();
     }
 
+    /// Test that the array of type `data_type` that has invalid utf8 data errors
+    fn check_utf8_validation<T: ArrowNativeType>(data_type: DataType) {
+        // 0x80 is a utf8 continuation sequence and is not a valid utf8 sequence itself
+        let data_buffer = Buffer::from_slice_ref(&[b'a', b'a', 0x80, 0x00]);
+        let offsets: Vec<T> = [0, 2, 3]
+            .iter()
+            .map(|&v| T::from_usize(v).unwrap())
+            .collect();
+
+        let offsets_buffer = Buffer::from_slice_ref(&offsets);
+        ArrayData::try_new(
+            data_type,
+            2,
+            None,
+            None,
+            0,
+            vec![offsets_buffer, data_buffer],
+            vec![],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    #[should_panic(expected = "Invalid UTF8 sequence at string index 1 (2..3)")]
+    fn test_validate_utf8_content() {
+        check_utf8_validation::<i32>(DataType::Utf8);
+    }
+
+    #[test]
+    #[should_panic(expected = "Invalid UTF8 sequence at string index 1 (2..3)")]
+    fn test_validate_large_utf8_content() {
+        check_utf8_validation::<i64>(DataType::LargeUtf8);
+    }
+
+    /// Test that the array of type `data_type` that has invalid indexes (out of bounds)
+    fn check_index_out_of_bounds_validation<T: ArrowNativeType>(data_type: DataType) {
+        let data_buffer = Buffer::from_slice_ref(&[b'a', b'b', b'c', b'd']);
+        // First two offsets are fine, then 5 is out of bounds
+        let offsets: Vec<T> = [0, 1, 2, 5, 2]
+            .iter()
+            .map(|&v| T::from_usize(v).unwrap())
+            .collect();
+
+        let offsets_buffer = Buffer::from_slice_ref(&offsets);
+        ArrayData::try_new(
+            data_type,
+            4,
+            None,
+            None,
+            0,
+            vec![offsets_buffer, data_buffer],
+            vec![],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: offset for slot 2 out of bounds: 5 > 4"
+    )]
+    fn test_validate_utf8_out_of_bounds() {
+        check_index_out_of_bounds_validation::<i32>(DataType::Utf8);
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: offset for slot 2 out of bounds: 5 > 4"
+    )]
+    fn test_validate_large_utf8_out_of_bounds() {
+        check_index_out_of_bounds_validation::<i64>(DataType::LargeUtf8);
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: offset for slot 2 out of bounds: 5 > 4"
+    )]
+    fn test_validate_binary_out_of_bounds() {
+        check_index_out_of_bounds_validation::<i32>(DataType::Binary);
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: offset for slot 2 out of bounds: 5 > 4"
+    )]
+    fn test_validate_large_binary_out_of_bounds() {
+        check_index_out_of_bounds_validation::<i64>(DataType::LargeBinary);
+    }
+
+    // validate that indexes don't go bacwards check indexes that go backwards
+    fn check_index_backwards_validation<T: ArrowNativeType>(data_type: DataType) {
+        let data_buffer = Buffer::from_slice_ref(&[b'a', b'b', b'c', b'd']);
+        // First three offsets are fine, then 1 goes backwards
+        let offsets: Vec<T> = [0, 1, 2, 2, 1]
+            .iter()
+            .map(|&v| T::from_usize(v).unwrap())
+            .collect();
+
+        let offsets_buffer = Buffer::from_slice_ref(&offsets);
+        ArrayData::try_new(
+            data_type,
+            4,
+            None,
+            None,
+            0,
+            vec![offsets_buffer, data_buffer],
+            vec![],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: non-monotonic offset at slot 3: 2 > 1"
+    )]
+    fn test_validate_utf8_index_backwards() {
+        check_index_backwards_validation::<i32>(DataType::Utf8);
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: non-monotonic offset at slot 3: 2 > 1"
+    )]
+    fn test_validate_large_utf8_index_backwards() {
+        check_index_backwards_validation::<i64>(DataType::LargeUtf8);
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: non-monotonic offset at slot 3: 2 > 1"
+    )]
+    fn test_validate_binary_index_backwards() {
+        check_index_backwards_validation::<i32>(DataType::Binary);
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: non-monotonic offset at slot 3: 2 > 1"
+    )]
+    fn test_validate_large_binary_index_backwards() {
+        check_index_backwards_validation::<i64>(DataType::LargeBinary);
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Value at position 1 out of bounds: 3 (should be in [0, 1])"
+    )]
+    fn test_validate_dictionary_index_too_large() {
+        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
+
+        // 3 is not a valid index into the values (only 0 and 1)
+        let keys: Int32Array = [Some(1), Some(3)].into_iter().collect();
+
+        let data_type = DataType::Dictionary(
+            Box::new(keys.data_type().clone()),
+            Box::new(values.data_type().clone()),
+        );
+
+        ArrayData::try_new(
+            data_type,
+            2,
+            None,
+            None,
+            0,
+            vec![keys.data().buffers[0].clone()],
+            vec![values.data().clone()],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Value at position 1 out of bounds: -1 (should be in [0, 1]"
+    )]
+    fn test_validate_dictionary_index_negative() {
+        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
+
+        // -1 is not a valid index at all!
+        let keys: Int32Array = [Some(1), Some(-1)].into_iter().collect();
+
+        let data_type = DataType::Dictionary(
+            Box::new(keys.data_type().clone()),
+            Box::new(values.data_type().clone()),
+        );
+
+        ArrayData::try_new(
+            data_type,
+            2,
+            None,
+            None,
+            0,
+            vec![keys.data().buffers[0].clone()],
+            vec![values.data().clone()],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    fn test_validate_dictionary_index_negative_but_not_referenced() {
+        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
+
+        // -1 is not a valid index at all, but the array is length 1
+        // so the -1 should not be looked at
+        let keys: Int32Array = [Some(1), Some(-1)].into_iter().collect();
+
+        let data_type = DataType::Dictionary(
+            Box::new(keys.data_type().clone()),
+            Box::new(values.data_type().clone()),
+        );
+
+        // Expect this not to panic
+        ArrayData::try_new(
+            data_type,
+            1,
+            None,
+            None,
+            0,
+            vec![keys.data().buffers[0].clone()],
+            vec![values.data().clone()],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Value at position 0 out of bounds: 18446744073709551615 (can not convert to i64)"
+    )]
+    fn test_validate_dictionary_index_giant_negative() {
+        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
+
+        // -1 is not a valid index at all!
+        let keys: UInt64Array = [Some(u64::MAX), Some(1)].into_iter().collect();
+
+        let data_type = DataType::Dictionary(
+            Box::new(keys.data_type().clone()),
+            Box::new(values.data_type().clone()),
+        );
+
+        ArrayData::try_new(
+            data_type,
+            2,
+            None,
+            None,
+            0,
+            vec![keys.data().buffers[0].clone()],
+            vec![values.data().clone()],
+        )
+        .unwrap();
+    }
+
+    /// Test that the list of type `data_type` generates correct offset out of bounds errors
+    fn check_list_offsets<T: ArrowNativeType>(data_type: DataType) {
+        let values: Int32Array =
+            [Some(1), Some(2), Some(3), Some(4)].into_iter().collect();
+
+        // 5 is an invalid offset into a list of only three values
+        let offsets: Vec<T> = [0, 2, 5, 4]
+            .iter()
+            .map(|&v| T::from_usize(v).unwrap())
+            .collect();
+        let offsets_buffer = Buffer::from_slice_ref(&offsets);
+
+        ArrayData::try_new(
+            data_type,
+            3,
+            None,
+            None,
+            0,
+            vec![offsets_buffer],
+            vec![values.data().clone()],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: offset for slot 1 out of bounds: 5 > 4"
+    )]
+    fn test_validate_list_offsets() {
+        let field_type = Field::new("f", DataType::Int32, true);
+        check_list_offsets::<i32>(DataType::List(Box::new(field_type)));
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: offset for slot 1 out of bounds: 5 > 4"
+    )]
+    fn test_validate_large_list_offsets() {
+        let field_type = Field::new("f", DataType::Int32, true);
+        check_list_offsets::<i64>(DataType::LargeList(Box::new(field_type)));
+    }
+
+    /// Test that the list of type `data_type` generates correct errors for negative offsets
+    #[test]
+    #[should_panic(
+        expected = "Offset invariant failure: Could not convert end_offset -1 to usize in slot 2"
+    )]
+    fn test_validate_list_negative_offsets() {
+        let values: Int32Array =
+            [Some(1), Some(2), Some(3), Some(4)].into_iter().collect();
+        let field_type = Field::new("f", values.data_type().clone(), true);
+        let data_type = DataType::List(Box::new(field_type));
+
+        // -1 is an invalid offset any way you look at it
+        let offsets: Vec<i32> = vec![0, 2, -1, 4];
+        let offsets_buffer = Buffer::from_slice_ref(&offsets);
+
+        ArrayData::try_new(
+            data_type,
+            3,
+            None,
+            None,
+            0,
+            vec![offsets_buffer],
+            vec![values.data().clone()],
+        )
+        .unwrap();
+    }
+
+    #[test]
+    #[should_panic(
+        expected = "child #0 invalid: Invalid argument error: Value at position 1 out of bounds: -1 (should be in [0, 1])"
+    )]
+    /// test that children are validated recursively (aka bugs in child data of struct also are flagged)
+    fn test_validate_recursive() {
+        // Form invalid dictionary array
+        let values: StringArray = [Some("foo"), Some("bar")].into_iter().collect();
+        // -1 is not a valid index
+        let keys: Int32Array = [Some(1), Some(-1), Some(1)].into_iter().collect();
+
+        let dict_data_type = DataType::Dictionary(
+            Box::new(keys.data_type().clone()),
+            Box::new(values.data_type().clone()),
+        );
+
+        // purposely create an invalid child data
+        let dict_data = unsafe {
+            ArrayData::new_unchecked(
+                dict_data_type,
+                2,
+                None,
+                None,
+                0,
+                vec![keys.data().buffers[0].clone()],
+                vec![values.data().clone()],
+            )
+        };
+
+        // Now, try and create a struct with this invalid child data (and expect an error)
+        let data_type =
+            DataType::Struct(vec![Field::new("d", dict_data.data_type().clone(), true)]);
+
+        ArrayData::try_new(data_type, 1, None, None, 0, vec![], vec![dict_data]).unwrap();
+    }
+
     /// returns a buffer initialized with some constant value for tests
     fn make_i32_buffer(n: usize) -> Buffer {
         Buffer::from_slice_ref(&vec![42i32; n])