-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-6226] Support parquet native bloom filters #8716
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
e2bd706
Make bloom working with the OP example
parisni 5c5f4de
Parse hadoop conf to infer bloom config
parisni c9d1268
Fix test
parisni 1e1a061
Add parquet bloom filter test
parisni 2b6691f
Also add other operations
parisni ae0d26f
Add UT
parisni 0d5a19f
Merge remote-tracking branch 'hudi/master' into parquet-bloom-suport
parisni bfa81a4
Fix style
parisni d51e9b6
Fix import static
parisni 0c2ed7d
Use reflexion to handle multiple parquet versions
parisni 8dcc242
Merge remote-tracking branch 'hudi/master' into parquet-bloom-suport
parisni 1351072
Adapt test for both spark2 and 3
parisni File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
123 changes: 123 additions & 0 deletions
123
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,123 @@ | ||
| /* | ||
| * 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. | ||
| */ | ||
|
|
||
| package org.apache.hudi | ||
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.hudi.HoodieSparkSessionExtension | ||
| import org.apache.spark.util.{AccumulatorV2} | ||
| import org.apache.spark.SparkContext | ||
|
|
||
| import org.apache.hudi.testutils.HoodieClientTestUtils.getSparkConfForTest | ||
| import org.apache.hudi.DataSourceWriteOptions | ||
| import org.apache.hudi.config.HoodieWriteConfig | ||
| import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType} | ||
|
|
||
|
|
||
| import org.junit.jupiter.api.Assertions.{assertEquals} | ||
| import org.junit.jupiter.api.{BeforeEach} | ||
| import org.junit.jupiter.params.ParameterizedTest | ||
| import org.junit.jupiter.params.provider.{EnumSource} | ||
|
|
||
| class TestHoodieParquetBloomFilter { | ||
|
|
||
| var spark: SparkSession = _ | ||
| var sqlContext: SQLContext = _ | ||
| var sc: SparkContext = _ | ||
parisni marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| def initSparkContext(): Unit = { | ||
| val sparkConf = getSparkConfForTest(getClass.getSimpleName) | ||
|
|
||
| spark = SparkSession.builder() | ||
| .withExtensions(new HoodieSparkSessionExtension) | ||
| .config(sparkConf) | ||
| .getOrCreate() | ||
|
|
||
| sc = spark.sparkContext | ||
| sc.setLogLevel("ERROR") | ||
| sqlContext = spark.sqlContext | ||
| } | ||
|
|
||
| @BeforeEach | ||
| def setUp() { | ||
| initSparkContext() | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @EnumSource(value = classOf[WriteOperationType], names = Array("BULK_INSERT", "INSERT", "UPSERT", "INSERT_OVERWRITE")) | ||
| def testBloomFilter(operation: WriteOperationType): Unit = { | ||
| // setup hadoop conf with bloom col enabled | ||
| spark.sparkContext.hadoopConfiguration.set("parquet.bloom.filter.enabled#bloom_col", "true") | ||
|
|
||
| val basePath = java.nio.file.Files.createTempDirectory("hoodie_bloom_source_path").toAbsolutePath.toString | ||
| val opts = Map( | ||
| HoodieWriteConfig.TBL_NAME.key -> "hoodie_bloom", | ||
| DataSourceWriteOptions.TABLE_TYPE.key -> HoodieTableType.COPY_ON_WRITE.toString, | ||
| DataSourceWriteOptions.OPERATION.key -> operation.toString, | ||
| DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", | ||
| DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition" | ||
| ) | ||
| val inputDF = spark.sql( | ||
| """select '0' as _row_key, '1' as bloom_col, '2' as partition, '3' as ts | ||
| |union | ||
| |select '1', '2', '3', '4' | ||
| |""".stripMargin) | ||
| inputDF.write.format("hudi") | ||
| .options(opts) | ||
| .mode(SaveMode.Overwrite) | ||
| .save(basePath) | ||
|
|
||
| val accu = new NumRowGroupsAcc | ||
| spark.sparkContext.register(accu) | ||
|
|
||
| // this one shall skip partition scanning thanks to bloom when spark >=3 | ||
| spark.read.format("hudi").load(basePath).filter("bloom_col = '3'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) | ||
| assertEquals(if (currentSparkSupportParquetBloom()) 0 else 1, accu.value) | ||
|
|
||
| // this one will trigger one partition scan | ||
| spark.read.format("hudi").load(basePath).filter("bloom_col = '2'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) | ||
| assertEquals(1, accu.value) | ||
| } | ||
|
|
||
| def currentSparkSupportParquetBloom(): Boolean = { | ||
| Integer.valueOf(spark.version.charAt(0)) >= 3 | ||
| } | ||
| } | ||
|
|
||
| class NumRowGroupsAcc extends AccumulatorV2[Integer, Integer] { | ||
| private var _sum = 0 | ||
|
|
||
| override def isZero: Boolean = _sum == 0 | ||
|
|
||
| override def copy(): AccumulatorV2[Integer, Integer] = { | ||
| val acc = new NumRowGroupsAcc() | ||
| acc._sum = _sum | ||
| acc | ||
| } | ||
|
|
||
| override def reset(): Unit = _sum = 0 | ||
|
|
||
| override def add(v: Integer): Unit = _sum += v | ||
|
|
||
| override def merge(other: AccumulatorV2[Integer, Integer]): Unit = other match { | ||
| case a: NumRowGroupsAcc => _sum += a._sum | ||
| case _ => throw new UnsupportedOperationException( | ||
| s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") | ||
| } | ||
|
|
||
| override def value: Integer = _sum | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you show me the code how spark adapter to the Parquet SSBF support?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry not fully understood what you want. What is Parquet SSBF? Do you mean how spark itself handle parquet blooms ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, can you show me that code snippet?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
please see my notes for the lineage on spark: it basically use an other parquet class to write, which get the hadoop conf directly. (ie ParquetOutputFormat) while hudi uses ParquetWriter directly
asks for parquet:ParquetOutputFormat which get the bloom configs
has PrepareWrite function, which propagate to ParquetOurputWriter
has prepareWrite function, which propagate to ParquetUtils.prepareWrite
uses ParquetWrite
uses ParquetTable in getTable (then for read and write)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, another question is does Delta can gain benefits from these bloomfilters, how much regression for writer path when enabling the
BloomFilterfor parquet.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
"Delta" do you mean MOR logs files configured in parquet format ? If you mean delta-lake then yes, as well as iceberg they likely rely on spark writer so benefit from bloom.
By regression do you mean performance regression ? Each column configured with bloom will introduce overhead at write time, but faster subsequent reads with predicates on the column. I haven't benchmarked that but I can say blooms will faster reads significantly by skipping lot of parquet scan that hudi stats index won't cover. ie uuids, strings, high cardinality dictionaries
BTW in this PR up to the user to enable blooms so no regression is expected