Skip to content

In Iceberg catalog, JdbcTableOperations should catch and handle PostgresSQL's exception for duplicated keys #13924

@sqd

Description

@sqd

Apache Iceberg version

1.9.2 (latest release)

Query engine

None

Please describe the bug 🐞

When creating an iceberg table in the catalog, we catch unique key violation exception to handle scenarios where the table already existed. However, PostgresSQL throws a different exception that we do not currently handle (for reference, this is how Postgres's Java client handles the error). This will result in the following exception:

Caused by: org.postgresql.util.PSQLException: ERROR: duplicate key value violates unique constraint "iceberg_tables_pkey"
  Detail: Key (catalog_name, table_namespace, table_name)=(**, **, **) already exists.
	at org.postgresql.core.v3.QueryExecutorImpl.receiveErrorResponse(QueryExecutorImpl.java:2733)
	at org.postgresql.core.v3.QueryExecutorImpl.processResults(QueryExecutorImpl.java:2420)
	at org.postgresql.core.v3.QueryExecutorImpl.execute(QueryExecutorImpl.java:372)
	at org.postgresql.jdbc.PgStatement.executeInternal(PgStatement.java:517)
	at org.postgresql.jdbc.PgStatement.execute(PgStatement.java:434)
	at org.postgresql.jdbc.PgPreparedStatement.executeWithFlags(PgPreparedStatement.java:194)
	at org.postgresql.jdbc.PgPreparedStatement.executeUpdate(PgPreparedStatement.java:155)
	at org.apache.iceberg.jdbc.JdbcUtil.lambda$doCommitCreate$3(JdbcUtil.java:677)
	at org.apache.iceberg.ClientPoolImpl.run(ClientPoolImpl.java:72)
	at org.apache.iceberg.ClientPoolImpl.run(ClientPoolImpl.java:65)
	at org.apache.iceberg.jdbc.JdbcUtil.doCommitCreate(JdbcUtil.java:662)
	at org.apache.iceberg.jdbc.JdbcUtil.doCommitCreateTable(JdbcUtil.java:690)
	at org.apache.iceberg.jdbc.JdbcTableOperations.createTable(JdbcTableOperations.java:190)
	at org.apache.iceberg.jdbc.JdbcTableOperations.doCommit(JdbcTableOperations.java:120)
	... 41 more
	at org.apache.iceberg.rest.RESTCatalogAdapter.execute(RESTCatalogAdapter.java:635)
	at org.apache.iceberg.rest.RESTCatalogServlet.execute(RESTCatalogServlet.java:108)
	at org.apache.iceberg.rest.RESTCatalogServlet.doPost(RESTCatalogServlet.java:78)

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

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugSomething isn't working

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions