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



##########
File path: parquet/src/arrow/record_reader/buffer.rs
##########
@@ -0,0 +1,191 @@
+// 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 std::marker::PhantomData;
+use std::ops::Range;
+
+use arrow::buffer::{Buffer, MutableBuffer};
+
+/// A buffer that supports writing new data to the end, and removing data from 
the front
+///
+/// Used by [RecordReader](`super::RecordReader`) to buffer up values before 
returning a
+/// potentially smaller number of values, corresponding to a whole number of 
semantic records
+pub trait BufferQueue: Sized {
+    type Output: Sized;
+
+    type Slice: ?Sized;
+
+    /// Split out the first `len` items
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the length of 
[`BufferQueue`]
+    ///
+    fn split_off(&mut self, len: usize) -> Self::Output;
+
+    /// Returns a [`Self::Slice`] with at least `batch_size` capacity that can 
be used
+    /// to append data to the end of this [`BufferQueue`]
+    ///
+    /// NB: writes to the returned slice will not update the length of 
[`BufferQueue`]
+    /// instead a subsequent call should be made to [`BufferQueue::set_len`]
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice;
+
+    /// Sets the length of the [`BufferQueue`].
+    ///
+    /// Intended to be used in combination with 
[`BufferQueue::spare_capacity_mut`]
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the initialized length

Review comment:
       I don't understand the `must panic` bit here -- how would 
implementations know what the initialized length (data written to the location 
returned by `space_capacity_mut`) is? Or is this referring to the capacity ?

##########
File path: parquet/src/arrow/record_reader/buffer.rs
##########
@@ -0,0 +1,191 @@
+// 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 std::marker::PhantomData;
+use std::ops::Range;
+
+use arrow::buffer::{Buffer, MutableBuffer};
+
+/// A buffer that supports writing new data to the end, and removing data from 
the front
+///
+/// Used by [RecordReader](`super::RecordReader`) to buffer up values before 
returning a
+/// potentially smaller number of values, corresponding to a whole number of 
semantic records
+pub trait BufferQueue: Sized {
+    type Output: Sized;
+
+    type Slice: ?Sized;
+
+    /// Split out the first `len` items
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the length of 
[`BufferQueue`]
+    ///
+    fn split_off(&mut self, len: usize) -> Self::Output;
+
+    /// Returns a [`Self::Slice`] with at least `batch_size` capacity that can 
be used
+    /// to append data to the end of this [`BufferQueue`]
+    ///
+    /// NB: writes to the returned slice will not update the length of 
[`BufferQueue`]
+    /// instead a subsequent call should be made to [`BufferQueue::set_len`]
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice;
+
+    /// Sets the length of the [`BufferQueue`].
+    ///
+    /// Intended to be used in combination with 
[`BufferQueue::spare_capacity_mut`]
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the initialized length
+    ///
+    /// Implementations may panic if `set_len` is called with less than what 
has been written
+    ///
+    /// This distinction is to allow for implementations that return a default 
initialized
+    /// [BufferQueue::Slice`] which doesn't track capacity and length 
separately
+    ///
+    /// For example, [`TypedBuffer<T>`] returns a default-initialized `&mut 
[T]`, and does not
+    /// track how much of this slice is actually written to by the caller. 
This is still
+    /// safe as the slice is default-initialized.
+    ///
+    fn set_len(&mut self, len: usize);
+}
+
+/// A typed buffer similar to [`Vec<T>`] but using [`MutableBuffer`] for 
storage
+pub struct TypedBuffer<T> {
+    buffer: MutableBuffer,
+
+    /// Length in elements of size T
+    len: usize,
+
+    /// Placeholder to allow `T` as an invariant generic parameter
+    _phantom: PhantomData<*mut T>,
+}
+
+impl<T> Default for TypedBuffer<T> {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+impl<T> TypedBuffer<T> {
+    pub fn new() -> Self {
+        Self {
+            buffer: MutableBuffer::new(0),
+            len: 0,
+            _phantom: Default::default(),
+        }
+    }
+
+    pub fn len(&self) -> usize {
+        self.len
+    }
+
+    pub fn is_empty(&self) -> bool {
+        self.len == 0
+    }
+
+    #[inline]
+    pub fn as_slice(&self) -> &[T] {
+        let (prefix, buf, suffix) = unsafe { 
self.buffer.as_slice().align_to::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+
+    #[inline]
+    pub fn as_slice_mut(&mut self) -> &mut [T] {
+        let (prefix, buf, suffix) =
+            unsafe { self.buffer.as_slice_mut().align_to_mut::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+}
+
+impl<T> BufferQueue for TypedBuffer<T> {
+    type Output = Buffer;
+
+    type Slice = [T];
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        assert!(len <= self.len);
+
+        let num_bytes = len * std::mem::size_of::<T>();
+        let remaining_bytes = self.buffer.len() - num_bytes;
+        // TODO: Optimize to reduce the copy
+        // create an empty buffer, as it will be resized below
+        let mut remaining = MutableBuffer::new(0);
+        remaining.resize(remaining_bytes, 0);
+
+        let new_records = remaining.as_slice_mut();
+
+        new_records[0..remaining_bytes]
+            .copy_from_slice(&self.buffer.as_slice()[num_bytes..]);
+
+        self.buffer.resize(num_bytes, 0);
+        self.len -= len;
+
+        std::mem::replace(&mut self.buffer, remaining).into()

Review comment:
       TIL: `std::mem::replace`

##########
File path: parquet/src/column/reader.rs
##########
@@ -440,63 +435,29 @@ impl<T: DataType> ColumnReaderImpl<T> {
             Ok(true)
         }
     }
+}
 
-    #[inline]
-    fn read_rep_levels(&mut self, buffer: &mut [i16]) -> Result<usize> {
-        let level_decoder = self
-            .rep_level_decoder
-            .as_mut()
-            .expect("rep_level_decoder be set");
-        level_decoder.get(buffer)
-    }
-
-    #[inline]
-    fn read_def_levels(&mut self, buffer: &mut [i16]) -> Result<usize> {
-        let level_decoder = self
-            .def_level_decoder
-            .as_mut()
-            .expect("def_level_decoder be set");
-        level_decoder.get(buffer)
-    }
-
-    #[inline]
-    fn read_values(&mut self, buffer: &mut [T::T]) -> Result<usize> {
-        let encoding = self
-            .current_encoding
-            .expect("current_encoding should be set");
-        let current_decoder = self
-            .decoders
-            .get_mut(&encoding)
-            .unwrap_or_else(|| panic!("decoder for encoding {} should be set", 
encoding));
-        current_decoder.get(buffer)
-    }
-
-    #[inline]
-    fn configure_dictionary(&mut self, page: Page) -> Result<bool> {
-        let mut encoding = page.encoding();
-        if encoding == Encoding::PLAIN || encoding == 
Encoding::PLAIN_DICTIONARY {
-            encoding = Encoding::RLE_DICTIONARY
-        }
-
-        if self.decoders.contains_key(&encoding) {
-            return Err(general_err!("Column cannot have more than one 
dictionary"));
+fn parse_v1_level(
+    max_level: i16,
+    num_buffered_values: u32,
+    encoding: Encoding,
+    buf: ByteBufferPtr,
+) -> Result<ByteBufferPtr> {

Review comment:
       Is there a reason to replicate the logic in `LevelDecoder::v1(enc, 
max_level);` here ? Could that level decoder simply be reused? Especially since 
it already has tests, etc

##########
File path: parquet/src/arrow/record_reader.rs
##########
@@ -16,75 +16,248 @@
 // under the License.
 
 use std::cmp::{max, min};
-use std::mem::{replace, size_of};
-
-use crate::column::{page::PageReader, reader::ColumnReaderImpl};
+use std::marker::PhantomData;
+use std::mem::replace;
+use std::ops::Range;
+
+use crate::arrow::record_reader::private::{
+    DefinitionLevels, RecordBuffer, RepetitionLevels,
+};
+use crate::column::{
+    page::PageReader,
+    reader::{
+        private::{
+            ColumnLevelDecoder, ColumnLevelDecoderImpl, ColumnValueDecoder,
+            ColumnValueDecoderImpl,
+        },
+        GenericColumnReader,
+    },
+};
 use crate::data_type::DataType;
-use crate::errors::{ParquetError, Result};
+use crate::errors::Result;
 use crate::schema::types::ColumnDescPtr;
 use arrow::array::BooleanBufferBuilder;
 use arrow::bitmap::Bitmap;
 use arrow::buffer::{Buffer, MutableBuffer};
 
+pub(crate) mod private {
+    use super::*;
+
+    pub trait RecordBuffer: Sized + Default {
+        type Output: Sized;
+
+        type Writer: ?Sized;
+
+        /// Split out `len` items
+        fn split(&mut self, len: usize) -> Self::Output;
+
+        /// Get a writer with `batch_size` capacity
+        fn writer(&mut self, batch_size: usize) -> &mut Self::Writer;
+
+        /// Record a write of `len` items
+        fn commit(&mut self, len: usize);
+    }
+
+    pub trait RepetitionLevels: RecordBuffer {
+        /// Inspects the buffered repetition levels in `range` and returns the 
number of
+        /// "complete" records along with the corresponding number of values
+        ///
+        /// A "complete" record is one where the buffer contains a subsequent 
repetition level of 0
+        fn count_records(
+            &self,
+            range: Range<usize>,
+            max_records: usize,
+        ) -> (usize, usize);
+    }
+
+    pub trait DefinitionLevels: RecordBuffer {
+        /// Update the provided validity mask based on contained levels
+        fn update_valid_mask(
+            &self,
+            valid: &mut BooleanBufferBuilder,
+            range: Range<usize>,
+            max_level: i16,
+        );
+    }
+
+    pub struct TypedBuffer<T> {
+        buffer: MutableBuffer,
+
+        /// Length in elements of size T
+        len: usize,
+
+        /// Placeholder to allow `T` as an invariant generic parameter
+        _phantom: PhantomData<*mut T>,
+    }
+
+    impl<T> Default for TypedBuffer<T> {
+        fn default() -> Self {
+            Self {
+                buffer: MutableBuffer::new(0),
+                len: 0,
+                _phantom: Default::default(),
+            }
+        }
+    }
+
+    impl<T> RecordBuffer for TypedBuffer<T> {
+        type Output = Buffer;
+
+        type Writer = [T];
+
+        fn split(&mut self, len: usize) -> Self::Output {
+            let num_bytes = len * std::mem::size_of::<T>();
+            let remaining_bytes = self.buffer.len() - num_bytes;
+            // TODO: Optimize to reduce the copy
+            // create an empty buffer, as it will be resized below
+            let mut remaining = MutableBuffer::new(0);
+            remaining.resize(remaining_bytes, 0);
+
+            let new_records = remaining.as_slice_mut();
+
+            new_records[0..remaining_bytes]
+                .copy_from_slice(&self.buffer.as_slice()[num_bytes..]);
+
+            self.buffer.resize(num_bytes, 0);
+
+            replace(&mut self.buffer, remaining).into()
+        }
+
+        fn writer(&mut self, batch_size: usize) -> &mut Self::Writer {
+            self.buffer
+                .resize((self.len + batch_size) * std::mem::size_of::<T>(), 0);
+
+            let (prefix, values, suffix) =
+                unsafe { self.buffer.as_slice_mut().align_to_mut::<T>() };
+            assert!(prefix.is_empty() && suffix.is_empty());
+
+            &mut values[self.len..self.len + batch_size]
+        }
+
+        fn commit(&mut self, len: usize) {
+            self.len = len;
+
+            let new_bytes = self.len * std::mem::size_of::<T>();
+            assert!(new_bytes <= self.buffer.len());
+            self.buffer.resize(new_bytes, 0);
+        }
+    }
+
+    impl RepetitionLevels for TypedBuffer<i16> {
+        fn count_records(
+            &self,
+            range: Range<usize>,
+            max_records: usize,
+        ) -> (usize, usize) {
+            let (prefix, buf, suffix) =
+                unsafe { self.buffer.as_slice().align_to::<i16>() };
+            assert!(prefix.is_empty() && suffix.is_empty());
+
+            let start = range.start;
+            let mut records_read = 0;
+            let mut end_of_last_record = start;
+
+            for current in range {
+                if buf[current] == 0 && current != end_of_last_record {
+                    records_read += 1;
+                    end_of_last_record = current;
+
+                    if records_read == max_records {
+                        break;
+                    }
+                }
+            }
+
+            (records_read, end_of_last_record - start)
+        }
+    }
+
+    impl DefinitionLevels for TypedBuffer<i16> {
+        fn update_valid_mask(
+            &self,
+            null_mask: &mut BooleanBufferBuilder,
+            range: Range<usize>,
+            max_level: i16,
+        ) {
+            let (prefix, buf, suffix) =
+                unsafe { self.buffer.as_slice().align_to::<i16>() };
+            assert!(prefix.is_empty() && suffix.is_empty());
+
+            for i in &buf[range] {
+                null_mask.append(*i == max_level)
+            }
+        }
+    }
+}
+
 const MIN_BATCH_SIZE: usize = 1024;
 
 /// A `RecordReader` is a stateful column reader that delimits semantic 
records.
-pub struct RecordReader<T: DataType> {
+pub type RecordReader<T> = GenericRecordReader<
+    private::TypedBuffer<i16>,
+    private::TypedBuffer<i16>,
+    private::TypedBuffer<<T as DataType>::T>,
+    ColumnLevelDecoderImpl,
+    ColumnLevelDecoderImpl,
+    ColumnValueDecoderImpl<T>,
+>;
+
+#[doc(hidden)]

Review comment:
       ```suggestion
   #[doc(hidden)]
   /// This type is hidden from the docs, and the private module makes it 
impossible for users to 
   /// directly construct it. Thus, this type signature may be changed without 
breaking downstream
   /// users
   ```

##########
File path: parquet/src/arrow/record_reader/buffer.rs
##########
@@ -0,0 +1,191 @@
+// 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 std::marker::PhantomData;
+use std::ops::Range;
+
+use arrow::buffer::{Buffer, MutableBuffer};
+
+/// A buffer that supports writing new data to the end, and removing data from 
the front
+///
+/// Used by [RecordReader](`super::RecordReader`) to buffer up values before 
returning a
+/// potentially smaller number of values, corresponding to a whole number of 
semantic records
+pub trait BufferQueue: Sized {
+    type Output: Sized;
+
+    type Slice: ?Sized;
+
+    /// Split out the first `len` items
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the length of 
[`BufferQueue`]
+    ///
+    fn split_off(&mut self, len: usize) -> Self::Output;
+
+    /// Returns a [`Self::Slice`] with at least `batch_size` capacity that can 
be used
+    /// to append data to the end of this [`BufferQueue`]
+    ///
+    /// NB: writes to the returned slice will not update the length of 
[`BufferQueue`]
+    /// instead a subsequent call should be made to [`BufferQueue::set_len`]
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice;
+
+    /// Sets the length of the [`BufferQueue`].
+    ///
+    /// Intended to be used in combination with 
[`BufferQueue::spare_capacity_mut`]
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the initialized length
+    ///
+    /// Implementations may panic if `set_len` is called with less than what 
has been written
+    ///
+    /// This distinction is to allow for implementations that return a default 
initialized
+    /// [BufferQueue::Slice`] which doesn't track capacity and length 
separately
+    ///
+    /// For example, [`TypedBuffer<T>`] returns a default-initialized `&mut 
[T]`, and does not
+    /// track how much of this slice is actually written to by the caller. 
This is still
+    /// safe as the slice is default-initialized.
+    ///
+    fn set_len(&mut self, len: usize);
+}
+
+/// A typed buffer similar to [`Vec<T>`] but using [`MutableBuffer`] for 
storage
+pub struct TypedBuffer<T> {
+    buffer: MutableBuffer,
+
+    /// Length in elements of size T
+    len: usize,
+
+    /// Placeholder to allow `T` as an invariant generic parameter
+    _phantom: PhantomData<*mut T>,
+}
+
+impl<T> Default for TypedBuffer<T> {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+impl<T> TypedBuffer<T> {
+    pub fn new() -> Self {
+        Self {
+            buffer: MutableBuffer::new(0),
+            len: 0,
+            _phantom: Default::default(),
+        }
+    }
+
+    pub fn len(&self) -> usize {
+        self.len
+    }
+
+    pub fn is_empty(&self) -> bool {
+        self.len == 0
+    }
+
+    #[inline]
+    pub fn as_slice(&self) -> &[T] {
+        let (prefix, buf, suffix) = unsafe { 
self.buffer.as_slice().align_to::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+
+    #[inline]
+    pub fn as_slice_mut(&mut self) -> &mut [T] {
+        let (prefix, buf, suffix) =
+            unsafe { self.buffer.as_slice_mut().align_to_mut::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+}
+
+impl<T> BufferQueue for TypedBuffer<T> {
+    type Output = Buffer;
+
+    type Slice = [T];
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        assert!(len <= self.len);
+
+        let num_bytes = len * std::mem::size_of::<T>();
+        let remaining_bytes = self.buffer.len() - num_bytes;
+        // TODO: Optimize to reduce the copy
+        // create an empty buffer, as it will be resized below
+        let mut remaining = MutableBuffer::new(0);
+        remaining.resize(remaining_bytes, 0);
+
+        let new_records = remaining.as_slice_mut();
+
+        new_records[0..remaining_bytes]
+            .copy_from_slice(&self.buffer.as_slice()[num_bytes..]);
+
+        self.buffer.resize(num_bytes, 0);
+        self.len -= len;
+
+        std::mem::replace(&mut self.buffer, remaining).into()
+    }
+
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice {
+        self.buffer
+            .resize((self.len + batch_size) * std::mem::size_of::<T>(), 0);
+
+        let range = self.len..self.len + batch_size;
+        &mut self.as_slice_mut()[range]
+    }
+
+    fn set_len(&mut self, len: usize) {
+        self.len = len;
+
+        let new_bytes = self.len * std::mem::size_of::<T>();
+        assert!(new_bytes <= self.buffer.len());
+        self.buffer.resize(new_bytes, 0);
+    }
+}
+
+/// A [`BufferQueue`] capable of storing column values
+pub trait ValuesBuffer: BufferQueue {
+    /// Iterate through the indexes in `range` in reverse order, moving the 
value at each
+    /// index to the next index returned by `rev_valid_position_iter`

Review comment:
       the code also seems to assume that `rev_valid_position_iter` is sorted

##########
File path: parquet/src/column/reader/decoder.rs
##########
@@ -0,0 +1,253 @@
+// 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 std::collections::HashMap;
+use std::ops::Range;
+
+use crate::basic::Encoding;
+use crate::data_type::DataType;
+use crate::decoding::{get_decoder, Decoder, DictDecoder, PlainDecoder};
+use crate::encodings::rle::RleDecoder;
+use crate::errors::{ParquetError, Result};
+use crate::memory::ByteBufferPtr;
+use crate::schema::types::ColumnDescPtr;
+use crate::util::bit_util::BitReader;
+
+/// A slice of levels buffer data that is written to by a 
[`ColumnLevelDecoder`]
+pub trait LevelsBufferSlice {

Review comment:
       I think I missed it somewhere along the line -- what is the point of 
Generisizing (sp?) levels, rather than just using `[i16]`? Can definition or 
repetition levels ever be something other than `i16`?
   
   

##########
File path: parquet/src/column/reader.rs
##########
@@ -102,36 +101,65 @@ pub fn get_typed_column_reader<T: DataType>(
 }
 
 /// Typed value reader for a particular primitive column.
-pub struct ColumnReaderImpl<T: DataType> {
+pub type ColumnReaderImpl<T> = GenericColumnReader<
+    decoder::ColumnLevelDecoderImpl,
+    decoder::ColumnLevelDecoderImpl,
+    decoder::ColumnValueDecoderImpl<T>,
+>;
+
+#[doc(hidden)]
+/// Reads data for a given column chunk, using the provided decoders:
+///
+/// - R: [`ColumnLevelDecoder`] used to decode repetition levels
+/// - D: [`ColumnLevelDecoder`] used to decode definition levels
+/// - V: [`ColumnValueDecoder`] used to decode value data
+pub struct GenericColumnReader<R, D, V> {
     descr: ColumnDescPtr,
-    def_level_decoder: Option<LevelDecoder>,
-    rep_level_decoder: Option<LevelDecoder>,
+
     page_reader: Box<dyn PageReader>,
-    current_encoding: Option<Encoding>,
 
-    // The total number of values stored in the data page.
+    /// The total number of values stored in the data page.
     num_buffered_values: u32,
 
-    // The number of values from the current data page that has been decoded 
into memory
-    // so far.
+    /// The number of values from the current data page that has been decoded 
into memory
+    /// so far.
     num_decoded_values: u32,
 
-    // Cache of decoders for existing encodings
-    decoders: HashMap<Encoding, Box<dyn Decoder<T>>>,

Review comment:
       For anyone else following along, the cache is moved into 
`ColumnValueDecoderImpl` below

##########
File path: parquet/src/arrow/record_reader/buffer.rs
##########
@@ -0,0 +1,191 @@
+// 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 std::marker::PhantomData;
+use std::ops::Range;
+
+use arrow::buffer::{Buffer, MutableBuffer};
+
+/// A buffer that supports writing new data to the end, and removing data from 
the front
+///
+/// Used by [RecordReader](`super::RecordReader`) to buffer up values before 
returning a
+/// potentially smaller number of values, corresponding to a whole number of 
semantic records
+pub trait BufferQueue: Sized {
+    type Output: Sized;
+
+    type Slice: ?Sized;
+
+    /// Split out the first `len` items
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the length of 
[`BufferQueue`]
+    ///
+    fn split_off(&mut self, len: usize) -> Self::Output;
+
+    /// Returns a [`Self::Slice`] with at least `batch_size` capacity that can 
be used
+    /// to append data to the end of this [`BufferQueue`]
+    ///
+    /// NB: writes to the returned slice will not update the length of 
[`BufferQueue`]
+    /// instead a subsequent call should be made to [`BufferQueue::set_len`]
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice;
+
+    /// Sets the length of the [`BufferQueue`].
+    ///
+    /// Intended to be used in combination with 
[`BufferQueue::spare_capacity_mut`]
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the initialized length
+    ///
+    /// Implementations may panic if `set_len` is called with less than what 
has been written
+    ///
+    /// This distinction is to allow for implementations that return a default 
initialized
+    /// [BufferQueue::Slice`] which doesn't track capacity and length 
separately
+    ///
+    /// For example, [`TypedBuffer<T>`] returns a default-initialized `&mut 
[T]`, and does not
+    /// track how much of this slice is actually written to by the caller. 
This is still
+    /// safe as the slice is default-initialized.
+    ///
+    fn set_len(&mut self, len: usize);
+}
+
+/// A typed buffer similar to [`Vec<T>`] but using [`MutableBuffer`] for 
storage
+pub struct TypedBuffer<T> {
+    buffer: MutableBuffer,
+
+    /// Length in elements of size T
+    len: usize,
+
+    /// Placeholder to allow `T` as an invariant generic parameter
+    _phantom: PhantomData<*mut T>,
+}
+
+impl<T> Default for TypedBuffer<T> {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+impl<T> TypedBuffer<T> {
+    pub fn new() -> Self {
+        Self {
+            buffer: MutableBuffer::new(0),
+            len: 0,
+            _phantom: Default::default(),
+        }
+    }
+
+    pub fn len(&self) -> usize {
+        self.len
+    }
+
+    pub fn is_empty(&self) -> bool {
+        self.len == 0
+    }
+
+    #[inline]
+    pub fn as_slice(&self) -> &[T] {
+        let (prefix, buf, suffix) = unsafe { 
self.buffer.as_slice().align_to::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+
+    #[inline]
+    pub fn as_slice_mut(&mut self) -> &mut [T] {
+        let (prefix, buf, suffix) =
+            unsafe { self.buffer.as_slice_mut().align_to_mut::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+}
+
+impl<T> BufferQueue for TypedBuffer<T> {
+    type Output = Buffer;
+
+    type Slice = [T];
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        assert!(len <= self.len);
+
+        let num_bytes = len * std::mem::size_of::<T>();
+        let remaining_bytes = self.buffer.len() - num_bytes;
+        // TODO: Optimize to reduce the copy
+        // create an empty buffer, as it will be resized below
+        let mut remaining = MutableBuffer::new(0);
+        remaining.resize(remaining_bytes, 0);
+
+        let new_records = remaining.as_slice_mut();
+
+        new_records[0..remaining_bytes]
+            .copy_from_slice(&self.buffer.as_slice()[num_bytes..]);
+
+        self.buffer.resize(num_bytes, 0);
+        self.len -= len;
+
+        std::mem::replace(&mut self.buffer, remaining).into()
+    }
+
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice {
+        self.buffer
+            .resize((self.len + batch_size) * std::mem::size_of::<T>(), 0);

Review comment:
       Is it ok to initialize everything to `0`? I am wondering if `0` isn't a 
valid representation for some type `T`? Perhaps this should be `T::default()` 
instead?

##########
File path: parquet/src/arrow/record_reader/buffer.rs
##########
@@ -0,0 +1,191 @@
+// 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 std::marker::PhantomData;
+use std::ops::Range;
+
+use arrow::buffer::{Buffer, MutableBuffer};
+
+/// A buffer that supports writing new data to the end, and removing data from 
the front
+///
+/// Used by [RecordReader](`super::RecordReader`) to buffer up values before 
returning a
+/// potentially smaller number of values, corresponding to a whole number of 
semantic records
+pub trait BufferQueue: Sized {
+    type Output: Sized;
+
+    type Slice: ?Sized;
+
+    /// Split out the first `len` items
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the length of 
[`BufferQueue`]
+    ///
+    fn split_off(&mut self, len: usize) -> Self::Output;
+
+    /// Returns a [`Self::Slice`] with at least `batch_size` capacity that can 
be used
+    /// to append data to the end of this [`BufferQueue`]
+    ///
+    /// NB: writes to the returned slice will not update the length of 
[`BufferQueue`]
+    /// instead a subsequent call should be made to [`BufferQueue::set_len`]
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice;
+
+    /// Sets the length of the [`BufferQueue`].
+    ///
+    /// Intended to be used in combination with 
[`BufferQueue::spare_capacity_mut`]
+    ///
+    /// # Panics
+    ///
+    /// Implementations must panic if `len` is beyond the initialized length
+    ///
+    /// Implementations may panic if `set_len` is called with less than what 
has been written
+    ///
+    /// This distinction is to allow for implementations that return a default 
initialized
+    /// [BufferQueue::Slice`] which doesn't track capacity and length 
separately
+    ///
+    /// For example, [`TypedBuffer<T>`] returns a default-initialized `&mut 
[T]`, and does not
+    /// track how much of this slice is actually written to by the caller. 
This is still
+    /// safe as the slice is default-initialized.
+    ///
+    fn set_len(&mut self, len: usize);
+}
+
+/// A typed buffer similar to [`Vec<T>`] but using [`MutableBuffer`] for 
storage
+pub struct TypedBuffer<T> {
+    buffer: MutableBuffer,
+
+    /// Length in elements of size T
+    len: usize,
+
+    /// Placeholder to allow `T` as an invariant generic parameter
+    _phantom: PhantomData<*mut T>,
+}
+
+impl<T> Default for TypedBuffer<T> {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+impl<T> TypedBuffer<T> {
+    pub fn new() -> Self {
+        Self {
+            buffer: MutableBuffer::new(0),
+            len: 0,
+            _phantom: Default::default(),
+        }
+    }
+
+    pub fn len(&self) -> usize {
+        self.len
+    }
+
+    pub fn is_empty(&self) -> bool {
+        self.len == 0
+    }
+
+    #[inline]
+    pub fn as_slice(&self) -> &[T] {
+        let (prefix, buf, suffix) = unsafe { 
self.buffer.as_slice().align_to::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+
+    #[inline]
+    pub fn as_slice_mut(&mut self) -> &mut [T] {
+        let (prefix, buf, suffix) =
+            unsafe { self.buffer.as_slice_mut().align_to_mut::<T>() };
+        assert!(prefix.is_empty() && suffix.is_empty());
+        buf
+    }
+}
+
+impl<T> BufferQueue for TypedBuffer<T> {
+    type Output = Buffer;
+
+    type Slice = [T];
+
+    fn split_off(&mut self, len: usize) -> Self::Output {
+        assert!(len <= self.len);
+
+        let num_bytes = len * std::mem::size_of::<T>();
+        let remaining_bytes = self.buffer.len() - num_bytes;
+        // TODO: Optimize to reduce the copy
+        // create an empty buffer, as it will be resized below
+        let mut remaining = MutableBuffer::new(0);
+        remaining.resize(remaining_bytes, 0);
+
+        let new_records = remaining.as_slice_mut();
+
+        new_records[0..remaining_bytes]
+            .copy_from_slice(&self.buffer.as_slice()[num_bytes..]);
+
+        self.buffer.resize(num_bytes, 0);
+        self.len -= len;
+
+        std::mem::replace(&mut self.buffer, remaining).into()
+    }
+
+    fn spare_capacity_mut(&mut self, batch_size: usize) -> &mut Self::Slice {
+        self.buffer
+            .resize((self.len + batch_size) * std::mem::size_of::<T>(), 0);
+
+        let range = self.len..self.len + batch_size;
+        &mut self.as_slice_mut()[range]
+    }
+
+    fn set_len(&mut self, len: usize) {
+        self.len = len;
+
+        let new_bytes = self.len * std::mem::size_of::<T>();
+        assert!(new_bytes <= self.buffer.len());
+        self.buffer.resize(new_bytes, 0);
+    }
+}
+
+/// A [`BufferQueue`] capable of storing column values
+pub trait ValuesBuffer: BufferQueue {
+    /// Iterate through the indexes in `range` in reverse order, moving the 
value at each
+    /// index to the next index returned by `rev_valid_position_iter`
+    ///
+    /// It is guaranteed that the `i`th index returned by 
`rev_valid_position_iter` is greater

Review comment:
       ```suggestion
       /// panics if  the `i`th index returned by `rev_valid_position_iter` is 
not greater
   ```




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to