Description
Apache Iceberg version
1.9.1 (latest release)
Query engine
Spark
Problem
Creating a table using Spark with a name that matches a metadata table name (e.g., entries
or snapshots
) fails when using the Iceberg REST catalog. The error message is misleading and inconsistent depending on the operation.
Expected Behavior
The following Spark SQL command should create a table named entries
in the namespace nyc
:
CREATE TABLE demo.nyc.entries (
`VendorID` BIGINT,
`trip_distance` DOUBLE
)
Observed Behavior
Running the above command fails with a BadRequestException: Malformed request: <h1>Bad Message 400</h1><pre>reason: Ambiguous URI empty segment</pre>
exception
Similarly, running DROP TABLE IF EXISTS demo.nyc.entries
also fails with a similar BadRequestException
.
However, querying the table using the REST API returns a different and confusing error message:
Table does not exist: nyc
This error omits the actual table name (entries
) and suggests the namespace itself is missing.
Reproduction Steps
This issue can be reproduced by following the Iceberg Spark Quickstart and using the table name entries
.
This issue consistently reproduces across all the available Spark sessions: SparkSQL
, Spark-Shell
and PySpark
Current Workaround
Creating the table using PyIceberg works successfully. Once created, all further operations (including Table Maintenance) can be performed as expected from Spark.
Full stack trace
org.apache.iceberg.exceptions.BadRequestException: Malformed request: <h1>Bad Message 400</h1><pre>reason: Ambiguous URI empty segment</pre>
at org.apache.iceberg.rest.ErrorHandlers$DefaultErrorHandler.accept(ErrorHandlers.java:208)
at org.apache.iceberg.rest.ErrorHandlers$TableErrorHandler.accept(ErrorHandlers.java:118)
at org.apache.iceberg.rest.ErrorHandlers$TableErrorHandler.accept(ErrorHandlers.java:102)
at org.apache.iceberg.rest.HTTPClient.throwFailure(HTTPClient.java:224)
at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:308)
at org.apache.iceberg.rest.BaseHTTPClient.get(BaseHTTPClient.java:77)
at org.apache.iceberg.rest.RESTClient.get(RESTClient.java:97)
at org.apache.iceberg.rest.RESTSessionCatalog.loadInternal(RESTSessionCatalog.java:465)
at org.apache.iceberg.rest.RESTSessionCatalog.loadTable(RESTSessionCatalog.java:499)
at org.apache.iceberg.catalog.BaseSessionCatalog$AsCatalog.loadTable(BaseSessionCatalog.java:99)
at org.apache.iceberg.rest.RESTCatalog.loadTable(RESTCatalog.java:102)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2406)
at java.base/java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1916)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2404)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2387)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:108)
at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalManualCache.get(LocalManualCache.java:62)
at org.apache.iceberg.CachingCatalog.loadTable(CachingCatalog.java:147)
at org.apache.iceberg.spark.SparkCatalog.load(SparkCatalog.java:844)
at org.apache.iceberg.spark.SparkCatalog.loadTable(SparkCatalog.java:169)
at org.apache.spark.sql.connector.catalog.TableCatalog.tableExists(TableCatalog.java:185)
at org.apache.spark.sql.execution.datasources.v2.CreateTableExec.run(CreateTableExec.scala:42)
at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:107)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:107)
at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:98)
at org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:85)
at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:83)
at org.apache.spark.sql.Dataset.<init>(Dataset.scala:220)
at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:691)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:682)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:713)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:744)
... 38 elided
Willingness to contribute
- I can contribute a fix for this bug independently
- I would be willing to contribute a fix for this bug with guidance from the Iceberg community
- I cannot contribute a fix for this bug at this time