-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-30616][SQL] Introduce TTL config option for SQL Metadata Cache #28852
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
Conversation
|
ok to test |
|
Could you add tests? |
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
|
Test build #124182 has finished for PR 28852 at commit
|
|
@maropu regarding testing, I don't see a dedicated test suite for the Do you think I should create a new suite for |
|
Test build #124224 has finished for PR 28852 at commit
|
How about adding |
It looks like If we go ahead with |
d9c5bf7 to
28da5cf
Compare
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
Outdated
Show resolved
Hide resolved
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala
Outdated
Show resolved
Hide resolved
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala
Outdated
Show resolved
Hide resolved
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala
Outdated
Show resolved
Hide resolved
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala
Outdated
Show resolved
Hide resolved
sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala
Outdated
Show resolved
Hide resolved
|
Test build #124502 has finished for PR 28852 at commit
|
|
Test build #124504 has finished for PR 28852 at commit
|
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
Outdated
Show resolved
Hide resolved
|
Test build #124632 has finished for PR 28852 at commit
|
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
Outdated
Show resolved
Hide resolved
|
Looks okay. cc: @cloud-fan @dongjoon-hyun @HyukjinKwon |
|
Test build #124678 has finished for PR 28852 at commit
|
|
^ the test failure seems to be unrelated, I see similar failures in other branches... |
|
retest this please |
|
Test build #124702 has finished for PR 28852 at commit
|
|
retest this please |
|
Test build #124709 has finished for PR 28852 at commit
|
| if (conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT) | ||
| } | ||
|
|
||
| private val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = { |
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 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.
Ah, I see. As you suggested, the most painful part (listed files) has already been cached there. But, it seems some datasources still has somewhat processing costs when resolving a relation (e.g., JDBC datasources send a query to an external database for schema resolution), so I think we need to carefully check performance impacts for removing this cache.
spark/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
Line 256 in fb51925
| LogicalRelation(dataSource.resolveRelation(checkFilesExist = false), table) |
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.
For external data sources, it's common that data are changed outside of Spark. I think it's more important to make sure we get the latest data in a new query. Maybe we should disable this relation cache by default.
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.
Hmm, I think this cache is still useful for avoiding inferring schema again. This is also an expensive operation.
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.
ah that's a good point. We should probably investigate how to design the data source API so that sources don't need to infer schema can skip this cache. It's hard to use the JDBC data source as we need to run REFRESH TABLE (or wait for TTL after this PR) once the table is changed outside of spark (which is common to JDBC source).
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
Outdated
Show resolved
Hide resolved
|
Test build #126027 has finished for PR 28852 at commit
|
|
retest this please |
|
Test build #126042 has finished for PR 28852 at commit
|
|
thanks, merging to master! |
### What changes were proposed in this pull request? This is a follow-up of #28852. This PR to use only config name; otherwise the doc for the config entry shows the entire details of the referring configs. ### Why are the changes needed? The doc for the newly introduced config entry shows the entire details of the referring configs. ### Does this PR introduce _any_ user-facing change? The doc for the config entry will show only the referring config keys. ### How was this patch tested? Existing tests. Closes #29194 from ueshin/issues/SPARK-30616/fup. Authored-by: Takuya UESHIN <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
| "session catalog cache. This configuration only has an effect when this value having " + | ||
| "a positive value (> 0). It also requires setting " + | ||
| s"${StaticSQLConf.CATALOG_IMPLEMENTATION} to `hive`, setting " + | ||
| s"${SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE} > 0 and setting " + |
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.
Could you update the message by using ${SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key} ?
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.
That was done in #29194 :)
What changes were proposed in this pull request?
New
spark.sql.metadataCacheTTLSecondsoption that adds time-to-live cache behaviour to the existing caches inFileStatusCacheandSessionCatalog.Why are the changes needed?
Currently Spark caches file listing for tables and requires issuing
REFRESH TABLEany time the file listing has changed outside of Spark. Unfortunately, simply submittingREFRESH TABLEcommands could be very cumbersome. Assuming frequently added files, hundreds of tables and dozens of users querying the data (and expecting up-to-date results), manually refreshing metadata for each table is not a solution.This is a pretty common use-case for streaming ingestion of data, which can be done outside of Spark (with tools like Kafka Connect, etc.).
A similar feature exists in Presto:
hive.file-status-cache-expire-timecan be found here.Does this PR introduce any user-facing change?
Yes, it's controlled with the new
spark.sql.metadataCacheTTLSecondsoption.When it's set to
-1(by default), the behaviour of caches doesn't change, so it stays backwards-compatible.Otherwise, you can specify a value in seconds, for example
spark.sql.metadataCacheTTLSeconds: 60means 1-minute cache TTL.How was this patch tested?
Added new tests in: