sqd opened a new issue, #13924:
URL: https://github.com/apache/iceberg/issues/13924

   ### 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](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java#L139-L143)
 to handle scenarios where the table already existed. However, [PostgresSQL 
throws a different 
exception](https://github.com/postgres/postgres/blob/master/src/backend/access/nbtree/nbtinsert.c#L669)
 that we do not currently handle (for reference, [this is how Postgres's Java 
client handles the 
error](https://github.com/pgjdbc/pgjdbc/blob/master/pgjdbc/src/main/java/org/postgresql/core/v3/QueryExecutorImpl.java#L2736)).
 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)=(iceberg_us, 
starfleet.SkippyEngineEvent, PyramidGenStaticConfigEventAvro) 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
   
   - [x] 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


-- 
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

Reply via email to