Hello again,

I took a look through the mail archives and found a little more information
in this and a few other threads.

http://mail-archives.apache.org/mod_mbox//parquet-dev/201707.mbox/%3CCAO4re1k8-bZZZWBRuLCnm1V7AoJE1fdunSuBn%2BecRuFGPgcXnA%40mail.gmail.com%3E

While I do understand the benefits for federating out the reading of
footers for the sake of not worrying about synchronization between the
cached metadata and any changes to the files on disk, it does appear there
is still a use case that isn't solved well with this design, needle in a
haystack selective filter queries, where the data is sorted by the filter
column. For example in the tests I ran with queries against lots of parquet
files where the vast majority are pruned by a bunch of small tasks, it
takes 33 seconds vs just 1-2 seconds with driver side pruning using the
summary file (requires a small spark changet).

In our use case we are never going to be replacing contents of existing
parquet files (with a delete and rewrite on HDFS) or appending new row
groups onto existing files. In that case I don't believe we should
experience any correctness problems, but I wanted to confirm if there is
something I am missing. I am
using readAllFootersInParallelUsingSummaryFiles which does fall back to
read individual footers if they are missing from the summary file.

I am also curious if a solution to the correctness problems could be to
include a file length and/or last modified time into the summary file,
which could confirm against FS metadata that the files on disk are still in
sync with the metadata summary relatively quickly. Would it be possible to
consider avoiding this deprecation if I was to work on an update to
implement this?

- Jason Altekruse


On Tue, Sep 15, 2020 at 8:52 PM Jason Altekruse <altekruseja...@gmail.com>
wrote:

> Hello all,
>
> I have been working on optimizing reads in spark to avoid spinning up lots
> of short lived tasks that just perform row group pruning in selective
> filter queries.
>
> My high level question is why metadata summary files were marked
> deprecated in this Parquet changeset? There isn't much explanation given
> or a description of what should be used instead.
> https://github.com/apache/parquet-mr/pull/429
>
> There are other members of the broader parquet community that are also
> confused by this deprecation, see this discussion in an arrow PR.
> https://github.com/apache/arrow/pull/4166
>
> In the course of making my small prototype I got an extra performance
> boost by making spark write out metadata summary files, rather than having
> to read all footers on the driver. This effect would be even more
> pronounced on a completely remote storage system like S3. Writing these
> summary files was disabled by default in SPARK-15719, because of the
> performance impact of appending a small number of new files to an existing
> dataset with many files.
>
> https://issues.apache.org/jira/browse/SPARK-15719
>
> This spark JIRA does make decent points considering how spark operates
> today, but I think that there is a performance optimization opportunity
> that is missed because the row group pruning is deferred to a bunch of
> separate short lived tasks rather than done upfront, currently spark only
> uses footers on the driver for schema merging.
>
> Thanks for the help!
> Jason Altekruse
>

Reply via email to