-
Notifications
You must be signed in to change notification settings - Fork 1.7k
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
Vacuum is very slow #395
Comments
Hey @lyogev Have you tried setting a lower value for park.sql.sources.parallelPartitionDiscovery.parallelism, it is 10000 by default in Apache Spark. |
@lyogev If you're dealing with smaller tables, you can definitely reduce that configuration. It doesn't launch 1 Spark job per file, it actually launches 1 Spark job with 200 partitions that's supposed to list your table directory. It pulls them to the Driver 1 by 1 to avoid OOMing the Driver, causing Vacuum to launch 200 Spark jobs. There is room for optimization here. We cache the file listing. We can count the number of files, and then repartition the result according to a heuristic to avoid launching 200 small jobs. Is this something you would consider contributing? :) |
Hi @brkyvz I would love to contibute. Pros for delta
Cons for delta
So basically the slow query performance + missing hive is what makes this unattractive for now, although I would love that ingestion speed! |
@brkyvz If I understand the code correctly, it would be a matter of changing
to something like this
Or would you rather envision something more sophisticated? |
So when you said no hive metastore support - what do you mean exactly?
|
@prasadvaze I mean you can't register tables yet per #85 |
@lyogev per comment by @ abiratsis in #375 , looks like he was able to register delta table in hiveMetastore val someDf = spark.emptyDataset[SomeSchema].toDF The only strange behaviour is the unexpected path of the Delta table as already mentioned in issue #85 |
@lyogev @prasadvaze MetaStore support should be coming in the next two weeks. We're working on getting Delta to compile with Spark 3.0 this week. Then we can release the support for Hive MetaStore tables. @redsk I would do something very similar to that a bit later in the code (right before the delete). |
* PR_20_UpdateFlinkSource_JavaDoc - changes to Javadocs Signed-off-by: Krzysztof Chmielewski <krzysztof.chmielewski@getindata.com> * PR_20_UpdateFlinkSource_JavaDoc - changes to Javadocs Signed-off-by: Krzysztof Chmielewski <krzysztof.chmielewski@getindata.com> * PR_20_UpdateFlinkSource_JavaDoc - changes to Javadocs Signed-off-by: Krzysztof Chmielewski <krzysztof.chmielewski@getindata.com> Co-authored-by: Krzysztof Chmielewski <krzysztof.chmielewski@getindata.com>
Hi,
I just started using delta lake and noticed that vacuum creates a single job per file in spark, does this make any sense?
The text was updated successfully, but these errors were encountered: