Hi Micah,

Answers inline.

Another item that we need to think about is whether we want to
guarantee that all rows that are returned are matching? The stronger
guarantee means that we need to check each and every value, rather than
just relying on the min/max in the page headers, but relieves the caller
from re-filtering the values. I am inclining towards a stronger guarantee.

On Mon, Oct 31, 2022 at 10:58 PM Micah Kornfield <[email protected]>
wrote:

> Hi Fatemah,
> I think there are likely two things to consider here:
> 1.  How will expressions be modeled?  There are already some examples of
> using expressions in Arrow for pruning predicates [1].  Do you plan to
> re-use them?
>
Thanks for pointing this out. Yes, I will consider re-using the
arrow::compute::Expression for specifying the filter.


> 2.  Along these lines is the proposed approach taken because the API to
> expose the raw data necessary and filter externally too unwieldy?
>
I think it makes sense for parquet readers to support the filtering
functionality, rather than each client implementing their own filtering.
This would allow for more optimizations as well. For example, right now,
the page header metadata is not exposed, and the caller simply calls
ReadBatch, without caring which page they are in. To use the min/max in the
page headers for filtering, we would need to expose this and the read logic
would be more complicated.

>
> Thanks,
> Micah
>
>
> [1]
>
> https://github.com/apache/arrow/blob/5e49174d69deb9d1cbbdf82bc8041b90098f560b/cpp/src/arrow/dataset/file_parquet.cc
>
> On Mon, Oct 31, 2022 at 9:50 AM Fatemah Panahi <[email protected]>
> wrote:
>
> > -- Sending as an email in case Jira messages are filtered out. Please let
> > me know your thoughts on this. Thanks!
> >
> > Jira ticket: https://issues.apache.org/jira/browse/PARQUET-2210
> >
> > Currently, we do not use the statistics that is stored in the page
> headers
> > for pruning the rows that we read. Row group pruning is very
> coarse-grained
> > and in many cases does not prune the row group. I propose adding a
> > FilteredPageReader that would accept a filter and would not return the
> > pages that do not match the filter based on page statistics.
> >
> > Initial set of filters can be: EQUALS, IS NULL, IS NOT NULL.
> >
> > Also, the FilteredPageReader will keep track of what row ranges matched
> and
> > not matched. We could use this to skip reading rows that do not match
> from
> > the rest of the columns. Note that the SkipRecords API is being added to
> > the Parquet reader (https://issues.apache.org/jira/browse/PARQUET-2188)
> >
>

Reply via email to