You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "AnubhavSiddharth (via GitHub)" <gi...@apache.org> on 2023/02/28 01:15:52 UTC

[GitHub] [iceberg] AnubhavSiddharth opened a new pull request, #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

AnubhavSiddharth opened a new pull request, #6952:
URL: https://github.com/apache/iceberg/pull/6952

   Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions
   
   The Snowflake JDBC Client was throwing generic exceptions in some scenarios (Eg: UncheckedSQLException), this change maps them to the appropriate Iceberg Exceptions to help the retry/error-handling mechanism in Iceberg. 
   
   Added test cases as well. 


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] AnubhavSiddharth commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "AnubhavSiddharth (via GitHub)" <gi...@apache.org>.
AnubhavSiddharth commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1125063665


##########
snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java:
##########
@@ -367,15 +535,66 @@ public void testListIcebergTablesInSchema() throws SQLException {
 
   /**
    * Any unexpected SQLException from the underlying connection will propagate out as an
-   * UncheckedSQLException when listing tables.
+   * UncheckedSQLException when listing tables at Root level
    */
   @Test
-  public void testListIcebergTablesSQLException() throws SQLException, InterruptedException {
-    when(mockClientPool.run(any(ClientPool.Action.class)))
-        .thenThrow(new SQLException("Fake SQL exception"));
-    Assertions.assertThatExceptionOfType(UncheckedSQLException.class)
-        .isThrownBy(() -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofDatabase("DB_1")))
-        .withStackTraceContaining("Fake SQL exception");
+  public void testListIcebergTablesSQLExceptionAtRootLevel()
+      throws SQLException, InterruptedException {
+    generateExceptionAndAssert(
+        () -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofRoot()),
+        new SQLException(String.format("SQL exception with Error Code %d", 0), "2000", 0, null),
+        UncheckedSQLException.class);
+  }
+
+  /**
+   * Any unexpected SQLException with specific error codes from the underlying connection will
+   * propagate out as a NoSuchNamespaceException when listing tables at Database level
+   */
+  @Test
+  public void testListIcebergTablesSQLExceptionAtDatabaseLevel()
+      throws SQLException, InterruptedException {
+    for (Integer errorCode : DATABASE_NOT_FOUND_ERROR_CODES) {
+      generateExceptionAndAssert(
+          () -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofDatabase("DB_1")),
+          new SQLException(
+              String.format("SQL exception with Error Code %d", errorCode),
+              "2000",
+              errorCode,
+              null),
+          NoSuchNamespaceException.class);
+    }
+  }
+
+  /**
+   * Any unexpected SQLException with specific error codes from the underlying connection will
+   * propagate out as a NoSuchNamespaceException when listing tables at Schema level
+   */
+  @Test
+  public void testListIcebergTablesSQLExceptionAtSchemaLevel()
+      throws SQLException, InterruptedException {
+    for (Integer errorCode : SCHEMA_NOT_FOUND_ERROR_CODES) {

Review Comment:
   Makes sense, I've made the suggested changes



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1186409992


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -125,6 +130,15 @@ public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, Strin
   private final JdbcClientPool connectionPool;
   private QueryHarness queryHarness;
 
+  @VisibleForTesting

Review Comment:
   I would move these static declarations to the top of the class.  The ordering here is a little awkward.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] AnubhavSiddharth commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "AnubhavSiddharth (via GitHub)" <gi...@apache.org>.
AnubhavSiddharth commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1125064058


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -208,6 +219,7 @@ public List<SnowflakeIdentifier> listDatabases() {
                   queryHarness.query(
                       conn, "SHOW DATABASES IN ACCOUNT", DATABASE_RESULT_SET_HANDLER));
     } catch (SQLException e) {
+      tryMapSnowflakeExceptionToIcebergException(SnowflakeIdentifier.ofRoot(), e);

Review Comment:
   Yes, I agree this looks better semantically, made the changes, thanks!



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] AnubhavSiddharth commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "AnubhavSiddharth (via GitHub)" <gi...@apache.org>.
AnubhavSiddharth commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1126825655


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -125,6 +130,15 @@ public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, Strin
   private final JdbcClientPool connectionPool;
   private QueryHarness queryHarness;
 
+  @VisibleForTesting
+  static final Set<Integer> DATABASE_NOT_FOUND_ERROR_CODES = ImmutableSet.of(2001, 2003, 2043);
+
+  @VisibleForTesting
+  static final Set<Integer> SCHEMA_NOT_FOUND_ERROR_CODES = ImmutableSet.of(2001, 2003, 2043);
+
+  @VisibleForTesting
+  static final Set<Integer> TABLE_NOT_FOUND_ERROR_CODES = ImmutableSet.of(2001, 2003, 2043);

Review Comment:
   The 3 sets could possibly diverge in the future, yes. 



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1186415027


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -343,4 +366,32 @@ public SnowflakeTableMetadata loadTableMetadata(SnowflakeIdentifier tableIdentif
   public void close() {
     connectionPool.close();
   }
+
+  private Optional<RuntimeException> snowflakeExceptionToIcebergException(

Review Comment:
   I think you could avoid the optional with all the additional `orElseGet` statements above by just passing in the generic text and falling back to an UncheckedSQLException.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1122838263


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -125,6 +129,13 @@ public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, Strin
   private final JdbcClientPool connectionPool;
   private QueryHarness queryHarness;
 
+  protected static final Set<Integer> DATABASE_NOT_FOUND_ERROR_CODES =

Review Comment:
   It seems these are only used in tests, so should all of these maybe be package-private and have a `@VisibleForTesting` annotation?



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java:
##########
@@ -367,15 +535,66 @@ public void testListIcebergTablesInSchema() throws SQLException {
 
   /**
    * Any unexpected SQLException from the underlying connection will propagate out as an
-   * UncheckedSQLException when listing tables.
+   * UncheckedSQLException when listing tables at Root level
    */
   @Test
-  public void testListIcebergTablesSQLException() throws SQLException, InterruptedException {
-    when(mockClientPool.run(any(ClientPool.Action.class)))
-        .thenThrow(new SQLException("Fake SQL exception"));
-    Assertions.assertThatExceptionOfType(UncheckedSQLException.class)
-        .isThrownBy(() -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofDatabase("DB_1")))
-        .withStackTraceContaining("Fake SQL exception");
+  public void testListIcebergTablesSQLExceptionAtRootLevel()
+      throws SQLException, InterruptedException {
+    generateExceptionAndAssert(
+        () -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofRoot()),
+        new SQLException(String.format("SQL exception with Error Code %d", 0), "2000", 0, null),
+        UncheckedSQLException.class);
+  }
+
+  /**
+   * Any unexpected SQLException with specific error codes from the underlying connection will
+   * propagate out as a NoSuchNamespaceException when listing tables at Database level
+   */
+  @Test
+  public void testListIcebergTablesSQLExceptionAtDatabaseLevel()
+      throws SQLException, InterruptedException {
+    for (Integer errorCode : DATABASE_NOT_FOUND_ERROR_CODES) {
+      generateExceptionAndAssert(
+          () -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofDatabase("DB_1")),
+          new SQLException(
+              String.format("SQL exception with Error Code %d", errorCode),
+              "2000",
+              errorCode,
+              null),
+          NoSuchNamespaceException.class);
+    }
+  }
+
+  /**
+   * Any unexpected SQLException with specific error codes from the underlying connection will
+   * propagate out as a NoSuchNamespaceException when listing tables at Schema level
+   */
+  @Test
+  public void testListIcebergTablesSQLExceptionAtSchemaLevel()
+      throws SQLException, InterruptedException {
+    for (Integer errorCode : SCHEMA_NOT_FOUND_ERROR_CODES) {

Review Comment:
   TBH using `generateExceptionAndAssert()` makes the code quite difficult to read and reason about. I understand the argument for reusing the same code, but I think tests are clearer to read when we're asserting everything in the test method, similar to below: 
   
   ```
    @Test
     public void testListIcebergTablesSQLExceptionAtSchemaLevel()
         throws SQLException, InterruptedException {
       for (Integer errorCode : SCHEMA_NOT_FOUND_ERROR_CODES) {
         SQLException injectedException =
             new SQLException(
                 String.format("SQL exception with Error Code %d", errorCode),
                 "2000",
                 errorCode,
                 null);
         when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException);
   
         Assertions.assertThatExceptionOfType(NoSuchNamespaceException.class)
             .isThrownBy(
                 () ->
                     snowflakeClient.listIcebergTables(
                         SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_1")))
             .withMessageContaining("Identifier not found: 'SCHEMA: 'DB_1.SCHEMA_1''")
             .withCause(injectedException);
       }
     }
   ```
   
   Note that we're using `.withMessageContaining` to make sure we're getting the right error message, and `.withCause` allows to make sure we get the correct exception in the cause.
   



##########
snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java:
##########
@@ -520,14 +757,12 @@ public void testGetTableMetadataSQLException() throws SQLException, InterruptedE
    */
   @Test
   public void testGetTableMetadataInterruptedException() throws SQLException, InterruptedException {
-    when(mockClientPool.run(any(ClientPool.Action.class)))

Review Comment:
   after reading through the entire changes in this class I don't think we should be introducing `generateExceptionAndAssert` as this makes it more difficult to read the code



##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -208,6 +219,7 @@ public List<SnowflakeIdentifier> listDatabases() {
                   queryHarness.query(
                       conn, "SHOW DATABASES IN ACCOUNT", DATABASE_RESULT_SET_HANDLER));
     } catch (SQLException e) {
+      tryMapSnowflakeExceptionToIcebergException(SnowflakeIdentifier.ofRoot(), e);

Review Comment:
   reading through the catch clause here and seeing this,  one wouldn't expect `tryMapSnowflakeExceptionToIcebergException` to actually throw. I think it would be good to rewrite the code so that it's more obvious.
   
   What about having `snowflakeExceptionToIcebergException` return an `Optional<RuntimeException>`? Then we could simply have  `throw snowflakeExceptionToIcebergException(SnowflakeIdentifier.ofRoot(), e).orElseGet(() -> new UncheckedSQLException(e, "Failed to list databases"));` here.



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] AnubhavSiddharth commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "AnubhavSiddharth (via GitHub)" <gi...@apache.org>.
AnubhavSiddharth commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1125063482


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -125,6 +129,13 @@ public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, Strin
   private final JdbcClientPool connectionPool;
   private QueryHarness queryHarness;
 
+  protected static final Set<Integer> DATABASE_NOT_FOUND_ERROR_CODES =

Review Comment:
   Done!



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #6952:
URL: https://github.com/apache/iceberg/pull/6952#discussion_r1126045030


##########
snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java:
##########
@@ -125,6 +130,15 @@ public <T> T query(Connection conn, String sql, ResultSetParser<T> parser, Strin
   private final JdbcClientPool connectionPool;
   private QueryHarness queryHarness;
 
+  @VisibleForTesting
+  static final Set<Integer> DATABASE_NOT_FOUND_ERROR_CODES = ImmutableSet.of(2001, 2003, 2043);
+
+  @VisibleForTesting
+  static final Set<Integer> SCHEMA_NOT_FOUND_ERROR_CODES = ImmutableSet.of(2001, 2003, 2043);
+
+  @VisibleForTesting
+  static final Set<Integer> TABLE_NOT_FOUND_ERROR_CODES = ImmutableSet.of(2001, 2003, 2043);

Review Comment:
   are we expecting those 3 sets to eventually contain different error codes?



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks merged pull request #6952: Improve Error Handling to map Snowflake Exceptions into Iceberg Exceptions

Posted by "danielcweeks (via GitHub)" <gi...@apache.org>.
danielcweeks merged PR #6952:
URL: https://github.com/apache/iceberg/pull/6952


-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org