Skip to content
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

Add manifest file caching support for Iceberg Hive Metastore catalog #22376

Closed
wants to merge 1 commit into from

Conversation

hackeryang
Copy link
Member

@hackeryang hackeryang commented Jun 13, 2024

Description

Iceberg library has a manifest caching feature contributed by Cloudera:
https://blog.cloudera.com/12-times-faster-query-planning-with-iceberg-manifest-caching-in-impala/

PrestoDB also has similar configurations to turn on this feature: prestodb/presto#21399

Iceberg manifest caching can accelerate query planning.

Additional context and related issues

https://prestodb.io/docs/current/connector/iceberg.html#manifest-file-caching

Release notes

( ) This is not user-visible or is docs only, and no release notes are required.
(x) Release notes are required. Please propose a release note for me.
( ) Release notes are required, with the following suggested text:

# Section
* Add manifest file caching support for Iceberg Hive Metastore catalog. ({issue}`22376`)

@cla-bot cla-bot bot added the cla-signed label Jun 13, 2024
@hackeryang hackeryang self-assigned this Jun 13, 2024
@github-actions github-actions bot added docs iceberg Iceberg connector labels Jun 13, 2024
@cla-bot cla-bot bot added the cla-signed label Jun 13, 2024
@github-actions github-actions bot added the docs label Jun 13, 2024
@hackeryang hackeryang force-pushed the iceberg_manifest_cache branch 3 times, most recently from b040afc to cfc4df9 Compare June 13, 2024 08:30

public static Map<String, String> loadManifestCachingProperties(Map<String, String> properties, IcebergConfig icebergConfig)
{
properties.put(IO_MANIFEST_CACHE_ENABLED, "true");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is configuring org.apache.iceberg.ManifestFiles#CONTENT_CACHES, right?
the cache is keyed by FileIO and our ForwardingFileIo is typically short-lived. how effective is this caching?

cc @alexjo2144 @electrum

Copy link
Member Author

@hackeryang hackeryang Jun 14, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, in one of our production environments, a big sql which was planned for 12s has been reduced to 8s~

In our internal implementation, we also used a Caffeine cache with the key CatalogType and the value FileIo to make a ForwardingFileIo live longer, i can also add relevant codes to our PR

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the 12s -> 8s improvement -- was it with or without fileio caching?

what would the numbers be if we don't do this PR and instead enable filesystem caching that we already have in trino?

Copy link
Member Author

@hackeryang hackeryang Jun 18, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the 12s -> 8s improvement -- was it with or without fileio caching?

what would the numbers be if we don't do this PR and instead enable filesystem caching that we already have in trino?

@findepi the 12s -> 8s improvement was with fileio/manifest caching.

Sorry our newest production environment is 423, so i couldn't test the performance about filesystem caching, but i saw that PrestoDB have manifest caching and filesystem caching both, so i thought that they may not conflict with each other~ The metadata caching in memory is also faster than metadata caching in disks in most cases.

Thank you for your advice, I have modified some codes accordingly, please review again when you have time~ The CICD error seems not related to our PR

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@findepi the 12s -> 8s improvement was with fileio/manifest caching.

thanks!

before merging this PR, we should understand what's the level of improvement that it brings (in isolation).

cc @sopel39 @raunaqmorarka

@findinpath
Copy link
Contributor

@hackeryang pls try adding a test into io.trino.plugin.iceberg.TestIcebergFileOperations

I had taken a stint on integrating this functionality into Iceberg as well some time ago in #20862 and the limitation on which I've stumbled was

At the time of this writing, the main limitation in Trino to use this feature is that each time a storage operation is done, a new ForwardingFileIo instance is being created, while the CONTENT_CACHES static field from Iceberg awaits the same FileIO instance to retrieve a certain content cache.

I'm hopeful though that I missed something in my original PR and that your feature is effective for making use of the manifest cache.

@hackeryang hackeryang changed the title Add Manifest file caching support for Iceberg Catalogs Add manifest file caching support for Iceberg Hive Metastore catalog Jun 18, 2024
@hackeryang
Copy link
Member Author

hackeryang commented Jun 18, 2024

@hackeryang pls try adding a test into io.trino.plugin.iceberg.TestIcebergFileOperations

I had taken a stint on integrating this functionality into Iceberg as well some time ago in #20862 and the limitation on which I've stumbled was

At the time of this writing, the main limitation in Trino to use this feature is that each time a storage operation is done, a new ForwardingFileIo instance is being created, while the CONTENT_CACHES static field from Iceberg awaits the same FileIO instance to retrieve a certain content cache.

I'm hopeful though that I missed something in my original PR and that your feature is effective for making use of the manifest cache.

We used a Caffeine cache in HiveMetastoreTableOperationsProvider with the key CatalogType and the value FileIO to reduce the frequency of temporary creations of fileio~

Sorry i haven't figure out good logic and APIs to write the ut yet, i will try to add one if i have a better idea, i saw that PrestoDB didn't add it

@lozbrown
Copy link

Will there be ways to flush this cache (specific to a table) in the same way the metadata cache can be via SQL

This allows you to have long cache times and still have data available as soon as it ingests by flushing the cache for tables after cache.

its something that's currently missing from the file status cache

Copy link

github-actions bot commented Jul 9, 2024

This pull request has gone a while without any activity. Tagging the Trino developer relations team: @bitsondatadev @colebow @mosabua

@github-actions github-actions bot added the stale label Jul 9, 2024
@mosabua
Copy link
Member

mosabua commented Jul 22, 2024

fyi @cwsteinbach .. maybe you can help here.

@raunaqmorarka
Copy link
Member

raunaqmorarka commented Jul 23, 2024

I have another PR adding iceberg metadata caching through a different approach #22739
That PR avoids depending on iceberg's manifest file caching and just implements the caching within Trino. I had attempted using iceberg manifest file caching as well, it forced changes around lifecycle of FileIO object because of the caching being tied to FileIO instance in iceberg. It's also harder to re-use across queries as we'd have to retain FileIO objects across queries to make that work and that may not be appropriate as each of those objects may have different ConnectorIdentity per query. Implementing it in Trino also makes it possible to potentially reuse in delta connector.

@mosabua
Copy link
Member

mosabua commented Jul 23, 2024

That sounds great @raunaqmorarka - can we conclude that we should therefore close this PR and help with reviews and input on yours?

@raunaqmorarka
Copy link
Member

That sounds great @raunaqmorarka - can we conclude that we should therefore close this PR and help with reviews and input on yours?

yup, i think so.
@hackeryang please try out #22739 and let me know if you run into any problems with that.

@mosabua mosabua closed this Jul 23, 2024
@mosabua
Copy link
Member

mosabua commented Jul 23, 2024

Closing as discussed in the previous comments since the approach in this PR is inferior to the linked PR from @raunaqmorarka - we will continue to collaborate their and would love to get review and testing feedback.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Development

Successfully merging this pull request may close these issues.

7 participants