You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by is...@apache.org on 2021/10/14 16:22:34 UTC

[ignite] branch master updated: IGNITE-15204 Add SQLSTATE to thin client SQL error message

This is an automated email from the ASF dual-hosted git repository.

isapego pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new c83b794  IGNITE-15204 Add SQLSTATE to thin client SQL error message
c83b794 is described below

commit c83b7943046616eae71f45e1c2827a86b355ce5c
Author: Igor Sapego <is...@apache.org>
AuthorDate: Thu Oct 14 18:20:53 2021 +0200

    IGNITE-15204 Add SQLSTATE to thin client SQL error message
    
    This closes #9487
---
 .../internal/processors/odbc/SqlListenerUtils.java | 31 ++++++++++++++++
 .../processors/odbc/jdbc/JdbcRequestHandler.java   | 23 +-----------
 .../processors/odbc/odbc/OdbcRequestHandler.java   | 20 ++--------
 .../internal/processors/odbc/odbc/OdbcUtils.java   | 18 ---------
 .../platform/client/ClientRequestHandler.java      | 16 +++++++-
 .../thin-client-test/src/sql_fields_query_test.cpp | 43 ++++++++++++++++++++++
 .../src/impl/cache/cache_client_impl.cpp           | 23 +++++++++++-
 .../thin-client/src/impl/cache/cache_client_impl.h | 12 ++++++
 8 files changed, 128 insertions(+), 58 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java
index cd83308..4d1cf81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java
@@ -23,11 +23,19 @@ import java.sql.Timestamp;
 import java.util.UUID;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.transactions.TransactionAlreadyCompletedException;
+import org.apache.ignite.transactions.TransactionDuplicateKeyException;
+import org.apache.ignite.transactions.TransactionMixedModeException;
+import org.apache.ignite.transactions.TransactionSerializationException;
+import org.apache.ignite.transactions.TransactionUnsupportedConcurrencyException;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -285,6 +293,29 @@ public abstract class SqlListenerUtils {
     }
 
     /**
+     * @param e Exception to convert.
+     * @return IgniteQueryErrorCode.
+     */
+    public static int exceptionToSqlErrorCode(Throwable e) {
+        if (e instanceof QueryCancelledException)
+            return IgniteQueryErrorCode.QUERY_CANCELED;
+        if (e instanceof TransactionSerializationException)
+            return IgniteQueryErrorCode.TRANSACTION_SERIALIZATION_ERROR;
+        if (e instanceof TransactionAlreadyCompletedException)
+            return IgniteQueryErrorCode.TRANSACTION_COMPLETED;
+        if (e instanceof TransactionDuplicateKeyException)
+            return IgniteQueryErrorCode.DUPLICATE_KEY;
+        if (e instanceof TransactionMixedModeException)
+            return IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH;
+        if (e instanceof TransactionUnsupportedConcurrencyException)
+            return IgniteQueryErrorCode.UNSUPPORTED_OPERATION;
+        if (e instanceof IgniteSQLException)
+            return ((IgniteSQLException)e).statusCode();
+        else
+            return IgniteQueryErrorCode.UNKNOWN;
+    }
+
+    /**
      * <p>Converts sql pattern wildcards into java regex wildcards.</p>
      * <p>Translates "_" to "." and "%" to ".*" if those are not escaped with "\" ("\_" or "\%").</p>
      * <p>All other characters are considered normal and will be escaped if necessary.</p>
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index e048208..d6869aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -61,6 +61,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponseSender;
+import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.NestedTxMode;
@@ -76,11 +77,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.MarshallerContext;
-import org.apache.ignite.transactions.TransactionAlreadyCompletedException;
-import org.apache.ignite.transactions.TransactionDuplicateKeyException;
-import org.apache.ignite.transactions.TransactionMixedModeException;
-import org.apache.ignite.transactions.TransactionSerializationException;
-import org.apache.ignite.transactions.TransactionUnsupportedConcurrencyException;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_CONTINUE;
@@ -1322,22 +1318,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
      * @return resulting {@link JdbcResponse}.
      */
     private JdbcResponse exceptionToResult(Throwable e) {
-        if (e instanceof QueryCancelledException)
-            return new JdbcResponse(IgniteQueryErrorCode.QUERY_CANCELED, e.getMessage());
-        if (e instanceof TransactionSerializationException)
-            return new JdbcResponse(IgniteQueryErrorCode.TRANSACTION_SERIALIZATION_ERROR, e.getMessage());
-        if (e instanceof TransactionAlreadyCompletedException)
-            return new JdbcResponse(IgniteQueryErrorCode.TRANSACTION_COMPLETED, e.getMessage());
-        if (e instanceof TransactionDuplicateKeyException)
-            return new JdbcResponse(IgniteQueryErrorCode.DUPLICATE_KEY, e.getMessage());
-        if (e instanceof TransactionMixedModeException)
-            return new JdbcResponse(IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH, e.getMessage());
-        if (e instanceof TransactionUnsupportedConcurrencyException)
-            return new JdbcResponse(IgniteQueryErrorCode.UNSUPPORTED_OPERATION, e.getMessage());
-        if (e instanceof IgniteSQLException)
-            return new JdbcResponse(((IgniteSQLException)e).statusCode(), e.getMessage());
-        else
-            return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, e.getMessage());
+        return new JdbcResponse(SqlListenerUtils.exceptionToSqlErrorCode(e), e.getMessage());
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index d808fbd..3e8f9e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponseSender;
+import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcParameterMeta;
 import org.apache.ignite.internal.processors.odbc.odbc.escape.OdbcEscapeUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
@@ -62,11 +63,6 @@ import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.transactions.TransactionAlreadyCompletedException;
-import org.apache.ignite.transactions.TransactionDuplicateKeyException;
-import org.apache.ignite.transactions.TransactionMixedModeException;
-import org.apache.ignite.transactions.TransactionSerializationException;
-import org.apache.ignite.transactions.TransactionUnsupportedConcurrencyException;
 
 import static java.sql.ResultSetMetaData.columnNoNulls;
 import static java.sql.ResultSetMetaData.columnNullable;
@@ -1026,19 +1022,9 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
      */
     private static OdbcResponse exceptionToResult(Throwable e) {
         String msg = OdbcUtils.tryRetrieveH2ErrorMessage(e);
+        int errorCode = SqlListenerUtils.exceptionToSqlErrorCode(e);
 
-        if (e instanceof TransactionSerializationException)
-            return new OdbcResponse(IgniteQueryErrorCode.TRANSACTION_SERIALIZATION_ERROR, msg);
-        if (e instanceof TransactionAlreadyCompletedException)
-            return new OdbcResponse(IgniteQueryErrorCode.TRANSACTION_COMPLETED, msg);
-        if (e instanceof TransactionMixedModeException)
-            return new OdbcResponse(IgniteQueryErrorCode.TRANSACTION_TYPE_MISMATCH, msg);
-        if (e instanceof TransactionUnsupportedConcurrencyException)
-            return new OdbcResponse(IgniteQueryErrorCode.UNSUPPORTED_OPERATION, msg);
-        if (e instanceof TransactionDuplicateKeyException)
-            return new OdbcResponse(IgniteQueryErrorCode.DUPLICATE_KEY, msg);
-
-        return new OdbcResponse(OdbcUtils.tryRetrieveSqlErrorCode(e), msg);
+        return new OdbcResponse(errorCode, msg);
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java
index 926524a..4bdad60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcUtils.java
@@ -24,11 +24,9 @@ import java.util.List;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
-import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.odbc.SqlListenerDataTypes;
 import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
-import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 
@@ -179,22 +177,6 @@ public class OdbcUtils {
     }
 
     /**
-     * Tries to retrieve SQL error code of the exception. If the exception is not {@link IgniteSQLException} returns
-     * {@link IgniteQueryErrorCode#UNKNOWN}.
-     *
-     * @param err Error to retrieve code from.
-     * @return Error code.
-     */
-    public static int tryRetrieveSqlErrorCode(Throwable err) {
-        int errorCode = IgniteQueryErrorCode.UNKNOWN;
-
-        if (err instanceof IgniteSQLException)
-            errorCode = ((IgniteSQLException) err).statusCode();
-
-        return errorCode;
-    }
-
-    /**
      * Tries to retrieve H2 engine error message from exception. If the exception is not of type
      * "org.h2.jdbc.JdbcSQLException" returns original error message.
      *
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
index 55a151a..4eaf887 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
@@ -21,10 +21,15 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
+import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
+import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheQueryNextPageRequest;
+import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheSqlFieldsQueryRequest;
+import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheSqlQueryRequest;
 import org.apache.ignite.internal.processors.platform.client.tx.ClientTxAwareRequest;
 import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext;
 import org.apache.ignite.plugin.security.SecurityException;
@@ -108,8 +113,17 @@ public class ClientRequestHandler implements ClientListenerRequestHandler {
         assert e != null;
 
         int status = getStatus(e);
+        String msg = e.getMessage();
 
-        return new ClientResponse(req.requestId(), status, e.getMessage());
+        if (req instanceof ClientCacheSqlQueryRequest ||
+            req instanceof ClientCacheSqlFieldsQueryRequest ||
+            req instanceof ClientCacheQueryNextPageRequest) {
+
+            String sqlState = IgniteQueryErrorCode.codeToSqlState(SqlListenerUtils.exceptionToSqlErrorCode(e));
+            msg = sqlState + ": " + msg;
+        }
+
+        return new ClientResponse(req.requestId(), status, msg);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/platforms/cpp/thin-client-test/src/sql_fields_query_test.cpp b/modules/platforms/cpp/thin-client-test/src/sql_fields_query_test.cpp
index b7a55b1..cbc1a4f 100644
--- a/modules/platforms/cpp/thin-client-test/src/sql_fields_query_test.cpp
+++ b/modules/platforms/cpp/thin-client-test/src/sql_fields_query_test.cpp
@@ -17,6 +17,7 @@
 
 #include <boost/test/unit_test.hpp>
 
+#include <ignite/ignite_error.h>
 #include <ignite/ignition.h>
 
 #include <ignite/thin/ignite_client_configuration.h>
@@ -55,6 +56,30 @@ public:
         ignite::Ignition::StopAll(false);
     }
 
+    /**
+     * Perform an invalid SQL query and check whether returned SQLSTATE is expected.
+     */
+    void CheckSqlStateForQuery(const std::string& sql, const std::string& expectedSqlState)
+    {
+        SqlFieldsQuery testQry(sql);
+        testQry.SetSchema("PUBLIC");
+
+        try
+        {
+            cacheAllFields.Query(testQry);
+
+            BOOST_FAIL("Expected to get SQL error here");
+        }
+        catch (ignite::IgniteError& err)
+        {
+            std::string msg(err.GetText());
+
+            std::string sqlState(msg.substr(0, msg.find(':')));
+
+            BOOST_CHECK_EQUAL(expectedSqlState, sqlState);
+        }
+    }
+
 protected:
     /** Server node. */
     ignite::Ignite serverNode;
@@ -494,4 +519,22 @@ BOOST_AUTO_TEST_CASE(TestInsertNull)
     CheckCursorEmpty(cursor);
 }
 
+
+/**
+ * Test that SQL errors contain SQLSTATE with cause.
+ */
+BOOST_AUTO_TEST_CASE(TestSqlStateOnErrors)
+{
+    CheckSqlStateForQuery("select * from \"UnknownCache\".UNKNOWN_TABLE", "42000");
+    CheckSqlStateForQuery("select * from UNKNOWN_TABLE", "42000");
+    CheckSqlStateForQuery("insert into \"cacheAllFields\".TestType(_key) values(null)", "22004");
+    CheckSqlStateForQuery("insert into \"cacheAllFields\".TestType(_key) values('abc')", "0700B");
+    CheckSqlStateForQuery("insert into \"cacheAllFields\".TestType(_key, _val) values(1, null)", "0A000");
+
+    SqlFieldsQuery qry("CREATE TABLE PUBLIC.varchar_table(id INT PRIMARY KEY, str VARCHAR(5))");
+    cacheAllFields.Query(qry);
+
+    CheckSqlStateForQuery("insert into varchar_table(id, str) values(1, 'too_long')", "23000");
+}
+
 BOOST_AUTO_TEST_SUITE_END()
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp
index d66b9e6..6be7064 100644
--- a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp
@@ -102,6 +102,27 @@ namespace ignite
                 }
 
                 template<typename ReqT, typename RspT>
+                SP_DataChannel CacheClientImpl::SyncMessageSql(const ReqT& req, RspT& rsp)
+                {
+                    SP_DataChannel channel;
+                    try {
+                        channel = router.Get()->SyncMessage(req, rsp);
+                    }
+                    catch (IgniteError& err)
+                    {
+                        std::string msg("08001: ");
+                        msg += err.GetText();
+
+                        throw IgniteError(err.GetCode(), msg.c_str());
+                    }
+
+                    if (rsp.GetStatus() != ResponseStatus::SUCCESS)
+                        throw IgniteError(IgniteError::IGNITE_ERR_CACHE, rsp.GetError().c_str());
+
+                    return channel;
+                }
+
+                template<typename ReqT, typename RspT>
                 void CacheClientImpl::TransactionalSyncCacheKeyMessage(const WritableKey &key, ReqT &req,
                     RspT &rsp)
                 {
@@ -334,7 +355,7 @@ namespace ignite
                     SqlFieldsQueryRequest req(id, qry);
                     SqlFieldsQueryResponse rsp;
 
-                    SP_DataChannel channel = SyncMessage(req, rsp);
+                    SP_DataChannel channel = SyncMessageSql(req, rsp);
 
                     query::SP_QueryFieldsCursorImpl cursorImpl(
                         new query::QueryFieldsCursorImpl(
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h
index 4efefd7..a7ab0be 100644
--- a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h
@@ -322,6 +322,18 @@ namespace ignite
                     SP_DataChannel SyncMessage(const ReqT& req, RspT& rsp);
 
                     /**
+                     * Synchronously send message and receive response.
+                     * Modified to properly set SQL state on connection errors.
+                     *
+                     * @param req Request message.
+                     * @param rsp Response message.
+                     * @return Channel that was used for request.
+                     * @throw IgniteError on error.
+                     */
+                    template<typename ReqT, typename RspT>
+                    SP_DataChannel SyncMessageSql(const ReqT& req, RspT& rsp);
+
+                    /**
                      * Synchronously send request message and receive response taking in account that it can be
                      * transactional.
                      *