[SPARK-55622][SQL][TESTS] Add test for DSV2 Tables with multi-part names on SessionCatalog#54411
[SPARK-55622][SQL][TESTS] Add test for DSV2 Tables with multi-part names on SessionCatalog#54411szehon-ho wants to merge 4 commits intoapache:masterfrom
Conversation
|
cc @cloud-fan @manuzhang @pan3793 do you think this will help? Thanks |
| sql(s"INSERT INTO $t1 VALUES (3, 'third')") | ||
|
|
||
| // Query the metadata table using multi-part identifier | ||
| val snapshots = sql(s"SELECT * FROM default.$t1.snapshots") |
There was a problem hiding this comment.
do you also want to test
SELECT * FROM $t1.snapshots;
SELECT * FROM spark_catalog.default.$t1.snapshots
?
| StructField("snapshot_id", LongType, nullable = false) | ||
| )) | ||
|
|
||
| override def capabilities(): util.Set[TableCapability] = { |
There was a problem hiding this comment.
a couple of questions that mgiht go beyond this topic:
- seems there is no "metadata table" concept in Spark so far, do we need a new
TableCapabilityto signal that a table supports that? how should the permission check be done for metadata tables? suppose they're read-only, does it mean users have read permission on metadata tables as long as they are granted to read the base table?
There was a problem hiding this comment.
Yea, as I guess you know, Iceberg implements the metadata table behind loadTable, matching the tableIdentifier with the form $table.$metadata_table_name.
It'd be nice for Spark to support metadata table officially in DSV2 one day. Then its more clear to Spark code base.
Until then, this test mocks current Iceberg behavior, is because Spark is often not aware of this unexpected behavior in Iceberg and breaks it. So its a regression test for #54247 case.
By the way, what do you mean read only? (I dont think there's such DSV2 concept yet?)
There was a problem hiding this comment.
btw, I also changed the test name / comments and JIRA title to reflect that it's more about supporting DSV2 loadTable with complex table name, and metadata table is just an example.
There was a problem hiding this comment.
By the way, what do you mean read only? (I dont think there's such DSV2 concept yet?)
Just ignore it, I might be getting bogged down in the implementation details of Iceberg. It's an invalid question for a general multi-part namespace table name, from Spark DSv2 perspective.
| override def loadTable(ident: Identifier): Table = { | ||
| // Check for metadata table pattern: namespace = [db, tableName], name = "snapshots" | ||
| // This simulates how Iceberg handles metadata tables like db.table.snapshots | ||
| if (ident.name() == "snapshots" && ident.namespace().length >= 1) { |
There was a problem hiding this comment.
identToUse should be used instead of ident, otherwise, customIdentifierResolution does not take effect.
I checked the iceberg code base, it first tries to load the ident as the table, fallback to resolve metadata table only when NoSuchTableException, which is more reasonable
There was a problem hiding this comment.
done. In Iceberg, it catch the Iceberg NoSuchTableException.
one small difference, our test V2SessionCatalog is not as detailed , so I end up catching AnalysisException, else we have to implement all the wrapper exceptions.
| verify(sql(s"SELECT * FROM $t1.snapshots"), "table.snapshots") | ||
| verify(sql(s"SELECT * FROM default.$t1.snapshots"), "default.table.snapshots") | ||
| verify( | ||
| sql(s"SELECT * FROM spark_catalog.default.$t1.snapshots"), | ||
| "spark_catalog.default.table.snapshots") | ||
| } |
There was a problem hiding this comment.
Seq("$t1.snapshots", "default.$t1.snapshots", "spark_catalog.default.$t1.snapshots").foreach { tblName =>
verify ...
}
| try { | ||
| super.loadTable(identToUse) | ||
| } catch { | ||
| case _: AnalysisException if identToUse.name().toLowerCase(Locale.ROOT) == "snapshots" => |
There was a problem hiding this comment.
the api say it should throw NoSuchTableException, but actually AnalysisException?
There was a problem hiding this comment.
So in this code path, we get AnlaysisException like:
[REQUIRES_SINGLE_PART_NAMESPACE] spark_catalog requires a single-part namespace, but got identifier `default`.`metadata_test_tbl`.`snapshots`. SQLSTATE: 42K05
org.apache.spark.sql.AnalysisException: [REQUIRES_SINGLE_PART_NAMESPACE] spark_catalog requires a single-part namespace, but got identifier `default`.`metadata_test_tbl`.`snapshots`. SQLSTATE: 42K05
at org.apache.spark.sql.errors.QueryCompilationErrors$.requiresSinglePartNamespaceError(QueryCompilationErrors.scala:1550)
at org.apache.spark.sql.connector.catalog.CatalogV2Implicits$IdentifierHelper.asTableIdentifier(CatalogV2Implicits.scala:171)
at org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.loadTable(V2SessionCatalog.scala:91)
at org.apache.spark.sql.connector.catalog.DelegatingCatalogExtension.loadTable(DelegatingCatalogExtension.java:73)
at org.apache.spark.sql.connector.InMemoryTableSessionCatalog.org$apache$spark$sql$connector$TestV2SessionCatalogBase$$super$loadTable(DataSourceV2DataFrameSessionCatalogSuite.scala:103)
at org.apache.spark.sql.connector.TestV2SessionCatalogBase.loadTable(TestV2SessionCatalogBase.scala:69)
at org.apache.spark.sql.connector.TestV2SessionCatalogBase.loadTable$(TestV2SessionCatalogBase.scala:62)
at org.apache.spark.sql.connector.InMemoryTableSessionCatalog.loadTable(DataSourceV2DataFrameSessionCatalogSuite.scala:120)
at org.apache.spark.sql.connector.catalog.CatalogV2Util$.getTable(CatalogV2Util.scala:483)
at org.apache.spark.sql.connector.catalog.CatalogV2Util$.loadTable(CatalogV2Util.scala:458)
at org.apache.spark.sql.catalyst.analysis.RelationResolution.$anonfun$resolveRelation$4(RelationResolution.scala:131)
at scala.Option.orElse(Option.scala:477)
Probably we can change the TestV2SessionCatalog (or V2SessionCatalog itself) to catch it and return the right error, but it may be too much.
What changes were proposed in this pull request?
Add a unit test for Iceberg's case of supporting multi part identifiers in SessionCatalog (for metadata tables). Add a fake metadata table to InMemoryDataSource.
Why are the changes needed?
It can increase Spark coverage to catch issue like: #54247
Does this PR introduce any user-facing change?
No
How was this patch tested?
Ran the added test
Was this patch authored or co-authored using generative AI tooling?
Yes, cursor claude 4.5 opus