Skip to content

Conversation

@blcksrx
Copy link

@blcksrx blcksrx commented Oct 29, 2025

Add configurable catalog cache expiration policy (expire-after-access vs expire-after-write)

This change introduces a configurable expiration policy for Iceberg’s catalog-level cache, allowing users to choose between:

  • EXPIRE_AFTER_ACCESS (current behavior: “hot” entries may never expire in continuously-accessed workloads)
  • EXPIRE_AFTER_WRITE (TTL from creation/write time: entries refresh periodically even if accessed frequently)

It addresses the pain described in Issue #14417: long-running Spark Structured Streaming jobs that repeatedly access reference tables can keep cache entries perpetually alive under expireAfterAccess, preventing periodic refreshes unless caching is disabled (which is expensive).


Background (why this exists)

In streaming patterns like stream-to-static joins against slowly changing reference data, you often want:

  • caching to avoid re-planning/re-loading metadata every microbatch, and
  • periodic refresh to pick up new snapshots (e.g., once per day)

With expireAfterAccess, frequent access can prevent expiry entirely, which makes “TTL” effectively meaningless in continuous workloads. Issue #14417 proposes making cache strategy configurable (or adding a smarter refresh mechanism).


What changed in this PR

✅ New catalog property: cache.expiration-policy

A new catalog property is added to choose the cache expiration strategy:

Property Default Description
cache.expiration-policy EXPIRE_AFTER_ACCESS Determines the expiration strategy for catalog cache entries. Supported values: EXPIRE_AFTER_ACCESS, EXPIRE_AFTER_WRITE.

This works alongside the existing catalog cache TTL property:

  • cache.expiration-interval-ms (existing) controls how long entries are cached
  • cache.expiration-policy (new) controls what “expiration” means (access-based vs write-based)

CachingCatalog supports policy selection

CachingCatalog.wrap(...) is extended so callers can pass a CacheExpirationPolicy, enabling expireAfterWrite semantics when requested.

✅ Engines pass the policy through (Flink + Spark integration points)

Flink catalog creation is updated to parse cache.expiration-policy and pass it into CachingCatalog.wrap(...).

The PR also touches Spark integration (SparkCatalog.java for Spark 3.4 / 3.5 / 4.0 are included in the file list), indicating the same policy is wired through there as well.

✅ Tests cover EXPIRE_AFTER_WRITE

A test validates that with EXPIRE_AFTER_WRITE, accessing an entry does not extend its lifetime, and it expires based on write/creation time as expected.


How to use it

Spark example

Use the property under your Spark catalog config:

spark.sql.catalog.<catalog-name>.cache-enabled=true
spark.sql.catalog.<catalog-name>.cache.expiration-interval-ms=86400000
spark.sql.catalog.<catalog-name>.cache.expiration-policy=EXPIRE_AFTER_WRITE

@blcksrx blcksrx force-pushed the hossein/cache_strategy branch 3 times, most recently from e315119 to e2bbabf Compare October 29, 2025 15:44
Copy link
Collaborator

@gaborkaszab gaborkaszab left a comment

Choose a reason for hiding this comment

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

Thank you for the contribution @blcksrx !
Just thinking out loud but I'm wondering if this is something that requires configurability or if there is any benefit to keep the expireAfterAccess functionality instead of simply switching to expireAfterWrite without any new configs. Let's see what others say.

@blcksrx
Copy link
Author

blcksrx commented Oct 30, 2025

Thank you for the contribution @blcksrx ! Just thinking out loud but I'm wondering if this is something that requires configurability or if there is any benefit to keep the expireAfterAccess functionality instead of simply switching to expireAfterWrite without any new configs. Let's see what others say.

Thank you very much. well my intention was not change the existing cache strategy, thats why I made it like that.
Also, I made some comments in your replies. If I could have answer for them. I'd be able to adjust my PR.
Thanks a lot

@Tommo56700
Copy link

Tommo56700 commented Nov 4, 2025

Looking good, I do have a couple of questions. Would it make sense to provide a similar configuration option to the executor cache?
https://github.com/apache/iceberg/blob/main/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java#L164

Also, not sure on the etiquette in this repository, but it might be appropriate to tag some of the original reviewers from this PR - they might have some more context/opinions on how this should take shape #3543

I also quite like your suggestion of a pluggable caching strategy, but I think it is great to introduce some standard defaults for now and should solve my use case.

@blcksrx
Copy link
Author

blcksrx commented Nov 5, 2025

#3543

Thank you very much for the input. I can provide the same PR on the ExecutorCache too. To getting this PR merge I need the community to vote and engage on it so I appreciate your support on it (please check the dev mailing archive of Iceberg).

Regarding pluggable cache strategy, I saw that i would be much work to implement it at first, so I saw it as a MVP. First provide the solution then we can make it pluggable

@blcksrx
Copy link
Author

blcksrx commented Nov 5, 2025

@jackye1995 @rdblue @findepi Since you have engaged on #3543, I do appreciate your input on this PR. thanks

@blcksrx blcksrx force-pushed the hossein/cache_strategy branch 8 times, most recently from 88270d4 to a8721f9 Compare November 10, 2025 11:28
@blcksrx
Copy link
Author

blcksrx commented Nov 11, 2025

@findepi @gaborkaszab I have applied the suggested changes, appreciate your review on it

@gaborkaszab
Copy link
Collaborator

Let's invite @nastra too who chimed in on the Slack conversation.

For me in general, expireAfterWrite makes a lot of sense over expireAfterAccess. For instance, the table could come with credentials that are valid when populating the cache, but could expire later, however, frequent reads could prevent the table creds to be refreshed. So making this configurable while keeping the original behavior as default seems reasonable. I wonder what others think before going into deeper code inspections.

@findepi
Copy link
Member

findepi commented Nov 11, 2025

@gaborkaszab that's a good point and a reason for pairing the two. I.e. likely anyone wanting to use "expire after access", would also want some cap on "expire after write" as well.

@blcksrx
Copy link
Author

blcksrx commented Nov 24, 2025

@nastra I would appreciate your inputs on this

@nastra
Copy link
Contributor

nastra commented Nov 25, 2025

@blcksrx I think this is a big enough change that would be good to bring up on the DEV mailing list in order to get more opinions about whether the community wants to make the caching policy pluggable or not

@blcksrx
Copy link
Author

blcksrx commented Nov 25, 2025

@blcksrx I think this is a big enough change that would be good to bring up on the DEV mailing list in order to get more opinions about whether the community wants to make the caching policy pluggable or not

I already did that and there were no reaction yet there. engagement here were much more

@nastra
Copy link
Contributor

nastra commented Nov 25, 2025

I must have missed the ML thread. Could you please link it here?

@blcksrx
Copy link
Author

blcksrx commented Nov 25, 2025

@pvary
Copy link
Contributor

pvary commented Dec 1, 2025

If we want this feature, then I'm fairly confident that we are going to the right direction with the change.
I would try to resurrect the dev list thread, and get more people involved in the discussion. Maybe raise the question on the community sync.

@blcksrx blcksrx force-pushed the hossein/cache_strategy branch 6 times, most recently from 3dd3313 to 9c7f67f Compare December 3, 2025 18:32
@blcksrx blcksrx requested review from gaborkaszab and pvary December 4, 2025 19:24
@pvary
Copy link
Contributor

pvary commented Dec 5, 2025

@blcksrx: If we want this feature included, I recommend raising it during the community sync—the next one is scheduled for December 17—or possibly in today’s Catalog Sync.
In the meantime, would periodically calling refresh on the table help address your scenario?

@blcksrx
Copy link
Author

blcksrx commented Dec 5, 2025

Thanks, for now we have to disable the cache to works.
BTW, I can not see scheduled meeting on thecalendar

@pvary
Copy link
Contributor

pvary commented Dec 5, 2025

Thanks, for now we have to disable the cache to works. BTW, I can not see scheduled meeting on thecalendar

Here is the link for the calendar: https://calendar.google.com/calendar/u/0/r/month/2025/12/1?cid=MzkwNWQ0OTJmMWI0NTBiYTA3MTJmMmFlNmFmYTc2ZWI3NTdmMTNkODUyMjBjYzAzYWE0NTI3ODg1YWRjNTYyOUBncm91cC5jYWxlbmRhci5nb29nbGUuY29t

It appears that the Catalog Sync was either removed or I may have made an error when checking for the next scheduled session. Sorry

Copy link
Collaborator

@gaborkaszab gaborkaszab left a comment

Choose a reason for hiding this comment

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

Thank you for the change @blcksrx ! For me the code seems fine at this point. I've just left some minor nits.
In general I think the enhancement makes sense, however, I'd try to ask for some feedback on the broader community as @pvary also suggested.

public static final boolean CACHE_CASE_SENSITIVE_DEFAULT = true;

/**
* Controls policy of table cache
Copy link
Collaborator

Choose a reason for hiding this comment

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

nit: Controls policy of tale cache expiration

/**
* Caching Catalog
*
* @deprecated will be removed in 1.12.0; use {@link #CachingCatalog(Catalog, boolean, long,
Copy link
Collaborator

Choose a reason for hiding this comment

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

nit: I think instead of #CachingCatalog( simply CachingCatalog( also works.

case EXPIRE_AFTER_WRITE:
return cacheBuilder.expireAfterWrite(Duration.ofMillis(expirationIntervalMillis)).build();
case EXPIRE_AFTER_ACCESS:
return cacheBuilder.expireAfterAccess(Duration.ofMillis(expirationIntervalMillis)).build();
Copy link
Member

Choose a reason for hiding this comment

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

i don't think this was answered to #14440 (comment)

when using "expire after access" there still should be a way to set "expire after write" in order to limit staleness of values

Copy link
Author

Choose a reason for hiding this comment

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

Got it, you mean using both of them?

Copy link
Collaborator

Choose a reason for hiding this comment

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

Giving this a second thought I think using both could make sense. We could cover the following setup:

  1. Have a "longer" ExpireAfterWrite that is aligned with the credential expiration time for the table obj (e.g. half of the credential expiration). This guarantees that the table won't be kept in cache so long that the credentials expire in the meantime.
  2. Have a "shorter" ExpireAfterAccess together with the above so that when there is no access for the table we could evict it sooner.

I'm wondering if softValues() is also something we want to tune, @findepi . E.g. now if the engine holds a table object only as long as aa query (or maybe query planning) lasts then if we don't have overlapping queries for the same table, the above setup won't do much. The table will be evicted from cache between two queries regardless of the above setups.

@github-actions
Copy link

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@iceberg.apache.org list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Jan 18, 2026
@blcksrx blcksrx force-pushed the hossein/cache_strategy branch 2 times, most recently from a7ba422 to 9644e64 Compare January 19, 2026 18:13
@github-actions github-actions bot removed the stale label Jan 20, 2026
@blcksrx blcksrx force-pushed the hossein/cache_strategy branch 3 times, most recently from 550b466 to 2cad72d Compare January 20, 2026 11:07
Signed-off-by: Hossein Torabi <sayed.torabi@ing.com>
Signed-off-by: Hossein Torabi <blcksrx@pm.me>
Signed-off-by: Hossein Torabi <sayed.torabi@ing.com>
@blcksrx blcksrx force-pushed the hossein/cache_strategy branch from 2cad72d to 8903e25 Compare January 20, 2026 11:17
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants