Skip to content

Pass full session to avoid Unknown connector errors#24803

Merged
jaystarshot merged 1 commit intoprestodb:masterfrom
denodo-research-labs:nessie_unknown_connector
Apr 4, 2025
Merged

Pass full session to avoid Unknown connector errors#24803
jaystarshot merged 1 commit intoprestodb:masterfrom
denodo-research-labs:nessie_unknown_connector

Conversation

@denodo-research-labs
Copy link
Contributor

@denodo-research-labs denodo-research-labs commented Mar 26, 2025

Description

The session passed to the system connector did not retain any connector properties, so an Unknown connector exception is thrown when trying to browse an Iceberg catalog connected to Nessie with a JDBC client.

Motivation and Context

Same issue as addressed here #22936, but when connecting to the Nessie catalog.

Iceberg catalog properties:

connector.name=iceberg
iceberg.catalog.type=nessie
iceberg.catalog.warehouse=/tmp
iceberg.nessie.uri=http://localhost:19120/api/v1

SchemaJdbcTable.cursor related trace:

com.facebook.presto.spi.PrestoException: Unknown connector iceberg_nessie
    at com.facebook.presto.metadata.SessionPropertyManager.getConnectorSessionPropertyMetadata(SessionPropertyManager.java:207)
    at com.facebook.presto.metadata.SessionPropertyManager.decodeCatalogPropertyValue(SessionPropertyManager.java:295)
    at com.facebook.presto.FullConnectorSession.getProperty(FullConnectorSession.java:160)
    at com.facebook.presto.iceberg.IcebergSessionProperties.getNessieReferenceName(IcebergSessionProperties.java:321)
    at com.facebook.presto.iceberg.nessie.IcebergNessieCatalogFactory.getCatalogCacheKey(IcebergNessieCatalogFactory.java:88)
    at com.facebook.presto.iceberg.IcebergNativeCatalogFactory.getCacheKey(IcebergNativeCatalogFactory.java:118)
    at com.facebook.presto.iceberg.IcebergNativeCatalogFactory.getCatalog(IcebergNativeCatalogFactory.java:85)
    at com.facebook.presto.iceberg.IcebergNativeCatalogFactory.getNamespaces(IcebergNativeCatalogFactory.java:102)
    at com.facebook.presto.iceberg.IcebergNativeMetadata.listSchemaNames(IcebergNativeMetadata.java:150)
    at com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorMetadata.listSchemaNames(ClassLoaderSafeConnectorMetadata.java:211)
    at com.facebook.presto.metadata.MetadataManager.listSchemaNames(MetadataManager.java:314)
    at com.facebook.presto.metadata.MetadataListing.listSchemas(MetadataListing.java:58)
    at com.facebook.presto.connector.system.jdbc.**SchemaJdbcTable.cursor(SchemaJdbcTable.java:74)**

TableJdbcTable.cursor related trace:

com.facebook.presto.spi.PrestoException: Unknown connector iceberg_nessie
    at com.facebook.presto.metadata.SessionPropertyManager.getConnectorSessionPropertyMetadata(SessionPropertyManager.java:207)
    at com.facebook.presto.metadata.SessionPropertyManager.decodeCatalogPropertyValue(SessionPropertyManager.java:295)
    at com.facebook.presto.FullConnectorSession.getProperty(FullConnectorSession.java:160)
    at com.facebook.presto.iceberg.IcebergSessionProperties.getNessieReferenceName(IcebergSessionProperties.java:321)
    at com.facebook.presto.iceberg.nessie.IcebergNessieCatalogFactory.getCatalogCacheKey(IcebergNessieCatalogFactory.java:88)
    at com.facebook.presto.iceberg.IcebergNativeCatalogFactory.getCacheKey(IcebergNativeCatalogFactory.java:118)
    at com.facebook.presto.iceberg.IcebergNativeCatalogFactory.getCatalog(IcebergNativeCatalogFactory.java:85)
    at com.facebook.presto.iceberg.IcebergNativeMetadata.listViews(IcebergNativeMetadata.java:241)
    at com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorMetadata.listViews(ClassLoaderSafeConnectorMetadata.java:506)
    at com.facebook.presto.metadata.MetadataManager.listViews(MetadataManager.java:964)
    at com.facebook.presto.metadata.MetadataListing.listViews(MetadataListing.java:72)
    at com.facebook.presto.connector.system.jdbc.**TableJdbcTable.cursor(TableJdbcTable.java:95)**

Impact

Cannot connect Presto to a Nessie catalog.

Contributor checklist

Release Notes

Please follow release notes guidelines and fill in the release notes below.

== RELEASE NOTES ==

Iceberg Connector Changes
* Fix to pass full session to avoid ``Unknown connector`` errors using the Nessie catalog.

@steveburnett
Copy link
Contributor

Thanks for the release note entry! Suggestion to follow the Order of changes in the Release Notes Guidelines, and a category change suggestion:

== RELEASE NOTES ==

Iceberg Connector Changes
* Fix to pass full session to avoid `Unknown connector` errors using the Nessie catalog.

@denodo-research-labs denodo-research-labs force-pushed the nessie_unknown_connector branch 3 times, most recently from edf42f7 to 621b072 Compare April 1, 2025 07:32
@jaystarshot
Copy link
Member

Is it possible to add a test case?

@denodo-research-labs
Copy link
Contributor Author

In #22936 we talked about this, but couldn't find a way to test this scenario. Do you have any suggestions?

@hantangwangd
Copy link
Member

Could we test the following statements in Nessie related test classes like TestIcebergDistributedNessie or TestIcebergSmokeNessie to reproduce this case?

select * from system.jdbc.schemas where TABLE_CATALOG = 'iceberg';
select * from system.jdbc.tables where TABLE_CAT = 'iceberg' and TABLE_SCHEM = 'tpch';

@denodo-research-labs
Copy link
Contributor Author

Added, thanks for the suggestion @hantangwangd

@steveburnett
Copy link
Contributor

Thanks for the release note! Nit of formatting suggestion.

== RELEASE NOTES ==

Iceberg Connector Changes
* Fix to pass full session to avoid ``Unknown connector`` errors using the Nessie catalog.

Copy link
Member

@hantangwangd hantangwangd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the fix, lgtm!

@jaystarshot jaystarshot merged commit 162d632 into prestodb:master Apr 4, 2025
97 checks passed
@ZacBlanco ZacBlanco mentioned this pull request May 29, 2025
21 tasks
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants