mayankvadariya opened a new issue, #11539: URL: https://github.com/apache/iceberg/issues/11539
### Apache Iceberg version 1.7.0 (latest release) ### Query engine Trino ### Please describe the bug 🐞 Nested namespace request to Iceberg catalog server is sent incorrectly after upgrading Iceberg library from 1.6.1 to 1.7.0 in Trino. I've provided further analysis in https://github.com/apache/iceberg/pull/10858#issuecomment-2471482802 After https://github.com/trinodb/trino/commit/ead6d9f7dc7fe94321bcf452dc90f26eb0d3a2f5 commit(which bumps Iceberg from 1.61. to 1.7.0), `show schemas` fails in Iceberg connector with below error. ```properties iceberg.catalog.type=rest iceberg.rest-catalog.uri= iceberg.rest-catalog.warehouse= iceberg.rest-catalog.security=OAUTH2 iceberg.rest-catalog.oauth2.credential= iceberg.rest-catalog.oauth2.scope=PRINCIPAL_ROLE:ALL iceberg.rest-catalog.nested-namespace-enabled=true ``` ```java trino:tpch> create schema level1; CREATE SCHEMA trino:tpch> create schema iceberg."level1.level2"; CREATE SCHEMA trino:tpch> show schemas; Query 20241113_175503_00027_msjru, FAILED, 3 nodes http://localhost:8080/ui/query.html?20241113_175503_00027_msjru Splits: 17 total, 1 done (5.88%) CPU Time: 0.0s total, 0 rows/s, 0B/s, 20% active Per Node: 0.0 parallelism, 0 rows/s, 0B/s Parallelism: 0.1 Peak Memory: 984B 0.07 [0 rows, 0B] [0 rows/s, 0B/s] Query 20241113_175503_00027_msjru failed: Error listing schemas for catalog iceberg: Namespace does not exist: level1%1Flevel2 io.trino.spi.TrinoException: Error listing schemas for catalog iceberg: Namespace does not exist: level1%1Flevel2 at io.trino.metadata.MetadataListing.handleListingException(MetadataListing.java:358) at io.trino.metadata.MetadataListing.listSchemas(MetadataListing.java:99) at io.trino.metadata.MetadataListing.listSchemas(MetadataListing.java:90) at io.trino.connector.informationschema.InformationSchemaPageSource.addSchemataRecords(InformationSchemaPageSource.java:331) at io.trino.connector.informationschema.InformationSchemaPageSource.buildPages(InformationSchemaPageSource.java:227) at io.trino.connector.informationschema.InformationSchemaPageSource.getNextPage(InformationSchemaPageSource.java:185) at io.trino.operator.TableScanOperator.getOutput(TableScanOperator.java:268) at io.trino.operator.Driver.processInternal(Driver.java:403) at io.trino.operator.Driver.lambda$process$8(Driver.java:306) at io.trino.operator.Driver.tryWithLock(Driver.java:709) at io.trino.operator.Driver.process(Driver.java:298) at io.trino.operator.Driver.processForDuration(Driver.java:269) at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:890) at io.trino.execution.executor.dedicated.SplitProcessor.run(SplitProcessor.java:77) at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.lambda$run$0(TaskEntry.java:201) at io.trino.$gen.Trino_testversion____20241113_175430_71.run(Unknown Source) at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.run(TaskEntry.java:202) at io.trino.execution.executor.scheduler.FairScheduler.runTask(FairScheduler.java:172) at io.trino.execution.executor.scheduler.FairScheduler.lambda$submit$0(FairScheduler.java:159) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572) at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131) at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:76) at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642) at java.base/java.lang.Thread.run(Thread.java:1575) Caused by: org.apache.iceberg.exceptions.NoSuchNamespaceException: Namespace does not exist: level1%1Flevel2 at org.apache.iceberg.rest.ErrorHandlers$NamespaceErrorHandler.accept(ErrorHandlers.java:173) at org.apache.iceberg.rest.ErrorHandlers$NamespaceErrorHandler.accept(ErrorHandlers.java:166) at org.apache.iceberg.rest.HTTPClient.throwFailure(HTTPClient.java:211) at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:323) at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:262) at org.apache.iceberg.rest.HTTPClient.get(HTTPClient.java:358) at org.apache.iceberg.rest.RESTClient.get(RESTClient.java:96) at org.apache.iceberg.rest.RESTSessionCatalog.listNamespaces(RESTSessionCatalog.java:630) at io.trino.plugin.iceberg.catalog.rest.TrinoRestCatalog.collectNamespaces(TrinoRestCatalog.java:170) at io.trino.plugin.iceberg.catalog.rest.TrinoRestCatalog.lambda$collectNamespaces$0(TrinoRestCatalog.java:173) at java.base/java.util.stream.ReferencePipeline$7$1FlatMap.accept(ReferencePipeline.java:289) at java.base/java.util.Collections$2.tryAdvance(Collections.java:5075) at java.base/java.util.Collections$2.forEachRemaining(Collections.java:5083) at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:570) at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:560) at java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921) at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:265) at java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:727) at io.trino.plugin.iceberg.catalog.rest.TrinoRestCatalog.collectNamespaces(TrinoRestCatalog.java:174) at io.trino.plugin.iceberg.catalog.rest.TrinoRestCatalog.lambda$collectNamespaces$0(TrinoRestCatalog.java:173) at java.base/java.util.stream.ReferencePipeline$7$1FlatMap.accept(ReferencePipeline.java:289) at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:1024) at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:570) at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:560) at java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921) at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:265) at java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:727) at io.trino.plugin.iceberg.catalog.rest.TrinoRestCatalog.collectNamespaces(TrinoRestCatalog.java:174) at io.trino.plugin.iceberg.catalog.rest.TrinoRestCatalog.listNamespaces(TrinoRestCatalog.java:161) at io.trino.plugin.iceberg.IcebergMetadata.listSchemaNames(IcebergMetadata.java:436) at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.listSchemaNames(ClassLoaderSafeConnectorMetadata.java:200) at io.trino.tracing.TracingConnectorMetadata.listSchemaNames(TracingConnectorMetadata.java:133) at io.trino.metadata.MetadataManager.listSchemaNames(MetadataManager.java:260) at io.trino.tracing.TracingMetadata.listSchemaNames(TracingMetadata.java:172) at io.trino.metadata.MetadataListing.doListSchemas(MetadataListing.java:105) at io.trino.metadata.MetadataListing.listSchemas(MetadataListing.java:96) ... 24 more ``` ### Willingness to contribute - [ ] I can contribute a fix for this bug independently - [X] 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 -- 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: issues-unsubscr...@iceberg.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org