You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hudi.apache.org by Nicolas Paris <ni...@riseup.net> on 2022/03/28 16:57:24 UTC

spark 3.2.1 built-in bloom filters

Hi,

spark 3.2 ships parquet 1.12 which provides built-in bloom filters on
arbirtrary columns. I wonder if:

- hudi can benefit from them ? (likely in 0.11, but not with MOR tables)
- would make sense to replace the hudi blooms with them ?
- what would be the advantage of storing our blooms in hfiles (AFAIK
  this is the future expected implementation) over the parquet built-in.


here is the syntax:

    .option("parquet.bloom.filter.enabled#favorite_color", "true")
    .option("parquet.bloom.filter.expected.ndv#favorite_color", "1000000")


and here some code to illustrate :

https://github.com/apache/spark/blob/e40fce919ab77f5faeb0bbd34dc86c56c04adbaa/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala#L1654



thx

Re: spark 3.2.1 built-in bloom filters

Posted by Nicolas Paris <ni...@riseup.net>.
As we now got hudi 0.11 with multiple columns bloom indexes thougth
`hoodie.metadata.index.bloom.filter.column.list`, the question is wether
those bloom are used by query planner for e.g id=19

The spark built-in blooms are used in this case, maybe that's also the
hudi multi-bloom purpose as well ? (there is no mention about their use)


thanks




On Wed Mar 30, 2022 at 11:36 PM CEST, Vinoth Chandar wrote:
> Hi,
>
> I noticed that it finally landed. We actually began tracking that JIRA
> while initially writing Hudi at Uber.. Parquet + Bloom Filters has taken
> just a few years :)
> I think we could switch out to reading the built-in bloom filters as
> well.
> it could make the footer reading lighter potentially.
>
> Few things that Hudi has built on top would be missing
>
> - Dynamic bloom filter support, where we auto size current bloom filters
> based on number of records, given a fpp target
> - Our current DAG that optimizes for checking records against bloom
> filters
> is still needed on writer side. Checking bloom filters for a given
> predicate e.g id=19, is much simpler compared to matching say a 100k ids
> against 1000 files. We need to be able to amortize the cost of these
> 100M
> comparisons.
>
> On the future direction, with 0.11, we are enabling storing of bloom
> filters and column ranges inside the Hudi metadata table (MDT). *(what
> we
> call multi modal indexes).
> This helps us make the access more resilient towards cloud storage
> throttling and also more performant (we need to read much fewer files)
>
> Over time, when this mechanism is stable, we plan to stop writing out
> bloom
> filters in parquet and also integrate the Hudi MDT with different query
> engines for point-ish lookups.
>
> Hope that helps
>
> Thanks
> Vinoth
>
>
>
>
> On Mon, Mar 28, 2022 at 9:57 AM Nicolas Paris <ni...@riseup.net>
> wrote:
>
> > Hi,
> >
> > spark 3.2 ships parquet 1.12 which provides built-in bloom filters on
> > arbirtrary columns. I wonder if:
> >
> > - hudi can benefit from them ? (likely in 0.11, but not with MOR tables)
> > - would make sense to replace the hudi blooms with them ?
> > - what would be the advantage of storing our blooms in hfiles (AFAIK
> >   this is the future expected implementation) over the parquet built-in.
> >
> >
> > here is the syntax:
> >
> >     .option("parquet.bloom.filter.enabled#favorite_color", "true")
> >     .option("parquet.bloom.filter.expected.ndv#favorite_color", "1000000")
> >
> >
> > and here some code to illustrate :
> >
> >
> > https://github.com/apache/spark/blob/e40fce919ab77f5faeb0bbd34dc86c56c04adbaa/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala#L1654
> >
> >
> >
> > thx
> >


Re: spark 3.2.1 built-in bloom filters

Posted by Vinoth Chandar <vi...@apache.org>.
By all means. That would be great.

Always looking for helping hand in improving docs

On Sat, Apr 2, 2022 at 6:18 AM Nicolas Paris <ni...@riseup.net>
wrote:

> Hi Vinoth,
>
> Thanks for your in depth explanations. I think those details could be
> of interest in the documentation. I can work on this if agreed
>
> On Wed, 2022-03-30 at 14:36 -0700, Vinoth Chandar wrote:
> > Hi,
> >
> > I noticed that it finally landed. We actually began tracking that
> > JIRA
> > while initially writing Hudi at Uber.. Parquet + Bloom Filters has
> > taken
> > just a few years :)
> > I think we could switch out to reading the built-in bloom filters as
> > well.
> > it could make the footer reading lighter potentially.
> >
> > Few things that Hudi has built on top would be missing
> >
> > - Dynamic bloom filter support, where we auto size current bloom
> > filters
> > based on number of records, given a fpp target
> > - Our current DAG that optimizes for checking records against bloom
> > filters
> > is still needed on writer side. Checking bloom filters for a given
> > predicate e.g id=19, is much simpler compared to matching say a 100k
> > ids
> > against 1000 files. We need to be able to amortize the cost of these
> > 100M
> > comparisons.
> >
> > On the future direction, with 0.11, we are enabling storing of bloom
> > filters and column ranges inside the Hudi metadata table (MDT).
> > *(what we
> > call multi modal indexes).
> > This helps us make the access more resilient towards cloud storage
> > throttling and also more performant (we need to read much fewer
> > files)
> >
> > Over time, when this mechanism is stable, we plan to stop writing out
> > bloom
> > filters in parquet and also integrate the Hudi MDT with different
> > query
> > engines for point-ish lookups.
> >
> > Hope that helps
> >
> > Thanks
> > Vinoth
> >
> >
> >
> >
> > On Mon, Mar 28, 2022 at 9:57 AM Nicolas Paris
> > <ni...@riseup.net>
> > wrote:
> >
> > > Hi,
> > >
> > > spark 3.2 ships parquet 1.12 which provides built-in bloom filters
> > > on
> > > arbirtrary columns. I wonder if:
> > >
> > > - hudi can benefit from them ? (likely in 0.11, but not with MOR
> > > tables)
> > > - would make sense to replace the hudi blooms with them ?
> > > - what would be the advantage of storing our blooms in hfiles
> > > (AFAIK
> > >   this is the future expected implementation) over the parquet
> > > built-in.
> > >
> > >
> > > here is the syntax:
> > >
> > >     .option("parquet.bloom.filter.enabled#favorite_color", "true")
> > >     .option("parquet.bloom.filter.expected.ndv#favorite_color",
> > > "1000000")
> > >
> > >
> > > and here some code to illustrate :
> > >
> > >
> > >
> https://github.com/apache/spark/blob/e40fce919ab77f5faeb0bbd34dc86c56c04adbaa/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala#L1654
> > >
> > >
> > >
> > > thx
> > >
>

Re: spark 3.2.1 built-in bloom filters

Posted by Nicolas Paris <ni...@riseup.net>.
Hi Vinoth,

Thanks for your in depth explanations. I think those details could be
of interest in the documentation. I can work on this if agreed

On Wed, 2022-03-30 at 14:36 -0700, Vinoth Chandar wrote:
> Hi,
> 
> I noticed that it finally landed. We actually began tracking that
> JIRA
> while initially writing Hudi at Uber.. Parquet + Bloom Filters has
> taken
> just a few years :)
> I think we could switch out to reading the built-in bloom filters as
> well.
> it could make the footer reading lighter potentially.
> 
> Few things that Hudi has built on top would be missing
> 
> - Dynamic bloom filter support, where we auto size current bloom
> filters
> based on number of records, given a fpp target
> - Our current DAG that optimizes for checking records against bloom
> filters
> is still needed on writer side. Checking bloom filters for a given
> predicate e.g id=19, is much simpler compared to matching say a 100k
> ids
> against 1000 files. We need to be able to amortize the cost of these
> 100M
> comparisons.
> 
> On the future direction, with 0.11, we are enabling storing of bloom
> filters and column ranges inside the Hudi metadata table (MDT).
> *(what we
> call multi modal indexes).
> This helps us make the access more resilient towards cloud storage
> throttling and also more performant (we need to read much fewer
> files)
> 
> Over time, when this mechanism is stable, we plan to stop writing out
> bloom
> filters in parquet and also integrate the Hudi MDT with different
> query
> engines for point-ish lookups.
> 
> Hope that helps
> 
> Thanks
> Vinoth
> 
> 
> 
> 
> On Mon, Mar 28, 2022 at 9:57 AM Nicolas Paris
> <ni...@riseup.net>
> wrote:
> 
> > Hi,
> > 
> > spark 3.2 ships parquet 1.12 which provides built-in bloom filters
> > on
> > arbirtrary columns. I wonder if:
> > 
> > - hudi can benefit from them ? (likely in 0.11, but not with MOR
> > tables)
> > - would make sense to replace the hudi blooms with them ?
> > - what would be the advantage of storing our blooms in hfiles
> > (AFAIK
> >   this is the future expected implementation) over the parquet
> > built-in.
> > 
> > 
> > here is the syntax:
> > 
> >     .option("parquet.bloom.filter.enabled#favorite_color", "true")
> >     .option("parquet.bloom.filter.expected.ndv#favorite_color",
> > "1000000")
> > 
> > 
> > and here some code to illustrate :
> > 
> > 
> > https://github.com/apache/spark/blob/e40fce919ab77f5faeb0bbd34dc86c56c04adbaa/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala#L1654
> > 
> > 
> > 
> > thx
> > 

Re: spark 3.2.1 built-in bloom filters

Posted by Vinoth Chandar <vi...@apache.org>.
Hi,

I noticed that it finally landed. We actually began tracking that JIRA
while initially writing Hudi at Uber.. Parquet + Bloom Filters has taken
just a few years :)
I think we could switch out to reading the built-in bloom filters as well.
it could make the footer reading lighter potentially.

Few things that Hudi has built on top would be missing

- Dynamic bloom filter support, where we auto size current bloom filters
based on number of records, given a fpp target
- Our current DAG that optimizes for checking records against bloom filters
is still needed on writer side. Checking bloom filters for a given
predicate e.g id=19, is much simpler compared to matching say a 100k ids
against 1000 files. We need to be able to amortize the cost of these 100M
comparisons.

On the future direction, with 0.11, we are enabling storing of bloom
filters and column ranges inside the Hudi metadata table (MDT). *(what we
call multi modal indexes).
This helps us make the access more resilient towards cloud storage
throttling and also more performant (we need to read much fewer files)

Over time, when this mechanism is stable, we plan to stop writing out bloom
filters in parquet and also integrate the Hudi MDT with different query
engines for point-ish lookups.

Hope that helps

Thanks
Vinoth




On Mon, Mar 28, 2022 at 9:57 AM Nicolas Paris <ni...@riseup.net>
wrote:

> Hi,
>
> spark 3.2 ships parquet 1.12 which provides built-in bloom filters on
> arbirtrary columns. I wonder if:
>
> - hudi can benefit from them ? (likely in 0.11, but not with MOR tables)
> - would make sense to replace the hudi blooms with them ?
> - what would be the advantage of storing our blooms in hfiles (AFAIK
>   this is the future expected implementation) over the parquet built-in.
>
>
> here is the syntax:
>
>     .option("parquet.bloom.filter.enabled#favorite_color", "true")
>     .option("parquet.bloom.filter.expected.ndv#favorite_color", "1000000")
>
>
> and here some code to illustrate :
>
>
> https://github.com/apache/spark/blob/e40fce919ab77f5faeb0bbd34dc86c56c04adbaa/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala#L1654
>
>
>
> thx
>