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 2022/08/05 14:30:24 UTC

[GitHub] [arrow-rs] tustvold opened a new pull request, #2335: RFC: Add RowFilter API

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

   _Draft as needs a lot more test coverage and general cleanup_
   
   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #2270
   
   # Rationale for this change
    
   <!--
   Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
   Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.
   -->
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   This adds a `RowFilter` API and refines the existing `RowSelection` API. There are a couple of things worth highlighting here
   
   * The `RowFilter` is pushed down to the IO level. @crepererum gave a good use case of if this allows eliminating an entire column chunk from consideration, etc...
   * I think we need a `RecordBatchReaderBuilder` as the current API can't really be used for this purpose
   
   # Are there any user-facing changes?
   
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!---
   If there are any breaking changes to public APIs, please add the `breaking change` label.
   -->
   


-- 
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 diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r944656102


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,426 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelector`] represents a range of rows to scan from a parquet file
+#[derive(Debug, Clone, Copy, PartialEq, Eq)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,
+
+    /// If true, skip `row_count` rows
+    pub skip: bool,
+}
+
+impl RowSelector {
+    /// Select `row_count` rows
+    pub fn select(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: false,
+        }
+    }
+
+    /// Skip `row_count` rows
+    pub fn skip(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: true,
+        }
+    }
+}
+
+/// [`RowSelection`] allows selecting or skipping a provided number of rows
+/// when scanning the parquet file.
+///
+/// This is applied prior to reading column data, and can therefore
+/// be used to skip IO to fetch data into memory
+///
+/// A typical use-case would be using the [`PageIndex`] to filter out rows
+/// that don't satisfy a predicate
+///
+/// [`PageIndex`]: [crate::file::page_index::index::PageIndex]
+#[derive(Debug, Clone, Default, Eq, PartialEq)]
+pub struct RowSelection {
+    selectors: Vec<RowSelector>,
+}
+
+impl RowSelection {
+    /// Creates a [`RowSelection`] from a slice of [`BooleanArray`]
+    ///
+    /// # Panic
+    ///
+    /// Panics if any of the [`BooleanArray`] contain nulls
+    pub fn from_filters(filters: &[BooleanArray]) -> Self {
+        let mut next_offset = 0;
+        let total_rows = filters.iter().map(|x| x.len()).sum();
+
+        let iter = filters.iter().flat_map(|filter| {
+            let offset = next_offset;
+            next_offset += filter.len();
+            assert_eq!(filter.null_count(), 0);
+            SlicesIterator::new(filter)
+                .map(move |(start, end)| start + offset..end + offset)
+        });
+
+        Self::from_consecutive_ranges(iter, total_rows)
+    }
+
+    /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep
+    fn from_consecutive_ranges<I: Iterator<Item = Range<usize>>>(
+        ranges: I,
+        total_rows: usize,
+    ) -> Self {
+        let mut selectors: Vec<RowSelector> = Vec::with_capacity(ranges.size_hint().0);
+        let mut last_end = 0;
+        for range in ranges {
+            let len = range.end - range.start;
+
+            match range.start.cmp(&last_end) {
+                Ordering::Equal => match selectors.last_mut() {
+                    Some(last) => last.row_count += len,
+                    None => selectors.push(RowSelector::select(len)),
+                },
+                Ordering::Greater => {
+                    selectors.push(RowSelector::skip(range.start - last_end));
+                    selectors.push(RowSelector::select(len))
+                }
+                Ordering::Less => panic!("out of order"),
+            }
+            last_end = range.end;
+        }
+
+        if last_end != total_rows {
+            selectors.push(RowSelector::skip(total_rows - last_end))
+        }
+
+        Self { selectors }
+    }
+
+    /// Splits off the first `row_count` from this [`RowSelection`]
+    pub fn split_off(&mut self, row_count: usize) -> Self {
+        let mut total_count = 0;
+
+        // Find the index where the selector exceeds the row count
+        let find = self.selectors.iter().enumerate().find(|(_, selector)| {
+            total_count += selector.row_count;
+            total_count > row_count
+        });
+
+        let split_idx = match find {
+            Some((idx, _)) => idx,
+            None => {
+                let selectors = std::mem::take(&mut self.selectors);
+                return Self { selectors };
+            }
+        };
+
+        let mut remaining = self.selectors.split_off(split_idx);
+
+        // Always present as `split_idx < self.selectors.len`
+        let next = remaining.first_mut().unwrap();
+        let overflow = total_count - row_count;
+
+        if next.row_count != overflow {
+            self.selectors.push(RowSelector {
+                row_count: next.row_count - overflow,
+                skip: next.skip,
+            })
+        }
+        next.row_count = overflow;
+
+        std::mem::swap(&mut remaining, &mut self.selectors);
+        Self {
+            selectors: remaining,
+        }
+    }
+
+    /// Given a [`RowSelection`] computed under `self`, returns the [`RowSelection`]
+    /// representing their conjunction

Review Comment:
   Yeah, I think the wording saying this is a `conjunction` is misleading, it is more like the conjunction of the subsequent filters only with the rows that were selected previously



-- 
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 diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939508363


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   If a column appears in multiple predicates and/or the final projection, it will need to be decoded multiple times. I don't really see a way around this, keeping the data around and doing `take + concat` adds significant complexity, and it is unclear that it would necessarily be faster.
   
   Eventually it might be possible to push simple predicates down to operate directly on the encoded data, which would avoid this



-- 
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] thinkharderdev commented on a diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939516218


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   Yeah, I think in many cases the difference would be negligible but in the degenerate cases (lots of predicates, filters which don't do much filtering, etc) I think it could potentially add up. The reason I worry about it in general is that we would have to rely on the engine to determine which predicates to apply and in which order. And in a situation where all we have is row group metadata we don't have a ton to go on. 
   
   I took a crack at seeing what it might look like preserving the decoded arrays and came up with https://github.com/tustvold/arrow-rs/pull/24. It certainly involves a lot of array slicing and dicing but the complexity seems manageable and would help ensure that applying filters doesn't ever come with a significant performance cost. 



-- 
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 diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939508363


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   If a column appears in multiple predicates and/or the final projection, it will need to be decoded multiple times. I don't really see a way around this, keeping the data around and doing `take + concat` adds significant complexity, and it is unclear that it would necessarily be faster.
   
   Eventually it might be possible to push the predicates down to operate directly on the encoded data, this would eliminate this



-- 
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] Ted-Jiang commented on a diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939545227


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -478,9 +550,56 @@ struct InMemoryRowGroup {
     row_count: usize,
 }
 
+impl InMemoryRowGroup {
+    /// Fetches the necessary column data into memory
+    async fn fetch<T: AsyncFileReader + Send>(
+        &mut self,
+        input: &mut T,
+        metadata: &RowGroupMetaData,
+        projection: &ProjectionMask,
+        _selection: Option<&RowSelection>,
+    ) -> Result<()> {
+        // TODO: Use OffsetIndex and selection to prune pages

Review Comment:
   👍 this avoid huge IO work in some situation make pageIndex more useful !
   I think it needs takes a lot of testing to decide when use random skip reads



-- 
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 merged pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold merged PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335


-- 
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 diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939508363


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   If a column appears in multiple predicates and/or the final projection, it will need to be decoded multiple times. I don't really see a way around this, keeping the data around and doing `take + concat` adds significant complexity, and it is unclear that it would necessarily 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


[GitHub] [arrow-rs] tustvold commented on a diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939508363


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   If a column appears in multiple predicates and/or the final projection, it will need to be decoded multiple times. I don't really see a way around this, keeping the data around and doing `take + concat` adds significant complexity, and it is unclear that it would necessarily be faster.
   
   Eventually it might be possible to push simple predicates down to operate directly on the encoded data, which would avoid this. But that is a wee ways off 😅



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r940088468


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   I will create a follow up ticket to investigate 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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r940088098


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,

Review Comment:
   > previous each filter rate
   
   I'm not sure what you mean?



-- 
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 diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r943609275


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,417 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelector`] represents a range of rows to scan from a parquet file
+#[derive(Debug, Clone, Copy, PartialEq, Eq)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,
+
+    /// If true, skip `row_count` rows
+    pub skip: bool,
+}
+
+impl RowSelector {
+    /// Select `row_count` rows
+    pub fn select(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: false,
+        }
+    }
+
+    /// Skip `row_count` rows
+    pub fn skip(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: true,
+        }
+    }
+}
+
+/// [`RowSelection`] allows selecting or skipping a provided number of rows
+/// when scanning the parquet file.
+///
+/// This is applied prior to reading column data, and can therefore
+/// be used to skip IO to fetch data into memory
+///
+/// A typical use-case would be using the [`PageIndex`] to filter out rows
+/// that don't satisfy a predicate
+///
+/// [`PageIndex`]: [crate::file::page_index::index::PageIndex]
+#[derive(Debug, Clone, Default, Eq, PartialEq)]
+pub struct RowSelection {
+    selectors: Vec<RowSelector>,
+}
+
+impl RowSelection {
+    /// Creates a [`RowSelection`] from a slice of [`BooleanArray`]
+    ///
+    /// # Panic
+    ///
+    /// Panics if any of the [`BooleanArray`] contain nulls
+    pub fn from_filters(filters: &[BooleanArray]) -> Self {
+        let mut next_offset = 0;
+        let total_rows = filters.iter().map(|x| x.len()).sum();
+
+        let iter = filters.iter().flat_map(|filter| {
+            let offset = next_offset;
+            next_offset += filter.len();
+            assert_eq!(filter.null_count(), 0);
+            SlicesIterator::new(filter)
+                .map(move |(start, end)| start + offset..end + offset)
+        });
+
+        Self::from_consecutive_ranges(iter, total_rows)
+    }
+
+    /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep
+    fn from_consecutive_ranges<I: Iterator<Item = Range<usize>>>(
+        ranges: I,
+        total_rows: usize,
+    ) -> Self {
+        let mut selectors: Vec<RowSelector> = Vec::with_capacity(ranges.size_hint().0);
+        let mut last_end = 0;
+        for range in ranges {
+            let len = range.end - range.start;
+
+            match range.start.cmp(&last_end) {
+                Ordering::Equal => match selectors.last_mut() {
+                    Some(last) => last.row_count += len,
+                    None => selectors.push(RowSelector::select(len)),
+                },
+                Ordering::Greater => {
+                    selectors.push(RowSelector::skip(range.start - last_end));
+                    selectors.push(RowSelector::select(len))
+                }
+                Ordering::Less => panic!("out of order"),
+            }
+            last_end = range.end;
+        }
+
+        if last_end != total_rows {
+            selectors.push(RowSelector::skip(total_rows - last_end))
+        }
+
+        Self { selectors }
+    }
+
+    /// Splits off the first `row_count` from this [`RowSelection`]
+    pub fn split_off(&mut self, row_count: usize) -> Self {
+        let mut total_count = 0;
+
+        // Find the index where the selector exceeds the row count
+        let find = self.selectors.iter().enumerate().find(|(_, selector)| {
+            total_count += selector.row_count;
+            total_count > row_count
+        });
+
+        let split_idx = match find {
+            Some((idx, _)) => idx,
+            None => {
+                let selectors = std::mem::take(&mut self.selectors);
+                return Self { selectors };
+            }
+        };
+
+        let mut remaining = self.selectors.split_off(split_idx);
+
+        // Always present as `split_idx < self.selectors.len`
+        let next = remaining.first_mut().unwrap();
+        let overflow = total_count - row_count;
+
+        if next.row_count != overflow {
+            self.selectors.push(RowSelector {
+                row_count: next.row_count - overflow,
+                skip: next.skip,
+            })
+        }
+        next.row_count = overflow;
+
+        std::mem::swap(&mut remaining, &mut self.selectors);
+        Self {
+            selectors: remaining,
+        }
+    }
+
+    /// Given a [`RowSelection`] computed under `self` returns the [`RowSelection`]
+    /// representing their conjunction
+    pub fn and(&self, other: &Self) -> Self {
+        let mut selectors = vec![];
+        let mut first = self.selectors.iter().cloned().peekable();
+        let mut second = other.selectors.iter().cloned().peekable();
+
+        let mut to_skip = 0;
+        while let Some(b) = second.peek_mut() {
+            let a = first.peek_mut().unwrap();
+
+            if b.row_count == 0 {
+                second.next().unwrap();
+                continue;
+            }
+
+            if a.row_count == 0 {
+                first.next().unwrap();
+                continue;
+            }
+
+            if a.skip {
+                // Records were skipped when producing second
+                to_skip += a.row_count;
+                first.next().unwrap();
+                continue;
+            }
+
+            let skip = b.skip;
+            let to_process = a.row_count.min(b.row_count);
+
+            a.row_count -= to_process;
+            b.row_count -= to_process;
+
+            match skip {
+                true => to_skip += to_process,
+                false => {
+                    if to_skip != 0 {
+                        selectors.push(RowSelector::skip(to_skip));
+                        to_skip = 0;
+                    }
+                    selectors.push(RowSelector::select(to_process))
+                }
+            }
+        }
+
+        for v in first {
+            if v.row_count != 0 {
+                assert!(v.skip);
+                to_skip += v.row_count
+            }
+        }
+
+        if to_skip != 0 {
+            selectors.push(RowSelector::skip(to_skip));
+        }
+
+        Self { selectors }
+    }
+
+    /// Returns `true` if this [`RowSelection`] selects any rows
+    pub fn selects_any(&self) -> bool {
+        self.selectors.iter().any(|x| !x.skip)
+    }
+}
+
+impl From<Vec<RowSelector>> for RowSelection {
+    fn from(selectors: Vec<RowSelector>) -> Self {
+        Self { selectors }
+    }
+}
+
+impl From<RowSelection> for VecDeque<RowSelector> {
+    fn from(r: RowSelection) -> Self {
+        r.selectors.into()
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use rand::{thread_rng, Rng};
+
+    #[test]
+    fn test_from_filters() {
+        let filters = vec![
+            BooleanArray::from(vec![false, false, false, true, true, true, true]),
+            BooleanArray::from(vec![true, true, false, false, true, true, true]),
+            BooleanArray::from(vec![false, false, false, false]),
+            BooleanArray::from(Vec::<bool>::new()),
+        ];
+
+        let selection = RowSelection::from_filters(&filters[..1]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(3), RowSelector::select(4)]
+        );
+
+        let selection = RowSelection::from_filters(&filters[..2]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3),
+                RowSelector::skip(4)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters[2..3]);
+        assert!(!selection.selects_any());
+        assert_eq!(selection.selectors, vec![RowSelector::skip(4)]);
+    }
+
+    #[test]
+    fn test_split_off() {
+        let mut selection = RowSelection::from(vec![
+            RowSelector::skip(34),
+            RowSelector::select(12),
+            RowSelector::skip(3),
+            RowSelector::select(35),
+        ]);
+
+        let split = selection.split_off(34);
+        assert_eq!(split.selectors, vec![RowSelector::skip(34)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(12),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(5);
+        assert_eq!(split.selectors, vec![RowSelector::select(5)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(7),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(8);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::select(7), RowSelector::skip(1)]
+        );
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+
+        let split = selection.split_off(200);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+        assert!(selection.selectors.is_empty());
+    }
+
+    #[test]
+    fn test_and() {
+        let mut a = RowSelection::from(vec![
+            RowSelector::skip(12),
+            RowSelector::select(23),
+            RowSelector::skip(3),
+            RowSelector::select(5),
+        ]);
+
+        let b = RowSelection::from(vec![
+            RowSelector::select(5),
+            RowSelector::skip(4),
+            RowSelector::select(15),
+            RowSelector::skip(4),
+        ]);
+
+        let mut expected = RowSelection::from(vec![

Review Comment:
   Perhaps `and` is the wrong name for this function? I will incorporate @thinkharderdev 's example as a comment as I think it is helpful :+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] ursabot commented on pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#issuecomment-1212433276

   Benchmark runs are scheduled for baseline = 4481993a0421e686f9767848843a0fb0370a78b1 and contender = 21ba02eff1ce356fc06c2949afe2ad4c84553ce9. 21ba02eff1ce356fc06c2949afe2ad4c84553ce9 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on ec2-t3-xlarge-us-east-2] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/2b45c6ac79a14289b3262ee71c0756ac...409581f4f2f44b7a8a95710520001adc/)
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on test-mac-arm] [test-mac-arm](https://conbench.ursa.dev/compare/runs/06c699d0bb644065801d8f06eb9f358b...613f066107b34081b4c5b9ad426ce6a4/)
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on ursa-i9-9960x] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/68673cc1f0394496b801352c73dd34f7...533c0e88ddc34328a4c35c3d531166ed/)
   [Skipped :warning: Benchmarking of arrow-rs-commits is not supported on ursa-thinkcentre-m75q] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/0d0f8cca6e5942db93d2a4af7f8a68e6...f913a2e7b6d84fd492feca725e340ad9/)
   Buildkite builds:
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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] thinkharderdev commented on a diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939103004


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   This would require decoding the filter columns twice (or multiple times in the case where we have multiple predicates) right?



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r940086934


##########
parquet/src/arrow/arrow_reader/mod.rs:
##########
@@ -375,11 +340,36 @@ impl ParquetRecordBatchReader {
             batch_size,
             array_reader,
             schema: Arc::new(schema),
-            selection,
+            selection: selection.map(Into::into),
         }
     }
 }
 
+/// Evaluates an [`ArrowPredicate`] returning the [`RowSelection`]
+///
+/// If this [`ParquetRecordBatchReader`] has a [`RowSelection`], the
+/// returned [`RowSelection`] will be the conjunction of this and
+/// the rows selected by `predicate`
+pub(crate) fn evaluate_predicate(
+    batch_size: usize,
+    array_reader: Box<dyn ArrayReader>,
+    selection: Option<RowSelection>,
+    predicate: &mut dyn ArrowPredicate,
+) -> Result<RowSelection> {
+    let reader =
+        ParquetRecordBatchReader::new(batch_size, array_reader, selection.clone());
+    let mut filters = vec![];
+    for maybe_batch in reader {
+        filters.push(predicate.filter(maybe_batch?)?);

Review Comment:
   The passed `array_reader` has the filter's projection mask applied



-- 
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] thinkharderdev commented on a diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939516218


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   Yeah, I think in many cases the difference would be negligible but in the degenerate cases (lots of predicates, filters which do much filtering, etc) I think it could potentially add up. The reason I worry about it in general is that we would have to rely on the engine to determine which predicates to apply and in which order. And in a situation where all we have is row group metadata we don't have a ton to go on. 
   
   I took a crack at seeing what it might look like preserving the decoded arrays and came up with https://github.com/tustvold/arrow-rs/pull/24. It certainly involves a lot of array slicing and dicing but the complexity seems manageable and would help ensure that applying filters doesn't ever come with a significant performance cost. 



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r940084025


##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls

Review Comment:
   It was more I didn't realise https://docs.rs/arrow/latest/arrow/compute/kernels/filter/fn.prep_null_mask_filter.html was public :+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] Ted-Jiang commented on a diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939543038


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,

Review Comment:
   Is there a situation we need read row groups with previous row group selection? 🤔



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r940087849


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -253,6 +262,30 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
         }
     }
 
+    /// Provide a [`RowSelection] to filter out rows, and avoid fetching their
+    /// data into memory
+    ///
+    /// Row group filtering is applied prior to this, and rows from skipped
+    /// row groups should not be included in the [`RowSelection`]
+    ///
+    /// TODO: Make public once stable (#1792)

Review Comment:
   The doc links should do this automatically?



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939535613


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when
+/// scanning a parquet file
+#[derive(Debug, Clone, Copy)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,

Review Comment:
   makes sense 



-- 
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] thinkharderdev commented on a diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939540049


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   > Conveniently, it seems like nothing in the API of this PR requires decoding multiple times, so I think we could also potentially implement the 'use take rather than redundant decode' in a follow on PR as well.
   
   I agree. As @tustvold points out, we need to be careful about memory overhead so maybe the best course is to go with the current approach and tackle avoiding the redundant decoding in a follow up. I prototyped the DataFusion piece (based on my other draft PR but it should be roughly similar) to see how it affected our internal benchmarks and saw a roughly 50% improvement on queries with reasonably selective predicates. We went from being mostly CPU bound with parquet decoding to being mostly IO bound which means I expect there is even more room for improvement once we are using the selection and the page index to avoid IO altogether. That's all to say I'm super excited about this work and think it will be a huge step forward!



-- 
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] Ted-Jiang commented on a diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939546886


##########
parquet/src/arrow/arrow_reader/mod.rs:
##########
@@ -375,11 +340,36 @@ impl ParquetRecordBatchReader {
             batch_size,
             array_reader,
             schema: Arc::new(schema),
-            selection,
+            selection: selection.map(Into::into),
         }
     }
 }
 
+/// Evaluates an [`ArrowPredicate`] returning the [`RowSelection`]
+///
+/// If this [`ParquetRecordBatchReader`] has a [`RowSelection`], the
+/// returned [`RowSelection`] will be the conjunction of this and
+/// the rows selected by `predicate`
+pub(crate) fn evaluate_predicate(
+    batch_size: usize,
+    array_reader: Box<dyn ArrayReader>,
+    selection: Option<RowSelection>,
+    predicate: &mut dyn ArrowPredicate,
+) -> Result<RowSelection> {
+    let reader =
+        ParquetRecordBatchReader::new(batch_size, array_reader, selection.clone());
+    let mut filters = vec![];
+    for maybe_batch in reader {
+        filters.push(predicate.filter(maybe_batch?)?);

Review Comment:
   I think here, we use the whole `project_mask` to generate  batch, maybe we can pass the 'filter' 's `project_mask` get better performance 🤔



-- 
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] thinkharderdev commented on pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#issuecomment-1212135931

   > I think this PR looks good to go to me so we can start hooking it up and getting everything ready.
   > 
   > I definitely have some questions about the `and` test -- but since this code isn't used yet I don't think it would block merge.
   > 
   > @Ted-Jiang or @thinkharderdev do you have any other comments or suggestions? We can also address any additional changes in a follow on PR as this one is already fairly large
   
   I think this should is good to merge. 


-- 
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 diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r941824173


##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,110 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the [`ProjectionMask`] that describes the columns required
+    /// to evaluate this predicate. All projected columns will be provided in the `batch`
+    /// passed to [`filter`](Self::filter)
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::projection`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// All row that are `true` in returned [`BooleanArray`] will be returned to the reader.

Review Comment:
   minor typo: `All row` is probably meant to be `All rows` (missing "s")



-- 
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 diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r941827220


##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,110 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the [`ProjectionMask`] that describes the columns required
+    /// to evaluate this predicate. All projected columns will be provided in the `batch`
+    /// passed to [`filter`](Self::filter)
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::projection`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// All row that are `true` in returned [`BooleanArray`] will be returned to the reader.
+    /// Any rows that are `false` or `Null` will not be
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;

Review Comment:
   should this be named `fn filter_array` instead to better indicate that the result would be a Boolean filter array instead of actually filtering the RecordBatch passed in as the batch parameter. 



-- 
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 #2335: RFC: Add RowFilter API

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

   FYI @thinkharderdev @alamb @crepererum @Ted-Jiang I'd appreciate any feedback you might have on this


-- 
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] thinkharderdev commented on a diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939108683


##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;
+}
+
+/// An [`ArrowPredicate`] created from an [`FnMut`]
+pub struct ArrowPredicateFn<F> {
+    f: F,
+    projection: ProjectionMask,
+}
+
+impl<F> ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    /// Create a new [`ArrowPredicateFn`]
+    pub fn new(projection: ProjectionMask, f: F) -> Self {
+        Self { f, projection }
+    }
+}
+
+impl<F> ArrowPredicate for ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    fn projection(&self) -> &ProjectionMask {
+        &self.projection
+    }
+
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray> {
+        (self.f)(batch)
+    }
+}
+
+/// A [`RowFilter`] allows pushing down a filter predicate to skip IO and decode
+///
+/// This consists of a list of [`ArrowPredicate`] where only the rows that satisfy all
+/// of the predicates will be returned. Any [`RowSelection`] will be applied prior
+/// to the first predicate, and each predicate in turn will then be used to compute
+/// a more refined [`RowSelection`] to use when evaluating the subsequent predicates.
+///
+/// Once all predicates have been evaluated, the resulting [`RowSelection`] will be
+/// used to return just the desired rows.
+///
+/// This design has a couple of implications:

Review Comment:
   Nice, I like this



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939534771


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when
+/// scanning a parquet file
+#[derive(Debug, Clone, Copy)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,

Review Comment:
   Parquet can't provide random access, and so RLE is not an option. We could use a smaller size, but I'd wager the overheads are completely insignificant compared to the overhead of processing a given skip



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939531214


##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls

Review Comment:
   ```suggestion
       /// The returned [`BooleanArray`] must not contain any nulls otherwise or else XXX will happen
   ```
   
   What will happen if the array has nulls?



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;
+}
+
+/// An [`ArrowPredicate`] created from an [`FnMut`]
+pub struct ArrowPredicateFn<F> {
+    f: F,
+    projection: ProjectionMask,
+}
+
+impl<F> ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    /// Create a new [`ArrowPredicateFn`]
+    pub fn new(projection: ProjectionMask, f: F) -> Self {
+        Self { f, projection }
+    }
+}
+
+impl<F> ArrowPredicate for ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    fn projection(&self) -> &ProjectionMask {
+        &self.projection
+    }
+
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray> {
+        (self.f)(batch)
+    }
+}
+
+/// A [`RowFilter`] allows pushing down a filter predicate to skip IO and decode
+///
+/// This consists of a list of [`ArrowPredicate`] where only the rows that satisfy all
+/// of the predicates will be returned. Any [`RowSelection`] will be applied prior
+/// to the first predicate, and each predicate in turn will then be used to compute
+/// a more refined [`RowSelection`] to use when evaluating the subsequent predicates.
+///
+/// Once all predicates have been evaluated, the resulting [`RowSelection`] will be
+/// used to return just the desired rows.
+///
+/// This design has a couple of implications:
+///
+/// * [`RowFilter`] can be used to skip fetching IO, in addition to decode overheads

Review Comment:
   ```suggestion
   /// * [`RowFilter`] can be used to skip entire pages, and thus IO, in addition to CPU decode overheads
   ```



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;
+}
+
+/// An [`ArrowPredicate`] created from an [`FnMut`]
+pub struct ArrowPredicateFn<F> {
+    f: F,
+    projection: ProjectionMask,
+}
+
+impl<F> ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    /// Create a new [`ArrowPredicateFn`]

Review Comment:
   ```suggestion
       /// Create a new [`ArrowPredicateFn`]. `f` will be passed batches
       /// that contains the columns specified in `projection`
       /// and returns a [`BooleanArray`] that describes which rows should
       /// be passed along.
   ```



##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when

Review Comment:
   ```suggestion
   /// [`RowSelector`] represents range of rows
   ```



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;
+}
+
+/// An [`ArrowPredicate`] created from an [`FnMut`]
+pub struct ArrowPredicateFn<F> {
+    f: F,
+    projection: ProjectionMask,
+}
+
+impl<F> ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    /// Create a new [`ArrowPredicateFn`]
+    pub fn new(projection: ProjectionMask, f: F) -> Self {
+        Self { f, projection }
+    }
+}
+
+impl<F> ArrowPredicate for ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    fn projection(&self) -> &ProjectionMask {
+        &self.projection
+    }
+
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray> {
+        (self.f)(batch)
+    }
+}
+
+/// A [`RowFilter`] allows pushing down a filter predicate to skip IO and decode
+///
+/// This consists of a list of [`ArrowPredicate`] where only the rows that satisfy all
+/// of the predicates will be returned. Any [`RowSelection`] will be applied prior
+/// to the first predicate, and each predicate in turn will then be used to compute
+/// a more refined [`RowSelection`] to use when evaluating the subsequent predicates.
+///
+/// Once all predicates have been evaluated, the resulting [`RowSelection`] will be
+/// used to return just the desired rows.

Review Comment:
   ```suggestion
   /// Once all predicates have been evaluated, the final [`RowSelection`] is applied
   /// to the overall [`ColumnProjection`] to produce the final output [`RecordBatch`]es.
   ```



##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when
+/// scanning a parquet file
+#[derive(Debug, Clone, Copy)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,

Review Comment:
   Alternately, perhaps we could represent Row selection by RLE runs of the rows to keep:
   
   (position, count)
   
   Again, using i32 or something smaller that might be much more efficient to process 🤷 



##########
parquet/src/arrow/async_reader.rs:
##########
@@ -253,6 +262,30 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
         }
     }
 
+    /// Provide a [`RowSelection] to filter out rows, and avoid fetching their
+    /// data into memory
+    ///
+    /// Row group filtering is applied prior to this, and rows from skipped
+    /// row groups should not be included in the [`RowSelection`]
+    ///
+    /// TODO: Make public once stable (#1792)

Review Comment:
   Probably also a good idea to link to the docs that describe the order of filter application in decoding (RowSelection followed by RowFilter)



##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   I think trying to eliminate redundant decoding is a good idea for the reasons that @thinkharderdev  
   
   Conveniently, it seems like nothing in the API of this PR requires decoding multiple times, so I think we could also potentially implement the 'use take rather than redundant decode' in a follow on PR as well.
   
   In terms of "Eventually it might be possible to push simple predicates down to operate directly on the encoded data, which would avoid this." I agree it is a ways off however, I think it could fit into this API with something like adding a list of `ParquetFilter`s to apply during the decode itself that could be efficiently implemented
   
   
   ```rust
   /// Filter that is applied during decoding of a single column
   /// semantically takes the form <col> <op> <constant>
   struct ParquetFilter {
     op: ParquetOp
     left: ParquetConst,
   }
   enum ParquetFilterOp { EQ, NEQ }  
   enum ParqeutConst {
     Int64(i64)
     Float(f64)
     ...
   }
   ```



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate

Review Comment:
   ```suggestion
       /// Returns the [`ProjectionMask`] that describes the columns required
       /// to evaluate this predicate. All projected columns will be provided in the `batch`
       /// passed to [`filter`](Self::filter)
   ```



##########
parquet/src/arrow/arrow_reader/mod.rs:
##########
@@ -375,11 +340,36 @@ impl ParquetRecordBatchReader {
             batch_size,
             array_reader,
             schema: Arc::new(schema),
-            selection,
+            selection: selection.map(Into::into),
         }
     }
 }
 
+/// Evaluates an [`ArrowPredicate`] returning the [`RowSelection`]
+///
+/// If this [`ParquetRecordBatchReader`] has a [`RowSelection`], the
+/// returned [`RowSelection`] will be the conjunction of this and
+/// the rows selected by `predicate`
+pub(crate) fn evaluate_predicate(
+    batch_size: usize,
+    array_reader: Box<dyn ArrayReader>,
+    selection: Option<RowSelection>,

Review Comment:
   It might make the code clearer if this were called `input_selection` to distinguish it from the output selection
   
   ```suggestion
       input_selection: Option<RowSelection>,
   ```



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;
+}
+
+/// An [`ArrowPredicate`] created from an [`FnMut`]
+pub struct ArrowPredicateFn<F> {
+    f: F,
+    projection: ProjectionMask,
+}
+
+impl<F> ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    /// Create a new [`ArrowPredicateFn`]
+    pub fn new(projection: ProjectionMask, f: F) -> Self {
+        Self { f, projection }
+    }
+}
+
+impl<F> ArrowPredicate for ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    fn projection(&self) -> &ProjectionMask {
+        &self.projection
+    }
+
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray> {
+        (self.f)(batch)
+    }
+}
+
+/// A [`RowFilter`] allows pushing down a filter predicate to skip IO and decode
+///
+/// This consists of a list of [`ArrowPredicate`] where only the rows that satisfy all
+/// of the predicates will be returned. Any [`RowSelection`] will be applied prior
+/// to the first predicate, and each predicate in turn will then be used to compute
+/// a more refined [`RowSelection`] to use when evaluating the subsequent predicates.
+///
+/// Once all predicates have been evaluated, the resulting [`RowSelection`] will be
+/// used to return just the desired rows.
+///
+/// This design has a couple of implications:
+///
+/// * [`RowFilter`] can be used to skip fetching IO, in addition to decode overheads
+/// * Columns may be decoded multiple times if they appear in multiple [`ProjectionMask`]
+/// * IO will be deferred until needed by a [`ProjectionMask`]
+///
+/// As such there is a trade-off between a single large predicate, or multiple predicates,
+/// that will depend on the shape of the data. Whilst multiple smaller predicates may
+/// minimise the amount of data scanned/decoded, it may not be faster overall.
+///

Review Comment:
   ```suggestion
   ///
   /// For example, if a predicate that needs a single column of data filters out all but
   /// 1% of the rows, applying it as one of the early `ArrowPredicateFn` will likely signficantly
   /// improve performance. 
   ///
   /// As a counter example, if a predicate needs several columns of data to evaluate but
   /// leaves 99% of the rows, it may be better to not filter the data from parquet and
   /// apply the filter after the RecordBatch has been fully decoded. 
   /// 
   ```



##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when
+/// scanning a parquet file
+#[derive(Debug, Clone, Copy)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,

Review Comment:
   I wonder if we could improve this to take less space in memory (and potentially be faster to process) by packing the row_count into a smaller size. As it is, I think this will take up 16 bytes:
   
   https://play.rust-lang.org/?version=stable&mode=debug&edition=2018&gist=3f7e94978d00ba227c594714b836e41d



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls

Review Comment:
   I recommend a slightly different semantic, consistent with SQL, that would allow more flexibility from users
   
   ```suggestion
       /// All row that are `true` in returned [`BooleanArray`] will be returned to the reader.
       /// Any rows that are `false` or `Null` will not be
   ```
   
   If performance is a concern, perhaps we could  provide specialized paths for a `BooleanArray` that has no nulls and note that in the comments



##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when
+/// scanning a parquet file
+#[derive(Debug, Clone, Copy)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,
+
+    /// If true, skip `row_count` rows
+    pub skip: bool,
+}
+
+impl RowSelector {
+    /// Select `row_count` rows
+    pub fn select(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: false,
+        }
+    }
+
+    /// Skip `row_count` rows
+    pub fn skip(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: true,
+        }
+    }
+}
+
+/// [`RowSelection`] allows selecting or skipping a provided number of rows
+/// when scanning the parquet file.
+///
+/// This is applied prior to reading column data, and can therefore
+/// be used to skip IO to fetch data into memory
+///
+/// A typical use-case would be using the [`PageIndex`] to filter out rows
+/// that don't satisfy a predicate
+///
+/// [`PageIndex`]: [crate::file::page_index::index::PageIndex]
+#[derive(Debug, Clone, Default)]
+pub struct RowSelection {
+    selectors: Vec<RowSelector>,
+}
+
+impl RowSelection {
+    /// Creates a [`RowSelection`] from a slice of [`BooleanArray`]
+    ///
+    /// # Panic
+    ///
+    /// Panics if any of the [`BooleanArray`] contain nulls
+    pub fn from_filters(filters: &[BooleanArray]) -> Self {
+        let mut next_offset = 0;
+        let total_rows = filters.iter().map(|x| x.len()).sum();
+
+        let iter = filters.iter().flat_map(|filter| {
+            let offset = next_offset;
+            next_offset += filter.len();
+            assert_eq!(filter.null_count(), 0);
+            SlicesIterator::new(filter)
+                .map(move |(start, end)| start + offset..end + offset)
+        });
+
+        Self::from_consecutive_ranges(iter, total_rows)
+    }
+
+    /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep
+    fn from_consecutive_ranges<I: Iterator<Item = Range<usize>>>(
+        ranges: I,
+        total_rows: usize,
+    ) -> Self {
+        let mut selectors: Vec<RowSelector> = Vec::with_capacity(ranges.size_hint().0);
+        let mut last_end = 0;
+        for range in ranges {
+            let len = range.end - range.start;
+
+            match range.start.cmp(&last_end) {
+                Ordering::Equal => match selectors.last_mut() {
+                    Some(last) => last.row_count += len,
+                    None => selectors.push(RowSelector::select(len)),
+                },
+                Ordering::Greater => {
+                    selectors.push(RowSelector::skip(range.start - last_end));
+                    selectors.push(RowSelector::select(len))
+                }
+                Ordering::Less => panic!("out of order"),
+            }
+            last_end = range.end;
+        }
+
+        if last_end != total_rows {
+            selectors.push(RowSelector::skip(total_rows - last_end))
+        }
+
+        Self { selectors }
+    }
+
+    /// Splits off `row_count` from this [`RowSelection`]
+    pub fn split_off(&mut self, row_count: usize) -> Self {
+        let mut total_count = 0;
+
+        // Find the index where the selector exceeds the row count
+        let find = self.selectors.iter().enumerate().find(|(_, selector)| {
+            total_count += selector.row_count;
+            total_count >= row_count
+        });
+
+        let split_idx = match find {
+            Some((idx, _)) => idx,
+            None => return Self::default(),
+        };
+
+        let mut remaining = self.selectors.split_off(split_idx);
+        if total_count != row_count {
+            let overflow = total_count - row_count;
+            let rem = remaining.first_mut().unwrap();
+            rem.row_count -= overflow;
+
+            self.selectors.push(RowSelector {
+                row_count,
+                skip: rem.skip,
+            })
+        }
+
+        std::mem::swap(&mut remaining, &mut self.selectors);
+        Self {
+            selectors: remaining,
+        }
+    }
+
+    /// Given a [`RowSelection`] computed under `self` returns the [`RowSelection`]
+    /// representing their conjunction
+    pub fn and(&self, other: &Self) -> Self {
+        let mut selectors = vec![];
+        let mut first = self.selectors.iter().cloned().peekable();
+        let mut second = other.selectors.iter().cloned().peekable();
+
+        let mut to_skip = 0;
+        while let (Some(a), Some(b)) = (first.peek_mut(), second.peek_mut()) {
+            if a.row_count == 0 {
+                first.next().unwrap();
+                continue;
+            }
+
+            if b.row_count == 0 {
+                second.next().unwrap();
+                continue;
+            }
+
+            if a.skip {
+                // Records were skipped when producing second
+                to_skip += a.row_count;
+                first.next().unwrap();
+                continue;
+            }
+
+            let skip = b.skip;
+            let to_process = a.row_count.min(b.row_count);
+
+            a.row_count -= to_process;
+            b.row_count -= to_process;
+
+            match skip {
+                true => to_skip += to_process,
+                false => {
+                    if to_skip != 0 {
+                        selectors.push(RowSelector::skip(to_skip));
+                        to_skip = 0;
+                    }
+                    selectors.push(RowSelector::select(to_process))
+                }
+            }
+        }
+        Self { selectors }
+    }
+
+    /// Returns `true` if this [`RowSelection`] selects any rows
+    pub fn selects_any(&self) -> bool {
+        self.selectors.iter().any(|x| !x.skip)
+    }
+}
+
+impl From<Vec<RowSelector>> for RowSelection {
+    fn from(selectors: Vec<RowSelector>) -> Self {
+        Self { selectors }
+    }
+}
+
+impl Into<VecDeque<RowSelector>> for RowSelection {
+    fn into(self) -> VecDeque<RowSelector> {
+        self.selectors.into()
+    }
+}

Review Comment:
   I agree -- I didn't review the logic in detail either as I figured we are just at the "API feedback" design phase



-- 
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 diff in pull request #2335: RFC: Add RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r938877205


##########
parquet/src/arrow/array_reader/builder.rs:
##########
@@ -39,20 +39,18 @@ use crate::data_type::{
     Int64Type, Int96Type,
 };
 use crate::errors::Result;
-use crate::schema::types::{ColumnDescriptor, ColumnPath, SchemaDescPtr, Type};
+use crate::schema::types::{ColumnDescriptor, ColumnPath, Type};
 
 /// Create array reader from parquet schema, projection mask, and parquet file reader.
 pub fn build_array_reader(
-    parquet_schema: SchemaDescPtr,
     arrow_schema: SchemaRef,
     mask: ProjectionMask,
-    row_groups: Box<dyn RowGroupCollection>,
+    row_groups: &dyn RowGroupCollection,

Review Comment:
   Drive by cleanup



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {

Review Comment:
   This is to make things more extensible in the long run



##########
parquet/src/arrow/array_reader/mod.rs:
##########
@@ -110,8 +110,8 @@ pub trait RowGroupCollection {
 }
 
 impl RowGroupCollection for Arc<dyn FileReader> {
-    fn schema(&self) -> Result<SchemaDescPtr> {
-        Ok(self.metadata().file_metadata().schema_descr_ptr())
+    fn schema(&self) -> SchemaDescPtr {

Review Comment:
   Drive by cleanup



##########
parquet/src/arrow/arrow_reader/mod.rs:
##########
@@ -128,18 +108,18 @@ impl ArrowReaderOptions {
 
     /// Scan rows from the parquet file according to the provided `selection`
     ///
-    /// TODO: Make public once row selection fully implemented (#1792)
-    pub(crate) fn with_row_selection(
-        self,
-        selection: impl Into<Vec<RowSelection>>,
-    ) -> Self {
+    /// TODO: Revisit this API, as [`Self`] is provided before the file metadata is available

Review Comment:
   I intend to revisit this as part of the next (21) arrow release, I suspect we can move to a builder and deprecate the current API which is quite clunky



##########
parquet/src/arrow/arrow_reader/mod.rs:
##########
@@ -375,11 +340,36 @@ impl ParquetRecordBatchReader {
             batch_size,
             array_reader,
             schema: Arc::new(schema),
-            selection,
+            selection: selection.map(Into::into),
         }
     }
 }
 
+/// Evaluates an [`ArrowPredicate`] returning the [`RowSelection`]
+///
+/// If this [`ParquetRecordBatchReader`] has a [`RowSelection`], the
+/// returned [`RowSelection`] will be the conjunction of this and
+/// the rows selected by `predicate`
+pub(crate) fn evaluate_predicate(
+    batch_size: usize,
+    array_reader: Box<dyn ArrayReader>,
+    selection: Option<RowSelection>,
+    predicate: &mut dyn ArrowPredicate,
+) -> Result<RowSelection> {
+    let reader =
+        ParquetRecordBatchReader::new(batch_size, array_reader, selection.clone());
+    let mut filters = vec![];

Review Comment:
   We could theoretically keep the decoded arrays around, but requires a lot of non-trivial `take` + `concat` in order to sync up the yielded batches. It also potentially balloons the memory consumption. I decided it was not worth it



##########
parquet/src/arrow/async_reader.rs:
##########
@@ -370,101 +503,40 @@ where
                         None => return Poll::Ready(None),
                     };
 
-                    let metadata = self.metadata.clone();
-                    let mut input = match self.input.take() {
-                        Some(input) => input,
-                        None => {
-                            self.state = StreamState::Error;
-                            return Poll::Ready(Some(Err(general_err!(
-                                "input stream lost"
-                            ))));
-                        }
-                    };
+                    let reader = self.reader.take().expect("lost reader");
 
-                    let projection = self.projection.clone();
-                    self.state = StreamState::Reading(
-                        async move {
-                            let row_group_metadata = metadata.row_group(row_group_idx);
-                            let mut column_chunks =
-                                vec![None; row_group_metadata.columns().len()];
-
-                            // TODO: Combine consecutive ranges
-                            let fetch_ranges = (0..column_chunks.len())

Review Comment:
   This logic is moved into `InMemoryRowGroup::fetch`



##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,95 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the projection mask for this predicate
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::mask`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// The returned [`BooleanArray`] must not contain any nulls
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;
+}
+
+/// An [`ArrowPredicate`] created from an [`FnMut`]
+pub struct ArrowPredicateFn<F> {
+    f: F,
+    projection: ProjectionMask,
+}
+
+impl<F> ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    /// Create a new [`ArrowPredicateFn`]
+    pub fn new(projection: ProjectionMask, f: F) -> Self {
+        Self { f, projection }
+    }
+}
+
+impl<F> ArrowPredicate for ArrowPredicateFn<F>
+where
+    F: FnMut(RecordBatch) -> ArrowResult<BooleanArray> + Send + 'static,
+{
+    fn projection(&self) -> &ProjectionMask {
+        &self.projection
+    }
+
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray> {
+        (self.f)(batch)
+    }
+}
+
+/// A [`RowFilter`] allows pushing down a filter predicate to skip IO and decode
+///
+/// This consists of a list of [`ArrowPredicate`] where only the rows that satisfy all
+/// of the predicates will be returned. Any [`RowSelection`] will be applied prior
+/// to the first predicate, and each predicate in turn will then be used to compute
+/// a more refined [`RowSelection`] to use when evaluating the subsequent predicates.
+///
+/// Once all predicates have been evaluated, the resulting [`RowSelection`] will be
+/// used to return just the desired rows.
+///
+/// This design has a couple of implications:

Review Comment:
   This is the major change vs #2310, FYI @thinkharderdev 



##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when
+/// scanning a parquet file
+#[derive(Debug, Clone, Copy)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,
+
+    /// If true, skip `row_count` rows
+    pub skip: bool,
+}
+
+impl RowSelector {
+    /// Select `row_count` rows
+    pub fn select(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: false,
+        }
+    }
+
+    /// Skip `row_count` rows
+    pub fn skip(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: true,
+        }
+    }
+}
+
+/// [`RowSelection`] allows selecting or skipping a provided number of rows
+/// when scanning the parquet file.
+///
+/// This is applied prior to reading column data, and can therefore
+/// be used to skip IO to fetch data into memory
+///
+/// A typical use-case would be using the [`PageIndex`] to filter out rows
+/// that don't satisfy a predicate
+///
+/// [`PageIndex`]: [crate::file::page_index::index::PageIndex]
+#[derive(Debug, Clone, Default)]
+pub struct RowSelection {
+    selectors: Vec<RowSelector>,
+}
+
+impl RowSelection {
+    /// Creates a [`RowSelection`] from a slice of [`BooleanArray`]
+    ///
+    /// # Panic
+    ///
+    /// Panics if any of the [`BooleanArray`] contain nulls
+    pub fn from_filters(filters: &[BooleanArray]) -> Self {
+        let mut next_offset = 0;
+        let total_rows = filters.iter().map(|x| x.len()).sum();
+
+        let iter = filters.iter().flat_map(|filter| {
+            let offset = next_offset;
+            next_offset += filter.len();
+            assert_eq!(filter.null_count(), 0);
+            SlicesIterator::new(filter)
+                .map(move |(start, end)| start + offset..end + offset)
+        });
+
+        Self::from_consecutive_ranges(iter, total_rows)
+    }
+
+    /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep
+    fn from_consecutive_ranges<I: Iterator<Item = Range<usize>>>(
+        ranges: I,
+        total_rows: usize,
+    ) -> Self {
+        let mut selectors: Vec<RowSelector> = Vec::with_capacity(ranges.size_hint().0);
+        let mut last_end = 0;
+        for range in ranges {
+            let len = range.end - range.start;
+
+            match range.start.cmp(&last_end) {
+                Ordering::Equal => match selectors.last_mut() {
+                    Some(last) => last.row_count += len,
+                    None => selectors.push(RowSelector::select(len)),
+                },
+                Ordering::Greater => {
+                    selectors.push(RowSelector::skip(range.start - last_end));
+                    selectors.push(RowSelector::select(len))
+                }
+                Ordering::Less => panic!("out of order"),
+            }
+            last_end = range.end;
+        }
+
+        if last_end != total_rows {
+            selectors.push(RowSelector::skip(total_rows - last_end))
+        }
+
+        Self { selectors }
+    }
+
+    /// Splits off `row_count` from this [`RowSelection`]
+    pub fn split_off(&mut self, row_count: usize) -> Self {
+        let mut total_count = 0;
+
+        // Find the index where the selector exceeds the row count
+        let find = self.selectors.iter().enumerate().find(|(_, selector)| {
+            total_count += selector.row_count;
+            total_count >= row_count
+        });
+
+        let split_idx = match find {
+            Some((idx, _)) => idx,
+            None => return Self::default(),
+        };
+
+        let mut remaining = self.selectors.split_off(split_idx);
+        if total_count != row_count {
+            let overflow = total_count - row_count;
+            let rem = remaining.first_mut().unwrap();
+            rem.row_count -= overflow;
+
+            self.selectors.push(RowSelector {
+                row_count,
+                skip: rem.skip,
+            })
+        }
+
+        std::mem::swap(&mut remaining, &mut self.selectors);
+        Self {
+            selectors: remaining,
+        }
+    }
+
+    /// Given a [`RowSelection`] computed under `self` returns the [`RowSelection`]
+    /// representing their conjunction
+    pub fn and(&self, other: &Self) -> Self {
+        let mut selectors = vec![];
+        let mut first = self.selectors.iter().cloned().peekable();
+        let mut second = other.selectors.iter().cloned().peekable();
+
+        let mut to_skip = 0;
+        while let (Some(a), Some(b)) = (first.peek_mut(), second.peek_mut()) {
+            if a.row_count == 0 {
+                first.next().unwrap();
+                continue;
+            }
+
+            if b.row_count == 0 {
+                second.next().unwrap();
+                continue;
+            }
+
+            if a.skip {
+                // Records were skipped when producing second
+                to_skip += a.row_count;
+                first.next().unwrap();
+                continue;
+            }
+
+            let skip = b.skip;
+            let to_process = a.row_count.min(b.row_count);
+
+            a.row_count -= to_process;
+            b.row_count -= to_process;
+
+            match skip {
+                true => to_skip += to_process,
+                false => {
+                    if to_skip != 0 {
+                        selectors.push(RowSelector::skip(to_skip));
+                        to_skip = 0;
+                    }
+                    selectors.push(RowSelector::select(to_process))
+                }
+            }
+        }
+        Self { selectors }
+    }
+
+    /// Returns `true` if this [`RowSelection`] selects any rows
+    pub fn selects_any(&self) -> bool {
+        self.selectors.iter().any(|x| !x.skip)
+    }
+}
+
+impl From<Vec<RowSelector>> for RowSelection {
+    fn from(selectors: Vec<RowSelector>) -> Self {
+        Self { selectors }
+    }
+}
+
+impl Into<VecDeque<RowSelector>> for RowSelection {
+    fn into(self) -> VecDeque<RowSelector> {
+        self.selectors.into()
+    }
+}

Review Comment:
   This file definitely needs some tests prior to merge. The code is largely lifted from #2201 



##########
parquet/src/arrow/arrow_reader/mod.rs:
##########
@@ -349,22 +323,13 @@ impl RecordBatchReader for ParquetRecordBatchReader {
 }
 
 impl ParquetRecordBatchReader {
-    pub fn try_new(

Review Comment:
   This module is not public, and this method was only being used in one place, so we can just remove 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 pull request #2335: Add Parquet RowFilter API

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

   I think this is now good for review, it isn't public (yet) and so doesn't need to be perfect, but I think we can continue to iterate on this. I will file follow up tickets for follow on work tomorrow.


-- 
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 diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r943596329


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,417 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelector`] represents a range of rows to scan from a parquet file
+#[derive(Debug, Clone, Copy, PartialEq, Eq)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,
+
+    /// If true, skip `row_count` rows
+    pub skip: bool,
+}
+
+impl RowSelector {
+    /// Select `row_count` rows
+    pub fn select(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: false,
+        }
+    }
+
+    /// Skip `row_count` rows
+    pub fn skip(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: true,
+        }
+    }
+}
+
+/// [`RowSelection`] allows selecting or skipping a provided number of rows
+/// when scanning the parquet file.
+///
+/// This is applied prior to reading column data, and can therefore
+/// be used to skip IO to fetch data into memory
+///
+/// A typical use-case would be using the [`PageIndex`] to filter out rows
+/// that don't satisfy a predicate
+///
+/// [`PageIndex`]: [crate::file::page_index::index::PageIndex]
+#[derive(Debug, Clone, Default, Eq, PartialEq)]
+pub struct RowSelection {
+    selectors: Vec<RowSelector>,
+}
+
+impl RowSelection {
+    /// Creates a [`RowSelection`] from a slice of [`BooleanArray`]
+    ///
+    /// # Panic
+    ///
+    /// Panics if any of the [`BooleanArray`] contain nulls
+    pub fn from_filters(filters: &[BooleanArray]) -> Self {
+        let mut next_offset = 0;
+        let total_rows = filters.iter().map(|x| x.len()).sum();
+
+        let iter = filters.iter().flat_map(|filter| {
+            let offset = next_offset;
+            next_offset += filter.len();
+            assert_eq!(filter.null_count(), 0);
+            SlicesIterator::new(filter)
+                .map(move |(start, end)| start + offset..end + offset)
+        });
+
+        Self::from_consecutive_ranges(iter, total_rows)
+    }
+
+    /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep
+    fn from_consecutive_ranges<I: Iterator<Item = Range<usize>>>(
+        ranges: I,
+        total_rows: usize,
+    ) -> Self {
+        let mut selectors: Vec<RowSelector> = Vec::with_capacity(ranges.size_hint().0);
+        let mut last_end = 0;
+        for range in ranges {
+            let len = range.end - range.start;
+
+            match range.start.cmp(&last_end) {
+                Ordering::Equal => match selectors.last_mut() {
+                    Some(last) => last.row_count += len,
+                    None => selectors.push(RowSelector::select(len)),
+                },
+                Ordering::Greater => {
+                    selectors.push(RowSelector::skip(range.start - last_end));
+                    selectors.push(RowSelector::select(len))
+                }
+                Ordering::Less => panic!("out of order"),
+            }
+            last_end = range.end;
+        }
+
+        if last_end != total_rows {
+            selectors.push(RowSelector::skip(total_rows - last_end))
+        }
+
+        Self { selectors }
+    }
+
+    /// Splits off the first `row_count` from this [`RowSelection`]
+    pub fn split_off(&mut self, row_count: usize) -> Self {
+        let mut total_count = 0;
+
+        // Find the index where the selector exceeds the row count
+        let find = self.selectors.iter().enumerate().find(|(_, selector)| {
+            total_count += selector.row_count;
+            total_count > row_count
+        });
+
+        let split_idx = match find {
+            Some((idx, _)) => idx,
+            None => {
+                let selectors = std::mem::take(&mut self.selectors);
+                return Self { selectors };
+            }
+        };
+
+        let mut remaining = self.selectors.split_off(split_idx);
+
+        // Always present as `split_idx < self.selectors.len`
+        let next = remaining.first_mut().unwrap();
+        let overflow = total_count - row_count;
+
+        if next.row_count != overflow {
+            self.selectors.push(RowSelector {
+                row_count: next.row_count - overflow,
+                skip: next.skip,
+            })
+        }
+        next.row_count = overflow;
+
+        std::mem::swap(&mut remaining, &mut self.selectors);
+        Self {
+            selectors: remaining,
+        }
+    }
+
+    /// Given a [`RowSelection`] computed under `self` returns the [`RowSelection`]
+    /// representing their conjunction
+    pub fn and(&self, other: &Self) -> Self {
+        let mut selectors = vec![];
+        let mut first = self.selectors.iter().cloned().peekable();
+        let mut second = other.selectors.iter().cloned().peekable();
+
+        let mut to_skip = 0;
+        while let Some(b) = second.peek_mut() {
+            let a = first.peek_mut().unwrap();
+
+            if b.row_count == 0 {
+                second.next().unwrap();
+                continue;
+            }
+
+            if a.row_count == 0 {
+                first.next().unwrap();
+                continue;
+            }
+
+            if a.skip {
+                // Records were skipped when producing second
+                to_skip += a.row_count;
+                first.next().unwrap();
+                continue;
+            }
+
+            let skip = b.skip;
+            let to_process = a.row_count.min(b.row_count);
+
+            a.row_count -= to_process;
+            b.row_count -= to_process;
+
+            match skip {
+                true => to_skip += to_process,
+                false => {
+                    if to_skip != 0 {
+                        selectors.push(RowSelector::skip(to_skip));
+                        to_skip = 0;
+                    }
+                    selectors.push(RowSelector::select(to_process))
+                }
+            }
+        }
+
+        for v in first {
+            if v.row_count != 0 {
+                assert!(v.skip);
+                to_skip += v.row_count
+            }
+        }
+
+        if to_skip != 0 {
+            selectors.push(RowSelector::skip(to_skip));
+        }
+
+        Self { selectors }
+    }
+
+    /// Returns `true` if this [`RowSelection`] selects any rows
+    pub fn selects_any(&self) -> bool {
+        self.selectors.iter().any(|x| !x.skip)
+    }
+}
+
+impl From<Vec<RowSelector>> for RowSelection {
+    fn from(selectors: Vec<RowSelector>) -> Self {
+        Self { selectors }
+    }
+}
+
+impl From<RowSelection> for VecDeque<RowSelector> {
+    fn from(r: RowSelection) -> Self {
+        r.selectors.into()
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use rand::{thread_rng, Rng};
+
+    #[test]
+    fn test_from_filters() {
+        let filters = vec![
+            BooleanArray::from(vec![false, false, false, true, true, true, true]),
+            BooleanArray::from(vec![true, true, false, false, true, true, true]),
+            BooleanArray::from(vec![false, false, false, false]),
+            BooleanArray::from(Vec::<bool>::new()),
+        ];
+
+        let selection = RowSelection::from_filters(&filters[..1]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(3), RowSelector::select(4)]
+        );
+
+        let selection = RowSelection::from_filters(&filters[..2]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3),
+                RowSelector::skip(4)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters[2..3]);
+        assert!(!selection.selects_any());
+        assert_eq!(selection.selectors, vec![RowSelector::skip(4)]);
+    }
+
+    #[test]
+    fn test_split_off() {
+        let mut selection = RowSelection::from(vec![
+            RowSelector::skip(34),
+            RowSelector::select(12),
+            RowSelector::skip(3),
+            RowSelector::select(35),
+        ]);
+
+        let split = selection.split_off(34);
+        assert_eq!(split.selectors, vec![RowSelector::skip(34)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(12),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(5);
+        assert_eq!(split.selectors, vec![RowSelector::select(5)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(7),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(8);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::select(7), RowSelector::skip(1)]
+        );
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+
+        let split = selection.split_off(200);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+        assert!(selection.selectors.is_empty());
+    }
+
+    #[test]
+    fn test_and() {
+        let mut a = RowSelection::from(vec![
+            RowSelector::skip(12),
+            RowSelector::select(23),
+            RowSelector::skip(3),
+            RowSelector::select(5),
+        ]);
+
+        let b = RowSelection::from(vec![
+            RowSelector::select(5),
+            RowSelector::skip(4),
+            RowSelector::select(15),
+            RowSelector::skip(4),
+        ]);
+
+        let mut expected = RowSelection::from(vec![

Review Comment:
   The expected answer doesn't make sense to me. When I did it out there seems to be something wrong
   
   N = Skip
   Y = Select
   
   ```text
   a: NNNNNNNNNNNNYYYYYYYYYYYYYYYYYYYYYYNNNYYYYY
   b: YYYYYNNNNYYYYYYYYYYYYYYYNNN
   
   What is here:
   e: NNNNNNNNNNNNYYYYYNNNNYYYYYYYYYYYYYYNNYNNNN
   
   What I think the answer should be:
   e: NNNNNNNNNNNNYYYYYYYYYYYYNNNYYYYYYYNNNYYYYY
   ```
   
   Though to be honest I am not sure sure what an `AND` and nulls should be
   
   I am probably missing something obvious 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] thinkharderdev commented on a diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r943607581


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,417 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelector`] represents a range of rows to scan from a parquet file
+#[derive(Debug, Clone, Copy, PartialEq, Eq)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,
+
+    /// If true, skip `row_count` rows
+    pub skip: bool,
+}
+
+impl RowSelector {
+    /// Select `row_count` rows
+    pub fn select(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: false,
+        }
+    }
+
+    /// Skip `row_count` rows
+    pub fn skip(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: true,
+        }
+    }
+}
+
+/// [`RowSelection`] allows selecting or skipping a provided number of rows
+/// when scanning the parquet file.
+///
+/// This is applied prior to reading column data, and can therefore
+/// be used to skip IO to fetch data into memory
+///
+/// A typical use-case would be using the [`PageIndex`] to filter out rows
+/// that don't satisfy a predicate
+///
+/// [`PageIndex`]: [crate::file::page_index::index::PageIndex]
+#[derive(Debug, Clone, Default, Eq, PartialEq)]
+pub struct RowSelection {
+    selectors: Vec<RowSelector>,
+}
+
+impl RowSelection {
+    /// Creates a [`RowSelection`] from a slice of [`BooleanArray`]
+    ///
+    /// # Panic
+    ///
+    /// Panics if any of the [`BooleanArray`] contain nulls
+    pub fn from_filters(filters: &[BooleanArray]) -> Self {
+        let mut next_offset = 0;
+        let total_rows = filters.iter().map(|x| x.len()).sum();
+
+        let iter = filters.iter().flat_map(|filter| {
+            let offset = next_offset;
+            next_offset += filter.len();
+            assert_eq!(filter.null_count(), 0);
+            SlicesIterator::new(filter)
+                .map(move |(start, end)| start + offset..end + offset)
+        });
+
+        Self::from_consecutive_ranges(iter, total_rows)
+    }
+
+    /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep
+    fn from_consecutive_ranges<I: Iterator<Item = Range<usize>>>(
+        ranges: I,
+        total_rows: usize,
+    ) -> Self {
+        let mut selectors: Vec<RowSelector> = Vec::with_capacity(ranges.size_hint().0);
+        let mut last_end = 0;
+        for range in ranges {
+            let len = range.end - range.start;
+
+            match range.start.cmp(&last_end) {
+                Ordering::Equal => match selectors.last_mut() {
+                    Some(last) => last.row_count += len,
+                    None => selectors.push(RowSelector::select(len)),
+                },
+                Ordering::Greater => {
+                    selectors.push(RowSelector::skip(range.start - last_end));
+                    selectors.push(RowSelector::select(len))
+                }
+                Ordering::Less => panic!("out of order"),
+            }
+            last_end = range.end;
+        }
+
+        if last_end != total_rows {
+            selectors.push(RowSelector::skip(total_rows - last_end))
+        }
+
+        Self { selectors }
+    }
+
+    /// Splits off the first `row_count` from this [`RowSelection`]
+    pub fn split_off(&mut self, row_count: usize) -> Self {
+        let mut total_count = 0;
+
+        // Find the index where the selector exceeds the row count
+        let find = self.selectors.iter().enumerate().find(|(_, selector)| {
+            total_count += selector.row_count;
+            total_count > row_count
+        });
+
+        let split_idx = match find {
+            Some((idx, _)) => idx,
+            None => {
+                let selectors = std::mem::take(&mut self.selectors);
+                return Self { selectors };
+            }
+        };
+
+        let mut remaining = self.selectors.split_off(split_idx);
+
+        // Always present as `split_idx < self.selectors.len`
+        let next = remaining.first_mut().unwrap();
+        let overflow = total_count - row_count;
+
+        if next.row_count != overflow {
+            self.selectors.push(RowSelector {
+                row_count: next.row_count - overflow,
+                skip: next.skip,
+            })
+        }
+        next.row_count = overflow;
+
+        std::mem::swap(&mut remaining, &mut self.selectors);
+        Self {
+            selectors: remaining,
+        }
+    }
+
+    /// Given a [`RowSelection`] computed under `self` returns the [`RowSelection`]
+    /// representing their conjunction
+    pub fn and(&self, other: &Self) -> Self {
+        let mut selectors = vec![];
+        let mut first = self.selectors.iter().cloned().peekable();
+        let mut second = other.selectors.iter().cloned().peekable();
+
+        let mut to_skip = 0;
+        while let Some(b) = second.peek_mut() {
+            let a = first.peek_mut().unwrap();
+
+            if b.row_count == 0 {
+                second.next().unwrap();
+                continue;
+            }
+
+            if a.row_count == 0 {
+                first.next().unwrap();
+                continue;
+            }
+
+            if a.skip {
+                // Records were skipped when producing second
+                to_skip += a.row_count;
+                first.next().unwrap();
+                continue;
+            }
+
+            let skip = b.skip;
+            let to_process = a.row_count.min(b.row_count);
+
+            a.row_count -= to_process;
+            b.row_count -= to_process;
+
+            match skip {
+                true => to_skip += to_process,
+                false => {
+                    if to_skip != 0 {
+                        selectors.push(RowSelector::skip(to_skip));
+                        to_skip = 0;
+                    }
+                    selectors.push(RowSelector::select(to_process))
+                }
+            }
+        }
+
+        for v in first {
+            if v.row_count != 0 {
+                assert!(v.skip);
+                to_skip += v.row_count
+            }
+        }
+
+        if to_skip != 0 {
+            selectors.push(RowSelector::skip(to_skip));
+        }
+
+        Self { selectors }
+    }
+
+    /// Returns `true` if this [`RowSelection`] selects any rows
+    pub fn selects_any(&self) -> bool {
+        self.selectors.iter().any(|x| !x.skip)
+    }
+}
+
+impl From<Vec<RowSelector>> for RowSelection {
+    fn from(selectors: Vec<RowSelector>) -> Self {
+        Self { selectors }
+    }
+}
+
+impl From<RowSelection> for VecDeque<RowSelector> {
+    fn from(r: RowSelection) -> Self {
+        r.selectors.into()
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use rand::{thread_rng, Rng};
+
+    #[test]
+    fn test_from_filters() {
+        let filters = vec![
+            BooleanArray::from(vec![false, false, false, true, true, true, true]),
+            BooleanArray::from(vec![true, true, false, false, true, true, true]),
+            BooleanArray::from(vec![false, false, false, false]),
+            BooleanArray::from(Vec::<bool>::new()),
+        ];
+
+        let selection = RowSelection::from_filters(&filters[..1]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(3), RowSelector::select(4)]
+        );
+
+        let selection = RowSelection::from_filters(&filters[..2]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3),
+                RowSelector::skip(4)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters[2..3]);
+        assert!(!selection.selects_any());
+        assert_eq!(selection.selectors, vec![RowSelector::skip(4)]);
+    }
+
+    #[test]
+    fn test_split_off() {
+        let mut selection = RowSelection::from(vec![
+            RowSelector::skip(34),
+            RowSelector::select(12),
+            RowSelector::skip(3),
+            RowSelector::select(35),
+        ]);
+
+        let split = selection.split_off(34);
+        assert_eq!(split.selectors, vec![RowSelector::skip(34)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(12),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(5);
+        assert_eq!(split.selectors, vec![RowSelector::select(5)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(7),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(8);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::select(7), RowSelector::skip(1)]
+        );
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+
+        let split = selection.split_off(200);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+        assert!(selection.selectors.is_empty());
+    }
+
+    #[test]
+    fn test_and() {
+        let mut a = RowSelection::from(vec![
+            RowSelector::skip(12),
+            RowSelector::select(23),
+            RowSelector::skip(3),
+            RowSelector::select(5),
+        ]);
+
+        let b = RowSelection::from(vec![
+            RowSelector::select(5),
+            RowSelector::skip(4),
+            RowSelector::select(15),
+            RowSelector::skip(4),
+        ]);
+
+        let mut expected = RowSelection::from(vec![

Review Comment:
   That looks right to me. The `and` should just be giving you the result of applying the filters sequentially. Visually, it makes sense (to me) like
   ```
   a: NNNNNNNNNNNNYYYYYYYYYYYYYYYYYYYYYYNNNYYYYY
   b:             YYYYYNNNNYYYYYYYYYYYYY   YYNNN
   
      NNNNNNNNNNNNYYYYYNNNNYYYYYYYYYYYYYYNNYNNNN
   ```



-- 
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 diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r943610452


##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,110 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the [`ProjectionMask`] that describes the columns required
+    /// to evaluate this predicate. All projected columns will be provided in the `batch`
+    /// passed to [`filter`](Self::filter)
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::projection`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// All row that are `true` in returned [`BooleanArray`] will be returned to the reader.
+    /// Any rows that are `false` or `Null` will not be
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;

Review Comment:
   I think both names are kind of confusing tbh, I'll rename it to `evaluate` as I think that should be clear



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
tustvold commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939534771


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,215 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelection`] is a collection of [`RowSelect`] used to skip rows when
+/// scanning a parquet file
+#[derive(Debug, Clone, Copy)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,

Review Comment:
   Parquet can't provide random access, and so RLE is not an option



-- 
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] Ted-Jiang commented on a diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
Ted-Jiang commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r939543038


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,

Review Comment:
   Is there a situation we need read row groups with previous row group selection? 🤔
   
   Edit: sorry it may come from `pageIndex`, forgive me



##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,

Review Comment:
   Maybe we need the previous each filter rate 😂(just a idea)



-- 
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 diff in pull request #2335: RFC: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r940678795


##########
parquet/src/arrow/async_reader.rs:
##########
@@ -271,25 +304,122 @@ impl<T: AsyncFileReader> ParquetRecordBatchStreamBuilder<T> {
             None => (0..self.metadata.row_groups().len()).collect(),
         };
 
+        let reader = ReaderFactory {
+            input: self.input,
+            filter: self.filter,
+            metadata: self.metadata.clone(),
+            schema: self.schema.clone(),
+        };
+
         Ok(ParquetRecordBatchStream {
+            metadata: self.metadata,
+            batch_size: self.batch_size,
             row_groups,
             projection: self.projection,
-            batch_size: self.batch_size,
-            metadata: self.metadata,
+            selection: self.selection,
             schema: self.schema,
-            input: Some(self.input),
+            reader: Some(reader),
             state: StreamState::Init,
         })
     }
 }
 
+type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;
+
+/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
+/// [`ParquetRecordBatchReader`]
+struct ReaderFactory<T> {
+    metadata: Arc<ParquetMetaData>,
+
+    schema: SchemaRef,
+
+    input: T,
+
+    filter: Option<RowFilter>,
+}
+
+impl<T> ReaderFactory<T>
+where
+    T: AsyncFileReader + Send,
+{
+    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
+    ///
+    /// Note: this captures self so that the resulting future has a static lifetime
+    async fn read_row_group(
+        mut self,
+        row_group_idx: usize,
+        mut selection: Option<RowSelection>,
+        projection: ProjectionMask,
+        batch_size: usize,
+    ) -> ReadResult<T> {
+        // TODO: calling build_array multiple times is wasteful
+        let selects_any = |selection: Option<&RowSelection>| {
+            selection.map(|x| x.selects_any()).unwrap_or(true)
+        };
+
+        let meta = self.metadata.row_group(row_group_idx);
+        let mut row_group = InMemoryRowGroup {
+            schema: meta.schema_descr_ptr(),
+            row_count: meta.num_rows() as usize,
+            column_chunks: vec![None; meta.columns().len()],
+        };
+
+        if let Some(filter) = self.filter.as_mut() {
+            for predicate in filter.predicates.iter_mut() {
+                if !selects_any(selection.as_ref()) {
+                    return Ok((self, None));
+                }
+
+                let predicate_projection = predicate.projection().clone();
+                row_group
+                    .fetch(
+                        &mut self.input,
+                        meta,
+                        &predicate_projection,
+                        selection.as_ref(),
+                    )
+                    .await?;
+
+                let array_reader = build_array_reader(
+                    self.schema.clone(),
+                    predicate_projection,
+                    &row_group,
+                )?;
+
+                selection = Some(evaluate_predicate(
+                    batch_size,
+                    array_reader,
+                    selection,
+                    predicate.as_mut(),
+                )?);
+            }
+        }
+
+        if !selects_any(selection.as_ref()) {
+            return Ok((self, None));
+        }
+
+        row_group
+            .fetch(&mut self.input, meta, &projection, selection.as_ref())
+            .await?;
+
+        let reader = ParquetRecordBatchReader::new(
+            batch_size,
+            build_array_reader(self.schema.clone(), projection, &row_group)?,
+            selection,
+        );

Review Comment:
   > We went from being mostly CPU bound with parquet decoding to being mostly IO bound which means I expect there is even more room for improvement once we are using the selection and the page index to avoid IO altogether. That's all to say I'm super excited about this work and think it will be a huge step forward!
   
   That is terrific news 🎉 



-- 
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 diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
yordan-pavlov commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r941827220


##########
parquet/src/arrow/arrow_reader/filter.rs:
##########
@@ -0,0 +1,110 @@
+// 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::ProjectionMask;
+use arrow::array::BooleanArray;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+/// A predicate operating on [`RecordBatch`]
+pub trait ArrowPredicate: Send + 'static {
+    /// Returns the [`ProjectionMask`] that describes the columns required
+    /// to evaluate this predicate. All projected columns will be provided in the `batch`
+    /// passed to [`filter`](Self::filter)
+    fn projection(&self) -> &ProjectionMask;
+
+    /// Called with a [`RecordBatch`] containing the columns identified by [`Self::projection`],
+    /// with `true` values in the returned [`BooleanArray`] indicating rows
+    /// matching the predicate.
+    ///
+    /// All row that are `true` in returned [`BooleanArray`] will be returned to the reader.
+    /// Any rows that are `false` or `Null` will not be
+    fn filter(&mut self, batch: RecordBatch) -> ArrowResult<BooleanArray>;

Review Comment:
   should this be named `fn filter_array` instead to better indicate that the result would be a Boolean filter array instead of actually filtering the RecordBatch passed in as the batch parameter?



-- 
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] thinkharderdev commented on a diff in pull request #2335: Add Parquet RowFilter API

Posted by GitBox <gi...@apache.org>.
thinkharderdev commented on code in PR #2335:
URL: https://github.com/apache/arrow-rs/pull/2335#discussion_r943623221


##########
parquet/src/arrow/arrow_reader/selection.rs:
##########
@@ -0,0 +1,417 @@
+// 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 arrow::array::{Array, BooleanArray};
+use arrow::compute::SlicesIterator;
+use std::cmp::Ordering;
+use std::collections::VecDeque;
+use std::ops::Range;
+
+/// [`RowSelector`] represents a range of rows to scan from a parquet file
+#[derive(Debug, Clone, Copy, PartialEq, Eq)]
+pub struct RowSelector {
+    /// The number of rows
+    pub row_count: usize,
+
+    /// If true, skip `row_count` rows
+    pub skip: bool,
+}
+
+impl RowSelector {
+    /// Select `row_count` rows
+    pub fn select(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: false,
+        }
+    }
+
+    /// Skip `row_count` rows
+    pub fn skip(row_count: usize) -> Self {
+        Self {
+            row_count,
+            skip: true,
+        }
+    }
+}
+
+/// [`RowSelection`] allows selecting or skipping a provided number of rows
+/// when scanning the parquet file.
+///
+/// This is applied prior to reading column data, and can therefore
+/// be used to skip IO to fetch data into memory
+///
+/// A typical use-case would be using the [`PageIndex`] to filter out rows
+/// that don't satisfy a predicate
+///
+/// [`PageIndex`]: [crate::file::page_index::index::PageIndex]
+#[derive(Debug, Clone, Default, Eq, PartialEq)]
+pub struct RowSelection {
+    selectors: Vec<RowSelector>,
+}
+
+impl RowSelection {
+    /// Creates a [`RowSelection`] from a slice of [`BooleanArray`]
+    ///
+    /// # Panic
+    ///
+    /// Panics if any of the [`BooleanArray`] contain nulls
+    pub fn from_filters(filters: &[BooleanArray]) -> Self {
+        let mut next_offset = 0;
+        let total_rows = filters.iter().map(|x| x.len()).sum();
+
+        let iter = filters.iter().flat_map(|filter| {
+            let offset = next_offset;
+            next_offset += filter.len();
+            assert_eq!(filter.null_count(), 0);
+            SlicesIterator::new(filter)
+                .map(move |(start, end)| start + offset..end + offset)
+        });
+
+        Self::from_consecutive_ranges(iter, total_rows)
+    }
+
+    /// Creates a [`RowSelection`] from an iterator of consecutive ranges to keep
+    fn from_consecutive_ranges<I: Iterator<Item = Range<usize>>>(
+        ranges: I,
+        total_rows: usize,
+    ) -> Self {
+        let mut selectors: Vec<RowSelector> = Vec::with_capacity(ranges.size_hint().0);
+        let mut last_end = 0;
+        for range in ranges {
+            let len = range.end - range.start;
+
+            match range.start.cmp(&last_end) {
+                Ordering::Equal => match selectors.last_mut() {
+                    Some(last) => last.row_count += len,
+                    None => selectors.push(RowSelector::select(len)),
+                },
+                Ordering::Greater => {
+                    selectors.push(RowSelector::skip(range.start - last_end));
+                    selectors.push(RowSelector::select(len))
+                }
+                Ordering::Less => panic!("out of order"),
+            }
+            last_end = range.end;
+        }
+
+        if last_end != total_rows {
+            selectors.push(RowSelector::skip(total_rows - last_end))
+        }
+
+        Self { selectors }
+    }
+
+    /// Splits off the first `row_count` from this [`RowSelection`]
+    pub fn split_off(&mut self, row_count: usize) -> Self {
+        let mut total_count = 0;
+
+        // Find the index where the selector exceeds the row count
+        let find = self.selectors.iter().enumerate().find(|(_, selector)| {
+            total_count += selector.row_count;
+            total_count > row_count
+        });
+
+        let split_idx = match find {
+            Some((idx, _)) => idx,
+            None => {
+                let selectors = std::mem::take(&mut self.selectors);
+                return Self { selectors };
+            }
+        };
+
+        let mut remaining = self.selectors.split_off(split_idx);
+
+        // Always present as `split_idx < self.selectors.len`
+        let next = remaining.first_mut().unwrap();
+        let overflow = total_count - row_count;
+
+        if next.row_count != overflow {
+            self.selectors.push(RowSelector {
+                row_count: next.row_count - overflow,
+                skip: next.skip,
+            })
+        }
+        next.row_count = overflow;
+
+        std::mem::swap(&mut remaining, &mut self.selectors);
+        Self {
+            selectors: remaining,
+        }
+    }
+
+    /// Given a [`RowSelection`] computed under `self` returns the [`RowSelection`]
+    /// representing their conjunction
+    pub fn and(&self, other: &Self) -> Self {
+        let mut selectors = vec![];
+        let mut first = self.selectors.iter().cloned().peekable();
+        let mut second = other.selectors.iter().cloned().peekable();
+
+        let mut to_skip = 0;
+        while let Some(b) = second.peek_mut() {
+            let a = first.peek_mut().unwrap();
+
+            if b.row_count == 0 {
+                second.next().unwrap();
+                continue;
+            }
+
+            if a.row_count == 0 {
+                first.next().unwrap();
+                continue;
+            }
+
+            if a.skip {
+                // Records were skipped when producing second
+                to_skip += a.row_count;
+                first.next().unwrap();
+                continue;
+            }
+
+            let skip = b.skip;
+            let to_process = a.row_count.min(b.row_count);
+
+            a.row_count -= to_process;
+            b.row_count -= to_process;
+
+            match skip {
+                true => to_skip += to_process,
+                false => {
+                    if to_skip != 0 {
+                        selectors.push(RowSelector::skip(to_skip));
+                        to_skip = 0;
+                    }
+                    selectors.push(RowSelector::select(to_process))
+                }
+            }
+        }
+
+        for v in first {
+            if v.row_count != 0 {
+                assert!(v.skip);
+                to_skip += v.row_count
+            }
+        }
+
+        if to_skip != 0 {
+            selectors.push(RowSelector::skip(to_skip));
+        }
+
+        Self { selectors }
+    }
+
+    /// Returns `true` if this [`RowSelection`] selects any rows
+    pub fn selects_any(&self) -> bool {
+        self.selectors.iter().any(|x| !x.skip)
+    }
+}
+
+impl From<Vec<RowSelector>> for RowSelection {
+    fn from(selectors: Vec<RowSelector>) -> Self {
+        Self { selectors }
+    }
+}
+
+impl From<RowSelection> for VecDeque<RowSelector> {
+    fn from(r: RowSelection) -> Self {
+        r.selectors.into()
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use rand::{thread_rng, Rng};
+
+    #[test]
+    fn test_from_filters() {
+        let filters = vec![
+            BooleanArray::from(vec![false, false, false, true, true, true, true]),
+            BooleanArray::from(vec![true, true, false, false, true, true, true]),
+            BooleanArray::from(vec![false, false, false, false]),
+            BooleanArray::from(Vec::<bool>::new()),
+        ];
+
+        let selection = RowSelection::from_filters(&filters[..1]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(3), RowSelector::select(4)]
+        );
+
+        let selection = RowSelection::from_filters(&filters[..2]);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters);
+        assert!(selection.selects_any());
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::skip(3),
+                RowSelector::select(6),
+                RowSelector::skip(2),
+                RowSelector::select(3),
+                RowSelector::skip(4)
+            ]
+        );
+
+        let selection = RowSelection::from_filters(&filters[2..3]);
+        assert!(!selection.selects_any());
+        assert_eq!(selection.selectors, vec![RowSelector::skip(4)]);
+    }
+
+    #[test]
+    fn test_split_off() {
+        let mut selection = RowSelection::from(vec![
+            RowSelector::skip(34),
+            RowSelector::select(12),
+            RowSelector::skip(3),
+            RowSelector::select(35),
+        ]);
+
+        let split = selection.split_off(34);
+        assert_eq!(split.selectors, vec![RowSelector::skip(34)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(12),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(5);
+        assert_eq!(split.selectors, vec![RowSelector::select(5)]);
+        assert_eq!(
+            selection.selectors,
+            vec![
+                RowSelector::select(7),
+                RowSelector::skip(3),
+                RowSelector::select(35)
+            ]
+        );
+
+        let split = selection.split_off(8);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::select(7), RowSelector::skip(1)]
+        );
+        assert_eq!(
+            selection.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+
+        let split = selection.split_off(200);
+        assert_eq!(
+            split.selectors,
+            vec![RowSelector::skip(2), RowSelector::select(35)]
+        );
+        assert!(selection.selectors.is_empty());
+    }
+
+    #[test]
+    fn test_and() {
+        let mut a = RowSelection::from(vec![
+            RowSelector::skip(12),
+            RowSelector::select(23),
+            RowSelector::skip(3),
+            RowSelector::select(5),
+        ]);
+
+        let b = RowSelection::from(vec![
+            RowSelector::select(5),
+            RowSelector::skip(4),
+            RowSelector::select(15),
+            RowSelector::skip(4),
+        ]);
+
+        let mut expected = RowSelection::from(vec![

Review Comment:
   FWIW I think `and` is a little confusing since we're dealing with a boolean array and it is not clear from the name that this is essentially a composition operator. Maybe `and_then`?



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