-
Notifications
You must be signed in to change notification settings - Fork 76
Cache iceberg metadata to reduce redundant requests to storage #509
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
base: main
Are you sure you want to change the base?
Changes from all commits
552381d
9b92ce1
a7c4ab9
ba759a1
58179c7
f5aa788
836c68f
226ec0a
37ab1d0
498b1d7
34bf955
303470a
d7f1e80
622850b
fbf3a65
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,35 @@ | ||
| package com.linkedin.openhouse.internal.catalog.cache; | ||
|
|
||
| import com.github.benmanes.caffeine.cache.Caffeine; | ||
| import com.linkedin.openhouse.internal.catalog.config.InternalCatalogSettings; | ||
| import java.util.List; | ||
| import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean; | ||
| import org.springframework.cache.CacheManager; | ||
| import org.springframework.cache.annotation.EnableCaching; | ||
| import org.springframework.cache.caffeine.CaffeineCacheManager; | ||
| import org.springframework.context.annotation.Bean; | ||
| import org.springframework.context.annotation.Configuration; | ||
|
|
||
| @Configuration | ||
| @EnableCaching | ||
| public class CacheConfiguration { | ||
|
|
||
| @Bean | ||
| @ConditionalOnMissingBean(InternalCatalogSettings.class) | ||
| public InternalCatalogSettings internalCatalogSettings() { | ||
| return new InternalCatalogSettings(); | ||
| } | ||
|
|
||
| @Bean | ||
| public CacheManager internalCatalogCacheManager(InternalCatalogSettings settings) { | ||
| CaffeineCacheManager cacheManager = new CaffeineCacheManager(); | ||
| cacheManager.setAllowNullValues(false); | ||
| cacheManager.setCacheNames(List.of("tableMetadata")); | ||
| cacheManager.setCaffeine( | ||
| Caffeine.newBuilder() | ||
| .expireAfterWrite(settings.getMetadataCache().getTtl()) | ||
| .maximumSize(settings.getMetadataCache().getMaxSize()) | ||
| .recordStats()); | ||
| return cacheManager; | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| package com.linkedin.openhouse.internal.catalog.cache; | ||
|
|
||
| import java.util.function.Supplier; | ||
| import org.apache.iceberg.TableMetadata; | ||
| import org.springframework.cache.annotation.CachePut; | ||
| import org.springframework.cache.annotation.Cacheable; | ||
| import org.springframework.stereotype.Component; | ||
|
|
||
| @Component | ||
| public class SpringTableMetadataCache implements TableMetadataCache { | ||
|
|
||
| @Override | ||
| @Cacheable( | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what would happen if there are concurrent requests for same metadata location? Probably both could miss and read from HDFS. Consider https://docs.spring.io/spring-framework/reference/integration/cache/annotations.html#cache-annotations-cacheable-synchronized ?
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's fine isn't it? This strictly reduces but does not eliminate HDFS load. |
||
| cacheManager = "internalCatalogCacheManager", | ||
| cacheNames = "tableMetadata", | ||
| key = "#metadataLocation") | ||
| public TableMetadata load(String metadataLocation, Supplier<TableMetadata> metadataLoader) { | ||
| return metadataLoader.get(); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is the caching here per request basis? What happens for the scenario immediate get after put or multiople concurrent get after put?
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Per Pod is my understanding.
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. PUT seeds cache with newest UUID after inserting to HTS. Caching table metadata by file location is safe because the locations are immutable, UUID-based paths. Each commit writes a new file like 00003-.metadata.json. The content at a given path never changes, so the cache key always maps to the same value. There's no invalidation problem because a new version is a new key, not an update to an old one. Stale reads are harmless because the commit path uses compare-and-swap, not the cache. if you try to commit based on an old version, the CAS check rejects it at HTS layer. The cache only affects read performance and not correctness. |
||
| } | ||
|
|
||
| @Override | ||
| @CachePut( | ||
| cacheManager = "internalCatalogCacheManager", | ||
| cacheNames = "tableMetadata", | ||
| key = "#metadataLocation") | ||
|
Comment on lines
+22
to
+25
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. SO this abstracts the cache as a spring component?
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea. And
so the end result is that callers stay behind the TableMetadataCache interface and don’t need to know about the concrete cache impl and the impl is mostly framework config. |
||
| public TableMetadata seed(String metadataLocation, TableMetadata tableMetadata) { | ||
| return tableMetadata; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,11 @@ | ||
| package com.linkedin.openhouse.internal.catalog.cache; | ||
|
|
||
| import java.util.function.Supplier; | ||
| import org.apache.iceberg.TableMetadata; | ||
|
|
||
| public interface TableMetadataCache { | ||
|
|
||
| TableMetadata load(String metadataLocation, Supplier<TableMetadata> metadataLoader); | ||
|
|
||
| TableMetadata seed(String metadataLocation, TableMetadata tableMetadata); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,19 @@ | ||
| package com.linkedin.openhouse.internal.catalog.config; | ||
|
|
||
| import java.time.Duration; | ||
| import lombok.Getter; | ||
| import lombok.Setter; | ||
|
|
||
| @Getter | ||
| @Setter | ||
| public class InternalCatalogSettings { | ||
|
|
||
| private MetadataCache metadataCache = new MetadataCache(); | ||
|
|
||
| @Getter | ||
| @Setter | ||
| public static class MetadataCache { | ||
| private Duration ttl = Duration.ofMinutes(5); | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Was any profiling was done on per-entry TableMetadata memory size to validate the 1000-entry max ?
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, but it is in progress, I will post the results on the PR as a heatmap (x axis ttl, y axis size, z is hit rate)
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I also am working on a way to visualize our efficiency intra-request. the goal is to have the right size / ttl for cache to last for whole request where request may be up to 3 min |
||
| private long maxSize = 1000; | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If this is a per request cache, why do we need more than one entity stored and a TTL?
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the per request cache is not able to emit prometheus metrics, so we can't monitor it. hence I'm changed strategy to use per-pod cache and key with UUID -- we get metrics, better cache-hit ratio, and don't affect our concurrency model.
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I'm not sure. T0 - Table Manifest State X A per-request cache limited lifecycle makes this a non-issue.
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. "the manifest" refers to a manifest file (the Avro file that tracks a list of data files). this is distinct from "the metadata" which specifically refers to a json file that represents table state e.g. and the metadata is immutable, once the UUID is generated and file is written it can never change. so in the same format listed above it more accurately this:
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Trying to understand the approach here and why per pod cache? As there are 56 tables service pod, the issue with per pod cache is cache miss scenario could be more as most of the requests could land on different pods. What are we missing with request level cache? I believe idea was to optimize multiple metadata calls for a given requests.
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the primary reason is because per request cache is too ephemeral to capture any metrics. and, our logs are only retained for 14days. so with 14d lookback we can't see a trend of before/after impact of caching. second reason is because per pod cache has a benefit for the getTable path. overall our solutions looks like this in increasing order of complexity and value.
and this PR sets the right abstractions to get to step 5if we need to in future |
||
| } | ||
| } | ||
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.
This can be a generic file cache no? I might call it a file cache but the variable name be metadata.
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.
there are two layers here:
internalCatalogCacheManagercan be generic for any file since it is just the Spring CacheManager, but the injected wrapper is still intentionallyTableMetadataCacheand specific to tablemetdata.Right now the cache namespace is tableMetadata and the value type is still TableMetadata
We can easily reuse the generic internalCatalogCacheManager for other use cases later like this