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/21 16:14:18 UTC

[GitHub] [arrow-rs] tustvold opened a new pull request #1082: Optimized ByteArrayReader (#1040)

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


   **Extremely experimental**, builds on #1054 
   
   **Creating this PR now to provide context, this is not read for review**
   
   # Which issue does this PR close?
   
   Adds an optimized ByteArrayReader as part of proving out the generics added in #1041, and as a precursor to #171
   
   # Rationale for this change
    
   Depending on the benchmark, this can be anything from approximately the same to significantly (2x) faster than the ArrowArrayReader implementation added in #384. This is largely down to slightly more efficient null padding, and avoiding dynamic dispatch. The dominating factor in the benchmarks is the string value copy, which is makes me optimistic for the returns #171 wil yield.
   
   _I didn't benchmark the results for `DELTA_BYTE_ARRAY` encoding but the returns are likely to be even more significant, as the layout is more optimal for decode_
   
   The major benefit over the ArrowArrayReader implementation, aside from the speed bump, is the ability to share the existing ColumnReaderImpl and RecordReader logic, and the ability to work with all types of variable length strings and byte arrays. I also expect to be able to reuse some of the logic for #171 - in particular `OffsetBuffer`.
   
   # What changes are included in this PR?
   
   Adds a new `ByteArrayReader` that implements `ArrayReader` for variable length byte arrays
   
   # Are there any user-facing changes?
   
   No
   


-- 
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 pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999707518


   > would you be able to share some performance benchmark results?
   
   They're very preliminary at this stage, I'm not totally confident this code is correct nor have I spent any time trying to optimise it, but here you go. My primary focus has been proving out the interface from #1041, not polishing up the specific optimisations yet.
   
   "Old" is the new ByteArrayReader, "new" is the StringArrayReader
   
   ```
   arrow_array_reader/read StringArray, plain encoded, mandatory, no NULLs - old                                                                            
                           time:   [110.98 us 111.00 us 111.03 us]
   arrow_array_reader/read StringArray, plain encoded, mandatory, no NULLs - new                                                                            
                           time:   [124.77 us 124.99 us 125.32 us]
   arrow_array_reader/read StringArray, plain encoded, optional, no NULLs - old                                                                            
                           time:   [122.15 us 122.17 us 122.20 us]
   arrow_array_reader/read StringArray, plain encoded, optional, no NULLs - new                                                                            
                           time:   [136.72 us 136.76 us 136.81 us]
   arrow_array_reader/read StringArray, plain encoded, optional, half NULLs - old                                                                            
                           time:   [117.26 us 117.35 us 117.43 us]
   arrow_array_reader/read StringArray, plain encoded, optional, half NULLs - new                                                                            
                           time:   [258.05 us 258.17 us 258.30 us]
   arrow_array_reader/read StringArray, dictionary encoded, mandatory, no NULLs - old                                                                            
                           time:   [145.30 us 145.35 us 145.41 us]
   arrow_array_reader/read StringArray, dictionary encoded, mandatory, no NULLs - new                                                                            
                           time:   [117.14 us 117.18 us 117.22 us]
   arrow_array_reader/read StringArray, dictionary encoded, optional, no NULLs - old                                                                            
                           time:   [159.07 us 159.11 us 159.15 us]
   arrow_array_reader/read StringArray, dictionary encoded, optional, no NULLs - new                                                                            
                           time:   [134.39 us 134.41 us 134.43 us]
   arrow_array_reader/read StringArray, dictionary encoded, optional, half NULLs - old                                                                            
                           time:   [108.28 us 108.30 us 108.33 us]
   arrow_array_reader/read StringArray, dictionary encoded, optional, half NULLs - new                                                                            
                           time:   [230.15 us 230.23 us 230.32 us]
   ```
   
   Aside from dictionary encoded columns with no nulls, it performs better. This is probably just something suboptimal in the way I'm decoding RLE data, and should be rectifiable.


-- 
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] Dandandan commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785434772



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -192,7 +211,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         self.offsets.len() - 1
     }
 
-    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+    fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Err(e) = std::str::from_utf8(data) {

Review comment:
       I think we tried to do something similar with parquet2 but concluded that the individual strings should be checked instead. `simdutf8` is more impressive at checking non ASCII strings btw (e.g. try Chinese or emojis)
   Checking the code points at the offsets seems an interesting approach!
   Also FYI @jorgecarleitao




-- 
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 change in pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r786177173



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,669 @@
+// 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::arrow::array_reader::offset_buffer::OffsetBuffer;
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::ScalarValue;
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::{ConvertedType, Encoding};
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::ColumnValueDecoder;
+use crate::data_type::Int32Type;
+use crate::encodings::{
+    decoding::{Decoder, DeltaBitPackDecoder},
+    rle::RleDecoder,
+};
+use crate::errors::{ParquetError, Result};
+use crate::schema::types::ColumnDescPtr;
+use crate::util::memory::ByteBufferPtr;
+use arrow::array::{ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+
+/// Returns an [`ArrayReader`] that decodes the provided byte array column
+pub fn make_byte_array_reader(
+    pages: Box<dyn PageIterator>,
+    column_desc: ColumnDescPtr,
+    arrow_type: Option<ArrowType>,
+    null_mask_only: bool,
+) -> Result<Box<dyn ArrayReader>> {
+    // Check if Arrow type is specified, else create it from Parquet type
+    let data_type = match arrow_type {
+        Some(t) => t,
+        None => parquet_to_arrow_field(column_desc.as_ref())?
+            .data_type()
+            .clone(),
+    };
+
+    match data_type {
+        ArrowType::Binary | ArrowType::Utf8 => {
+            let reader =
+                GenericRecordReader::new_with_options(column_desc, null_mask_only);
+            Ok(Box::new(ByteArrayReader::<i32>::new(
+                pages, data_type, reader,
+            )))
+        }
+        ArrowType::LargeUtf8 | ArrowType::LargeBinary => {
+            let reader =
+                GenericRecordReader::new_with_options(column_desc, null_mask_only);
+            Ok(Box::new(ByteArrayReader::<i64>::new(
+                pages, data_type, reader,
+            )))
+        }
+        _ => Err(general_err!(
+            "invalid data type for byte array reader - {}",
+            data_type
+        )),
+    }
+}
+
+/// An [`ArrayReader`] for variable length byte arrays
+struct ByteArrayReader<I: ScalarValue> {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    record_reader: GenericRecordReader<OffsetBuffer<I>, ByteArrayColumnValueDecoder<I>>,
+}
+
+impl<I: ScalarValue> ByteArrayReader<I> {
+    fn new(
+        pages: Box<dyn PageIterator>,
+        data_type: ArrowType,
+        record_reader: GenericRecordReader<
+            OffsetBuffer<I>,
+            ByteArrayColumnValueDecoder<I>,
+        >,
+    ) -> Self {
+        Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            record_reader,
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ArrayReader for ByteArrayReader<I> {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> Result<ArrayRef> {
+        read_records(&mut self.record_reader, self.pages.as_mut(), batch_size)?;
+        let buffer = self.record_reader.consume_record_data()?;
+        let null_buffer = self.record_reader.consume_bitmap_buffer()?;
+        self.def_levels_buffer = self.record_reader.consume_def_levels()?;
+        self.rep_levels_buffer = self.record_reader.consume_rep_levels()?;
+        self.record_reader.reset();
+
+        Ok(buffer.into_array(null_buffer, self.data_type.clone()))
+    }
+
+    fn get_def_levels(&self) -> Option<&[i16]> {
+        self.def_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+
+    fn get_rep_levels(&self) -> Option<&[i16]> {
+        self.rep_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+}
+
+/// A [`ColumnValueDecoder`] for variable length byte arrays
+struct ByteArrayColumnValueDecoder<I: ScalarValue> {
+    dict: Option<OffsetBuffer<I>>,
+    decoder: Option<ByteArrayDecoder>,
+    validate_utf8: bool,
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ColumnValueDecoder
+    for ByteArrayColumnValueDecoder<I>
+{
+    type Slice = OffsetBuffer<I>;
+
+    fn new(desc: &ColumnDescPtr) -> Self {
+        let validate_utf8 = desc.converted_type() == ConvertedType::UTF8;
+        Self {
+            dict: None,
+            decoder: None,
+            validate_utf8,
+        }
+    }
+
+    fn set_dict(
+        &mut self,
+        buf: ByteBufferPtr,
+        num_values: u32,
+        encoding: Encoding,
+        _is_sorted: bool,
+    ) -> Result<()> {
+        if !matches!(
+            encoding,
+            Encoding::PLAIN | Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY
+        ) {
+            return Err(nyi_err!(
+                "Invalid/Unsupported encoding type for dictionary: {}",
+                encoding
+            ));
+        }
+
+        let mut buffer = OffsetBuffer::default();
+        let mut decoder = ByteArrayDecoderPlain::new(
+            buf,
+            num_values as usize,
+            Some(num_values as usize),
+            self.validate_utf8,
+        );
+        decoder.read(&mut buffer, usize::MAX)?;
+        self.dict = Some(buffer);
+        Ok(())
+    }
+
+    fn set_data(
+        &mut self,
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+    ) -> Result<()> {
+        self.decoder = Some(ByteArrayDecoder::new(
+            encoding,
+            data,
+            num_levels,
+            num_values,
+            self.validate_utf8,
+        )?);
+        Ok(())
+    }
+
+    fn read(&mut self, out: &mut Self::Slice, range: Range<usize>) -> Result<usize> {
+        self.decoder.as_mut().expect("decoder set").read(
+            out,
+            range.end - range.start,
+            self.dict.as_ref(),
+        )
+    }
+}
+
+/// A generic decoder from uncompressed parquet value data to [`OffsetBuffer`]
+pub enum ByteArrayDecoder {
+    Plain(ByteArrayDecoderPlain),
+    Dictionary(ByteArrayDecoderDictionary),
+    DeltaLength(ByteArrayDecoderDeltaLength),
+    DeltaByteArray(ByteArrayDecoderDelta),
+}
+
+impl ByteArrayDecoder {
+    pub fn new(
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+        validate_utf8: bool,
+    ) -> Result<Self> {
+        let decoder = match encoding {
+            Encoding::PLAIN => ByteArrayDecoder::Plain(ByteArrayDecoderPlain::new(
+                data,
+                num_levels,
+                num_values,
+                validate_utf8,
+            )),
+            Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY => {
+                ByteArrayDecoder::Dictionary(ByteArrayDecoderDictionary::new(
+                    data, num_levels, num_values,
+                ))
+            }
+            Encoding::DELTA_LENGTH_BYTE_ARRAY => ByteArrayDecoder::DeltaLength(
+                ByteArrayDecoderDeltaLength::new(data, validate_utf8)?,
+            ),
+            Encoding::DELTA_BYTE_ARRAY => ByteArrayDecoder::DeltaByteArray(
+                ByteArrayDecoderDelta::new(data, validate_utf8)?,
+            ),
+            _ => {
+                return Err(general_err!(
+                    "unsupported encoding for byte array: {}",
+                    encoding
+                ))
+            }
+        };
+
+        Ok(decoder)
+    }
+
+    /// Read up to `len` values to `out` with the optional dictionary
+    pub fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        out: &mut OffsetBuffer<I>,
+        len: usize,
+        dict: Option<&OffsetBuffer<I>>,
+    ) -> Result<usize> {
+        match self {
+            ByteArrayDecoder::Plain(d) => d.read(out, len),
+            ByteArrayDecoder::Dictionary(d) => {
+                let dict = dict.expect("dictionary set");
+                d.read(out, dict, len)
+            }
+            ByteArrayDecoder::DeltaLength(d) => d.read(out, len),
+            ByteArrayDecoder::DeltaByteArray(d) => d.read(out, len),
+        }
+    }
+}
+
+/// Decoder from [`Encoding::PLAIN`] data to [`OffsetBuffer`]
+pub struct ByteArrayDecoderPlain {
+    buf: ByteBufferPtr,
+    offset: usize,
+    validate_utf8: bool,
+
+    /// This is a maximum as the null count is not always known, e.g. value data from
+    /// a v1 data page
+    max_remaining_values: usize,
+}
+
+impl ByteArrayDecoderPlain {
+    pub fn new(
+        buf: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+        validate_utf8: bool,
+    ) -> Self {
+        Self {
+            buf,
+            validate_utf8,
+            offset: 0,
+            max_remaining_values: num_values.unwrap_or(num_levels),
+        }
+    }
+
+    pub fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+
+        let to_read = len.min(self.max_remaining_values);
+        output.offsets.reserve(to_read);
+
+        let remaining_bytes = self.buf.len() - self.offset;
+        if remaining_bytes == 0 {
+            return Ok(0);
+        }
+
+        let estimated_bytes = remaining_bytes
+            .checked_mul(to_read)
+            .map(|x| x / self.max_remaining_values)
+            .unwrap_or_default();
+
+        output.values.reserve(estimated_bytes);
+
+        let mut read = 0;
+
+        let buf = self.buf.as_ref();
+        while self.offset < self.buf.len() && read != to_read {
+            if self.offset + 4 > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+            let len_bytes: [u8; 4] =
+                buf[self.offset..self.offset + 4].try_into().unwrap();
+            let len = u32::from_le_bytes(len_bytes);
+
+            let start_offset = self.offset + 4;
+            let end_offset = start_offset + len as usize;
+            if end_offset > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            output.try_push(&buf[start_offset..end_offset], self.validate_utf8)?;
+
+            self.offset = end_offset;
+            read += 1;
+        }
+        self.max_remaining_values -= to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::DELTA_LENGTH_BYTE_ARRAY`] data to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDeltaLength {
+    lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    validate_utf8: bool,
+}
+
+impl ByteArrayDecoderDeltaLength {
+    fn new(data: ByteBufferPtr, validate_utf8: bool) -> Result<Self> {
+        let mut len_decoder = DeltaBitPackDecoder::<Int32Type>::new();
+        len_decoder.set_data(data.all(), 0)?;
+        let values = len_decoder.values_left();
+
+        let mut lengths = vec![0; values];
+        len_decoder.get(&mut lengths)?;
+
+        Ok(Self {
+            lengths,
+            data,
+            validate_utf8,
+            length_offset: 0,
+            data_offset: len_decoder.get_offset(),
+        })
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+
+        let to_read = len.min(self.lengths.len() - self.length_offset);
+        output.offsets.reserve(to_read);
+
+        let src_lengths = &self.lengths[self.length_offset..self.length_offset + to_read];
+
+        let total_bytes: usize = src_lengths.iter().map(|x| *x as usize).sum();
+        output.values.reserve(total_bytes);
+
+        if self.data_offset + total_bytes > self.data.len() {
+            return Err(ParquetError::EOF(
+                "Insufficient delta length byte array bytes".to_string(),
+            ));
+        }
+
+        let mut start_offset = self.data_offset;
+        for length in src_lengths {
+            let end_offset = start_offset + *length as usize;
+            output.try_push(
+                &self.data.as_ref()[start_offset..end_offset],
+                self.validate_utf8,
+            )?;
+            start_offset = end_offset;
+        }
+
+        self.data_offset = start_offset;
+        self.length_offset += to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::DELTA_BYTE_ARRAY`] to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDelta {
+    prefix_lengths: Vec<i32>,
+    suffix_lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    last_value: Vec<u8>,
+    validate_utf8: bool,
+}
+
+impl ByteArrayDecoderDelta {
+    fn new(data: ByteBufferPtr, validate_utf8: bool) -> Result<Self> {
+        let mut prefix = DeltaBitPackDecoder::<Int32Type>::new();
+        prefix.set_data(data.all(), 0)?;
+
+        let num_prefix = prefix.values_left();
+        let mut prefix_lengths = vec![0; num_prefix];
+        assert_eq!(prefix.get(&mut prefix_lengths)?, num_prefix);
+
+        let mut suffix = DeltaBitPackDecoder::<Int32Type>::new();
+        suffix.set_data(data.start_from(prefix.get_offset()), 0)?;
+
+        let num_suffix = suffix.values_left();
+        let mut suffix_lengths = vec![0; num_suffix];
+        assert_eq!(suffix.get(&mut suffix_lengths)?, num_suffix);
+
+        if num_prefix != num_suffix {
+            return Err(general_err!(format!(
+                "inconsistent DELTA_BYTE_ARRAY lengths, prefixes: {}, suffixes: {}",
+                num_prefix, num_suffix
+            )));
+        }
+
+        Ok(Self {
+            prefix_lengths,
+            suffix_lengths,
+            data,
+            length_offset: 0,
+            data_offset: prefix.get_offset() + suffix.get_offset(),
+            last_value: vec![],
+            validate_utf8,
+        })
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+        assert_eq!(self.prefix_lengths.len(), self.suffix_lengths.len());
+
+        let to_read = len.min(self.prefix_lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let length_range = self.length_offset..self.length_offset + to_read;
+        let iter = self.prefix_lengths[length_range.clone()]
+            .iter()
+            .zip(&self.suffix_lengths[length_range]);
+
+        let data = self.data.as_ref();
+
+        for (prefix_length, suffix_length) in iter {
+            let prefix_length = *prefix_length as usize;
+            let suffix_length = *suffix_length as usize;
+
+            if self.data_offset + suffix_length > self.data.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            self.last_value.truncate(prefix_length);
+            self.last_value.extend_from_slice(
+                &data[self.data_offset..self.data_offset + suffix_length],
+            );
+            output.try_push(&self.last_value, self.validate_utf8)?;
+
+            self.data_offset += suffix_length;
+        }
+
+        self.length_offset += to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::RLE_DICTIONARY`] to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDictionary {
+    decoder: RleDecoder,
+
+    index_buf: Box<[i32; 1024]>,
+    index_buf_len: usize,
+    index_offset: usize,
+
+    /// This is a maximum as the null count is not always known, e.g. value data from
+    /// a v1 data page
+    max_remaining_values: usize,
+}
+
+impl ByteArrayDecoderDictionary {
+    fn new(data: ByteBufferPtr, num_levels: usize, num_values: Option<usize>) -> Self {
+        let bit_width = data[0];
+        let mut decoder = RleDecoder::new(bit_width);
+        decoder.set_data(data.start_from(1));
+
+        Self {
+            decoder,
+            index_buf: Box::new([0; 1024]),
+            index_buf_len: 0,
+            index_offset: 0,
+            max_remaining_values: num_values.unwrap_or(num_levels),
+        }
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        dict: &OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let mut values_read = 0;
+
+        while values_read != len && self.max_remaining_values != 0 {
+            if self.index_offset == self.index_buf_len {
+                let read = self.decoder.get_batch(self.index_buf.as_mut())?;
+                if read == 0 {
+                    break;
+                }
+                self.index_buf_len = read;
+                self.index_offset = 0;
+            }
+
+            let to_read = (len - values_read)
+                .min(self.index_buf_len - self.index_offset)
+                .min(self.max_remaining_values);
+
+            output.extend_from_dictionary(
+                &self.index_buf[self.index_offset..self.index_offset + to_read],
+                dict.offsets.as_slice(),
+                dict.values.as_slice(),
+            )?;
+
+            self.index_offset += to_read;
+            values_read += to_read;
+            self.max_remaining_values -= to_read;
+        }
+        Ok(values_read)
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::basic::Type as PhysicalType;
+    use crate::data_type::{ByteArray, ByteArrayType};
+    use crate::encodings::encoding::{get_encoder, DictEncoder, Encoder};
+    use crate::schema::types::{ColumnDescriptor, ColumnPath, Type};
+    use crate::util::memory::MemTracker;
+    use std::sync::Arc;
+
+    fn column() -> ColumnDescPtr {
+        let t = Type::primitive_type_builder("col", PhysicalType::BYTE_ARRAY)
+            .with_converted_type(ConvertedType::UTF8)
+            .build()
+            .unwrap();
+
+        Arc::new(ColumnDescriptor::new(
+            Arc::new(t),
+            1,
+            0,
+            ColumnPath::new(vec![]),
+        ))
+    }
+
+    fn get_encoded(encoding: Encoding, data: &[ByteArray]) -> ByteBufferPtr {
+        let descriptor = column();
+        let mem_tracker = Arc::new(MemTracker::new());
+        let mut encoder =
+            get_encoder::<ByteArrayType>(descriptor, encoding, mem_tracker).unwrap();
+
+        encoder.put(data).unwrap();
+        encoder.flush_buffer().unwrap()
+    }
+
+    #[test]
+    fn test_byte_array_decoder() {

Review comment:
       The null padding is technically handled and tested as part of OffsetBuffer, but I'll add something here
   
   Edit: added

##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    /// Extends this buffer with a list of keys
+    ///
+    /// For each value `key` in `keys` this will insert
+    /// `&dict_values[dict_offsets[key]..dict_offsets[key+1]]`
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(
+        &mut self,
+        keys: &[K],
+        dict_offsets: &[V],
+        dict_values: &[u8],
+    ) -> Result<()> {
+        for key in keys {
+            let index = key.to_usize().unwrap();
+            if index + 1 >= dict_offsets.len() {
+                return Err(general_err!("invalid offset in byte array: {}", index));
+            }
+            let start_offset = dict_offsets[index].to_usize().unwrap();
+            let end_offset = dict_offsets[index + 1].to_usize().unwrap();
+
+            // Dictionary values are verified when decoding dictionary page
+            self.try_push(&dict_values[start_offset..end_offset], false)?;
+        }
+        Ok(())
+    }
+
+    /// Validates that `&self.values[start_offset..]` is a valid UTF-8 sequence
+    ///
+    /// This MUST be combined with validating that the offsets start on a character
+    /// boundary, otherwise it would be possible for the values array to be a valid UTF-8
+    /// sequence, but not the individual string slices it contains
+    ///
+    /// [`Self::try_push`] can perform this validation check on insertion
+    pub fn check_valid_utf8(&self, start_offset: usize) -> Result<()> {
+        match std::str::from_utf8(&self.values.as_slice()[start_offset..]) {
+            Ok(_) => Ok(()),
+            Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)),
+        }
+    }
+
+    /// Converts this into an [`ArrayRef`] with the provided `data_type` and `null_buffer`
+    pub fn into_array(
+        self,
+        null_buffer: Option<Buffer>,
+        data_type: ArrowType,
+    ) -> ArrayRef {
+        let mut array_data_builder = ArrayDataBuilder::new(data_type)
+            .len(self.len())
+            .add_buffer(self.offsets.into())
+            .add_buffer(self.values.into());
+
+        if let Some(buffer) = null_buffer {
+            array_data_builder = array_data_builder.null_bit_buffer(buffer);
+        }
+
+        let data = match cfg!(debug_assertions) {
+            true => array_data_builder.build().unwrap(),
+            false => unsafe { array_data_builder.build_unchecked() },
+        };
+
+        make_array(data)
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> BufferQueue for OffsetBuffer<I> {
+    type Output = Self;
+    type Slice = Self;
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;

Review comment:
       Added




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785499327



##########
File path: parquet/benches/arrow_array_reader.rs
##########
@@ -368,10 +366,10 @@ fn add_benches(c: &mut Criterion) {
                     mandatory_int32_column_desc.clone(),
                 );
                 count = bench_array_reader(array_reader);
-            })
+            });
+            assert_eq!(count, EXPECTED_VALUE_COUNT);

Review comment:
       This change allows for running a subset of the benchmarks, without this the assertion fails if the bench function is filtered out.
   
   For example, this would run just the string array benchmarks
   ```
   cargo criterion --bench arrow_array_reader --features test_common,experimental -- StringArray
   ```




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785498287



##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -68,6 +77,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         Ok(())
     }
 
+    /// Returns the values buffer as a string slice, returning an error
+    /// if it is invalid UTF-8
+    ///
+    /// `start_offset` is the offset in bytes from the start
+    pub fn values_as_str(&self, start_offset: usize) -> Result<&str> {
+        std::str::from_utf8(&self.values.as_slice()[start_offset..]).map_err(|e| {

Review comment:
       Done




-- 
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 change in pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r786194424



##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    /// Extends this buffer with a list of keys
+    ///
+    /// For each value `key` in `keys` this will insert
+    /// `&dict_values[dict_offsets[key]..dict_offsets[key+1]]`
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(
+        &mut self,
+        keys: &[K],
+        dict_offsets: &[V],
+        dict_values: &[u8],
+    ) -> Result<()> {
+        for key in keys {
+            let index = key.to_usize().unwrap();
+            if index + 1 >= dict_offsets.len() {
+                return Err(general_err!("invalid offset in byte array: {}", index));
+            }
+            let start_offset = dict_offsets[index].to_usize().unwrap();
+            let end_offset = dict_offsets[index + 1].to_usize().unwrap();
+
+            // Dictionary values are verified when decoding dictionary page
+            self.try_push(&dict_values[start_offset..end_offset], false)?;
+        }
+        Ok(())
+    }
+
+    /// Validates that `&self.values[start_offset..]` is a valid UTF-8 sequence
+    ///
+    /// This MUST be combined with validating that the offsets start on a character
+    /// boundary, otherwise it would be possible for the values array to be a valid UTF-8
+    /// sequence, but not the individual string slices it contains
+    ///
+    /// [`Self::try_push`] can perform this validation check on insertion
+    pub fn check_valid_utf8(&self, start_offset: usize) -> Result<()> {
+        match std::str::from_utf8(&self.values.as_slice()[start_offset..]) {
+            Ok(_) => Ok(()),
+            Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)),
+        }
+    }
+
+    /// Converts this into an [`ArrayRef`] with the provided `data_type` and `null_buffer`
+    pub fn into_array(
+        self,
+        null_buffer: Option<Buffer>,
+        data_type: ArrowType,
+    ) -> ArrayRef {
+        let mut array_data_builder = ArrayDataBuilder::new(data_type)
+            .len(self.len())
+            .add_buffer(self.offsets.into())
+            .add_buffer(self.values.into());
+
+        if let Some(buffer) = null_buffer {
+            array_data_builder = array_data_builder.null_bit_buffer(buffer);
+        }
+
+        let data = match cfg!(debug_assertions) {
+            true => array_data_builder.build().unwrap(),
+            false => unsafe { array_data_builder.build_unchecked() },
+        };
+
+        make_array(data)
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> BufferQueue for OffsetBuffer<I> {
+    type Output = Self;
+    type Slice = Self;
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = ScalarBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)
+        }
+
+        self.offsets.resize(len + 1);
+
+        Self {
+            offsets: std::mem::replace(&mut self.offsets, new_offsets),
+            values: self.values.take(end_offset.to_usize().unwrap()),

Review comment:
       This is a backwards compatibility thing, `ScalarBuffer::Output` must be Buffer to avoid changing the API of `ColumnReaderImpl`. Perhaps this could be included in a future breaking change cleanup PR :thinking: 




-- 
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 pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-1011005560


   I've added UTF-8 validation, including @jorgecarleitao 's very helpful test case, so this should fix that also :tada:


-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r783394641



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -192,7 +211,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         self.offsets.len() - 1
     }
 
-    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+    fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Err(e) = std::str::from_utf8(data) {

Review comment:
       I wonder if something like https://github.com/rusticstuff/simdutf8 could be used for faster UTF8 validation




-- 
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 pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

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


   https://github.com/apache/arrow-rs/issues/1197 tracks ArrowArrayReader removal


-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r776110779



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+            Reader::LargeBinary(r) | Reader::LargeUtf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+        };
+
+        Ok(match &self.record_reader {
+            Reader::Binary(_) => Arc::new(BinaryArray::from(data)),

Review comment:
       I wonder if the the type of array to create should be a generic parameter as well?




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r774005033



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {

Review comment:
       I wasn't sure how to achieve that without either leaking the index type into ByteArrayReader, or creating some sort of trait and using dynamic dispatch. An enumeration seemed like the simplest approach, and is likely to have negligible performance implications?




-- 
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] yordan-pavlov edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999877477


   @tustvold you are probably aware of this, but just to make sure it's not missed, when I run this branch with datafusion against a parquet file I get an error `Parquet argument error: Parquet error: unsupported encoding for byte array: PLAIN_DICTIONARY`
   
   Other than that, the performance benchmark results look impressive - I was able to run the benchmark and this branch is faster than the `ArrowArrayReader`, sometimes several times faster, in almost all cases (exceptions listed below). And the `ArrowArrayReader` was already several times faster in many cases than the old array reader implementation, making these performance results even more impressive.
   
   A major reason, why I only implemented `ArrowArrayReader` for string arrays is because I have been struggling to make it faster for dictionary-encoded primitive arrays, but it looks like this isn't going to be a problem with this new implementation.
   So if we can make it faster in all benchmarks, I am happy to abandon the `ArrowArrayReader` in favor of this new implementation.
   
   Where it is still a bit slower is in these two cases:
   
   read StringArray, plain encoded, mandatory, no NULLs - old: time:   [306.10 us 342.14 us 377.28 us]
   read StringArray, plain encoded, mandatory, no NULLs - new: time:   [310.84 us 337.49 us 368.74 us]
   
   read StringArray, dictionary encoded, mandatory, no NULLs - old: time:   [286.61 us 320.07 us 354.74 us]
   read StringArray, dictionary encoded, mandatory, no NULLs - new: time:   [222.87 us 240.56 us 260.93 us]
   
   The reason why `ArrowArrayReader` is fast in those cases, I suspect, is because when there are no nulls / def levels, the def level buffers are not read or processed at all, see here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L566 . This also means that the bit of code that produces the null bitmap also doesn't run, see here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L595 and the main path in the code is not concerned with null values at all, which is why it's so fast when there are no null / def levels, see here: https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L592 , see string converter here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L1164 .
   


-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785454161



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -192,7 +211,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         self.offsets.len() - 1
     }
 
-    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+    fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Err(e) = std::str::from_utf8(data) {

Review comment:
       I _think_ this should be sufficient, but I'm not an expert on UTF-8. My reasoning is that when you slice a `str` all it validates are that the start and end offsets pass `std::str::is_char_boundary` - [here](https://doc.rust-lang.org/std/primitive.str.html#panics-3). Taking that the standard library is correct, and the only invariant of `str` is that the bytes are UTF-8 as a whole, I think this is no different?




-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r773510590



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+            Reader::LargeBinary(r) | Reader::LargeUtf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+        };
+
+        Ok(match &self.record_reader {
+            Reader::Binary(_) => Arc::new(BinaryArray::from(data)),
+            Reader::LargeBinary(_) => Arc::new(LargeBinaryArray::from(data)),
+            Reader::Utf8(_) => Arc::new(StringArray::from(data)),
+            Reader::LargeUtf8(_) => Arc::new(LargeStringArray::from(data)),
+        })
+    }
+
+    fn get_def_levels(&self) -> Option<&[i16]> {
+        self.def_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+
+    fn get_rep_levels(&self) -> Option<&[i16]> {
+        self.rep_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+}
+
+struct OffsetBuffer<I> {
+    offsets: TypedBuffer<I>,
+    values: TypedBuffer<u8>,
+}
+
+impl<I> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = TypedBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: TypedBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait> OffsetBuffer<I> {
+    fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+}
+
+impl<I: OffsetSizeTrait> RecordBuffer for OffsetBuffer<I> {
+    type Output = Self;
+    type Writer = Self;
+
+    fn split(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = TypedBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)
+        }
+
+        self.offsets.resize(len + 1);
+
+        Self {
+            offsets: std::mem::replace(&mut self.offsets, new_offsets),
+            values: self.values.take(end_offset.to_usize().unwrap()),
+        }
+    }
+
+    fn writer(&mut self, _batch_size: usize) -> &mut Self::Writer {
+        self
+    }
+
+    fn commit(&mut self, len: usize) {
+        assert_eq!(self.offsets.len(), len + 1);
+    }
+}
+
+impl<I: OffsetSizeTrait> ValueBuffer for OffsetBuffer<I> {
+    fn pad_nulls(
+        &mut self,
+        values_range: Range<usize>,
+        levels_range: Range<usize>,
+        rev_position_iter: impl Iterator<Item = usize>,
+    ) {
+        assert_eq!(self.offsets.len(), values_range.end + 1);
+        self.offsets.resize(levels_range.end + 1);
+
+        let offsets = self.offsets.as_slice_mut();
+
+        let values_start = values_range.start;
+        let mut last_offset = levels_range.end + 1;
+
+        for (value_pos, level_pos) in values_range.rev().zip(rev_position_iter) {
+            assert!(level_pos >= value_pos);
+            assert!(level_pos < last_offset);
+
+            if level_pos == value_pos {
+                // Pad trailing nulls if necessary
+                if level_pos != last_offset && last_offset == levels_range.end + 1 {
+                    let value = offsets[value_pos];
+                    for x in &mut offsets[level_pos + 1..last_offset] {
+                        *x = value;
+                    }
+                }
+
+                // We are done
+                return;
+            }
+
+            // Fill in any nulls
+            let value_end = offsets[value_pos + 1];
+            let value_start = offsets[value_pos];
+
+            for x in &mut offsets[level_pos + 1..last_offset] {
+                *x = value_end;
+            }
+
+            offsets[level_pos] = value_start;
+            last_offset = level_pos;
+        }
+
+        // Pad leading nulls up to `last_offset`
+        let value = offsets[values_start];
+        for x in &mut offsets[values_start + 1..last_offset] {
+            *x = value
+        }
+    }
+}
+
+impl<I> ValuesWriter for OffsetBuffer<I> {
+    fn capacity(&self) -> usize {
+        usize::MAX
+    }
+}
+
+struct ByteArrayDecoder<I> {
+    dict: Option<OffsetBuffer<I>>,
+    decoder: Option<StringDecoder>,
+}
+
+impl<I: OffsetSizeTrait> ColumnValueDecoder for ByteArrayDecoder<I> {
+    type Writer = OffsetBuffer<I>;
+
+    fn create(_: &ColumnDescPtr) -> Self {
+        Self {
+            dict: None,
+            decoder: None,
+        }
+    }
+
+    fn set_dict(
+        &mut self,
+        buf: ByteBufferPtr,
+        num_values: u32,
+        encoding: Encoding,
+        _is_sorted: bool,
+    ) -> Result<()> {
+        if !matches!(
+            encoding,
+            Encoding::PLAIN | Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY
+        ) {
+            return Err(nyi_err!(
+                "Invalid/Unsupported encoding type for dictionary: {}",
+                encoding
+            ));
+        }
+
+        let mut buffer = OffsetBuffer::default();
+        let mut decoder = PlainDecoder::new(buf, num_values as usize);
+        decoder.read(&mut buffer, usize::MAX)?;
+        self.dict = Some(buffer);
+        Ok(())
+    }
+
+    fn set_data(
+        &mut self,
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_values: usize,
+    ) -> Result<()> {
+        let decoder = match encoding {
+            Encoding::PLAIN => StringDecoder::Plain(PlainDecoder::new(data, num_values)),
+            Encoding::RLE_DICTIONARY => {
+                StringDecoder::Dictionary(DictionaryDecoder::new(data))
+            }
+            Encoding::DELTA_LENGTH_BYTE_ARRAY => {
+                StringDecoder::DeltaLength(DeltaLengthDecoder::new(data, num_values)?)
+            }
+            Encoding::DELTA_BYTE_ARRAY => {
+                StringDecoder::DeltaStrings(DeltaStringsDecoder::new(data, num_values)?)
+            }
+            _ => {
+                return Err(general_err!(
+                    "unsupported encoding for byte array: {}",
+                    encoding
+                ))
+            }
+        };
+        self.decoder = Some(decoder);
+        Ok(())
+    }
+
+    fn read(&mut self, out: &mut Self::Writer, range: Range<usize>) -> Result<usize> {
+        let len = range.end - range.start;
+        match self.decoder.as_mut().expect("decoder set") {
+            StringDecoder::Plain(d) => d.read(out, len),
+            StringDecoder::Dictionary(d) => {
+                let dict = self.dict.as_ref().expect("dictionary set");
+                d.read(out, dict, len)
+            }
+            StringDecoder::DeltaLength(d) => d.read(out, len),
+            StringDecoder::DeltaStrings(d) => d.read(out, len),
+        }
+    }
+}
+
+enum StringDecoder {
+    Plain(PlainDecoder),
+    Dictionary(DictionaryDecoder),
+    DeltaLength(DeltaLengthDecoder),
+    DeltaStrings(DeltaStringsDecoder),
+}
+
+/// Decoder for [`Encoding::PLAIN`]
+struct PlainDecoder {
+    buf: ByteBufferPtr,
+    offset: usize,
+    remaining_values: usize,
+}
+
+impl PlainDecoder {
+    fn new(buf: ByteBufferPtr, values: usize) -> Self {
+        Self {
+            buf,
+            offset: 0,
+            remaining_values: values,
+        }
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let to_read = len.min(self.remaining_values);
+        output.offsets.reserve(to_read);
+
+        let remaining_bytes = self.buf.len() - self.offset;
+        if remaining_bytes == 0 {
+            return Ok(0);
+        }
+
+        let estimated_bytes = remaining_bytes
+            .checked_mul(to_read)
+            .map(|x| x / self.remaining_values)
+            .unwrap_or_default();
+
+        output.values.reserve(estimated_bytes);
+
+        let mut read = 0;
+
+        let buf = self.buf.as_ref();
+        while self.offset < self.buf.len() && read != to_read {
+            if self.offset + 4 > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+            let len_bytes: [u8; 4] =
+                buf[self.offset..self.offset + 4].try_into().unwrap();
+            let len = u32::from_le_bytes(len_bytes);
+
+            let start_offset = self.offset + 4;
+            let end_offset = start_offset + len as usize;
+            if end_offset > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            output.try_push(&buf[start_offset..end_offset])?;
+
+            self.offset = end_offset;
+            read += 1;
+        }
+        self.remaining_values -= to_read;
+        Ok(to_read)
+    }
+}
+
+/// Decoder for [`Encoding::DELTA_LENGTH_BYTE_ARRAY`]
+struct DeltaLengthDecoder {
+    lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+}
+
+impl DeltaLengthDecoder {
+    fn new(data: ByteBufferPtr, values: usize) -> Result<Self> {
+        let mut len_decoder = DeltaBitPackDecoder::<Int32Type>::new();
+        len_decoder.set_data(data.all(), values)?;
+        let mut lengths = vec![0; values];
+        len_decoder.get(&mut lengths)?;
+
+        Ok(Self {
+            lengths,
+            data,
+            length_offset: 0,
+            data_offset: len_decoder.get_offset(),
+        })
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let to_read = len.min(self.lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let mut to_read_bytes: usize = 0;
+        let mut offset = output.values.len();
+
+        for length in &self.lengths[self.length_offset..self.length_offset + to_read] {
+            offset = offset.saturating_add(*length as usize);
+            to_read_bytes += *length as usize;
+
+            let offset_i = I::from_usize(offset)
+                .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+            output.offsets.push(offset_i)
+        }
+
+        output.values.extend_from_slice(
+            &self.data.as_ref()[self.data_offset..self.data_offset + to_read_bytes],
+        );
+
+        self.data_offset += to_read_bytes;
+        self.length_offset += to_read;
+        Ok(to_read)
+    }
+}
+
+/// Decoder for [`Encoding::DELTA_BYTE_ARRAY`]
+struct DeltaStringsDecoder {
+    prefix_lengths: Vec<i32>,
+    suffix_lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    last_value: Vec<u8>,
+}
+
+impl DeltaStringsDecoder {
+    fn new(data: ByteBufferPtr, values: usize) -> Result<Self> {
+        let mut prefix = DeltaBitPackDecoder::<Int32Type>::new();
+        prefix.set_data(data.all(), values)?;
+        let mut prefix_lengths = vec![0; values];
+        prefix.get(&mut prefix_lengths)?;
+
+        let mut suffix = DeltaBitPackDecoder::<Int32Type>::new();
+        suffix.set_data(data.start_from(prefix.get_offset()), values)?;
+        let mut suffix_lengths = vec![0; values];
+        suffix.get(&mut suffix_lengths)?;
+
+        Ok(Self {
+            prefix_lengths,
+            suffix_lengths,
+            data,
+            length_offset: 0,
+            data_offset: prefix.get_offset() + suffix.get_offset(),
+            last_value: vec![],
+        })
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        assert_eq!(self.prefix_lengths.len(), self.suffix_lengths.len());
+
+        let to_read = len.min(self.prefix_lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let length_range = self.length_offset..self.length_offset + to_read;
+        let iter = self.prefix_lengths[length_range.clone()]
+            .iter()
+            .zip(&self.suffix_lengths[length_range]);
+
+        let mut offset = output.values.len();
+        let data = self.data.as_ref();
+
+        for (prefix_length, suffix_length) in iter {
+            let total_length = *prefix_length as usize + *suffix_length as usize;
+
+            if self.data_offset + total_length > self.data.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            offset = offset.saturating_add(total_length);
+
+            let offset_i = I::from_usize(offset)
+                .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+            output.offsets.push(offset_i);
+
+            self.last_value.truncate(*prefix_length as usize);
+            self.last_value.extend_from_slice(
+                &data[self.data_offset..self.data_offset + total_length],
+            );
+
+            output.values.reserve(total_length);
+            output.values.extend_from_slice(&self.last_value);
+
+            self.data_offset += total_length;
+        }
+
+        self.length_offset += to_read;
+        Ok(to_read)
+    }
+}
+
+struct DictionaryDecoder {
+    decoder: RleDecoder,
+    index_buf: Box<[i32; 1024]>,
+    index_offset: usize,
+}
+
+impl DictionaryDecoder {
+    fn new(data: ByteBufferPtr) -> Self {
+        let bit_width = data[0];
+        let mut decoder = RleDecoder::new(bit_width);
+        decoder.set_data(data.start_from(1));
+
+        Self {
+            decoder,
+            index_buf: Box::new([0; 1024]),
+            index_offset: 1024,
+        }
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        dict: &OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let mut values_read = 0;
+
+        while values_read != len {
+            if self.index_offset == self.index_buf.len() {
+                let decoded = self.decoder.get_batch(self.index_buf.as_mut())?;
+                self.index_offset = 0;
+                if decoded != self.index_buf.len() && decoded < len - values_read {
+                    return Err(ParquetError::EOF(

Review comment:
       isn't it a valid use case that less values can be returned than requested / `len`? (e.g. at end of file)




-- 
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] yordan-pavlov commented on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999877477


   @tustvold you are probably aware about this, but just to make sure it's not missed, when I run this branch with datafusion against a parquet file I get an error `Parquet argument error: Parquet error: unsupported encoding for byte array: PLAIN_DICTIONARY`


-- 
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] codecov-commenter commented on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-998923929


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1082](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a0d745f) into [master](https://codecov.io/gh/apache/arrow-rs/commit/99b7d01103495607932343146c973b6fba0eb8d5?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (99b7d01) will **decrease** coverage by `0.33%`.
   > The diff coverage is `79.17%`.
   
   > :exclamation: Current head a0d745f differs from pull request most recent head 56c3bdc. Consider uploading reports for the commit 56c3bdc to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1082/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1082      +/-   ##
   ==========================================
   - Coverage   82.31%   81.97%   -0.34%     
   ==========================================
     Files         168      172       +4     
     Lines       49056    49851     +795     
   ==========================================
   + Hits        40379    40865     +486     
   - Misses       8677     8986     +309     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [arrow/src/array/data.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2RhdGEucnM=) | `80.85% <0.00%> (-0.32%)` | :arrow_down: |
   | [arrow/src/array/equal/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2VxdWFsL21vZC5ycw==) | `93.13% <0.00%> (-0.17%)` | :arrow_down: |
   | [arrow/src/array/ord.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L29yZC5ycw==) | `67.15% <0.00%> (-0.50%)` | :arrow_down: |
   | [arrow/src/array/transform/mod.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L3RyYW5zZm9ybS9tb2QucnM=) | `84.73% <0.00%> (-0.24%)` | :arrow_down: |
   | [arrow/src/compute/kernels/sort.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2NvbXB1dGUva2VybmVscy9zb3J0LnJz) | `94.81% <0.00%> (-0.14%)` | :arrow_down: |
   | [arrow/src/datatypes/types.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2RhdGF0eXBlcy90eXBlcy5ycw==) | `88.88% <ø> (ø)` | |
   | [arrow/src/ipc/gen/Schema.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2lwYy9nZW4vU2NoZW1hLnJz) | `41.60% <ø> (ø)` | |
   | [arrow/src/ipc/reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2lwYy9yZWFkZXIucnM=) | `85.74% <ø> (ø)` | |
   | [arrow/src/util/display.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL3V0aWwvZGlzcGxheS5ycw==) | `19.81% <0.00%> (-0.19%)` | :arrow_down: |
   | [arrow/src/util/integration\_util.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL3V0aWwvaW50ZWdyYXRpb25fdXRpbC5ycw==) | `68.66% <0.00%> (-0.42%)` | :arrow_down: |
   | ... and [31 more](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [99b7d01...56c3bdc](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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 pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999804838


   > what about performance with primitive types (e.g. int32)
   
   This PR builds on #1054 which yields a 2-6x speed up when using PrimitiveArrayReader on non-nested columns compared to current master. This is purely through better null handling, which this PR also benefits from.
   
   I do have some reservations about drawing too much from these benchmarks, I have found them to have strange interactions with my system's memory allocator, but its certainly not slower and is likely significantly faster.
   
   > compared to old array reader implementation
   
   That's the key thing about #1041 it doesn't replace this array reader implementation, it just adds the ability to extend it. For primitive types the performance of #1041 is therefore unchanged, it just gives the ability to add optimisations such as #1054


-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r776516142



##########
File path: parquet/src/arrow/array_reader.rs
##########
@@ -1663,69 +1699,58 @@ impl<'a> ArrayReaderBuilder {
                     arrow_type,
                 )?))
             }
-            PhysicalType::FLOAT => Ok(Box::new(PrimitiveArrayReader::<FloatType>::new(
-                page_iterator,
-                column_desc,
-                arrow_type,
-            )?)),
-            PhysicalType::DOUBLE => {
-                Ok(Box::new(PrimitiveArrayReader::<DoubleType>::new(
+            PhysicalType::FLOAT => Ok(Box::new(
+                PrimitiveArrayReader::<FloatType>::new_with_options(
                     page_iterator,
                     column_desc,
                     arrow_type,
-                )?))
-            }
-            PhysicalType::BYTE_ARRAY => {
-                if cur_type.get_basic_info().converted_type() == ConvertedType::UTF8 {
-                    if let Some(ArrowType::LargeUtf8) = arrow_type {
-                        let converter =
-                            LargeUtf8Converter::new(LargeUtf8ArrayConverter {});
-                        Ok(Box::new(ComplexObjectArrayReader::<
-                            ByteArrayType,
-                            LargeUtf8Converter,
-                        >::new(
-                            page_iterator,
-                            column_desc,
-                            converter,
-                            arrow_type,
-                        )?))
-                    } else {
-                        use crate::arrow::arrow_array_reader::{
-                            ArrowArrayReader, StringArrayConverter,
-                        };
-                        let converter = StringArrayConverter::new();
-                        Ok(Box::new(ArrowArrayReader::try_new(
-                            *page_iterator,
-                            column_desc,
-                            converter,
-                            arrow_type,
-                        )?))
+                    null_mask_only,
+                )?,
+            )),
+            PhysicalType::DOUBLE => Ok(Box::new(
+                PrimitiveArrayReader::<DoubleType>::new_with_options(
+                    page_iterator,
+                    column_desc,
+                    arrow_type,
+                    null_mask_only,
+                )?,
+            )),
+            PhysicalType::BYTE_ARRAY => match arrow_type {
+                // TODO: Replace with optimised dictionary reader (#171)
+                Some(ArrowType::Dictionary(_, _)) => {
+                    match cur_type.get_basic_info().converted_type() {
+                        ConvertedType::UTF8 => {
+                            let converter = Utf8Converter::new(Utf8ArrayConverter {});
+                            Ok(Box::new(ComplexObjectArrayReader::<
+                                ByteArrayType,
+                                Utf8Converter,
+                            >::new(
+                                page_iterator,
+                                column_desc,
+                                converter,
+                                arrow_type,
+                            )?))
+                        }
+                        _ => {
+                            let converter = BinaryConverter::new(BinaryArrayConverter {});
+                            Ok(Box::new(ComplexObjectArrayReader::<
+                                ByteArrayType,
+                                BinaryConverter,
+                            >::new(
+                                page_iterator,
+                                column_desc,
+                                converter,
+                                arrow_type,
+                            )?))
+                        }
                     }
-                } else if let Some(ArrowType::LargeBinary) = arrow_type {
-                    let converter =
-                        LargeBinaryConverter::new(LargeBinaryArrayConverter {});
-                    Ok(Box::new(ComplexObjectArrayReader::<
-                        ByteArrayType,
-                        LargeBinaryConverter,
-                    >::new(
-                        page_iterator,
-                        column_desc,
-                        converter,
-                        arrow_type,
-                    )?))
-                } else {
-                    let converter = BinaryConverter::new(BinaryArrayConverter {});
-                    Ok(Box::new(ComplexObjectArrayReader::<
-                        ByteArrayType,
-                        BinaryConverter,
-                    >::new(
-                        page_iterator,
-                        column_desc,
-                        converter,
-                        arrow_type,
-                    )?))
                 }
-            }
+                _ => Ok(Box::new(ByteArrayReader::new(
+                    page_iterator,
+                    column_desc,
+                    arrow_type,

Review comment:
       Pure oversight - good spot :+1: 




-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r776110379



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {

Review comment:
       I think it would be natural to make `ByteArrayReader` generic on `I: OffsetSizeTrait` similar to `consume_array_data` - then creating it wouldn't be more complex than it already is (for `ComplexObjectArrayReader`) but the `ByteArrayReader` itself would become much simpler and probably a bit faster




-- 
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 edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-1011005560






-- 
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 pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-1014821537


   > What is the plan for the ArrowArrayReader implementation
   
   I don't think there is a particular reason for it to stay, but I defer the final decision to @yordan-pavlov 


-- 
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 pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

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


   I also ran the tests from the latest master branch of datafusion against this branch and they all passed. Not that it is the most thorough coverage of the parquet format, but it adds some.
   
   👍 


-- 
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] codecov-commenter edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-998923929


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1082](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (971622c) into [master](https://codecov.io/gh/apache/arrow-rs/commit/e45d1183aee00f27e79ca7acaed4d3d33c62ec72?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e45d118) will **decrease** coverage by `0.07%`.
   > The diff coverage is `82.80%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1082/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1082      +/-   ##
   ==========================================
   - Coverage   82.64%   82.57%   -0.08%     
   ==========================================
     Files         173      175       +2     
     Lines       50865    51254     +389     
   ==========================================
   + Hits        42037    42322     +285     
   - Misses       8828     8932     +104     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyLnJz) | `76.93% <37.03%> (-0.23%)` | :arrow_down: |
   | [parquet/src/column/reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvY29sdW1uL3JlYWRlci5ycw==) | `68.80% <41.17%> (-1.09%)` | :arrow_down: |
   | [parquet/src/arrow/record\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci5ycw==) | `94.02% <60.00%> (-0.73%)` | :arrow_down: |
   | [parquet/src/arrow/array\_reader/byte\_array.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL2J5dGVfYXJyYXkucnM=) | `83.56% <83.56%> (ø)` | |
   | [parquet/src/arrow/arrow\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfcmVhZGVyLnJz) | `91.86% <90.00%> (+0.25%)` | :arrow_up: |
   | [parquet/src/arrow/array\_reader/offset\_buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL29mZnNldF9idWZmZXIucnM=) | `93.05% <93.05%> (ø)` | |
   | [parquet/src/arrow/record\_reader/buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9idWZmZXIucnM=) | `89.23% <100.00%> (+3.23%)` | :arrow_up: |
   | [...rquet/src/arrow/record\_reader/definition\_levels.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9kZWZpbml0aW9uX2xldmVscy5ycw==) | `87.50% <100.00%> (-0.07%)` | :arrow_down: |
   | [parquet/src/column/reader/decoder.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvY29sdW1uL3JlYWRlci9kZWNvZGVyLnJz) | `76.27% <100.00%> (ø)` | |
   | [parquet/src/arrow/arrow\_array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfYXJyYXlfcmVhZGVyLnJz) | `73.66% <0.00%> (-5.59%)` | :arrow_down: |
   | ... and [8 more](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e45d118...971622c](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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 pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-1013914022


   I'm going to continue to write tests for various parts of this, but I think the main bulk is ready for review and has fairly good coverage from the fuzz tests.
   
   Unfortunately this ended up a wee bit chonkier than I had hoped, if this is a problem let me know and I can try to split some stuff out.


-- 
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 change in pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r786173636



##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {

Review comment:
       I thought similar, lifting this and ScalarBuffer into arrow-rs would likely remove a non-trivial amount of unsafe




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785470310



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+            Reader::LargeBinary(r) | Reader::LargeUtf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+        };
+
+        Ok(match &self.record_reader {
+            Reader::Binary(_) => Arc::new(BinaryArray::from(data)),
+            Reader::LargeBinary(_) => Arc::new(LargeBinaryArray::from(data)),
+            Reader::Utf8(_) => Arc::new(StringArray::from(data)),
+            Reader::LargeUtf8(_) => Arc::new(LargeStringArray::from(data)),
+        })
+    }
+
+    fn get_def_levels(&self) -> Option<&[i16]> {
+        self.def_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+
+    fn get_rep_levels(&self) -> Option<&[i16]> {
+        self.rep_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+}
+
+struct OffsetBuffer<I> {
+    offsets: TypedBuffer<I>,
+    values: TypedBuffer<u8>,
+}
+
+impl<I> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = TypedBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: TypedBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait> OffsetBuffer<I> {
+    fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+}
+
+impl<I: OffsetSizeTrait> RecordBuffer for OffsetBuffer<I> {
+    type Output = Self;
+    type Writer = Self;
+
+    fn split(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = TypedBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)
+        }
+
+        self.offsets.resize(len + 1);
+
+        Self {
+            offsets: std::mem::replace(&mut self.offsets, new_offsets),
+            values: self.values.take(end_offset.to_usize().unwrap()),
+        }
+    }
+
+    fn writer(&mut self, _batch_size: usize) -> &mut Self::Writer {
+        self
+    }
+
+    fn commit(&mut self, len: usize) {
+        assert_eq!(self.offsets.len(), len + 1);
+    }
+}
+
+impl<I: OffsetSizeTrait> ValueBuffer for OffsetBuffer<I> {
+    fn pad_nulls(
+        &mut self,
+        values_range: Range<usize>,
+        levels_range: Range<usize>,
+        rev_position_iter: impl Iterator<Item = usize>,
+    ) {
+        assert_eq!(self.offsets.len(), values_range.end + 1);
+        self.offsets.resize(levels_range.end + 1);
+
+        let offsets = self.offsets.as_slice_mut();
+
+        let values_start = values_range.start;
+        let mut last_offset = levels_range.end + 1;
+
+        for (value_pos, level_pos) in values_range.rev().zip(rev_position_iter) {
+            assert!(level_pos >= value_pos);
+            assert!(level_pos < last_offset);
+
+            if level_pos == value_pos {
+                // Pad trailing nulls if necessary
+                if level_pos != last_offset && last_offset == levels_range.end + 1 {
+                    let value = offsets[value_pos];
+                    for x in &mut offsets[level_pos + 1..last_offset] {
+                        *x = value;
+                    }
+                }
+
+                // We are done
+                return;
+            }
+
+            // Fill in any nulls
+            let value_end = offsets[value_pos + 1];
+            let value_start = offsets[value_pos];
+
+            for x in &mut offsets[level_pos + 1..last_offset] {
+                *x = value_end;
+            }
+
+            offsets[level_pos] = value_start;
+            last_offset = level_pos;
+        }
+
+        // Pad leading nulls up to `last_offset`
+        let value = offsets[values_start];
+        for x in &mut offsets[values_start + 1..last_offset] {
+            *x = value
+        }
+    }
+}
+
+impl<I> ValuesWriter for OffsetBuffer<I> {
+    fn capacity(&self) -> usize {
+        usize::MAX
+    }
+}
+
+struct ByteArrayDecoder<I> {
+    dict: Option<OffsetBuffer<I>>,
+    decoder: Option<StringDecoder>,
+}
+
+impl<I: OffsetSizeTrait> ColumnValueDecoder for ByteArrayDecoder<I> {
+    type Writer = OffsetBuffer<I>;
+
+    fn create(_: &ColumnDescPtr) -> Self {
+        Self {
+            dict: None,
+            decoder: None,
+        }
+    }
+
+    fn set_dict(
+        &mut self,
+        buf: ByteBufferPtr,
+        num_values: u32,
+        encoding: Encoding,
+        _is_sorted: bool,
+    ) -> Result<()> {
+        if !matches!(
+            encoding,
+            Encoding::PLAIN | Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY
+        ) {
+            return Err(nyi_err!(
+                "Invalid/Unsupported encoding type for dictionary: {}",
+                encoding
+            ));
+        }
+
+        let mut buffer = OffsetBuffer::default();
+        let mut decoder = PlainDecoder::new(buf, num_values as usize);
+        decoder.read(&mut buffer, usize::MAX)?;
+        self.dict = Some(buffer);
+        Ok(())
+    }
+
+    fn set_data(
+        &mut self,
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_values: usize,
+    ) -> Result<()> {
+        let decoder = match encoding {
+            Encoding::PLAIN => StringDecoder::Plain(PlainDecoder::new(data, num_values)),
+            Encoding::RLE_DICTIONARY => {
+                StringDecoder::Dictionary(DictionaryDecoder::new(data))
+            }
+            Encoding::DELTA_LENGTH_BYTE_ARRAY => {
+                StringDecoder::DeltaLength(DeltaLengthDecoder::new(data, num_values)?)
+            }
+            Encoding::DELTA_BYTE_ARRAY => {
+                StringDecoder::DeltaStrings(DeltaStringsDecoder::new(data, num_values)?)
+            }
+            _ => {
+                return Err(general_err!(
+                    "unsupported encoding for byte array: {}",
+                    encoding
+                ))
+            }
+        };
+        self.decoder = Some(decoder);
+        Ok(())
+    }
+
+    fn read(&mut self, out: &mut Self::Writer, range: Range<usize>) -> Result<usize> {
+        let len = range.end - range.start;
+        match self.decoder.as_mut().expect("decoder set") {
+            StringDecoder::Plain(d) => d.read(out, len),
+            StringDecoder::Dictionary(d) => {
+                let dict = self.dict.as_ref().expect("dictionary set");
+                d.read(out, dict, len)
+            }
+            StringDecoder::DeltaLength(d) => d.read(out, len),
+            StringDecoder::DeltaStrings(d) => d.read(out, len),
+        }
+    }
+}
+
+enum StringDecoder {
+    Plain(PlainDecoder),
+    Dictionary(DictionaryDecoder),
+    DeltaLength(DeltaLengthDecoder),
+    DeltaStrings(DeltaStringsDecoder),
+}
+
+/// Decoder for [`Encoding::PLAIN`]
+struct PlainDecoder {
+    buf: ByteBufferPtr,
+    offset: usize,
+    remaining_values: usize,
+}
+
+impl PlainDecoder {
+    fn new(buf: ByteBufferPtr, values: usize) -> Self {
+        Self {
+            buf,
+            offset: 0,
+            remaining_values: values,
+        }
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let to_read = len.min(self.remaining_values);
+        output.offsets.reserve(to_read);
+
+        let remaining_bytes = self.buf.len() - self.offset;
+        if remaining_bytes == 0 {
+            return Ok(0);
+        }
+
+        let estimated_bytes = remaining_bytes
+            .checked_mul(to_read)
+            .map(|x| x / self.remaining_values)
+            .unwrap_or_default();
+
+        output.values.reserve(estimated_bytes);
+
+        let mut read = 0;
+
+        let buf = self.buf.as_ref();
+        while self.offset < self.buf.len() && read != to_read {
+            if self.offset + 4 > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+            let len_bytes: [u8; 4] =
+                buf[self.offset..self.offset + 4].try_into().unwrap();
+            let len = u32::from_le_bytes(len_bytes);
+
+            let start_offset = self.offset + 4;
+            let end_offset = start_offset + len as usize;
+            if end_offset > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            output.try_push(&buf[start_offset..end_offset])?;
+
+            self.offset = end_offset;
+            read += 1;
+        }
+        self.remaining_values -= to_read;
+        Ok(to_read)
+    }
+}
+
+/// Decoder for [`Encoding::DELTA_LENGTH_BYTE_ARRAY`]
+struct DeltaLengthDecoder {
+    lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+}
+
+impl DeltaLengthDecoder {
+    fn new(data: ByteBufferPtr, values: usize) -> Result<Self> {
+        let mut len_decoder = DeltaBitPackDecoder::<Int32Type>::new();
+        len_decoder.set_data(data.all(), values)?;
+        let mut lengths = vec![0; values];
+        len_decoder.get(&mut lengths)?;
+
+        Ok(Self {
+            lengths,
+            data,
+            length_offset: 0,
+            data_offset: len_decoder.get_offset(),
+        })
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let to_read = len.min(self.lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let mut to_read_bytes: usize = 0;
+        let mut offset = output.values.len();
+
+        for length in &self.lengths[self.length_offset..self.length_offset + to_read] {
+            offset = offset.saturating_add(*length as usize);
+            to_read_bytes += *length as usize;
+
+            let offset_i = I::from_usize(offset)
+                .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+            output.offsets.push(offset_i)
+        }
+
+        output.values.extend_from_slice(
+            &self.data.as_ref()[self.data_offset..self.data_offset + to_read_bytes],
+        );
+
+        self.data_offset += to_read_bytes;
+        self.length_offset += to_read;
+        Ok(to_read)
+    }
+}
+
+/// Decoder for [`Encoding::DELTA_BYTE_ARRAY`]
+struct DeltaStringsDecoder {
+    prefix_lengths: Vec<i32>,
+    suffix_lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    last_value: Vec<u8>,
+}
+
+impl DeltaStringsDecoder {
+    fn new(data: ByteBufferPtr, values: usize) -> Result<Self> {
+        let mut prefix = DeltaBitPackDecoder::<Int32Type>::new();
+        prefix.set_data(data.all(), values)?;
+        let mut prefix_lengths = vec![0; values];
+        prefix.get(&mut prefix_lengths)?;
+
+        let mut suffix = DeltaBitPackDecoder::<Int32Type>::new();
+        suffix.set_data(data.start_from(prefix.get_offset()), values)?;
+        let mut suffix_lengths = vec![0; values];
+        suffix.get(&mut suffix_lengths)?;
+
+        Ok(Self {
+            prefix_lengths,
+            suffix_lengths,
+            data,
+            length_offset: 0,
+            data_offset: prefix.get_offset() + suffix.get_offset(),
+            last_value: vec![],
+        })
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        assert_eq!(self.prefix_lengths.len(), self.suffix_lengths.len());
+
+        let to_read = len.min(self.prefix_lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let length_range = self.length_offset..self.length_offset + to_read;
+        let iter = self.prefix_lengths[length_range.clone()]
+            .iter()
+            .zip(&self.suffix_lengths[length_range]);
+
+        let mut offset = output.values.len();
+        let data = self.data.as_ref();
+
+        for (prefix_length, suffix_length) in iter {
+            let total_length = *prefix_length as usize + *suffix_length as usize;
+
+            if self.data_offset + total_length > self.data.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            offset = offset.saturating_add(total_length);
+
+            let offset_i = I::from_usize(offset)
+                .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+            output.offsets.push(offset_i);
+
+            self.last_value.truncate(*prefix_length as usize);
+            self.last_value.extend_from_slice(
+                &data[self.data_offset..self.data_offset + total_length],
+            );
+
+            output.values.reserve(total_length);
+            output.values.extend_from_slice(&self.last_value);
+
+            self.data_offset += total_length;
+        }
+
+        self.length_offset += to_read;
+        Ok(to_read)
+    }
+}
+
+struct DictionaryDecoder {
+    decoder: RleDecoder,
+    index_buf: Box<[i32; 1024]>,
+    index_offset: usize,
+}
+
+impl DictionaryDecoder {
+    fn new(data: ByteBufferPtr) -> Self {
+        let bit_width = data[0];
+        let mut decoder = RleDecoder::new(bit_width);
+        decoder.set_data(data.start_from(1));
+
+        Self {
+            decoder,
+            index_buf: Box::new([0; 1024]),
+            index_offset: 1024,
+        }
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        dict: &OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let mut values_read = 0;
+
+        while values_read != len {
+            if self.index_offset == self.index_buf.len() {
+                let decoded = self.decoder.get_batch(self.index_buf.as_mut())?;
+                self.index_offset = 0;
+                if decoded != self.index_buf.len() && decoded < len - values_read {
+                    return Err(ParquetError::EOF(

Review comment:
       I removed this logic, and added checks higher up in RecordReader




-- 
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] Dandandan commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785434772



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -192,7 +211,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         self.offsets.len() - 1
     }
 
-    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+    fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Err(e) = std::str::from_utf8(data) {

Review comment:
       I think we tried to do something similar with parquet2 but concluded that the individual strings should be checked instead. `simdutf8` is more impressive at checking non ASCII strings btw.
   Checking the code points at the offsets seems an interesting approach!
   Also FYI @jorgecarleitao




-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785493522



##########
File path: parquet/benches/arrow_array_reader.rs
##########
@@ -366,10 +366,10 @@ fn add_benches(c: &mut Criterion) {
                     mandatory_int32_column_desc.clone(),
                 );
                 count = bench_array_reader(array_reader);
+                assert_eq!(count, EXPECTED_VALUE_COUNT);

Review comment:
       with this change, doesn't `assert_eq!(count, EXPECTED_VALUE_COUNT);` become part of the benchmarked code? yes, it should have minimal to no impact, but my concern is more about keeping the benchmark as clean as possible.




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785424844



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -192,7 +211,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         self.offsets.len() - 1
     }
 
-    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+    fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Err(e) = std::str::from_utf8(data) {

Review comment:
       So I did some experimentation:
   
   It is **significantly** faster to verify on push that the first byte is a valid start UTF-8 codepoint, and then do UTF-8 validation on the larger buffer in one go, it takes the performance hit on PLAIN encoded strings to ~1.1x down from ~2x. I have modified the code to do this.
   
   With this optimisation applied, changing to simdutf8 made only a very minor ~6% improvement on PLAIN encoded strings, which reduced to no appreciable difference with RLE encoded strings. This may be my machine, or the lack of non-ASCII characters in the input, but I'm going to leave this out for now. 




-- 
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 change in pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r786196266



##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    /// Extends this buffer with a list of keys
+    ///
+    /// For each value `key` in `keys` this will insert
+    /// `&dict_values[dict_offsets[key]..dict_offsets[key+1]]`
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(
+        &mut self,
+        keys: &[K],
+        dict_offsets: &[V],
+        dict_values: &[u8],
+    ) -> Result<()> {
+        for key in keys {
+            let index = key.to_usize().unwrap();
+            if index + 1 >= dict_offsets.len() {
+                return Err(general_err!("invalid offset in byte array: {}", index));
+            }
+            let start_offset = dict_offsets[index].to_usize().unwrap();
+            let end_offset = dict_offsets[index + 1].to_usize().unwrap();
+
+            // Dictionary values are verified when decoding dictionary page
+            self.try_push(&dict_values[start_offset..end_offset], false)?;
+        }
+        Ok(())
+    }
+
+    /// Validates that `&self.values[start_offset..]` is a valid UTF-8 sequence
+    ///
+    /// This MUST be combined with validating that the offsets start on a character
+    /// boundary, otherwise it would be possible for the values array to be a valid UTF-8
+    /// sequence, but not the individual string slices it contains
+    ///
+    /// [`Self::try_push`] can perform this validation check on insertion
+    pub fn check_valid_utf8(&self, start_offset: usize) -> Result<()> {
+        match std::str::from_utf8(&self.values.as_slice()[start_offset..]) {
+            Ok(_) => Ok(()),
+            Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)),
+        }
+    }
+
+    /// Converts this into an [`ArrayRef`] with the provided `data_type` and `null_buffer`
+    pub fn into_array(
+        self,
+        null_buffer: Option<Buffer>,
+        data_type: ArrowType,
+    ) -> ArrayRef {
+        let mut array_data_builder = ArrayDataBuilder::new(data_type)
+            .len(self.len())
+            .add_buffer(self.offsets.into())
+            .add_buffer(self.values.into());
+
+        if let Some(buffer) = null_buffer {
+            array_data_builder = array_data_builder.null_bit_buffer(buffer);
+        }
+
+        let data = match cfg!(debug_assertions) {
+            true => array_data_builder.build().unwrap(),
+            false => unsafe { array_data_builder.build_unchecked() },
+        };
+
+        make_array(data)
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> BufferQueue for OffsetBuffer<I> {
+    type Output = Self;
+    type Slice = Self;
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = ScalarBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)
+        }
+
+        self.offsets.resize(len + 1);
+
+        Self {
+            offsets: std::mem::replace(&mut self.offsets, new_offsets),
+            values: self.values.take(end_offset.to_usize().unwrap()),
+        }
+    }
+
+    fn spare_capacity_mut(&mut self, _batch_size: usize) -> &mut Self::Slice {
+        self
+    }
+
+    fn set_len(&mut self, len: usize) {
+        assert_eq!(self.offsets.len(), len + 1);
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ValuesBuffer for OffsetBuffer<I> {
+    fn pad_nulls(
+        &mut self,
+        read_offset: usize,
+        values_read: usize,
+        levels_read: usize,
+        rev_position_iter: impl Iterator<Item = usize>,
+    ) {
+        assert_eq!(self.offsets.len(), read_offset + values_read + 1);
+        self.offsets.resize(read_offset + levels_read + 1);
+
+        let offsets = self.offsets.as_slice_mut();
+
+        let mut last_pos = read_offset + levels_read + 1;
+        let mut last_start_offset = I::from_usize(self.values.len()).unwrap();
+
+        let values_range = read_offset..read_offset + values_read;
+        for (value_pos, level_pos) in values_range.clone().rev().zip(rev_position_iter) {
+            assert!(level_pos >= value_pos);

Review comment:
       There was much wailing and gnashing of teeth in its creation :sweat_smile: 




-- 
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] yordan-pavlov commented on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999798239


   @tustvold  what about performance with primitive types (e.g. int32)? - this is where I have been struggling to make the `ArrowArrayReader` faster (compare to old array reader) for dictionary-encoded primitive values


-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785454423



##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,211 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(

Review comment:
       This is split out so that it can be used by #1180 




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785470260



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {

Review comment:
       Switched to using generics as it made #1180 easier to implement, I ended up with a combinatorial explosion of key and value types :sweat_smile: 




-- 
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 edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999804838


   > what about performance with primitive types (e.g. int32)
   
   This PR builds on #1054 which yields a 2-6x speed up when using PrimitiveArrayReader on non-nested columns compared to current master. This is purely through better null handling, which this PR also benefits from.
   
   I do have some reservations about drawing too much from these benchmarks, I have found them to have strange interactions with my system's memory allocator, but its certainly not slower and is likely significantly faster.
   
   > compared to old array reader implementation
   
   That's the key thing about #1041 it doesn't replace this array reader implementation, it just adds the ability to extend it. For primitive types the performance of #1041 is therefore [unchanged](https://github.com/apache/arrow-rs/pull/1041#issuecomment-993816497), it just gives the ability to add optimisations such as #1054


-- 
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] yordan-pavlov edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999877477


   @tustvold you are probably aware about this, but just to make sure it's not missed, when I run this branch with datafusion against a parquet file I get an error `Parquet argument error: Parquet error: unsupported encoding for byte array: PLAIN_DICTIONARY`
   
   Other than that, the performance benchmark results look impressive - I was able to run the benchmark and this branch is faster than the `ArrowArrayReader`, sometimes several times faster, in almost all cases (exceptions listed below). And the `ArrowArrayReader` was already several times faster in many cases than the old array reader implementation, making these performance results even more impressive.
   
   A major reason, why I only implemented `ArrowArrayReader` for string arrays is because I have been struggling to make it faster for dictionary-encoded primitive arrays, but it looks like this isn't going to be a problem with this new implementation.
   So if we can make it faster in all benchmarks, I am happy to abandon the `ArrowArrayReader` in favor of this new implementation.
   
   Where it is still a bit slower is in these two cases:
   
   read StringArray, plain encoded, mandatory, no NULLs - old: time:   [306.10 us 342.14 us 377.28 us]
   read StringArray, plain encoded, mandatory, no NULLs - new: time:   [310.84 us 337.49 us 368.74 us]
   
   read StringArray, dictionary encoded, mandatory, no NULLs - old: time:   [286.61 us 320.07 us 354.74 us]
   read StringArray, dictionary encoded, mandatory, no NULLs - new: time:   [222.87 us 240.56 us 260.93 us]
   
   The reason why `ArrowArrayReader` is fast in those cases, I suspect, is because when there are no nulls / def levels, the def level buffer are not read or processed at all, see here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L566 . This also means that the bit of code that produces the null bitmap also doesn't run, see here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L595 and the main path in the code is not concerned with null values at all, which is why it's so fast when there are no null / def levels, see here: https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L592 , see string converter here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L1164 .
   


-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r776323820



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {

Review comment:
       Perhaps it is a hold over from my experience with C++ templates, but it has generally been my experience that generic code is tricky to use and even more tricky to maintain. I therefore am loathe to resort to it in places where it isn't strictly necessary. 
   
   Adding a generic argument to `ByteArrayReader` would require a conditional block in the caller, and therefore the additional code saving is likely to be small. Additionally the cost of the conditional jump in `next_batch` is going to be completely dwarfed by the conditionals and memory allocations inherent in reading variable length string data, especially if using a reasonable batch_size.




-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r776412307



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {

Review comment:
       My thinking in proposing adding generic type parameters, is that this `next_batch` method is part of the hot path in decoding arrow arrays from parquet files and the more conditions can be eliminated from the hot path, the better. I might be wrong, may be replacing the conditions with generic type parameters will not yield any performance improvements. But it should be easy to test - it should be a small code change and easy to run benchmarks against it.




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r774006088



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+            Reader::LargeBinary(r) | Reader::LargeUtf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+        };
+
+        Ok(match &self.record_reader {
+            Reader::Binary(_) => Arc::new(BinaryArray::from(data)),
+            Reader::LargeBinary(_) => Arc::new(LargeBinaryArray::from(data)),
+            Reader::Utf8(_) => Arc::new(StringArray::from(data)),
+            Reader::LargeUtf8(_) => Arc::new(LargeStringArray::from(data)),
+        })
+    }
+
+    fn get_def_levels(&self) -> Option<&[i16]> {
+        self.def_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+
+    fn get_rep_levels(&self) -> Option<&[i16]> {
+        self.rep_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+}
+
+struct OffsetBuffer<I> {
+    offsets: TypedBuffer<I>,
+    values: TypedBuffer<u8>,
+}
+
+impl<I> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = TypedBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: TypedBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait> OffsetBuffer<I> {
+    fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+}
+
+impl<I: OffsetSizeTrait> RecordBuffer for OffsetBuffer<I> {
+    type Output = Self;
+    type Writer = Self;
+
+    fn split(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = TypedBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)
+        }
+
+        self.offsets.resize(len + 1);
+
+        Self {
+            offsets: std::mem::replace(&mut self.offsets, new_offsets),
+            values: self.values.take(end_offset.to_usize().unwrap()),
+        }
+    }
+
+    fn writer(&mut self, _batch_size: usize) -> &mut Self::Writer {
+        self
+    }
+
+    fn commit(&mut self, len: usize) {
+        assert_eq!(self.offsets.len(), len + 1);
+    }
+}
+
+impl<I: OffsetSizeTrait> ValueBuffer for OffsetBuffer<I> {
+    fn pad_nulls(
+        &mut self,
+        values_range: Range<usize>,
+        levels_range: Range<usize>,
+        rev_position_iter: impl Iterator<Item = usize>,
+    ) {
+        assert_eq!(self.offsets.len(), values_range.end + 1);
+        self.offsets.resize(levels_range.end + 1);
+
+        let offsets = self.offsets.as_slice_mut();
+
+        let values_start = values_range.start;
+        let mut last_offset = levels_range.end + 1;
+
+        for (value_pos, level_pos) in values_range.rev().zip(rev_position_iter) {
+            assert!(level_pos >= value_pos);
+            assert!(level_pos < last_offset);
+
+            if level_pos == value_pos {
+                // Pad trailing nulls if necessary
+                if level_pos != last_offset && last_offset == levels_range.end + 1 {
+                    let value = offsets[value_pos];
+                    for x in &mut offsets[level_pos + 1..last_offset] {
+                        *x = value;
+                    }
+                }
+
+                // We are done
+                return;
+            }
+
+            // Fill in any nulls
+            let value_end = offsets[value_pos + 1];
+            let value_start = offsets[value_pos];
+
+            for x in &mut offsets[level_pos + 1..last_offset] {
+                *x = value_end;
+            }
+
+            offsets[level_pos] = value_start;
+            last_offset = level_pos;
+        }
+
+        // Pad leading nulls up to `last_offset`
+        let value = offsets[values_start];
+        for x in &mut offsets[values_start + 1..last_offset] {
+            *x = value
+        }
+    }
+}
+
+impl<I> ValuesWriter for OffsetBuffer<I> {
+    fn capacity(&self) -> usize {
+        usize::MAX
+    }
+}
+
+struct ByteArrayDecoder<I> {
+    dict: Option<OffsetBuffer<I>>,
+    decoder: Option<StringDecoder>,
+}
+
+impl<I: OffsetSizeTrait> ColumnValueDecoder for ByteArrayDecoder<I> {
+    type Writer = OffsetBuffer<I>;
+
+    fn create(_: &ColumnDescPtr) -> Self {
+        Self {
+            dict: None,
+            decoder: None,
+        }
+    }
+
+    fn set_dict(
+        &mut self,
+        buf: ByteBufferPtr,
+        num_values: u32,
+        encoding: Encoding,
+        _is_sorted: bool,
+    ) -> Result<()> {
+        if !matches!(
+            encoding,
+            Encoding::PLAIN | Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY
+        ) {
+            return Err(nyi_err!(
+                "Invalid/Unsupported encoding type for dictionary: {}",
+                encoding
+            ));
+        }
+
+        let mut buffer = OffsetBuffer::default();
+        let mut decoder = PlainDecoder::new(buf, num_values as usize);
+        decoder.read(&mut buffer, usize::MAX)?;
+        self.dict = Some(buffer);
+        Ok(())
+    }
+
+    fn set_data(
+        &mut self,
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_values: usize,
+    ) -> Result<()> {
+        let decoder = match encoding {
+            Encoding::PLAIN => StringDecoder::Plain(PlainDecoder::new(data, num_values)),
+            Encoding::RLE_DICTIONARY => {
+                StringDecoder::Dictionary(DictionaryDecoder::new(data))
+            }
+            Encoding::DELTA_LENGTH_BYTE_ARRAY => {
+                StringDecoder::DeltaLength(DeltaLengthDecoder::new(data, num_values)?)
+            }
+            Encoding::DELTA_BYTE_ARRAY => {
+                StringDecoder::DeltaStrings(DeltaStringsDecoder::new(data, num_values)?)
+            }
+            _ => {
+                return Err(general_err!(
+                    "unsupported encoding for byte array: {}",
+                    encoding
+                ))
+            }
+        };
+        self.decoder = Some(decoder);
+        Ok(())
+    }
+
+    fn read(&mut self, out: &mut Self::Writer, range: Range<usize>) -> Result<usize> {
+        let len = range.end - range.start;
+        match self.decoder.as_mut().expect("decoder set") {
+            StringDecoder::Plain(d) => d.read(out, len),
+            StringDecoder::Dictionary(d) => {
+                let dict = self.dict.as_ref().expect("dictionary set");
+                d.read(out, dict, len)
+            }
+            StringDecoder::DeltaLength(d) => d.read(out, len),
+            StringDecoder::DeltaStrings(d) => d.read(out, len),
+        }
+    }
+}
+
+enum StringDecoder {
+    Plain(PlainDecoder),
+    Dictionary(DictionaryDecoder),
+    DeltaLength(DeltaLengthDecoder),
+    DeltaStrings(DeltaStringsDecoder),
+}
+
+/// Decoder for [`Encoding::PLAIN`]
+struct PlainDecoder {
+    buf: ByteBufferPtr,
+    offset: usize,
+    remaining_values: usize,
+}
+
+impl PlainDecoder {
+    fn new(buf: ByteBufferPtr, values: usize) -> Self {
+        Self {
+            buf,
+            offset: 0,
+            remaining_values: values,
+        }
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let to_read = len.min(self.remaining_values);
+        output.offsets.reserve(to_read);
+
+        let remaining_bytes = self.buf.len() - self.offset;
+        if remaining_bytes == 0 {
+            return Ok(0);
+        }
+
+        let estimated_bytes = remaining_bytes
+            .checked_mul(to_read)
+            .map(|x| x / self.remaining_values)
+            .unwrap_or_default();
+
+        output.values.reserve(estimated_bytes);
+
+        let mut read = 0;
+
+        let buf = self.buf.as_ref();
+        while self.offset < self.buf.len() && read != to_read {
+            if self.offset + 4 > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+            let len_bytes: [u8; 4] =
+                buf[self.offset..self.offset + 4].try_into().unwrap();
+            let len = u32::from_le_bytes(len_bytes);
+
+            let start_offset = self.offset + 4;
+            let end_offset = start_offset + len as usize;
+            if end_offset > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            output.try_push(&buf[start_offset..end_offset])?;
+
+            self.offset = end_offset;
+            read += 1;
+        }
+        self.remaining_values -= to_read;
+        Ok(to_read)
+    }
+}
+
+/// Decoder for [`Encoding::DELTA_LENGTH_BYTE_ARRAY`]
+struct DeltaLengthDecoder {
+    lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+}
+
+impl DeltaLengthDecoder {
+    fn new(data: ByteBufferPtr, values: usize) -> Result<Self> {
+        let mut len_decoder = DeltaBitPackDecoder::<Int32Type>::new();
+        len_decoder.set_data(data.all(), values)?;
+        let mut lengths = vec![0; values];
+        len_decoder.get(&mut lengths)?;
+
+        Ok(Self {
+            lengths,
+            data,
+            length_offset: 0,
+            data_offset: len_decoder.get_offset(),
+        })
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let to_read = len.min(self.lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let mut to_read_bytes: usize = 0;
+        let mut offset = output.values.len();
+
+        for length in &self.lengths[self.length_offset..self.length_offset + to_read] {
+            offset = offset.saturating_add(*length as usize);
+            to_read_bytes += *length as usize;
+
+            let offset_i = I::from_usize(offset)
+                .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+            output.offsets.push(offset_i)
+        }
+
+        output.values.extend_from_slice(
+            &self.data.as_ref()[self.data_offset..self.data_offset + to_read_bytes],
+        );
+
+        self.data_offset += to_read_bytes;
+        self.length_offset += to_read;
+        Ok(to_read)
+    }
+}
+
+/// Decoder for [`Encoding::DELTA_BYTE_ARRAY`]
+struct DeltaStringsDecoder {
+    prefix_lengths: Vec<i32>,
+    suffix_lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    last_value: Vec<u8>,
+}
+
+impl DeltaStringsDecoder {
+    fn new(data: ByteBufferPtr, values: usize) -> Result<Self> {
+        let mut prefix = DeltaBitPackDecoder::<Int32Type>::new();
+        prefix.set_data(data.all(), values)?;
+        let mut prefix_lengths = vec![0; values];
+        prefix.get(&mut prefix_lengths)?;
+
+        let mut suffix = DeltaBitPackDecoder::<Int32Type>::new();
+        suffix.set_data(data.start_from(prefix.get_offset()), values)?;
+        let mut suffix_lengths = vec![0; values];
+        suffix.get(&mut suffix_lengths)?;
+
+        Ok(Self {
+            prefix_lengths,
+            suffix_lengths,
+            data,
+            length_offset: 0,
+            data_offset: prefix.get_offset() + suffix.get_offset(),
+            last_value: vec![],
+        })
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        assert_eq!(self.prefix_lengths.len(), self.suffix_lengths.len());
+
+        let to_read = len.min(self.prefix_lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let length_range = self.length_offset..self.length_offset + to_read;
+        let iter = self.prefix_lengths[length_range.clone()]
+            .iter()
+            .zip(&self.suffix_lengths[length_range]);
+
+        let mut offset = output.values.len();
+        let data = self.data.as_ref();
+
+        for (prefix_length, suffix_length) in iter {
+            let total_length = *prefix_length as usize + *suffix_length as usize;
+
+            if self.data_offset + total_length > self.data.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            offset = offset.saturating_add(total_length);
+
+            let offset_i = I::from_usize(offset)
+                .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+            output.offsets.push(offset_i);
+
+            self.last_value.truncate(*prefix_length as usize);
+            self.last_value.extend_from_slice(
+                &data[self.data_offset..self.data_offset + total_length],
+            );
+
+            output.values.reserve(total_length);
+            output.values.extend_from_slice(&self.last_value);
+
+            self.data_offset += total_length;
+        }
+
+        self.length_offset += to_read;
+        Ok(to_read)
+    }
+}
+
+struct DictionaryDecoder {
+    decoder: RleDecoder,
+    index_buf: Box<[i32; 1024]>,
+    index_offset: usize,
+}
+
+impl DictionaryDecoder {
+    fn new(data: ByteBufferPtr) -> Self {
+        let bit_width = data[0];
+        let mut decoder = RleDecoder::new(bit_width);
+        decoder.set_data(data.start_from(1));
+
+        Self {
+            decoder,
+            index_buf: Box::new([0; 1024]),
+            index_offset: 1024,
+        }
+    }
+
+    fn read<I: OffsetSizeTrait>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        dict: &OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let mut values_read = 0;
+
+        while values_read != len {
+            if self.index_offset == self.index_buf.len() {
+                let decoded = self.decoder.get_batch(self.index_buf.as_mut())?;
+                self.index_offset = 0;
+                if decoded != self.index_buf.len() && decoded < len - values_read {
+                    return Err(ParquetError::EOF(

Review comment:
       Quite possibly, although these structs are intended for general consumption - and RecordReader should be handling requesting only the correct amount. Further refinement is definitely likely if/when this gets closer to being review ready




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785424844



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -192,7 +211,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         self.offsets.len() - 1
     }
 
-    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+    fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Err(e) = std::str::from_utf8(data) {

Review comment:
       So I did some experimentation:
   
   It is **significantly** faster to verify on push that the first byte is a valid start UTF-8 codepoint, and then do UTF-8 validation on the larger buffer in one go, it takes the performance hit on PLAIN encoded strings to ~1.1x down from ~2x.
   
   With this optimisation applied, changing to simdutf8 made only a very minor ~6% improvement on PLAIN encoded strings, which reduced to no appreciable difference with RLE encoded strings. This may be my machine, or the lack of non-ASCII characters in the input, but I'm going to leave this out for now. 




-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785494166



##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -68,6 +77,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         Ok(())
     }
 
+    /// Returns the values buffer as a string slice, returning an error
+    /// if it is invalid UTF-8
+    ///
+    /// `start_offset` is the offset in bytes from the start
+    pub fn values_as_str(&self, start_offset: usize) -> Result<&str> {
+        std::str::from_utf8(&self.values.as_slice()[start_offset..]).map_err(|e| {

Review comment:
       if the &str result is not actually used, does this method need to return it? e.g. why not change to `-> Result<()>` and possibly even change the method name to something like `check_valid_utf8`.




-- 
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] codecov-commenter edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-998923929


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1082](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cf099c3) into [master](https://codecov.io/gh/apache/arrow-rs/commit/e45d1183aee00f27e79ca7acaed4d3d33c62ec72?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e45d118) will **decrease** coverage by `0.04%`.
   > The diff coverage is `84.48%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1082/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1082      +/-   ##
   ==========================================
   - Coverage   82.64%   82.59%   -0.05%     
   ==========================================
     Files         173      175       +2     
     Lines       50865    51302     +437     
   ==========================================
   + Hits        42037    42373     +336     
   - Misses       8828     8929     +101     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyLnJz) | `76.93% <37.03%> (-0.23%)` | :arrow_down: |
   | [parquet/src/column/reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvY29sdW1uL3JlYWRlci5ycw==) | `68.80% <41.17%> (-1.09%)` | :arrow_down: |
   | [parquet/src/arrow/record\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci5ycw==) | `94.02% <60.00%> (-0.73%)` | :arrow_down: |
   | [parquet/src/arrow/array\_reader/byte\_array.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL2J5dGVfYXJyYXkucnM=) | `86.51% <86.51%> (ø)` | |
   | [parquet/src/arrow/arrow\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfcmVhZGVyLnJz) | `91.86% <90.00%> (+0.25%)` | :arrow_up: |
   | [parquet/src/arrow/array\_reader/offset\_buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL29mZnNldF9idWZmZXIucnM=) | `93.05% <93.05%> (ø)` | |
   | [parquet/src/arrow/record\_reader/buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9idWZmZXIucnM=) | `89.23% <100.00%> (+3.23%)` | :arrow_up: |
   | [...rquet/src/arrow/record\_reader/definition\_levels.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9kZWZpbml0aW9uX2xldmVscy5ycw==) | `87.50% <100.00%> (-0.07%)` | :arrow_down: |
   | [parquet/src/column/reader/decoder.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvY29sdW1uL3JlYWRlci9kZWNvZGVyLnJz) | `76.27% <100.00%> (ø)` | |
   | [parquet/src/arrow/arrow\_array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfYXJyYXlfcmVhZGVyLnJz) | `73.66% <0.00%> (-5.59%)` | :arrow_down: |
   | ... and [9 more](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e45d118...cf099c3](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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] yordan-pavlov commented on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999156146


   @tustvold this sounds exciting, would you be able to share some performance benchmark results?


-- 
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 merged pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
alamb merged pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082


   


-- 
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] codecov-commenter edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-998923929


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1082](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a12babb) into [master](https://codecov.io/gh/apache/arrow-rs/commit/e45d1183aee00f27e79ca7acaed4d3d33c62ec72?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e45d118) will **decrease** coverage by `0.08%`.
   > The diff coverage is `80.05%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1082/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1082      +/-   ##
   ==========================================
   - Coverage   82.64%   82.55%   -0.09%     
   ==========================================
     Files         173      175       +2     
     Lines       50865    51188     +323     
   ==========================================
   + Hits        42037    42258     +221     
   - Misses       8828     8930     +102     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyLnJz) | `76.93% <37.03%> (-0.23%)` | :arrow_down: |
   | [parquet/src/column/reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvY29sdW1uL3JlYWRlci5ycw==) | `68.80% <41.17%> (-1.09%)` | :arrow_down: |
   | [parquet/src/arrow/record\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci5ycw==) | `94.02% <60.00%> (-0.73%)` | :arrow_down: |
   | [parquet/src/arrow/array\_reader/byte\_array.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL2J5dGVfYXJyYXkucnM=) | `83.56% <83.56%> (ø)` | |
   | [parquet/src/arrow/array\_reader/offset\_buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL29mZnNldF9idWZmZXIucnM=) | `88.46% <88.46%> (ø)` | |
   | [parquet/src/arrow/arrow\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfcmVhZGVyLnJz) | `91.86% <90.00%> (+0.25%)` | :arrow_up: |
   | [parquet/src/arrow/record\_reader/buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9idWZmZXIucnM=) | `89.23% <100.00%> (+3.23%)` | :arrow_up: |
   | [...rquet/src/arrow/record\_reader/definition\_levels.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9kZWZpbml0aW9uX2xldmVscy5ycw==) | `87.50% <100.00%> (-0.07%)` | :arrow_down: |
   | [parquet/src/column/reader/decoder.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvY29sdW1uL3JlYWRlci9kZWNvZGVyLnJz) | `76.27% <100.00%> (ø)` | |
   | [parquet/src/arrow/arrow\_array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfYXJyYXlfcmVhZGVyLnJz) | `73.66% <0.00%> (-5.59%)` | :arrow_down: |
   | ... and [9 more](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e45d118...a12babb](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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 edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-1013914022


   I'm going to continue to write tests for various parts of this, but I think the main bulk is ready for review and has fairly good coverage from the fuzz tests.
   
   Thank you @yordan-pavlov for your review, I think I have incorporated all of your suggestions, but let me know if I've missed something.
   
   Unfortunately this ended up a wee bit chonkier than I had hoped, if this is a problem let me know and I can try to split some stuff out.


-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785452762



##########
File path: parquet/src/column/reader.rs
##########
@@ -258,11 +258,15 @@ where
             // At this point we have read values, definition and repetition levels.
             // If both definition and repetition levels are defined, their counts
             // should be equal. Values count is always less or equal to definition levels.
-            if num_def_levels != 0 && num_rep_levels != 0 {
-                assert_eq!(

Review comment:
       Drive by fix, a bad parquet file shouldn't trigger a panic




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785501187



##########
File path: parquet/src/column/reader/decoder.rs
##########
@@ -95,11 +95,22 @@ pub trait ColumnValueDecoder {
     ) -> Result<()>;
 
     /// Set the current data page
+    ///
+    /// - `encoding` - the encoding of the page
+    /// - `data` - a point to the page's uncompressed value data
+    /// - `num_levels` - the number of levels contained within the page, i.e. values including nulls
+    /// - `num_values` - the number of non-null values contained within the page (V2 page only)
+    ///
+    /// Note: data encoded with [`Encoding::RLE`] may not know its exact length, as the final

Review comment:
       I wanted to be explicit about this to avoid a resurgence of this style of bug - https://github.com/apache/arrow-rs/issues/1111
   
   This is a crate-private API, and the necessary null counting dance is performed by RecordReader, but I wanted to call it out for the avoidance of confusion.




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785493832



##########
File path: parquet/benches/arrow_array_reader.rs
##########
@@ -366,10 +366,10 @@ fn add_benches(c: &mut Criterion) {
                     mandatory_int32_column_desc.clone(),
                 );
                 count = bench_array_reader(array_reader);
+                assert_eq!(count, EXPECTED_VALUE_COUNT);

Review comment:
       This was so that you can run a subset of the benchmarks, let me try lifting them out of the iter block but still within the bench_function :thinking: 
   
   Edit: done




-- 
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] codecov-commenter edited a comment on pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-998923929


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1082](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (22c090e) into [master](https://codecov.io/gh/apache/arrow-rs/commit/e45d1183aee00f27e79ca7acaed4d3d33c62ec72?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e45d118) will **increase** coverage by `0.01%`.
   > The diff coverage is `88.88%`.
   
   > :exclamation: Current head 22c090e differs from pull request most recent head c941606. Consider uploading reports for the commit c941606 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1082/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1082      +/-   ##
   ==========================================
   + Coverage   82.64%   82.65%   +0.01%     
   ==========================================
     Files         173      175       +2     
     Lines       50865    51512     +647     
   ==========================================
   + Hits        42037    42578     +541     
   - Misses       8828     8934     +106     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyLnJz) | `76.93% <37.03%> (-0.23%)` | :arrow_down: |
   | [parquet/src/column/reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvY29sdW1uL3JlYWRlci5ycw==) | `68.80% <41.17%> (-1.09%)` | :arrow_down: |
   | [parquet/src/arrow/record\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci5ycw==) | `94.02% <60.00%> (-0.73%)` | :arrow_down: |
   | [arrow/src/array/array\_string.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2FycmF5X3N0cmluZy5ycw==) | `97.61% <66.66%> (ø)` | |
   | [parquet/src/arrow/array\_reader/byte\_array.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL2J5dGVfYXJyYXkucnM=) | `87.06% <87.06%> (ø)` | |
   | [parquet/src/arrow/arrow\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfcmVhZGVyLnJz) | `91.86% <90.00%> (+0.25%)` | :arrow_up: |
   | [arrow/src/array/array\_binary.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2FycmF5L2FycmF5X2JpbmFyeS5ycw==) | `93.13% <92.30%> (-0.42%)` | :arrow_down: |
   | [parquet/src/arrow/array\_reader/offset\_buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL29mZnNldF9idWZmZXIucnM=) | `93.10% <93.10%> (ø)` | |
   | [parquet/src/arrow/levels.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvbGV2ZWxzLnJz) | `84.56% <95.45%> (+0.28%)` | :arrow_up: |
   | [arrow/src/compute/kernels/take.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2NvbXB1dGUva2VybmVscy90YWtlLnJz) | `95.35% <97.95%> (+0.18%)` | :arrow_up: |
   | ... and [16 more](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e45d118...c941606](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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] yordan-pavlov edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999798239


   @tustvold  what about performance with primitive types (e.g. int32)? - this is where I have been struggling to make the `ArrowArrayReader` faster (compared to old array reader implementation) for dictionary-encoded primitive values


-- 
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 edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999804838


   > what about performance with primitive types (e.g. int32)
   
   This PR builds on #1054 which yields a 2-6x speed up when using PrimitiveArrayReader on non-nested columns compared to current master. This is purely through better null handling, which this PR also benefits from.
   
   I do have some reservations about drawing too much from these benchmarks, I have found them to have strange interactions with my system's memory allocator, but its certainly not slower and is likely significantly faster.
   
   > compared to old array reader implementation
   
   That's the key thing about #1041 it doesn't replace this array reader implementation, it just adds the ability to extend it. For primitive types the performance of #1041 is therefore [unchanged](https://github.com/apache/arrow-rs/pull/1041#issuecomment-993816497), it just gives the ability to add optimisations such as #1054 and this PR


-- 
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 edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999804838






-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r776499818



##########
File path: parquet/src/arrow/array_reader.rs
##########
@@ -1663,69 +1699,58 @@ impl<'a> ArrayReaderBuilder {
                     arrow_type,
                 )?))
             }
-            PhysicalType::FLOAT => Ok(Box::new(PrimitiveArrayReader::<FloatType>::new(
-                page_iterator,
-                column_desc,
-                arrow_type,
-            )?)),
-            PhysicalType::DOUBLE => {
-                Ok(Box::new(PrimitiveArrayReader::<DoubleType>::new(
+            PhysicalType::FLOAT => Ok(Box::new(
+                PrimitiveArrayReader::<FloatType>::new_with_options(
                     page_iterator,
                     column_desc,
                     arrow_type,
-                )?))
-            }
-            PhysicalType::BYTE_ARRAY => {
-                if cur_type.get_basic_info().converted_type() == ConvertedType::UTF8 {
-                    if let Some(ArrowType::LargeUtf8) = arrow_type {
-                        let converter =
-                            LargeUtf8Converter::new(LargeUtf8ArrayConverter {});
-                        Ok(Box::new(ComplexObjectArrayReader::<
-                            ByteArrayType,
-                            LargeUtf8Converter,
-                        >::new(
-                            page_iterator,
-                            column_desc,
-                            converter,
-                            arrow_type,
-                        )?))
-                    } else {
-                        use crate::arrow::arrow_array_reader::{
-                            ArrowArrayReader, StringArrayConverter,
-                        };
-                        let converter = StringArrayConverter::new();
-                        Ok(Box::new(ArrowArrayReader::try_new(
-                            *page_iterator,
-                            column_desc,
-                            converter,
-                            arrow_type,
-                        )?))
+                    null_mask_only,
+                )?,
+            )),
+            PhysicalType::DOUBLE => Ok(Box::new(
+                PrimitiveArrayReader::<DoubleType>::new_with_options(
+                    page_iterator,
+                    column_desc,
+                    arrow_type,
+                    null_mask_only,
+                )?,
+            )),
+            PhysicalType::BYTE_ARRAY => match arrow_type {
+                // TODO: Replace with optimised dictionary reader (#171)
+                Some(ArrowType::Dictionary(_, _)) => {
+                    match cur_type.get_basic_info().converted_type() {
+                        ConvertedType::UTF8 => {
+                            let converter = Utf8Converter::new(Utf8ArrayConverter {});
+                            Ok(Box::new(ComplexObjectArrayReader::<
+                                ByteArrayType,
+                                Utf8Converter,
+                            >::new(
+                                page_iterator,
+                                column_desc,
+                                converter,
+                                arrow_type,
+                            )?))
+                        }
+                        _ => {
+                            let converter = BinaryConverter::new(BinaryArrayConverter {});
+                            Ok(Box::new(ComplexObjectArrayReader::<
+                                ByteArrayType,
+                                BinaryConverter,
+                            >::new(
+                                page_iterator,
+                                column_desc,
+                                converter,
+                                arrow_type,
+                            )?))
+                        }
                     }
-                } else if let Some(ArrowType::LargeBinary) = arrow_type {
-                    let converter =
-                        LargeBinaryConverter::new(LargeBinaryArrayConverter {});
-                    Ok(Box::new(ComplexObjectArrayReader::<
-                        ByteArrayType,
-                        LargeBinaryConverter,
-                    >::new(
-                        page_iterator,
-                        column_desc,
-                        converter,
-                        arrow_type,
-                    )?))
-                } else {
-                    let converter = BinaryConverter::new(BinaryArrayConverter {});
-                    Ok(Box::new(ComplexObjectArrayReader::<
-                        ByteArrayType,
-                        BinaryConverter,
-                    >::new(
-                        page_iterator,
-                        column_desc,
-                        converter,
-                        arrow_type,
-                    )?))
                 }
-            }
+                _ => Ok(Box::new(ByteArrayReader::new(
+                    page_iterator,
+                    column_desc,
+                    arrow_type,

Review comment:
       why not use `null_mask_only` here for `ByteArrayReader` as well?




-- 
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] codecov-commenter edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-998923929


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1082](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0ba1ca7) into [master](https://codecov.io/gh/apache/arrow-rs/commit/66b84f37e18e4ba5f9b3463f84f4bbbefb6c4341?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (66b84f3) will **decrease** coverage by `0.14%`.
   > The diff coverage is `73.12%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1082/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1082      +/-   ##
   ==========================================
   - Coverage   82.66%   82.51%   -0.15%     
   ==========================================
     Files         173      175       +2     
     Lines       50902    51200     +298     
   ==========================================
   + Hits        42077    42248     +171     
   - Misses       8825     8952     +127     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyLnJz) | `76.93% <37.03%> (-0.23%)` | :arrow_down: |
   | [parquet/src/arrow/record\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci5ycw==) | `94.02% <60.00%> (-0.73%)` | :arrow_down: |
   | [parquet/src/arrow/array\_reader/byte\_array.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL2J5dGVfYXJyYXkucnM=) | `69.08% <69.08%> (ø)` | |
   | [parquet/src/arrow/array\_reader/offset\_buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL29mZnNldF9idWZmZXIucnM=) | `88.46% <88.46%> (ø)` | |
   | [parquet/src/arrow/arrow\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfcmVhZGVyLnJz) | `91.86% <90.00%> (+0.25%)` | :arrow_up: |
   | [parquet/src/arrow/record\_reader/buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9idWZmZXIucnM=) | `89.23% <100.00%> (+3.23%)` | :arrow_up: |
   | [...rquet/src/arrow/record\_reader/definition\_levels.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9kZWZpbml0aW9uX2xldmVscy5ycw==) | `87.50% <100.00%> (-0.07%)` | :arrow_down: |
   | [parquet/src/arrow/arrow\_array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfYXJyYXlfcmVhZGVyLnJz) | `73.66% <0.00%> (-5.59%)` | :arrow_down: |
   | [parquet/src/arrow/converter.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvY29udmVydGVyLnJz) | `64.34% <0.00%> (-5.22%)` | :arrow_down: |
   | [parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvZmlsZS9zZXJpYWxpemVkX3JlYWRlci5ycw==) | `94.37% <0.00%> (-0.60%)` | :arrow_down: |
   | ... and [10 more](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [66b84f3...0ba1ca7](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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] codecov-commenter edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-998923929


   # [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1082](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (014be9f) into [master](https://codecov.io/gh/apache/arrow-rs/commit/66b84f37e18e4ba5f9b3463f84f4bbbefb6c4341?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (66b84f3) will **decrease** coverage by `0.13%`.
   > The diff coverage is `73.46%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-rs/pull/1082/graphs/tree.svg?width=650&height=150&src=pr&token=pq9V9qWZ1N&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #1082      +/-   ##
   ==========================================
   - Coverage   82.66%   82.52%   -0.14%     
   ==========================================
     Files         173      175       +2     
     Lines       50902    51201     +299     
   ==========================================
   + Hits        42077    42254     +177     
   - Misses       8825     8947     +122     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [parquet/src/arrow/array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyLnJz) | `76.93% <37.03%> (-0.23%)` | :arrow_down: |
   | [parquet/src/arrow/array\_reader/byte\_array.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL2J5dGVfYXJyYXkucnM=) | `69.08% <69.08%> (ø)` | |
   | [parquet/src/arrow/array\_reader/offset\_buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyYXlfcmVhZGVyL29mZnNldF9idWZmZXIucnM=) | `88.46% <88.46%> (ø)` | |
   | [parquet/src/arrow/arrow\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfcmVhZGVyLnJz) | `91.86% <90.00%> (+0.25%)` | :arrow_up: |
   | [parquet/src/arrow/record\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci5ycw==) | `94.79% <100.00%> (+0.03%)` | :arrow_up: |
   | [parquet/src/arrow/record\_reader/buffer.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvcmVjb3JkX3JlYWRlci9idWZmZXIucnM=) | `89.06% <100.00%> (+3.06%)` | :arrow_up: |
   | [parquet/src/arrow/arrow\_array\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvYXJyb3dfYXJyYXlfcmVhZGVyLnJz) | `73.66% <0.00%> (-5.59%)` | :arrow_down: |
   | [parquet/src/arrow/converter.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvYXJyb3cvY29udmVydGVyLnJz) | `64.34% <0.00%> (-5.22%)` | :arrow_down: |
   | [parquet/src/file/serialized\_reader.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGFycXVldC9zcmMvZmlsZS9zZXJpYWxpemVkX3JlYWRlci5ycw==) | `94.37% <0.00%> (-0.60%)` | :arrow_down: |
   | [arrow/src/datatypes/field.rs](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YXJyb3cvc3JjL2RhdGF0eXBlcy9maWVsZC5ycw==) | `53.79% <0.00%> (-0.31%)` | :arrow_down: |
   | ... and [9 more](https://codecov.io/gh/apache/arrow-rs/pull/1082/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [66b84f3...014be9f](https://codecov.io/gh/apache/arrow-rs/pull/1082?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785469777



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+            Reader::LargeBinary(r) | Reader::LargeUtf8(r) => {
+                read_records(r, self.pages.as_mut(), batch_size)?;
+                let data = consume_array_data(self.data_type.clone(), r)?;
+                self.def_levels_buffer = r.consume_def_levels()?;
+                self.rep_levels_buffer = r.consume_rep_levels()?;
+                r.reset();
+                data
+            }
+        };
+
+        Ok(match &self.record_reader {
+            Reader::Binary(_) => Arc::new(BinaryArray::from(data)),

Review comment:
       I ended up doing exactly this :+1: 




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785498869



##########
File path: parquet/benches/arrow_array_reader.rs
##########
@@ -273,7 +274,7 @@ fn build_dictionary_encoded_string_page_iterator(
     InMemoryPageIterator::new(schema, column_desc, pages)
 }
 
-fn bench_array_reader(mut array_reader: impl ArrayReader) -> usize {
+fn bench_array_reader(mut array_reader: Box<dyn ArrayReader>) -> usize {

Review comment:
       This change is necessary because byte_array_reader hides its implementing type, this is both to make the API more ergonomic for clients and also to aid future crate evolution




-- 
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] yordan-pavlov commented on pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-1014838087


   > What is the plan for the `ArrowArrayReader` implementation added in #384? Should we plan to remove it from this crate (if so I can file a ticket)
   
   I am happy for `ArrowArrayReader` to be removed - I have ran the benchmarks against the latest code and @tustvold 's work is now often several times faster in almost all cases and in the one or two cases where it isn't the difference is small - congratulations @tustvold ; plus I think @tustvold 's array reader could be made faster still


-- 
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] yordan-pavlov commented on a change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r773508032



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,639 @@
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::{RecordBuffer, TypedBuffer, ValueBuffer};
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::Encoding;
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::{ColumnValueDecoder, ValuesWriter};
+use crate::data_type::Int32Type;
+use crate::decoding::{Decoder, DeltaBitPackDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use arrow::array::{
+    ArrayData, ArrayDataBuilder, ArrayRef, BinaryArray, LargeBinaryArray,
+    LargeStringArray, OffsetSizeTrait, StringArray,
+};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+use std::sync::Arc;
+
+enum Reader {
+    Binary(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeBinary(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+    Utf8(GenericRecordReader<OffsetBuffer<i32>, ByteArrayDecoder<i32>>),
+    LargeUtf8(GenericRecordReader<OffsetBuffer<i64>, ByteArrayDecoder<i64>>),
+}
+
+fn consume_array_data<I: OffsetSizeTrait>(
+    data_type: ArrowType,
+    reader: &mut GenericRecordReader<OffsetBuffer<I>, ByteArrayDecoder<I>>,
+) -> Result<ArrayData> {
+    let buffer = reader.consume_record_data()?;
+    let mut array_data_builder = ArrayDataBuilder::new(data_type)
+        .len(buffer.len())
+        .add_buffer(buffer.offsets.into())
+        .add_buffer(buffer.values.into());
+
+    if let Some(buffer) = reader.consume_bitmap_buffer()? {
+        array_data_builder = array_data_builder.null_bit_buffer(buffer);
+    }
+    Ok(unsafe { array_data_builder.build_unchecked() })
+}
+
+pub struct ByteArrayReader {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    column_desc: ColumnDescPtr,
+    record_reader: Reader,
+}
+
+impl ByteArrayReader {
+    /// Construct primitive array reader.
+    pub fn new(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+    ) -> Result<Self> {
+        Self::new_with_options(pages, column_desc, arrow_type, false)
+    }
+
+    /// Construct primitive array reader with ability to only compute null mask and not
+    /// buffer level data
+    pub fn new_with_options(
+        pages: Box<dyn PageIterator>,
+        column_desc: ColumnDescPtr,
+        arrow_type: Option<ArrowType>,
+        null_mask_only: bool,
+    ) -> Result<Self> {
+        // Check if Arrow type is specified, else create it from Parquet type
+        let data_type = match arrow_type {
+            Some(t) => t,
+            None => parquet_to_arrow_field(column_desc.as_ref())?
+                .data_type()
+                .clone(),
+        };
+
+        let record_reader = match data_type {
+            ArrowType::Binary => Reader::Binary(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeBinary => {
+                Reader::LargeBinary(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            ArrowType::Utf8 => Reader::Utf8(GenericRecordReader::new_with_options(
+                column_desc.clone(),
+                null_mask_only,
+            )),
+            ArrowType::LargeUtf8 => {
+                Reader::LargeUtf8(GenericRecordReader::new_with_options(
+                    column_desc.clone(),
+                    null_mask_only,
+                ))
+            }
+            _ => {
+                return Err(general_err!(
+                    "invalid data type for ByteArrayReader - {}",
+                    data_type
+                ))
+            }
+        };
+
+        Ok(Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            column_desc,
+            record_reader,
+        })
+    }
+}
+
+impl ArrayReader for ByteArrayReader {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> crate::errors::Result<ArrayRef> {
+        let data = match &mut self.record_reader {
+            Reader::Binary(r) | Reader::Utf8(r) => {

Review comment:
       can't this choice be made when the `ByteArrayReader` is created instead of every time `next_batch` is called?




-- 
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] yordan-pavlov edited a comment on pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
yordan-pavlov edited a comment on pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#issuecomment-999877477


   @tustvold you are probably aware of this, but just to make sure it's not missed, when I run this branch with datafusion against a parquet file I get an error `Parquet argument error: Parquet error: unsupported encoding for byte array: PLAIN_DICTIONARY`
   
   Other than that, the performance benchmark results look impressive - I was able to run the benchmark and this branch is faster than the `ArrowArrayReader`, sometimes several times faster, in almost all cases (exceptions listed below). And the `ArrowArrayReader` was already several times faster in many cases than the old array reader implementation, making these performance results even more impressive.
   
   A major reason, why I only implemented `ArrowArrayReader` for string arrays is because I have been struggling to make it faster for dictionary-encoded primitive arrays, but it looks like this isn't going to be a problem with this new implementation.
   So if we can make it faster in all benchmarks, I am happy to abandon the `ArrowArrayReader` in favor of this new implementation.
   
   Where it is still a bit slower is in these two cases:
   
   read StringArray, plain encoded, mandatory, no NULLs - old: time:   [306.10 us 342.14 us 377.28 us]
   read StringArray, plain encoded, mandatory, no NULLs - new: time:   [310.84 us 337.49 us 368.74 us]
   
   read StringArray, dictionary encoded, mandatory, no NULLs - old: time:   [286.61 us 320.07 us 354.74 us]
   read StringArray, dictionary encoded, mandatory, no NULLs - new: time:   [222.87 us 240.56 us 260.93 us]
   
   The reason why `ArrowArrayReader` is fast in those cases, I suspect, is because when there are no nulls / def levels, the def level buffer are not read or processed at all, see here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L566 . This also means that the bit of code that produces the null bitmap also doesn't run, see here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L595 and the main path in the code is not concerned with null values at all, which is why it's so fast when there are no null / def levels, see here: https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L592 , see string converter here https://github.com/apache/arrow-rs/blob/master/parquet/src/arrow/arrow_array_reader.rs#L1164 .
   


-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785499448



##########
File path: parquet/src/arrow/array_reader.rs
##########
@@ -1749,57 +1753,43 @@ impl<'a> ArrayReaderBuilder {
                     null_mask_only,
                 )?,
             )),
-            PhysicalType::BYTE_ARRAY => {
-                if cur_type.get_basic_info().converted_type() == ConvertedType::UTF8 {
-                    if let Some(ArrowType::LargeUtf8) = arrow_type {
-                        let converter =
-                            LargeUtf8Converter::new(LargeUtf8ArrayConverter {});
-                        Ok(Box::new(ComplexObjectArrayReader::<
-                            ByteArrayType,
-                            LargeUtf8Converter,
-                        >::new(
-                            page_iterator,
-                            column_desc,
-                            converter,
-                            arrow_type,
-                        )?))
-                    } else {
-                        use crate::arrow::arrow_array_reader::{
-                            ArrowArrayReader, StringArrayConverter,
-                        };
-                        let converter = StringArrayConverter::new();
-                        Ok(Box::new(ArrowArrayReader::try_new(
-                            *page_iterator,
-                            column_desc,
-                            converter,
-                            arrow_type,
-                        )?))
+            PhysicalType::BYTE_ARRAY => match arrow_type {
+                // TODO: Replace with optimised dictionary reader (#171)

Review comment:
       See #1180 




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785493832



##########
File path: parquet/benches/arrow_array_reader.rs
##########
@@ -366,10 +366,10 @@ fn add_benches(c: &mut Criterion) {
                     mandatory_int32_column_desc.clone(),
                 );
                 count = bench_array_reader(array_reader);
+                assert_eq!(count, EXPECTED_VALUE_COUNT);

Review comment:
       This was so that you can run a subset of the benchmarks, let me try lifting them out of the iter block but still within the bench_function :thinking: 




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r785501427



##########
File path: parquet/src/arrow/arrow_reader.rs
##########
@@ -968,4 +969,40 @@ mod tests {
         assert_eq!(batch.num_rows(), 4);
         assert_eq!(batch.column(0).data().null_count(), 2);
     }
+
+    #[test]
+    fn test_invalid_utf8() {

Review comment:
       Test sourced from #786 




-- 
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 change in pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

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



##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {

Review comment:
       I almost wonder if this is valuable itself to put into the `arrow` crate and use to create `GenericStringArray`s from iterators of `&str` etc. Not for this PR, I am just musing

##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {

Review comment:
       https://doc.rust-lang.org/src/core/str/mod.rs.html#218 👍 

##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    /// Extends this buffer with a list of keys
+    ///
+    /// For each value `key` in `keys` this will insert
+    /// `&dict_values[dict_offsets[key]..dict_offsets[key+1]]`
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(
+        &mut self,
+        keys: &[K],
+        dict_offsets: &[V],
+        dict_values: &[u8],
+    ) -> Result<()> {
+        for key in keys {
+            let index = key.to_usize().unwrap();
+            if index + 1 >= dict_offsets.len() {
+                return Err(general_err!("invalid offset in byte array: {}", index));
+            }
+            let start_offset = dict_offsets[index].to_usize().unwrap();
+            let end_offset = dict_offsets[index + 1].to_usize().unwrap();
+
+            // Dictionary values are verified when decoding dictionary page
+            self.try_push(&dict_values[start_offset..end_offset], false)?;
+        }
+        Ok(())
+    }
+
+    /// Validates that `&self.values[start_offset..]` is a valid UTF-8 sequence
+    ///
+    /// This MUST be combined with validating that the offsets start on a character
+    /// boundary, otherwise it would be possible for the values array to be a valid UTF-8
+    /// sequence, but not the individual string slices it contains
+    ///
+    /// [`Self::try_push`] can perform this validation check on insertion
+    pub fn check_valid_utf8(&self, start_offset: usize) -> Result<()> {
+        match std::str::from_utf8(&self.values.as_slice()[start_offset..]) {
+            Ok(_) => Ok(()),
+            Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)),
+        }
+    }
+
+    /// Converts this into an [`ArrayRef`] with the provided `data_type` and `null_buffer`
+    pub fn into_array(
+        self,
+        null_buffer: Option<Buffer>,
+        data_type: ArrowType,
+    ) -> ArrayRef {
+        let mut array_data_builder = ArrayDataBuilder::new(data_type)
+            .len(self.len())
+            .add_buffer(self.offsets.into())
+            .add_buffer(self.values.into());
+
+        if let Some(buffer) = null_buffer {
+            array_data_builder = array_data_builder.null_bit_buffer(buffer);
+        }
+
+        let data = match cfg!(debug_assertions) {
+            true => array_data_builder.build().unwrap(),
+            false => unsafe { array_data_builder.build_unchecked() },
+        };
+
+        make_array(data)
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> BufferQueue for OffsetBuffer<I> {
+    type Output = Self;
+    type Slice = Self;
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;

Review comment:
       recommend an assert here that `self.offsets.len() > len` for clarity, but I think that the `offsets[len]` would panic below if this were not the case, so I don't think it is a safety issue

##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    /// Extends this buffer with a list of keys
+    ///
+    /// For each value `key` in `keys` this will insert
+    /// `&dict_values[dict_offsets[key]..dict_offsets[key+1]]`
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(
+        &mut self,
+        keys: &[K],
+        dict_offsets: &[V],
+        dict_values: &[u8],
+    ) -> Result<()> {
+        for key in keys {
+            let index = key.to_usize().unwrap();
+            if index + 1 >= dict_offsets.len() {
+                return Err(general_err!("invalid offset in byte array: {}", index));
+            }
+            let start_offset = dict_offsets[index].to_usize().unwrap();
+            let end_offset = dict_offsets[index + 1].to_usize().unwrap();
+
+            // Dictionary values are verified when decoding dictionary page
+            self.try_push(&dict_values[start_offset..end_offset], false)?;
+        }
+        Ok(())
+    }
+
+    /// Validates that `&self.values[start_offset..]` is a valid UTF-8 sequence
+    ///
+    /// This MUST be combined with validating that the offsets start on a character
+    /// boundary, otherwise it would be possible for the values array to be a valid UTF-8
+    /// sequence, but not the individual string slices it contains
+    ///
+    /// [`Self::try_push`] can perform this validation check on insertion
+    pub fn check_valid_utf8(&self, start_offset: usize) -> Result<()> {
+        match std::str::from_utf8(&self.values.as_slice()[start_offset..]) {
+            Ok(_) => Ok(()),
+            Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)),
+        }
+    }
+
+    /// Converts this into an [`ArrayRef`] with the provided `data_type` and `null_buffer`
+    pub fn into_array(
+        self,
+        null_buffer: Option<Buffer>,
+        data_type: ArrowType,
+    ) -> ArrayRef {
+        let mut array_data_builder = ArrayDataBuilder::new(data_type)
+            .len(self.len())
+            .add_buffer(self.offsets.into())
+            .add_buffer(self.values.into());
+
+        if let Some(buffer) = null_buffer {
+            array_data_builder = array_data_builder.null_bit_buffer(buffer);
+        }
+
+        let data = match cfg!(debug_assertions) {
+            true => array_data_builder.build().unwrap(),
+            false => unsafe { array_data_builder.build_unchecked() },
+        };
+
+        make_array(data)
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> BufferQueue for OffsetBuffer<I> {
+    type Output = Self;
+    type Slice = Self;
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = ScalarBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)
+        }
+
+        self.offsets.resize(len + 1);
+
+        Self {
+            offsets: std::mem::replace(&mut self.offsets, new_offsets),
+            values: self.values.take(end_offset.to_usize().unwrap()),

Review comment:
       I found it a little confusing that `values.take()` does the same thing as `split_off` -- maybe it is worth renaming `ScalarBuffer<T>::take()` to `ScalarBuffer<T>::split_off()`?

##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {

Review comment:
       for anyone else following along, I double checked the code and `validate_utf8` is disabled for `DataType::Binary` as one would expect. It is always enabled for `DataType::Utf8`

##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    /// Extends this buffer with a list of keys
+    ///
+    /// For each value `key` in `keys` this will insert
+    /// `&dict_values[dict_offsets[key]..dict_offsets[key+1]]`
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(
+        &mut self,
+        keys: &[K],
+        dict_offsets: &[V],
+        dict_values: &[u8],
+    ) -> Result<()> {
+        for key in keys {
+            let index = key.to_usize().unwrap();
+            if index + 1 >= dict_offsets.len() {
+                return Err(general_err!("invalid offset in byte array: {}", index));
+            }
+            let start_offset = dict_offsets[index].to_usize().unwrap();
+            let end_offset = dict_offsets[index + 1].to_usize().unwrap();
+
+            // Dictionary values are verified when decoding dictionary page
+            self.try_push(&dict_values[start_offset..end_offset], false)?;
+        }
+        Ok(())
+    }
+
+    /// Validates that `&self.values[start_offset..]` is a valid UTF-8 sequence
+    ///
+    /// This MUST be combined with validating that the offsets start on a character
+    /// boundary, otherwise it would be possible for the values array to be a valid UTF-8
+    /// sequence, but not the individual string slices it contains
+    ///
+    /// [`Self::try_push`] can perform this validation check on insertion
+    pub fn check_valid_utf8(&self, start_offset: usize) -> Result<()> {
+        match std::str::from_utf8(&self.values.as_slice()[start_offset..]) {
+            Ok(_) => Ok(()),
+            Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)),
+        }
+    }
+
+    /// Converts this into an [`ArrayRef`] with the provided `data_type` and `null_buffer`
+    pub fn into_array(
+        self,
+        null_buffer: Option<Buffer>,
+        data_type: ArrowType,
+    ) -> ArrayRef {
+        let mut array_data_builder = ArrayDataBuilder::new(data_type)
+            .len(self.len())
+            .add_buffer(self.offsets.into())
+            .add_buffer(self.values.into());
+
+        if let Some(buffer) = null_buffer {
+            array_data_builder = array_data_builder.null_bit_buffer(buffer);
+        }
+
+        let data = match cfg!(debug_assertions) {
+            true => array_data_builder.build().unwrap(),
+            false => unsafe { array_data_builder.build_unchecked() },
+        };
+
+        make_array(data)
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> BufferQueue for OffsetBuffer<I> {
+    type Output = Self;
+    type Slice = Self;
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = ScalarBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)
+        }
+
+        self.offsets.resize(len + 1);
+
+        Self {
+            offsets: std::mem::replace(&mut self.offsets, new_offsets),
+            values: self.values.take(end_offset.to_usize().unwrap()),
+        }
+    }
+
+    fn spare_capacity_mut(&mut self, _batch_size: usize) -> &mut Self::Slice {
+        self
+    }
+
+    fn set_len(&mut self, len: usize) {
+        assert_eq!(self.offsets.len(), len + 1);
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ValuesBuffer for OffsetBuffer<I> {
+    fn pad_nulls(
+        &mut self,
+        read_offset: usize,
+        values_read: usize,
+        levels_read: usize,
+        rev_position_iter: impl Iterator<Item = usize>,
+    ) {
+        assert_eq!(self.offsets.len(), read_offset + values_read + 1);
+        self.offsets.resize(read_offset + levels_read + 1);
+
+        let offsets = self.offsets.as_slice_mut();
+
+        let mut last_pos = read_offset + levels_read + 1;
+        let mut last_start_offset = I::from_usize(self.values.len()).unwrap();
+
+        let values_range = read_offset..read_offset + values_read;
+        for (value_pos, level_pos) in values_range.clone().rev().zip(rev_position_iter) {
+            assert!(level_pos >= value_pos);

Review comment:
       this is definitely a tricky bit of logic, looks reasonable to me

##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,669 @@
+// 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::arrow::array_reader::offset_buffer::OffsetBuffer;
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::ScalarValue;
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::{ConvertedType, Encoding};
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::ColumnValueDecoder;
+use crate::data_type::Int32Type;
+use crate::encodings::{
+    decoding::{Decoder, DeltaBitPackDecoder},
+    rle::RleDecoder,
+};
+use crate::errors::{ParquetError, Result};
+use crate::schema::types::ColumnDescPtr;
+use crate::util::memory::ByteBufferPtr;
+use arrow::array::{ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+
+/// Returns an [`ArrayReader`] that decodes the provided byte array column
+pub fn make_byte_array_reader(
+    pages: Box<dyn PageIterator>,
+    column_desc: ColumnDescPtr,
+    arrow_type: Option<ArrowType>,
+    null_mask_only: bool,
+) -> Result<Box<dyn ArrayReader>> {
+    // Check if Arrow type is specified, else create it from Parquet type
+    let data_type = match arrow_type {
+        Some(t) => t,
+        None => parquet_to_arrow_field(column_desc.as_ref())?
+            .data_type()
+            .clone(),
+    };
+
+    match data_type {
+        ArrowType::Binary | ArrowType::Utf8 => {
+            let reader =
+                GenericRecordReader::new_with_options(column_desc, null_mask_only);
+            Ok(Box::new(ByteArrayReader::<i32>::new(
+                pages, data_type, reader,
+            )))
+        }
+        ArrowType::LargeUtf8 | ArrowType::LargeBinary => {
+            let reader =
+                GenericRecordReader::new_with_options(column_desc, null_mask_only);
+            Ok(Box::new(ByteArrayReader::<i64>::new(
+                pages, data_type, reader,
+            )))
+        }
+        _ => Err(general_err!(
+            "invalid data type for byte array reader - {}",
+            data_type
+        )),
+    }
+}
+
+/// An [`ArrayReader`] for variable length byte arrays
+struct ByteArrayReader<I: ScalarValue> {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    record_reader: GenericRecordReader<OffsetBuffer<I>, ByteArrayColumnValueDecoder<I>>,
+}
+
+impl<I: ScalarValue> ByteArrayReader<I> {
+    fn new(
+        pages: Box<dyn PageIterator>,
+        data_type: ArrowType,
+        record_reader: GenericRecordReader<
+            OffsetBuffer<I>,
+            ByteArrayColumnValueDecoder<I>,
+        >,
+    ) -> Self {
+        Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            record_reader,
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ArrayReader for ByteArrayReader<I> {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> Result<ArrayRef> {
+        read_records(&mut self.record_reader, self.pages.as_mut(), batch_size)?;
+        let buffer = self.record_reader.consume_record_data()?;
+        let null_buffer = self.record_reader.consume_bitmap_buffer()?;
+        self.def_levels_buffer = self.record_reader.consume_def_levels()?;
+        self.rep_levels_buffer = self.record_reader.consume_rep_levels()?;
+        self.record_reader.reset();
+
+        Ok(buffer.into_array(null_buffer, self.data_type.clone()))
+    }
+
+    fn get_def_levels(&self) -> Option<&[i16]> {
+        self.def_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+
+    fn get_rep_levels(&self) -> Option<&[i16]> {
+        self.rep_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+}
+
+/// A [`ColumnValueDecoder`] for variable length byte arrays
+struct ByteArrayColumnValueDecoder<I: ScalarValue> {
+    dict: Option<OffsetBuffer<I>>,
+    decoder: Option<ByteArrayDecoder>,
+    validate_utf8: bool,
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ColumnValueDecoder
+    for ByteArrayColumnValueDecoder<I>
+{
+    type Slice = OffsetBuffer<I>;
+
+    fn new(desc: &ColumnDescPtr) -> Self {
+        let validate_utf8 = desc.converted_type() == ConvertedType::UTF8;
+        Self {
+            dict: None,
+            decoder: None,
+            validate_utf8,
+        }
+    }
+
+    fn set_dict(
+        &mut self,
+        buf: ByteBufferPtr,
+        num_values: u32,
+        encoding: Encoding,
+        _is_sorted: bool,
+    ) -> Result<()> {
+        if !matches!(
+            encoding,
+            Encoding::PLAIN | Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY
+        ) {
+            return Err(nyi_err!(
+                "Invalid/Unsupported encoding type for dictionary: {}",
+                encoding
+            ));
+        }
+
+        let mut buffer = OffsetBuffer::default();
+        let mut decoder = ByteArrayDecoderPlain::new(
+            buf,
+            num_values as usize,
+            Some(num_values as usize),
+            self.validate_utf8,
+        );
+        decoder.read(&mut buffer, usize::MAX)?;
+        self.dict = Some(buffer);
+        Ok(())
+    }
+
+    fn set_data(
+        &mut self,
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+    ) -> Result<()> {
+        self.decoder = Some(ByteArrayDecoder::new(
+            encoding,
+            data,
+            num_levels,
+            num_values,
+            self.validate_utf8,
+        )?);
+        Ok(())
+    }
+
+    fn read(&mut self, out: &mut Self::Slice, range: Range<usize>) -> Result<usize> {
+        self.decoder.as_mut().expect("decoder set").read(
+            out,
+            range.end - range.start,
+            self.dict.as_ref(),
+        )
+    }
+}
+
+/// A generic decoder from uncompressed parquet value data to [`OffsetBuffer`]
+pub enum ByteArrayDecoder {
+    Plain(ByteArrayDecoderPlain),
+    Dictionary(ByteArrayDecoderDictionary),
+    DeltaLength(ByteArrayDecoderDeltaLength),
+    DeltaByteArray(ByteArrayDecoderDelta),
+}
+
+impl ByteArrayDecoder {
+    pub fn new(
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+        validate_utf8: bool,
+    ) -> Result<Self> {
+        let decoder = match encoding {
+            Encoding::PLAIN => ByteArrayDecoder::Plain(ByteArrayDecoderPlain::new(
+                data,
+                num_levels,
+                num_values,
+                validate_utf8,
+            )),
+            Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY => {
+                ByteArrayDecoder::Dictionary(ByteArrayDecoderDictionary::new(
+                    data, num_levels, num_values,
+                ))
+            }
+            Encoding::DELTA_LENGTH_BYTE_ARRAY => ByteArrayDecoder::DeltaLength(
+                ByteArrayDecoderDeltaLength::new(data, validate_utf8)?,
+            ),
+            Encoding::DELTA_BYTE_ARRAY => ByteArrayDecoder::DeltaByteArray(
+                ByteArrayDecoderDelta::new(data, validate_utf8)?,
+            ),
+            _ => {
+                return Err(general_err!(
+                    "unsupported encoding for byte array: {}",
+                    encoding
+                ))
+            }
+        };
+
+        Ok(decoder)
+    }
+
+    /// Read up to `len` values to `out` with the optional dictionary
+    pub fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        out: &mut OffsetBuffer<I>,
+        len: usize,
+        dict: Option<&OffsetBuffer<I>>,
+    ) -> Result<usize> {
+        match self {
+            ByteArrayDecoder::Plain(d) => d.read(out, len),
+            ByteArrayDecoder::Dictionary(d) => {
+                let dict = dict.expect("dictionary set");
+                d.read(out, dict, len)
+            }
+            ByteArrayDecoder::DeltaLength(d) => d.read(out, len),
+            ByteArrayDecoder::DeltaByteArray(d) => d.read(out, len),
+        }
+    }
+}
+
+/// Decoder from [`Encoding::PLAIN`] data to [`OffsetBuffer`]
+pub struct ByteArrayDecoderPlain {
+    buf: ByteBufferPtr,
+    offset: usize,
+    validate_utf8: bool,
+
+    /// This is a maximum as the null count is not always known, e.g. value data from
+    /// a v1 data page
+    max_remaining_values: usize,
+}
+
+impl ByteArrayDecoderPlain {
+    pub fn new(
+        buf: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+        validate_utf8: bool,
+    ) -> Self {
+        Self {
+            buf,
+            validate_utf8,
+            offset: 0,
+            max_remaining_values: num_values.unwrap_or(num_levels),
+        }
+    }
+
+    pub fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+
+        let to_read = len.min(self.max_remaining_values);
+        output.offsets.reserve(to_read);
+
+        let remaining_bytes = self.buf.len() - self.offset;
+        if remaining_bytes == 0 {
+            return Ok(0);
+        }
+
+        let estimated_bytes = remaining_bytes
+            .checked_mul(to_read)
+            .map(|x| x / self.max_remaining_values)
+            .unwrap_or_default();
+
+        output.values.reserve(estimated_bytes);
+
+        let mut read = 0;
+
+        let buf = self.buf.as_ref();
+        while self.offset < self.buf.len() && read != to_read {
+            if self.offset + 4 > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+            let len_bytes: [u8; 4] =
+                buf[self.offset..self.offset + 4].try_into().unwrap();
+            let len = u32::from_le_bytes(len_bytes);
+
+            let start_offset = self.offset + 4;
+            let end_offset = start_offset + len as usize;
+            if end_offset > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            output.try_push(&buf[start_offset..end_offset], self.validate_utf8)?;
+
+            self.offset = end_offset;
+            read += 1;
+        }
+        self.max_remaining_values -= to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::DELTA_LENGTH_BYTE_ARRAY`] data to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDeltaLength {
+    lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    validate_utf8: bool,
+}
+
+impl ByteArrayDecoderDeltaLength {
+    fn new(data: ByteBufferPtr, validate_utf8: bool) -> Result<Self> {
+        let mut len_decoder = DeltaBitPackDecoder::<Int32Type>::new();
+        len_decoder.set_data(data.all(), 0)?;
+        let values = len_decoder.values_left();
+
+        let mut lengths = vec![0; values];
+        len_decoder.get(&mut lengths)?;
+
+        Ok(Self {
+            lengths,
+            data,
+            validate_utf8,
+            length_offset: 0,
+            data_offset: len_decoder.get_offset(),
+        })
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+
+        let to_read = len.min(self.lengths.len() - self.length_offset);
+        output.offsets.reserve(to_read);
+
+        let src_lengths = &self.lengths[self.length_offset..self.length_offset + to_read];
+
+        let total_bytes: usize = src_lengths.iter().map(|x| *x as usize).sum();
+        output.values.reserve(total_bytes);
+
+        if self.data_offset + total_bytes > self.data.len() {
+            return Err(ParquetError::EOF(
+                "Insufficient delta length byte array bytes".to_string(),
+            ));
+        }
+
+        let mut start_offset = self.data_offset;
+        for length in src_lengths {
+            let end_offset = start_offset + *length as usize;
+            output.try_push(
+                &self.data.as_ref()[start_offset..end_offset],
+                self.validate_utf8,
+            )?;
+            start_offset = end_offset;
+        }
+
+        self.data_offset = start_offset;
+        self.length_offset += to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::DELTA_BYTE_ARRAY`] to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDelta {
+    prefix_lengths: Vec<i32>,
+    suffix_lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    last_value: Vec<u8>,
+    validate_utf8: bool,
+}
+
+impl ByteArrayDecoderDelta {
+    fn new(data: ByteBufferPtr, validate_utf8: bool) -> Result<Self> {
+        let mut prefix = DeltaBitPackDecoder::<Int32Type>::new();
+        prefix.set_data(data.all(), 0)?;
+
+        let num_prefix = prefix.values_left();
+        let mut prefix_lengths = vec![0; num_prefix];
+        assert_eq!(prefix.get(&mut prefix_lengths)?, num_prefix);
+
+        let mut suffix = DeltaBitPackDecoder::<Int32Type>::new();
+        suffix.set_data(data.start_from(prefix.get_offset()), 0)?;
+
+        let num_suffix = suffix.values_left();
+        let mut suffix_lengths = vec![0; num_suffix];
+        assert_eq!(suffix.get(&mut suffix_lengths)?, num_suffix);
+
+        if num_prefix != num_suffix {
+            return Err(general_err!(format!(
+                "inconsistent DELTA_BYTE_ARRAY lengths, prefixes: {}, suffixes: {}",
+                num_prefix, num_suffix
+            )));
+        }
+
+        Ok(Self {
+            prefix_lengths,
+            suffix_lengths,
+            data,
+            length_offset: 0,
+            data_offset: prefix.get_offset() + suffix.get_offset(),
+            last_value: vec![],
+            validate_utf8,
+        })
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+        assert_eq!(self.prefix_lengths.len(), self.suffix_lengths.len());
+
+        let to_read = len.min(self.prefix_lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let length_range = self.length_offset..self.length_offset + to_read;
+        let iter = self.prefix_lengths[length_range.clone()]
+            .iter()
+            .zip(&self.suffix_lengths[length_range]);
+
+        let data = self.data.as_ref();
+
+        for (prefix_length, suffix_length) in iter {
+            let prefix_length = *prefix_length as usize;
+            let suffix_length = *suffix_length as usize;
+
+            if self.data_offset + suffix_length > self.data.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            self.last_value.truncate(prefix_length);
+            self.last_value.extend_from_slice(
+                &data[self.data_offset..self.data_offset + suffix_length],
+            );
+            output.try_push(&self.last_value, self.validate_utf8)?;
+
+            self.data_offset += suffix_length;
+        }
+
+        self.length_offset += to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::RLE_DICTIONARY`] to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDictionary {
+    decoder: RleDecoder,
+
+    index_buf: Box<[i32; 1024]>,
+    index_buf_len: usize,
+    index_offset: usize,
+
+    /// This is a maximum as the null count is not always known, e.g. value data from
+    /// a v1 data page
+    max_remaining_values: usize,
+}
+
+impl ByteArrayDecoderDictionary {
+    fn new(data: ByteBufferPtr, num_levels: usize, num_values: Option<usize>) -> Self {
+        let bit_width = data[0];
+        let mut decoder = RleDecoder::new(bit_width);
+        decoder.set_data(data.start_from(1));
+
+        Self {
+            decoder,
+            index_buf: Box::new([0; 1024]),
+            index_buf_len: 0,
+            index_offset: 0,
+            max_remaining_values: num_values.unwrap_or(num_levels),
+        }
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        dict: &OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let mut values_read = 0;
+
+        while values_read != len && self.max_remaining_values != 0 {
+            if self.index_offset == self.index_buf_len {
+                let read = self.decoder.get_batch(self.index_buf.as_mut())?;
+                if read == 0 {
+                    break;
+                }
+                self.index_buf_len = read;
+                self.index_offset = 0;
+            }
+
+            let to_read = (len - values_read)
+                .min(self.index_buf_len - self.index_offset)
+                .min(self.max_remaining_values);
+
+            output.extend_from_dictionary(
+                &self.index_buf[self.index_offset..self.index_offset + to_read],
+                dict.offsets.as_slice(),
+                dict.values.as_slice(),
+            )?;
+
+            self.index_offset += to_read;
+            values_read += to_read;
+            self.max_remaining_values -= to_read;
+        }
+        Ok(values_read)
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::basic::Type as PhysicalType;
+    use crate::data_type::{ByteArray, ByteArrayType};
+    use crate::encodings::encoding::{get_encoder, DictEncoder, Encoder};
+    use crate::schema::types::{ColumnDescriptor, ColumnPath, Type};
+    use crate::util::memory::MemTracker;
+    use std::sync::Arc;
+
+    fn column() -> ColumnDescPtr {
+        let t = Type::primitive_type_builder("col", PhysicalType::BYTE_ARRAY)
+            .with_converted_type(ConvertedType::UTF8)
+            .build()
+            .unwrap();
+
+        Arc::new(ColumnDescriptor::new(
+            Arc::new(t),
+            1,
+            0,
+            ColumnPath::new(vec![]),
+        ))
+    }
+
+    fn get_encoded(encoding: Encoding, data: &[ByteArray]) -> ByteBufferPtr {
+        let descriptor = column();
+        let mem_tracker = Arc::new(MemTracker::new());
+        let mut encoder =
+            get_encoder::<ByteArrayType>(descriptor, encoding, mem_tracker).unwrap();
+
+        encoder.put(data).unwrap();
+        encoder.flush_buffer().unwrap()
+    }
+
+    #[test]
+    fn test_byte_array_decoder() {

Review comment:
       Is NULL covered anywhere? If not I think that might be valuable to cover here too

##########
File path: parquet/src/arrow/array_reader/offset_buffer.rs
##########
@@ -0,0 +1,337 @@
+// 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::arrow::record_reader::buffer::{
+    BufferQueue, ScalarBuffer, ScalarValue, ValuesBuffer,
+};
+use crate::column::reader::decoder::ValuesBufferSlice;
+use crate::errors::{ParquetError, Result};
+use arrow::array::{make_array, ArrayDataBuilder, ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::{ArrowNativeType, DataType as ArrowType};
+
+/// A buffer of variable-sized byte arrays that can be converted into
+/// a corresponding [`ArrayRef`]
+pub struct OffsetBuffer<I: ScalarValue> {
+    pub offsets: ScalarBuffer<I>,
+    pub values: ScalarBuffer<u8>,
+}
+
+impl<I: ScalarValue> Default for OffsetBuffer<I> {
+    fn default() -> Self {
+        let mut offsets = ScalarBuffer::new();
+        offsets.resize(1);
+        Self {
+            offsets,
+            values: ScalarBuffer::new(),
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
+    /// Returns the number of byte arrays in this buffer
+    pub fn len(&self) -> usize {
+        self.offsets.len() - 1
+    }
+
+    /// If `validate_utf8` this verifies that the first character of `data` is
+    /// the start of a UTF-8 codepoint
+    ///
+    /// Note: This does not verify that the entirety of `data` is valid
+    /// UTF-8. This should be done by calling [`Self::values_as_str`] after
+    /// all data has been written
+    pub fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Some(&b) = data.first() {
+                // A valid code-point iff it does not start with 0b10xxxxxx
+                // Bit-magic taken from `std::str::is_char_boundary`
+                if (b as i8) < -0x40 {
+                    return Err(ParquetError::General(
+                        "encountered non UTF-8 data".to_string(),
+                    ));
+                }
+            }
+        }
+
+        self.values.extend_from_slice(data);
+
+        let index_offset = I::from_usize(self.values.len())
+            .ok_or_else(|| general_err!("index overflow decoding byte array"))?;
+
+        self.offsets.push(index_offset);
+        Ok(())
+    }
+
+    /// Extends this buffer with a list of keys
+    ///
+    /// For each value `key` in `keys` this will insert
+    /// `&dict_values[dict_offsets[key]..dict_offsets[key+1]]`
+    pub fn extend_from_dictionary<K: ArrowNativeType, V: ArrowNativeType>(
+        &mut self,
+        keys: &[K],
+        dict_offsets: &[V],
+        dict_values: &[u8],
+    ) -> Result<()> {
+        for key in keys {
+            let index = key.to_usize().unwrap();
+            if index + 1 >= dict_offsets.len() {
+                return Err(general_err!("invalid offset in byte array: {}", index));
+            }
+            let start_offset = dict_offsets[index].to_usize().unwrap();
+            let end_offset = dict_offsets[index + 1].to_usize().unwrap();
+
+            // Dictionary values are verified when decoding dictionary page
+            self.try_push(&dict_values[start_offset..end_offset], false)?;
+        }
+        Ok(())
+    }
+
+    /// Validates that `&self.values[start_offset..]` is a valid UTF-8 sequence
+    ///
+    /// This MUST be combined with validating that the offsets start on a character
+    /// boundary, otherwise it would be possible for the values array to be a valid UTF-8
+    /// sequence, but not the individual string slices it contains
+    ///
+    /// [`Self::try_push`] can perform this validation check on insertion
+    pub fn check_valid_utf8(&self, start_offset: usize) -> Result<()> {
+        match std::str::from_utf8(&self.values.as_slice()[start_offset..]) {
+            Ok(_) => Ok(()),
+            Err(e) => Err(general_err!("encountered non UTF-8 data: {}", e)),
+        }
+    }
+
+    /// Converts this into an [`ArrayRef`] with the provided `data_type` and `null_buffer`
+    pub fn into_array(
+        self,
+        null_buffer: Option<Buffer>,
+        data_type: ArrowType,
+    ) -> ArrayRef {
+        let mut array_data_builder = ArrayDataBuilder::new(data_type)
+            .len(self.len())
+            .add_buffer(self.offsets.into())
+            .add_buffer(self.values.into());
+
+        if let Some(buffer) = null_buffer {
+            array_data_builder = array_data_builder.null_bit_buffer(buffer);
+        }
+
+        let data = match cfg!(debug_assertions) {
+            true => array_data_builder.build().unwrap(),
+            false => unsafe { array_data_builder.build_unchecked() },
+        };
+
+        make_array(data)
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> BufferQueue for OffsetBuffer<I> {
+    type Output = Self;
+    type Slice = Self;
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        let remaining_offsets = self.offsets.len() - len - 1;
+        let offsets = self.offsets.as_slice();
+
+        let end_offset = offsets[len];
+
+        let mut new_offsets = ScalarBuffer::new();
+        new_offsets.reserve(remaining_offsets + 1);
+        for v in &offsets[len..] {
+            new_offsets.push(*v - end_offset)

Review comment:
       nice




-- 
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 change in pull request #1082: parquet: Optimized ByteArrayReader, Add UTF-8 Validation (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r786177173



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -0,0 +1,669 @@
+// 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::arrow::array_reader::offset_buffer::OffsetBuffer;
+use crate::arrow::array_reader::{read_records, ArrayReader};
+use crate::arrow::record_reader::buffer::ScalarValue;
+use crate::arrow::record_reader::GenericRecordReader;
+use crate::arrow::schema::parquet_to_arrow_field;
+use crate::basic::{ConvertedType, Encoding};
+use crate::column::page::PageIterator;
+use crate::column::reader::decoder::ColumnValueDecoder;
+use crate::data_type::Int32Type;
+use crate::encodings::{
+    decoding::{Decoder, DeltaBitPackDecoder},
+    rle::RleDecoder,
+};
+use crate::errors::{ParquetError, Result};
+use crate::schema::types::ColumnDescPtr;
+use crate::util::memory::ByteBufferPtr;
+use arrow::array::{ArrayRef, OffsetSizeTrait};
+use arrow::buffer::Buffer;
+use arrow::datatypes::DataType as ArrowType;
+use std::any::Any;
+use std::ops::Range;
+
+/// Returns an [`ArrayReader`] that decodes the provided byte array column
+pub fn make_byte_array_reader(
+    pages: Box<dyn PageIterator>,
+    column_desc: ColumnDescPtr,
+    arrow_type: Option<ArrowType>,
+    null_mask_only: bool,
+) -> Result<Box<dyn ArrayReader>> {
+    // Check if Arrow type is specified, else create it from Parquet type
+    let data_type = match arrow_type {
+        Some(t) => t,
+        None => parquet_to_arrow_field(column_desc.as_ref())?
+            .data_type()
+            .clone(),
+    };
+
+    match data_type {
+        ArrowType::Binary | ArrowType::Utf8 => {
+            let reader =
+                GenericRecordReader::new_with_options(column_desc, null_mask_only);
+            Ok(Box::new(ByteArrayReader::<i32>::new(
+                pages, data_type, reader,
+            )))
+        }
+        ArrowType::LargeUtf8 | ArrowType::LargeBinary => {
+            let reader =
+                GenericRecordReader::new_with_options(column_desc, null_mask_only);
+            Ok(Box::new(ByteArrayReader::<i64>::new(
+                pages, data_type, reader,
+            )))
+        }
+        _ => Err(general_err!(
+            "invalid data type for byte array reader - {}",
+            data_type
+        )),
+    }
+}
+
+/// An [`ArrayReader`] for variable length byte arrays
+struct ByteArrayReader<I: ScalarValue> {
+    data_type: ArrowType,
+    pages: Box<dyn PageIterator>,
+    def_levels_buffer: Option<Buffer>,
+    rep_levels_buffer: Option<Buffer>,
+    record_reader: GenericRecordReader<OffsetBuffer<I>, ByteArrayColumnValueDecoder<I>>,
+}
+
+impl<I: ScalarValue> ByteArrayReader<I> {
+    fn new(
+        pages: Box<dyn PageIterator>,
+        data_type: ArrowType,
+        record_reader: GenericRecordReader<
+            OffsetBuffer<I>,
+            ByteArrayColumnValueDecoder<I>,
+        >,
+    ) -> Self {
+        Self {
+            data_type,
+            pages,
+            def_levels_buffer: None,
+            rep_levels_buffer: None,
+            record_reader,
+        }
+    }
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ArrayReader for ByteArrayReader<I> {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn get_data_type(&self) -> &ArrowType {
+        &self.data_type
+    }
+
+    fn next_batch(&mut self, batch_size: usize) -> Result<ArrayRef> {
+        read_records(&mut self.record_reader, self.pages.as_mut(), batch_size)?;
+        let buffer = self.record_reader.consume_record_data()?;
+        let null_buffer = self.record_reader.consume_bitmap_buffer()?;
+        self.def_levels_buffer = self.record_reader.consume_def_levels()?;
+        self.rep_levels_buffer = self.record_reader.consume_rep_levels()?;
+        self.record_reader.reset();
+
+        Ok(buffer.into_array(null_buffer, self.data_type.clone()))
+    }
+
+    fn get_def_levels(&self) -> Option<&[i16]> {
+        self.def_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+
+    fn get_rep_levels(&self) -> Option<&[i16]> {
+        self.rep_levels_buffer
+            .as_ref()
+            .map(|buf| unsafe { buf.typed_data() })
+    }
+}
+
+/// A [`ColumnValueDecoder`] for variable length byte arrays
+struct ByteArrayColumnValueDecoder<I: ScalarValue> {
+    dict: Option<OffsetBuffer<I>>,
+    decoder: Option<ByteArrayDecoder>,
+    validate_utf8: bool,
+}
+
+impl<I: OffsetSizeTrait + ScalarValue> ColumnValueDecoder
+    for ByteArrayColumnValueDecoder<I>
+{
+    type Slice = OffsetBuffer<I>;
+
+    fn new(desc: &ColumnDescPtr) -> Self {
+        let validate_utf8 = desc.converted_type() == ConvertedType::UTF8;
+        Self {
+            dict: None,
+            decoder: None,
+            validate_utf8,
+        }
+    }
+
+    fn set_dict(
+        &mut self,
+        buf: ByteBufferPtr,
+        num_values: u32,
+        encoding: Encoding,
+        _is_sorted: bool,
+    ) -> Result<()> {
+        if !matches!(
+            encoding,
+            Encoding::PLAIN | Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY
+        ) {
+            return Err(nyi_err!(
+                "Invalid/Unsupported encoding type for dictionary: {}",
+                encoding
+            ));
+        }
+
+        let mut buffer = OffsetBuffer::default();
+        let mut decoder = ByteArrayDecoderPlain::new(
+            buf,
+            num_values as usize,
+            Some(num_values as usize),
+            self.validate_utf8,
+        );
+        decoder.read(&mut buffer, usize::MAX)?;
+        self.dict = Some(buffer);
+        Ok(())
+    }
+
+    fn set_data(
+        &mut self,
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+    ) -> Result<()> {
+        self.decoder = Some(ByteArrayDecoder::new(
+            encoding,
+            data,
+            num_levels,
+            num_values,
+            self.validate_utf8,
+        )?);
+        Ok(())
+    }
+
+    fn read(&mut self, out: &mut Self::Slice, range: Range<usize>) -> Result<usize> {
+        self.decoder.as_mut().expect("decoder set").read(
+            out,
+            range.end - range.start,
+            self.dict.as_ref(),
+        )
+    }
+}
+
+/// A generic decoder from uncompressed parquet value data to [`OffsetBuffer`]
+pub enum ByteArrayDecoder {
+    Plain(ByteArrayDecoderPlain),
+    Dictionary(ByteArrayDecoderDictionary),
+    DeltaLength(ByteArrayDecoderDeltaLength),
+    DeltaByteArray(ByteArrayDecoderDelta),
+}
+
+impl ByteArrayDecoder {
+    pub fn new(
+        encoding: Encoding,
+        data: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+        validate_utf8: bool,
+    ) -> Result<Self> {
+        let decoder = match encoding {
+            Encoding::PLAIN => ByteArrayDecoder::Plain(ByteArrayDecoderPlain::new(
+                data,
+                num_levels,
+                num_values,
+                validate_utf8,
+            )),
+            Encoding::RLE_DICTIONARY | Encoding::PLAIN_DICTIONARY => {
+                ByteArrayDecoder::Dictionary(ByteArrayDecoderDictionary::new(
+                    data, num_levels, num_values,
+                ))
+            }
+            Encoding::DELTA_LENGTH_BYTE_ARRAY => ByteArrayDecoder::DeltaLength(
+                ByteArrayDecoderDeltaLength::new(data, validate_utf8)?,
+            ),
+            Encoding::DELTA_BYTE_ARRAY => ByteArrayDecoder::DeltaByteArray(
+                ByteArrayDecoderDelta::new(data, validate_utf8)?,
+            ),
+            _ => {
+                return Err(general_err!(
+                    "unsupported encoding for byte array: {}",
+                    encoding
+                ))
+            }
+        };
+
+        Ok(decoder)
+    }
+
+    /// Read up to `len` values to `out` with the optional dictionary
+    pub fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        out: &mut OffsetBuffer<I>,
+        len: usize,
+        dict: Option<&OffsetBuffer<I>>,
+    ) -> Result<usize> {
+        match self {
+            ByteArrayDecoder::Plain(d) => d.read(out, len),
+            ByteArrayDecoder::Dictionary(d) => {
+                let dict = dict.expect("dictionary set");
+                d.read(out, dict, len)
+            }
+            ByteArrayDecoder::DeltaLength(d) => d.read(out, len),
+            ByteArrayDecoder::DeltaByteArray(d) => d.read(out, len),
+        }
+    }
+}
+
+/// Decoder from [`Encoding::PLAIN`] data to [`OffsetBuffer`]
+pub struct ByteArrayDecoderPlain {
+    buf: ByteBufferPtr,
+    offset: usize,
+    validate_utf8: bool,
+
+    /// This is a maximum as the null count is not always known, e.g. value data from
+    /// a v1 data page
+    max_remaining_values: usize,
+}
+
+impl ByteArrayDecoderPlain {
+    pub fn new(
+        buf: ByteBufferPtr,
+        num_levels: usize,
+        num_values: Option<usize>,
+        validate_utf8: bool,
+    ) -> Self {
+        Self {
+            buf,
+            validate_utf8,
+            offset: 0,
+            max_remaining_values: num_values.unwrap_or(num_levels),
+        }
+    }
+
+    pub fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+
+        let to_read = len.min(self.max_remaining_values);
+        output.offsets.reserve(to_read);
+
+        let remaining_bytes = self.buf.len() - self.offset;
+        if remaining_bytes == 0 {
+            return Ok(0);
+        }
+
+        let estimated_bytes = remaining_bytes
+            .checked_mul(to_read)
+            .map(|x| x / self.max_remaining_values)
+            .unwrap_or_default();
+
+        output.values.reserve(estimated_bytes);
+
+        let mut read = 0;
+
+        let buf = self.buf.as_ref();
+        while self.offset < self.buf.len() && read != to_read {
+            if self.offset + 4 > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+            let len_bytes: [u8; 4] =
+                buf[self.offset..self.offset + 4].try_into().unwrap();
+            let len = u32::from_le_bytes(len_bytes);
+
+            let start_offset = self.offset + 4;
+            let end_offset = start_offset + len as usize;
+            if end_offset > buf.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            output.try_push(&buf[start_offset..end_offset], self.validate_utf8)?;
+
+            self.offset = end_offset;
+            read += 1;
+        }
+        self.max_remaining_values -= to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::DELTA_LENGTH_BYTE_ARRAY`] data to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDeltaLength {
+    lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    validate_utf8: bool,
+}
+
+impl ByteArrayDecoderDeltaLength {
+    fn new(data: ByteBufferPtr, validate_utf8: bool) -> Result<Self> {
+        let mut len_decoder = DeltaBitPackDecoder::<Int32Type>::new();
+        len_decoder.set_data(data.all(), 0)?;
+        let values = len_decoder.values_left();
+
+        let mut lengths = vec![0; values];
+        len_decoder.get(&mut lengths)?;
+
+        Ok(Self {
+            lengths,
+            data,
+            validate_utf8,
+            length_offset: 0,
+            data_offset: len_decoder.get_offset(),
+        })
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+
+        let to_read = len.min(self.lengths.len() - self.length_offset);
+        output.offsets.reserve(to_read);
+
+        let src_lengths = &self.lengths[self.length_offset..self.length_offset + to_read];
+
+        let total_bytes: usize = src_lengths.iter().map(|x| *x as usize).sum();
+        output.values.reserve(total_bytes);
+
+        if self.data_offset + total_bytes > self.data.len() {
+            return Err(ParquetError::EOF(
+                "Insufficient delta length byte array bytes".to_string(),
+            ));
+        }
+
+        let mut start_offset = self.data_offset;
+        for length in src_lengths {
+            let end_offset = start_offset + *length as usize;
+            output.try_push(
+                &self.data.as_ref()[start_offset..end_offset],
+                self.validate_utf8,
+            )?;
+            start_offset = end_offset;
+        }
+
+        self.data_offset = start_offset;
+        self.length_offset += to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::DELTA_BYTE_ARRAY`] to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDelta {
+    prefix_lengths: Vec<i32>,
+    suffix_lengths: Vec<i32>,
+    data: ByteBufferPtr,
+    length_offset: usize,
+    data_offset: usize,
+    last_value: Vec<u8>,
+    validate_utf8: bool,
+}
+
+impl ByteArrayDecoderDelta {
+    fn new(data: ByteBufferPtr, validate_utf8: bool) -> Result<Self> {
+        let mut prefix = DeltaBitPackDecoder::<Int32Type>::new();
+        prefix.set_data(data.all(), 0)?;
+
+        let num_prefix = prefix.values_left();
+        let mut prefix_lengths = vec![0; num_prefix];
+        assert_eq!(prefix.get(&mut prefix_lengths)?, num_prefix);
+
+        let mut suffix = DeltaBitPackDecoder::<Int32Type>::new();
+        suffix.set_data(data.start_from(prefix.get_offset()), 0)?;
+
+        let num_suffix = suffix.values_left();
+        let mut suffix_lengths = vec![0; num_suffix];
+        assert_eq!(suffix.get(&mut suffix_lengths)?, num_suffix);
+
+        if num_prefix != num_suffix {
+            return Err(general_err!(format!(
+                "inconsistent DELTA_BYTE_ARRAY lengths, prefixes: {}, suffixes: {}",
+                num_prefix, num_suffix
+            )));
+        }
+
+        Ok(Self {
+            prefix_lengths,
+            suffix_lengths,
+            data,
+            length_offset: 0,
+            data_offset: prefix.get_offset() + suffix.get_offset(),
+            last_value: vec![],
+            validate_utf8,
+        })
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let initial_values_length = output.values.len();
+        assert_eq!(self.prefix_lengths.len(), self.suffix_lengths.len());
+
+        let to_read = len.min(self.prefix_lengths.len() - self.length_offset);
+
+        output.offsets.reserve(to_read);
+
+        let length_range = self.length_offset..self.length_offset + to_read;
+        let iter = self.prefix_lengths[length_range.clone()]
+            .iter()
+            .zip(&self.suffix_lengths[length_range]);
+
+        let data = self.data.as_ref();
+
+        for (prefix_length, suffix_length) in iter {
+            let prefix_length = *prefix_length as usize;
+            let suffix_length = *suffix_length as usize;
+
+            if self.data_offset + suffix_length > self.data.len() {
+                return Err(ParquetError::EOF("eof decoding byte array".into()));
+            }
+
+            self.last_value.truncate(prefix_length);
+            self.last_value.extend_from_slice(
+                &data[self.data_offset..self.data_offset + suffix_length],
+            );
+            output.try_push(&self.last_value, self.validate_utf8)?;
+
+            self.data_offset += suffix_length;
+        }
+
+        self.length_offset += to_read;
+
+        if self.validate_utf8 {
+            output.check_valid_utf8(initial_values_length)?;
+        }
+        Ok(to_read)
+    }
+}
+
+/// Decoder from [`Encoding::RLE_DICTIONARY`] to [`OffsetBuffer`]
+pub struct ByteArrayDecoderDictionary {
+    decoder: RleDecoder,
+
+    index_buf: Box<[i32; 1024]>,
+    index_buf_len: usize,
+    index_offset: usize,
+
+    /// This is a maximum as the null count is not always known, e.g. value data from
+    /// a v1 data page
+    max_remaining_values: usize,
+}
+
+impl ByteArrayDecoderDictionary {
+    fn new(data: ByteBufferPtr, num_levels: usize, num_values: Option<usize>) -> Self {
+        let bit_width = data[0];
+        let mut decoder = RleDecoder::new(bit_width);
+        decoder.set_data(data.start_from(1));
+
+        Self {
+            decoder,
+            index_buf: Box::new([0; 1024]),
+            index_buf_len: 0,
+            index_offset: 0,
+            max_remaining_values: num_values.unwrap_or(num_levels),
+        }
+    }
+
+    fn read<I: OffsetSizeTrait + ScalarValue>(
+        &mut self,
+        output: &mut OffsetBuffer<I>,
+        dict: &OffsetBuffer<I>,
+        len: usize,
+    ) -> Result<usize> {
+        let mut values_read = 0;
+
+        while values_read != len && self.max_remaining_values != 0 {
+            if self.index_offset == self.index_buf_len {
+                let read = self.decoder.get_batch(self.index_buf.as_mut())?;
+                if read == 0 {
+                    break;
+                }
+                self.index_buf_len = read;
+                self.index_offset = 0;
+            }
+
+            let to_read = (len - values_read)
+                .min(self.index_buf_len - self.index_offset)
+                .min(self.max_remaining_values);
+
+            output.extend_from_dictionary(
+                &self.index_buf[self.index_offset..self.index_offset + to_read],
+                dict.offsets.as_slice(),
+                dict.values.as_slice(),
+            )?;
+
+            self.index_offset += to_read;
+            values_read += to_read;
+            self.max_remaining_values -= to_read;
+        }
+        Ok(values_read)
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::basic::Type as PhysicalType;
+    use crate::data_type::{ByteArray, ByteArrayType};
+    use crate::encodings::encoding::{get_encoder, DictEncoder, Encoder};
+    use crate::schema::types::{ColumnDescriptor, ColumnPath, Type};
+    use crate::util::memory::MemTracker;
+    use std::sync::Arc;
+
+    fn column() -> ColumnDescPtr {
+        let t = Type::primitive_type_builder("col", PhysicalType::BYTE_ARRAY)
+            .with_converted_type(ConvertedType::UTF8)
+            .build()
+            .unwrap();
+
+        Arc::new(ColumnDescriptor::new(
+            Arc::new(t),
+            1,
+            0,
+            ColumnPath::new(vec![]),
+        ))
+    }
+
+    fn get_encoded(encoding: Encoding, data: &[ByteArray]) -> ByteBufferPtr {
+        let descriptor = column();
+        let mem_tracker = Arc::new(MemTracker::new());
+        let mut encoder =
+            get_encoder::<ByteArrayType>(descriptor, encoding, mem_tracker).unwrap();
+
+        encoder.put(data).unwrap();
+        encoder.flush_buffer().unwrap()
+    }
+
+    #[test]
+    fn test_byte_array_decoder() {

Review comment:
       The null padding is technically handled and tested as part of OffsetBuffer, but I'll add something here




-- 
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 change in pull request #1082: Optimized ByteArrayReader (#1040)

Posted by GitBox <gi...@apache.org>.
tustvold commented on a change in pull request #1082:
URL: https://github.com/apache/arrow-rs/pull/1082#discussion_r783415855



##########
File path: parquet/src/arrow/array_reader/byte_array.rs
##########
@@ -192,7 +211,16 @@ impl<I: OffsetSizeTrait + ScalarValue> OffsetBuffer<I> {
         self.offsets.len() - 1
     }
 
-    fn try_push(&mut self, data: &[u8]) -> Result<()> {
+    fn try_push(&mut self, data: &[u8], validate_utf8: bool) -> Result<()> {
+        if validate_utf8 {
+            if let Err(e) = std::str::from_utf8(data) {

Review comment:
       Definitely something to look into. It would also be interesting to see if it is faster to validate the entire string buffer and do codepoint validation at the offsets separately, or to validate each individual string as is done here. I'm not honestly sure which will be faster




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