Tommo56700 opened a new issue, #14417: URL: https://github.com/apache/iceberg/issues/14417
### Feature Request / Improvement ### Summary When using Iceberg with Spark Structured Streaming, the current caching behavior in Iceberg’s Spark integration makes it difficult to handle evolving reference data efficiently. Specifically, both the catalog and executor-level caches use expireAfterAccess semantics, meaning that frequently accessed data never expires — preventing periodic refreshes of Iceberg metadata or table data without fully disabling caching. This proposal suggests making the cache expiration strategy configurable (e.g. allowing expireAfterWrite), or introducing a smarter refresh mechanism that can detect and reload metadata when the underlying table changes. ### Background and Motivation In my use case, I’m running a Spark Structured Streaming job with a continuous Kafka source and performing stream-to-static joins against reference data stored in Iceberg tables. The reference data changes slowly — updated once per day — and the goal is for joins to automatically pick up the latest snapshot as the table evolves. This use case is similar to the approach described here: https://www.waitingforcode.com/apache-spark-structured-streaming/static-enrichment-dataset-delta-lake/read However, I found that Iceberg tables did not reflect new data unless I completely disabled caching using the following configurations: ``` spark.sql.catalog.<catalog-name>.cache-enabled=false spark.sql.iceberg.executor-cache.enabled=false ``` As described in the official docs: https://iceberg.apache.org/docs/latest/spark-configuration/ (As an aside, I think the config description for caching settings is currently misleading and lacking depth) While this workaround ensures freshness, it causes the Spark job to reload reference tables for every microbatch, which can occur multiple times per minute, leading to significant overhead. ### Problem Iceberg provides TTL-like configurations, such as: - `spark.sql.catalog.<catalog-name>.cache.expiration-interval-ms` - `spark.sql.iceberg.executor-cache.timeout` However, both are implemented using Caffeine’s expireAfterAccess, which means the cache entries are never evicted as long as they are being accessed frequently — effectively disabling periodic refresh for continuously queried datasets. ### References - [CachingCatalog.java](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/CachingCatalog.java#L106-L116) - [SparkExecutorCache.java](https://github.com/apache/iceberg/blob/main/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java#L162-L170) In continuous workloads, this means cached tables are never refreshed, even if the underlying metadata or table snapshots have changed. ### Proposed Improvement Add support for configurable cache expiration policies, for example: - Allow choosing between expireAfterAccess and expireAfterWrite for both catalog and executor caches. - OR implement a smarter refresh strategy that checks whether the underlying table metadata or snapshot ID has changed, and refreshes automatically when necessary. This would allow users to maintain efficient caching while still keeping reference data up to date in long-running streaming jobs. ### Benefits - Enables long-running structured streaming jobs to see updated Iceberg table data without fully disabling caching. - Reduces unnecessary metadata fetches while maintaining data freshness. - Provides flexibility for diverse caching needs (e.g., static joins vs. ad-hoc queries). - Aligns with caching capabilities in other data lake formats (e.g., Delta Lake TTL cache behavior). ### Query engine Spark ### Willingness to contribute - [ ] I can contribute this improvement/feature independently - [x] I would be willing to contribute this improvement/feature with guidance from the Iceberg community - [ ] I cannot contribute this improvement/feature at this time -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
