You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/12/01 22:00:31 UTC

[GitHub] [arrow-rs] alamb commented on a change in pull request #921: Add full data validation for ArrayData::try_new()

alamb commented on a change in pull request #921:
URL: https://github.com/apache/arrow-rs/pull/921#discussion_r760603402



##########
File path: arrow/src/array/data.rs
##########
@@ -878,6 +898,250 @@ 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..]) };

Review comment:
       Thank you @jhorstmann  -- I agree the indexes should be bounded to the length+offset rather than whatever is in the buffer. 
   
   I also agree that `buffer.typed_data` doesn't need to be marked as `unsafe` -- I'll file a separate ticket / PR for that change. 




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