You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pp...@apache.org on 2023/09/08 13:57:26 UTC

[ignite-3] branch main updated: IGNITE-20149 Sql. Revise the use of the INTERNAL_ERR code in the sql module (#2549)

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

ppa pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 673a0a99a7 IGNITE-20149 Sql. Revise the use of the INTERNAL_ERR code in the sql module (#2549)
673a0a99a7 is described below

commit 673a0a99a7f03374ad3ccebc877a68652da828cf
Author: ygerzhedovich <41...@users.noreply.github.com>
AuthorDate: Fri Sep 8 16:57:20 2023 +0300

    IGNITE-20149 Sql. Revise the use of the INTERNAL_ERR code in the sql module (#2549)
---
 .../apache/ignite/sql/CursorClosedException.java   |   2 +-
 .../client/handler/JdbcQueryEventHandlerImpl.java  |  38 ++----
 .../ignite/lang/IgniteExceptionMapperUtil.java     |   8 +-
 .../apache/ignite/jdbc/ItJdbcBatchSelfTest.java    |  27 ++--
 .../ignite/jdbc/ItJdbcStatementSelfTest.java       |   8 +-
 .../ignite/internal/sql/api/ItCommonApiTest.java   |   4 +-
 .../internal/sql/api/ItSqlAsynchronousApiTest.java | 141 +++++++++++----------
 .../internal/sql/api/ItSqlClientMetricsTest.java   |  10 +-
 .../internal/sql/api/ItSqlSynchronousApiTest.java  |  94 +++++++++-----
 .../internal/sql/engine/ItCreateTableDdlTest.java  |   1 +
 .../internal/sql/engine/ItDataTypesTest.java       |   5 +-
 .../ignite/internal/sql/engine/ItDmlTest.java      |   7 +-
 .../sql/engine/ItDynamicParameterTest.java         |   9 +-
 .../internal/sql/engine/ItLimitOffsetTest.java     |   3 +
 .../sql/engine/QueryCancelledException.java        |   9 ++
 .../ignite/internal/sql/engine/QueryContext.java   |   4 +-
 .../internal/sql/engine/SqlQueryProcessor.java     |   7 +-
 .../sql/engine/exec/ExchangeServiceImpl.java       |   3 +-
 .../engine/exec/ExecutionCancelledException.java   |  38 ------
 .../sql/engine/exec/ExecutionServiceImpl.java      |  21 +--
 .../sql/engine/exec/QueryValidationException.java  |  38 ------
 .../sql/engine/exec/exp/ExpressionFactoryImpl.java |   9 +-
 .../internal/sql/engine/exec/rel/AbstractNode.java |   4 +-
 .../sql/engine/exec/rel/AsyncRootNode.java         |   4 +-
 .../internal/sql/engine/exec/rel/RootNode.java     |   8 +-
 .../metadata/RemoteFragmentExecutionException.java |   7 +-
 .../sql/engine/prepare/PrepareServiceImpl.java     |   4 +-
 .../engine/util/SqlExceptionMapperProvider.java    |   7 +-
 .../engine/QueryTransactionWrapperSelfTest.java    |   1 +
 .../internal/sql/engine/StopCalciteModuleTest.java |   3 +-
 .../sql/engine/exec/ExecutionServiceImplTest.java  |   5 +-
 .../sql/engine/planner/PlannerTimeoutTest.java     |   1 +
 .../internal/sql/engine/util/SqlTestUtils.java     |  34 +++--
 33 files changed, 273 insertions(+), 291 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/sql/CursorClosedException.java b/modules/api/src/main/java/org/apache/ignite/sql/CursorClosedException.java
index d75d663f52..2bad187d4f 100644
--- a/modules/api/src/main/java/org/apache/ignite/sql/CursorClosedException.java
+++ b/modules/api/src/main/java/org/apache/ignite/sql/CursorClosedException.java
@@ -27,6 +27,6 @@ public class CursorClosedException extends SqlException {
      * Creates an exception instance.
      */
     public CursorClosedException() {
-        super(CURSOR_CLOSED_ERR);
+        super(CURSOR_CLOSED_ERR, "Cursor is closed");
     }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
index ea632c4f4d..4344eae430 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
@@ -23,8 +23,6 @@ import static org.apache.ignite.internal.util.ArrayUtils.OBJECT_EMPTY_ARRAY;
 import static org.apache.ignite.lang.ErrorGroups.Client.CONNECTION_ERR;
 
 import it.unimi.dsi.fastutil.ints.IntArrayList;
-import java.io.PrintWriter;
-import java.io.StringWriter;
 import java.sql.Statement;
 import java.util.List;
 import java.util.Set;
@@ -58,14 +56,12 @@ import org.apache.ignite.internal.sql.engine.AsyncSqlCursor;
 import org.apache.ignite.internal.sql.engine.QueryContext;
 import org.apache.ignite.internal.sql.engine.QueryProcessor;
 import org.apache.ignite.internal.sql.engine.SqlQueryType;
-import org.apache.ignite.internal.sql.engine.exec.QueryValidationException;
 import org.apache.ignite.internal.sql.engine.property.PropertiesHelper;
 import org.apache.ignite.internal.sql.engine.property.PropertiesHolder;
 import org.apache.ignite.internal.sql.engine.session.SessionId;
 import org.apache.ignite.internal.sql.engine.session.SessionNotFoundException;
 import org.apache.ignite.internal.util.ExceptionUtils;
 import org.apache.ignite.internal.util.Pair;
-import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteExceptionMapperUtil;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.apache.ignite.lang.IgniteInternalException;
@@ -137,9 +133,9 @@ public class JdbcQueryEventHandlerImpl implements JdbcQueryEventHandler {
 
             return CompletableFuture.completedFuture(new JdbcConnectResult(connectionId));
         } catch (IgniteInternalCheckedException exception) {
-            StringWriter sw = getWriterWithStackTrace(exception);
+            String msg = getErrorMessage(exception);
 
-            return CompletableFuture.completedFuture(new JdbcConnectResult(Response.STATUS_FAILED, "Unable to connect: " + sw));
+            return CompletableFuture.completedFuture(new JdbcConnectResult(Response.STATUS_FAILED, "Unable to connect: " + msg));
         }
     }
 
@@ -175,10 +171,10 @@ public class JdbcQueryEventHandlerImpl implements JdbcQueryEventHandler {
                 .exceptionally(t -> {
                     LOG.info("Exception while executing query [query=" + req.sqlQuery() + "]", ExceptionUtils.unwrapCause(t));
 
-                    StringWriter sw = getWriterWithStackTrace(t);
+                    String msg = getErrorMessage(t);
 
                     return new JdbcQueryExecuteResult(Response.STATUS_FAILED,
-                            "Exception while executing query [query=" + req.sqlQuery() + "]. Error message:" + sw);
+                            "Exception while executing query [query=" + req.sqlQuery() + "]. Error message:" + msg);
                 });
     }
 
@@ -282,14 +278,14 @@ public class JdbcQueryEventHandlerImpl implements JdbcQueryEventHandler {
     }
 
     private JdbcBatchExecuteResult handleBatchException(Throwable e, String query, int[] counters) {
-        StringWriter sw = getWriterWithStackTrace(e);
+        String msg = getErrorMessage(e);
 
         String error;
 
         if (e instanceof ClassCastException) {
-            error = "Unexpected result. Not an upsert statement? [query=" + query + "] Error message:" + sw;
+            error = "Unexpected result. Not an upsert statement? [query=" + query + "] Error message:" + msg;
         } else {
-            error = sw.toString();
+            error = msg;
         }
 
         return new JdbcBatchExecuteResult(Response.STATUS_FAILED, UNKNOWN, error, counters);
@@ -340,26 +336,14 @@ public class JdbcQueryEventHandlerImpl implements JdbcQueryEventHandler {
     }
 
     /**
-     * Serializes the stack trace of given exception for further sending to the client.
+     * Get a message of given exception for further sending to the client.
      *
      * @param t Throwable.
-     * @return StringWriter filled with exception.
+     * @return String filled with exception message.
      */
-    private StringWriter getWriterWithStackTrace(Throwable t) {
+    @Nullable private String getErrorMessage(Throwable t) {
         Throwable cause = ExceptionUtils.unwrapCause(t);
-        StringWriter sw = new StringWriter();
-
-        try (PrintWriter pw = new PrintWriter(sw)) {
-            // We need to remap QueryValidationException into a jdbc error.
-            if (cause instanceof QueryValidationException
-                    || (cause instanceof IgniteException && cause.getCause() instanceof QueryValidationException)) {
-                pw.print("Given statement type does not match that declared by JDBC driver.");
-            } else {
-                pw.print(cause.getMessage());
-            }
-
-            return sw;
-        }
+        return cause.getMessage();
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java
index 869daa8801..9ee7c4f988 100755
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteExceptionMapperUtil.java
@@ -88,10 +88,6 @@ public class IgniteExceptionMapperUtil {
             return origin;
         }
 
-        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
-            return origin;
-        }
-
         IgniteExceptionMapper<? extends Exception, ? extends Exception> m = EXCEPTION_CONVERTERS.get(origin.getClass());
         if (m != null) {
             Exception mapped = map(m, origin);
@@ -102,6 +98,10 @@ public class IgniteExceptionMapperUtil {
             return mapped;
         }
 
+        if (origin instanceof IgniteException || origin instanceof IgniteCheckedException) {
+            return origin;
+        }
+
         // There are no exception mappings for the given exception. This case should be considered as internal error.
         return new IgniteException(INTERNAL_ERR, origin);
     }
diff --git a/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcBatchSelfTest.java b/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcBatchSelfTest.java
index 9fa6cc9ba6..208a3a9847 100644
--- a/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcBatchSelfTest.java
+++ b/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcBatchSelfTest.java
@@ -172,7 +172,7 @@ public class ItJdbcBatchSelfTest extends AbstractJdbcSelfTest {
 
             assertEquals(0, updCnts.length, "Invalid update counts size");
 
-            assertThat(e.getMessage(), containsString("Given statement type does not match that declared by JDBC driver"));
+            assertThat(e.getMessage(), containsString("Invalid SQL statement type. Expected [DML, DDL] but got QUERY"));
 
             assertEquals(SqlStateCode.INTERNAL_ERROR, e.getSQLState(), "Invalid SQL state.");
             assertEquals(IgniteQueryErrorCode.UNKNOWN, e.getErrorCode(), "Invalid error code.");
@@ -217,24 +217,21 @@ public class ItJdbcBatchSelfTest extends AbstractJdbcSelfTest {
         stmt.addBatch("insert into Person (id, firstName, lastName, age) values "
                 + generateValues(100, 7));
 
-        try {
-            stmt.executeBatch();
-
-            fail("BatchUpdateException must be thrown");
-        } catch (BatchUpdateException e) {
-            int[] updCnts = e.getUpdateCounts();
-
-            assertEquals(successUpdates, updCnts.length, "Invalid update counts size");
+        BatchUpdateException e = JdbcTestUtils.assertThrowsSqlException(
+                BatchUpdateException.class,
+                "Invalid SQL statement type. Expected [DML, DDL] but got QUERY",
+                stmt::executeBatch);
 
-            for (int i = 0; i < successUpdates; ++i) {
-                assertEquals(i + 1, updCnts[i], "Invalid update count");
-            }
+        int[] updCnts = e.getUpdateCounts();
 
-            assertThat(e.getMessage(), containsString("Given statement type does not match that declared by JDBC driver"));
+        assertEquals(successUpdates, updCnts.length, "Invalid update counts size");
 
-            assertEquals(SqlStateCode.INTERNAL_ERROR, e.getSQLState(), "Invalid SQL state.");
-            assertEquals(IgniteQueryErrorCode.UNKNOWN, e.getErrorCode(), "Invalid error code.");
+        for (int i = 0; i < successUpdates; ++i) {
+            assertEquals(i + 1, updCnts[i], "Invalid update count");
         }
+
+        assertEquals(SqlStateCode.INTERNAL_ERROR, e.getSQLState(), "Invalid SQL state.");
+        assertEquals(IgniteQueryErrorCode.UNKNOWN, e.getErrorCode(), "Invalid error code.");
     }
 
     @Test
diff --git a/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcStatementSelfTest.java b/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcStatementSelfTest.java
index 0cae0b498f..5086ce05cb 100644
--- a/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcStatementSelfTest.java
+++ b/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcStatementSelfTest.java
@@ -482,13 +482,13 @@ public class ItJdbcStatementSelfTest extends ItJdbcAbstractStatementSelfTest {
         final String sqlText = "select * from TEST;";
 
         JdbcTestUtils.assertThrowsSqlException(
-                "Given statement type does not match that declared by JDBC driver",
+                "Invalid SQL statement type",
                 () -> stmt.executeUpdate(sqlText));
     }
 
     @Test
     public void testExecuteUpdateOnDdl() throws SQLException {
-        String tableName = "\"test_" + UUID.randomUUID().toString() + "\"";
+        String tableName = "\"test_" + UUID.randomUUID() + "\"";
 
         stmt.executeUpdate("CREATE TABLE " + tableName + "(id INT PRIMARY KEY, val VARCHAR)");
 
@@ -758,7 +758,7 @@ public class ItJdbcStatementSelfTest extends ItJdbcAbstractStatementSelfTest {
         stmt.executeQuery("select 1;");
 
         JdbcTestUtils.assertThrowsSqlException(
-                "Given statement type does not match that declared by JDBC driver",
+                "Invalid SQL statement type",
                 () -> stmt.executeUpdate("select 1;")
         );
 
@@ -768,7 +768,7 @@ public class ItJdbcStatementSelfTest extends ItJdbcAbstractStatementSelfTest {
     @Test
     public void testStatementTypeMismatchUpdate() throws Exception {
         JdbcTestUtils.assertThrowsSqlException(
-                "Given statement type does not match that declared by JDBC driver",
+                "Invalid SQL statement type",
                 () -> stmt.executeQuery("update TEST set NAME='28' where ID=1")
         );
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java
index adc3488085..53bff13425 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java
@@ -40,8 +40,8 @@ import org.apache.ignite.internal.schema.testutils.definition.ColumnType;
 import org.apache.ignite.internal.schema.testutils.definition.ColumnType.TemporalColumnType;
 import org.apache.ignite.internal.schema.testutils.definition.TableDefinition;
 import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.sql.engine.SqlQueryProcessor;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionCancelledException;
 import org.apache.ignite.internal.sql.engine.schema.SqlSchemaManager;
 import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.internal.testframework.IgniteTestUtils;
@@ -94,7 +94,7 @@ public class ItCommonApiTest extends ClusterPerClassIntegrationTest {
             while (rs1.hasNext()) {
                 rs1.next();
             }
-        }, ExecutionCancelledException.class);
+        }, QueryCancelledException.class);
 
         rs1.close();
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java
index c3023f41b3..ab9ef7a6be 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlAsynchronousApiTest.java
@@ -19,18 +19,10 @@ package org.apache.ignite.internal.sql.api;
 
 import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsIndexScan;
 import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsTableScan;
+import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.assertThrowsSqlException;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.await;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
-import static org.apache.ignite.lang.ErrorGroups.Sql.CONSTRAINT_VIOLATION_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Sql.CURSOR_CLOSED_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Sql.EXECUTION_CANCELLED_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Sql.QUERY_NO_RESULT_SET_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Sql.RUNTIME_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Sql.SESSION_CLOSED_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Sql.STMT_PARSE_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Sql.STMT_VALIDATION_ERR;
-import static org.apache.ignite.lang.ErrorGroups.Transactions.TX_FAILED_READ_WRITE_OPERATION_ERR;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -57,7 +49,7 @@ import org.apache.ignite.internal.catalog.commands.CatalogUtils;
 import org.apache.ignite.internal.client.sql.ClientSql;
 import org.apache.ignite.internal.sql.api.ColumnMetadataImpl.ColumnOriginImpl;
 import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.testframework.IgniteTestUtils;
 import org.apache.ignite.internal.testframework.IgniteTestUtils.RunnableX;
 import org.apache.ignite.internal.tx.TxManager;
@@ -66,6 +58,8 @@ import org.apache.ignite.lang.ColumnAlreadyExistsException;
 import org.apache.ignite.lang.ColumnNotFoundException;
 import org.apache.ignite.lang.ErrorGroups;
 import org.apache.ignite.lang.ErrorGroups.Index;
+import org.apache.ignite.lang.ErrorGroups.Sql;
+import org.apache.ignite.lang.ErrorGroups.Transactions;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IndexAlreadyExistsException;
 import org.apache.ignite.lang.IndexNotFoundException;
@@ -114,12 +108,12 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)");
         checkError(
                 TableAlreadyExistsException.class,
+                ErrorGroups.Table.TABLE_ALREADY_EXISTS_ERR,
                 "Table already exists [name=\"PUBLIC\".\"TEST\"]",
                 ses,
                 "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"
         );
-        checkError(
-                SqlException.class,
+        checkSqlError(
                 ErrorGroups.Table.TABLE_DEFINITION_ERR,
                 "Can't create table with duplicate columns: ID, VAL, VAL",
                 ses,
@@ -131,6 +125,7 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "ALTER TABLE TEST ADD COLUMN VAL1 VARCHAR");
         checkError(
                 TableNotFoundException.class,
+                ErrorGroups.Table.TABLE_NOT_FOUND_ERR,
                 "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]",
                 ses,
                 "ALTER TABLE NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR"
@@ -138,6 +133,7 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR");
         checkError(
                 ColumnAlreadyExistsException.class,
+                ErrorGroups.Table.COLUMN_ALREADY_EXISTS_ERR,
                 "Column already exists [name=\"VAL1\"]",
                 ses,
                 "ALTER TABLE TEST ADD COLUMN VAL1 INT"
@@ -147,6 +143,7 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "CREATE INDEX TEST_IDX ON TEST(VAL0)");
         checkError(
                 IndexAlreadyExistsException.class,
+                Index.INDEX_ALREADY_EXISTS_ERR,
                 "Index already exists [name=\"PUBLIC\".\"TEST_IDX\"]",
                 ses,
                 "CREATE INDEX TEST_IDX ON TEST(VAL1)"
@@ -160,25 +157,22 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "CREATE INDEX TEST_IDX1 ON TEST(VAL0)");
         checkDdl(true, ses, "CREATE INDEX TEST_IDX2 ON TEST(VAL0)");
         checkDdl(true, ses, "CREATE INDEX TEST_IDX3 ON TEST(ID, VAL0, VAL1)");
-        checkError(
-                SqlException.class,
+        checkSqlError(
                 Index.INVALID_INDEX_DEFINITION_ERR,
                 "Can't create index on duplicate columns: VAL0, VAL0",
                 ses,
                 "CREATE INDEX TEST_IDX4 ON TEST(VAL0, VAL0)"
         );
 
-        checkError(
-                SqlException.class,
-                STMT_VALIDATION_ERR,
+        checkSqlError(
+                Sql.STMT_VALIDATION_ERR,
                 "Can`t delete column(s). Column VAL1 is used by indexes [TEST_IDX3].",
                 ses,
                 "ALTER TABLE TEST DROP COLUMN val1"
         );
 
-        SqlException ex = checkError(
-                SqlException.class,
-                STMT_VALIDATION_ERR,
+        SqlException ex = checkSqlError(
+                Sql.STMT_VALIDATION_ERR,
                 "Can`t delete column(s).",
                 ses,
                 "ALTER TABLE TEST DROP COLUMN (val0, val1)"
@@ -191,9 +185,8 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         assertTrue(msg.contains("TEST_IDX1") && msg.contains("TEST_IDX2") && msg.contains("TEST_IDX3"), explainMsg);
         assertTrue(msg.contains("Column VAL1 is used by indexes [TEST_IDX3]"), explainMsg);
 
-        checkError(
-                SqlException.class,
-                STMT_VALIDATION_ERR,
+        checkSqlError(
+                Sql.STMT_VALIDATION_ERR,
                 "Can`t delete column, belongs to primary key: [name=ID]",
                 ses,
                 "ALTER TABLE TEST DROP COLUMN id"
@@ -207,6 +200,7 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "ALTER TABLE TEST DROP COLUMN VAL1");
         checkError(
                 TableNotFoundException.class,
+                ErrorGroups.Table.TABLE_NOT_FOUND_ERR,
                 "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]",
                 ses,
                 "ALTER TABLE NOT_EXISTS_TABLE DROP COLUMN VAL1"
@@ -214,6 +208,7 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE DROP COLUMN VAL1");
         checkError(
                 ColumnNotFoundException.class,
+                ErrorGroups.Table.COLUMN_NOT_FOUND_ERR,
                 "Column does not exist [tableName=\"PUBLIC\".\"TEST\", columnName=\"VAL1\"]",
                 ses,
                 "ALTER TABLE TEST DROP COLUMN VAL1"
@@ -225,6 +220,7 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "DROP TABLE TEST");
         checkError(
                 TableNotFoundException.class,
+                ErrorGroups.Table.TABLE_NOT_FOUND_ERR,
                 "The table does not exist [name=\"PUBLIC\".\"TEST\"]",
                 ses,
                 "DROP TABLE TEST"
@@ -234,6 +230,7 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
 
         checkError(
                 IndexNotFoundException.class,
+                Index.INDEX_NOT_FOUND_ERR,
                 "Index does not exist [name=\"PUBLIC\".\"TEST_IDX\"]", ses,
                 "DROP INDEX TEST_IDX"
         );
@@ -349,11 +346,11 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         Transaction outerTx0 = outerTx;
         IgniteException e = assertThrows(IgniteException.class,
                 () -> checkDml(1, ses, "INSERT INTO TEST VALUES (?, ?)", outerTx0, ROW_COUNT, Integer.MAX_VALUE));
-        assertEquals(TX_FAILED_READ_WRITE_OPERATION_ERR, e.code());
+        assertEquals(Transactions.TX_FAILED_READ_WRITE_OPERATION_ERR, e.code());
 
         e = assertThrows(SqlException.class,
                 () -> checkDml(1, ses, "INSERT INTO TEST VALUES (?, ?)", ROW_COUNT, Integer.MAX_VALUE));
-        assertEquals(CONSTRAINT_VIOLATION_ERR, e.code());
+        assertEquals(Sql.CONSTRAINT_VIOLATION_ERR, e.code());
 
         AsyncResultSet rs = await(ses.executeAsync(null, "SELECT VAL0 FROM TEST ORDER BY VAL0"));
 
@@ -603,40 +600,47 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         }
 
         // Parse error.
-        checkError(SqlException.class, STMT_PARSE_ERR, "Failed to parse query", ses, "SELECT ID FROM");
+        checkSqlError(Sql.STMT_PARSE_ERR, "Failed to parse query", ses, "SELECT ID FROM");
 
         // Validation errors.
-        checkError(SqlException.class, STMT_VALIDATION_ERR, "Column 'VAL0' does not allow NULLs", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Column 'VAL0' does not allow NULLs", ses,
                 "INSERT INTO TEST VALUES (2, NULL)");
 
-        checkError(SqlException.class, STMT_VALIDATION_ERR, "Object 'NOT_EXISTING_TABLE' not found", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Object 'NOT_EXISTING_TABLE' not found", ses,
                 "SELECT * FROM NOT_EXISTING_TABLE");
 
-        checkError(SqlException.class, STMT_VALIDATION_ERR, "Column 'NOT_EXISTING_COLUMN' not found", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Column 'NOT_EXISTING_COLUMN' not found", ses,
                 "SELECT NOT_EXISTING_COLUMN FROM TEST");
 
-        checkError(SqlException.class, STMT_VALIDATION_ERR, "Multiple statements are not allowed", ses, "SELECT 1; SELECT 2");
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Multiple statements are not allowed", ses, "SELECT 1; SELECT 2");
 
-        checkError(SqlException.class, STMT_VALIDATION_ERR, "Table without PRIMARY KEY is not supported", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Table without PRIMARY KEY is not supported", ses,
                 "CREATE TABLE TEST2 (VAL INT)");
 
         // Execute error.
-        checkError(SqlException.class, RUNTIME_ERR, "/ by zero", ses, "SELECT 1 / ?", 0);
-        checkError(SqlException.class, RUNTIME_ERR, "negative substring length not allowed", ses, "SELECT SUBSTRING('foo', 1, -3)");
+        checkSqlError(Sql.RUNTIME_ERR, "/ by zero", ses, "SELECT 1 / ?", 0);
+        checkSqlError(Sql.RUNTIME_ERR, "/ by zero", ses, "UPDATE TEST SET val0 = val0/(val0 - ?) + " + ROW_COUNT, 0);
+        checkSqlError(Sql.RUNTIME_ERR, "negative substring length not allowed", ses, "SELECT SUBSTRING('foo', 1, -3)");
 
         // No result set error.
         {
             AsyncResultSet ars = await(ses.executeAsync(null, "CREATE TABLE TEST3 (ID INT PRIMARY KEY)"));
-            assertThrowsPublicException(() -> await(ars.fetchNextPage()),
-                    NoRowSetExpectedException.class, QUERY_NO_RESULT_SET_ERR, "Query has no result set");
+            assertThrowsSqlException(
+                    NoRowSetExpectedException.class,
+                    Sql.QUERY_NO_RESULT_SET_ERR, "Query has no result set",
+                    () -> await(ars.fetchNextPage()));
         }
 
         // Cursor closed error.
         {
             AsyncResultSet ars = await(ses.executeAsync(null, "SELECT * FROM TEST"));
             await(ars.closeAsync());
-            assertThrowsPublicException(() -> await(ars.fetchNextPage()),
-                    CursorClosedException.class, CURSOR_CLOSED_ERR, null);
+
+            assertThrowsSqlException(
+                    CursorClosedException.class,
+                    Sql.CURSOR_CLOSED_ERR,
+                    "Cursor is closed",
+                    () -> await(ars.fetchNextPage()));
         }
     }
 
@@ -651,10 +655,10 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         {
             Transaction tx = igniteTx().begin();
             try {
-                assertThrowsPublicException(() -> await(ses.executeAsync(tx, "CREATE TABLE TEST2(ID INT PRIMARY KEY, VAL0 INT)")),
-                        SqlException.class,
-                        STMT_VALIDATION_ERR,
-                        "DDL doesn't support transactions."
+                assertThrowsSqlException(
+                        Sql.STMT_VALIDATION_ERR,
+                        "DDL doesn't support transactions.",
+                        () -> await(ses.executeAsync(tx, "CREATE TABLE TEST2(ID INT PRIMARY KEY, VAL0 INT)"))
                 );
             } finally {
                 tx.rollback();
@@ -665,10 +669,10 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
             AsyncResultSet<SqlRow> res = await(ses.executeAsync(tx, "INSERT INTO TEST VALUES (?, ?)", -1, -1));
             assertEquals(1, res.affectedRows());
 
-            assertThrowsPublicException(() -> await(ses.executeAsync(tx, "CREATE TABLE TEST2(ID INT PRIMARY KEY, VAL0 INT)")),
-                    SqlException.class,
-                    STMT_VALIDATION_ERR,
-                    "DDL doesn't support transactions."
+            assertThrowsSqlException(
+                    Sql.STMT_VALIDATION_ERR,
+                    "DDL doesn't support transactions.",
+                    () -> await(ses.executeAsync(tx, "CREATE TABLE TEST2(ID INT PRIMARY KEY, VAL0 INT)"))
             );
             tx.commit();
 
@@ -694,12 +698,13 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         // Fetched page is available after cancel.
         ars0.currentPage();
 
-        SqlException sqlEx = assertThrowsPublicException(() -> await(ars0.fetchNextPage()),
-                SqlException.class, EXECUTION_CANCELLED_ERR, null);
-        assertTrue(IgniteTestUtils.hasCause(sqlEx, ExecutionCancelledException.class, null));
+        SqlException sqlEx = assertThrowsSqlException(
+                Sql.EXECUTION_CANCELLED_ERR,
+                "The query was cancelled while executing",
+                () -> await(ars0.fetchNextPage()));
 
-        assertThrowsPublicException(() -> await(ses.executeAsync(null, "SELECT ID FROM TEST")),
-                SqlException.class, SESSION_CLOSED_ERR, "Session is closed");
+        assertTrue(IgniteTestUtils.hasCause(sqlEx, QueryCancelledException.class, null));
+        assertThrowsSqlException(Sql.SESSION_CLOSED_ERR, "Session is closed", () -> await(ses.executeAsync(null, "SELECT ID FROM TEST")));
     }
 
     @Test
@@ -724,11 +729,17 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         IntStream.range(0, ROW_COUNT).forEach(i -> assertEquals(i, res.get(i).get(0)));
 
         // Check invalid query type
-        assertThrowsPublicException(() -> await(ses.executeBatchAsync(null, "SELECT * FROM TEST", args)),
-                SqlBatchException.class, STMT_VALIDATION_ERR, "Invalid SQL statement type in the batch");
+        assertThrowsSqlException(
+                SqlBatchException.class,
+                Sql.STMT_VALIDATION_ERR,
+                "Invalid SQL statement type",
+                () -> await(ses.executeBatchAsync(null, "SELECT * FROM TEST", args)));
 
-        assertThrowsPublicException(() -> await(ses.executeBatchAsync(null, "CREATE TABLE TEST1(ID INT PRIMARY KEY, VAL0 INT)", args)),
-                SqlBatchException.class, STMT_VALIDATION_ERR, "Invalid SQL statement type in the batch");
+        assertThrowsSqlException(
+                SqlBatchException.class,
+                Sql.STMT_VALIDATION_ERR,
+                "Invalid SQL statement type",
+                () -> await(ses.executeBatchAsync(null, "CREATE TABLE TEST1(ID INT PRIMARY KEY, VAL0 INT)", args)));
     }
 
     @Test
@@ -750,11 +761,11 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
             }
         }
 
-        SqlBatchException ex = assertThrowsPublicException(
-                () -> await(ses.executeBatchAsync(null, "INSERT INTO TEST VALUES (?, ?)", args)),
+        SqlBatchException ex = assertThrowsSqlException(
                 SqlBatchException.class,
-                CONSTRAINT_VIOLATION_ERR,
-                null
+                Sql.CONSTRAINT_VIOLATION_ERR,
+                "PK unique constraint is violated",
+                () -> await(ses.executeBatchAsync(null, "INSERT INTO TEST VALUES (?, ?)", args))
         );
 
         assertEquals(err, ex.updateCounters().length);
@@ -826,19 +837,19 @@ public class ItSqlAsynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(expectedApplied, ses, sql, null);
     }
 
-    private static <T extends IgniteException> T checkError(Class<T> expCls, String msg, Session ses, String sql, Object... args) {
-        return checkError(expCls, null, msg, ses, sql, args);
+    private static <T extends IgniteException> T checkError(Class<T> expCls, Integer code, String msg, Session ses, String sql,
+            Object... args) {
+        return assertThrowsPublicException(() -> await(ses.executeAsync(null, sql, args)), expCls, code, msg);
     }
 
-    private static <T extends IgniteException> T checkError(
-            Class<T> expCls,
-            @Nullable Integer code,
-            @Nullable String msg,
+    private static SqlException checkSqlError(
+            int code,
+            String msg,
             Session ses,
             String sql,
             Object... args
     ) {
-        return assertThrowsPublicException(() -> await(ses.executeAsync(null, sql, args)), expCls, code, msg);
+        return assertThrowsSqlException(code, msg, () -> await(ses.executeAsync(null, sql, args)));
     }
 
     protected static void checkDml(int expectedAffectedRows, Session ses, String sql, Transaction tx, Object... args) {
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java
index 286645a65c..0c7b32fdf4 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java
@@ -121,14 +121,20 @@ public class ItSqlClientMetricsTest extends ClusterPerClassIntegrationTest {
     public void testErroneousFlow() throws Exception {
         Session session = sql.createSession();
 
-        assertThrowsSqlException(Sql.STMT_PARSE_ERR, () -> session.execute(null, "SELECT * ODINfrom " + DEFAULT_TABLE_NAME));
+        assertThrowsSqlException(
+                Sql.STMT_PARSE_ERR,
+                "Failed to parse query",
+                () -> session.execute(null, "SELECT * ODINfrom " + DEFAULT_TABLE_NAME));
         assertMetricValue(clientMetricSet, SqlClientMetricSource.METRIC_OPEN_CURSORS, 0);
 
         assertInternalSqlException("Column 'A' not found in any table", () -> session.execute(null, "SELECT a from " + DEFAULT_TABLE_NAME));
         assertMetricValue(clientMetricSet, SqlClientMetricSource.METRIC_OPEN_CURSORS, 0);
 
         session.close();
-        assertThrowsSqlException(Sql.SESSION_CLOSED_ERR, () -> session.execute(null, "SELECT * from " + DEFAULT_TABLE_NAME));
+        assertThrowsSqlException(
+                Sql.SESSION_CLOSED_ERR,
+                "Session is closed",
+                () -> session.execute(null, "SELECT * from " + DEFAULT_TABLE_NAME));
         assertMetricValue(clientMetricSet, SqlClientMetricSource.METRIC_OPEN_CURSORS, 0);
     }
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java
index 1a4bcf0263..e04c4250ba 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlSynchronousApiTest.java
@@ -55,8 +55,8 @@ import org.apache.ignite.sql.SqlRow;
 import org.apache.ignite.table.Table;
 import org.apache.ignite.tx.Transaction;
 import org.hamcrest.MatcherAssert;
-import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
@@ -82,12 +82,12 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)");
         checkError(
                 TableAlreadyExistsException.class,
+                ErrorGroups.Table.TABLE_ALREADY_EXISTS_ERR,
                 "Table already exists [name=\"PUBLIC\".\"TEST\"]",
                 ses,
                 "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"
         );
-        checkError(
-                SqlException.class,
+        checkSqlError(
                 ErrorGroups.Table.TABLE_DEFINITION_ERR,
                 "Can't create table with duplicate columns: ID, VAL, VAL",
                 ses,
@@ -99,6 +99,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "ALTER TABLE TEST ADD COLUMN VAL1 VARCHAR");
         checkError(
                 TableNotFoundException.class,
+                ErrorGroups.Table.TABLE_NOT_FOUND_ERR,
                 "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]",
                 ses,
                 "ALTER TABLE NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR"
@@ -106,6 +107,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR");
         checkError(
                 ColumnAlreadyExistsException.class,
+                ErrorGroups.Table.COLUMN_ALREADY_EXISTS_ERR,
                 "Column already exists [name=\"VAL1\"]",
                 ses,
                 "ALTER TABLE TEST ADD COLUMN VAL1 INT"
@@ -115,6 +117,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "CREATE INDEX TEST_IDX ON TEST(VAL0)");
         checkError(
                 IndexAlreadyExistsException.class,
+                Index.INDEX_ALREADY_EXISTS_ERR,
                 "Index already exists [name=\"PUBLIC\".\"TEST_IDX\"]",
                 ses,
                 "CREATE INDEX TEST_IDX ON TEST(VAL1)"
@@ -128,24 +131,21 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "CREATE INDEX TEST_IDX1 ON TEST(VAL0)");
         checkDdl(true, ses, "CREATE INDEX TEST_IDX2 ON TEST(VAL0)");
         checkDdl(true, ses, "CREATE INDEX TEST_IDX3 ON TEST(ID, VAL0, VAL1)");
-        checkError(
-                SqlException.class,
+        checkSqlError(
                 Index.INVALID_INDEX_DEFINITION_ERR,
                 "Can't create index on duplicate columns: VAL0, VAL0",
                 ses,
                 "CREATE INDEX TEST_IDX4 ON TEST(VAL0, VAL0)"
         );
 
-        checkError(
-                SqlException.class,
+        checkSqlError(
                 Sql.STMT_VALIDATION_ERR,
                 "Can`t delete column(s). Column VAL1 is used by indexes [TEST_IDX3].",
                 ses,
                 "ALTER TABLE TEST DROP COLUMN val1"
         );
 
-        SqlException ex = checkError(
-                SqlException.class,
+        SqlException ex = checkSqlError(
                 Sql.STMT_VALIDATION_ERR,
                 "Can`t delete column(s).",
                 ses,
@@ -159,8 +159,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         assertTrue(msg.contains("TEST_IDX1") && msg.contains("TEST_IDX2") && msg.contains("TEST_IDX3"), explainMsg);
         assertTrue(msg.contains("Column VAL1 is used by indexes [TEST_IDX3]"), explainMsg);
 
-        checkError(
-                SqlException.class,
+        checkSqlError(
                 Sql.STMT_VALIDATION_ERR,
                 "Can`t delete column, belongs to primary key: [name=ID]",
                 ses,
@@ -175,6 +174,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "ALTER TABLE TEST DROP COLUMN VAL1");
         checkError(
                 TableNotFoundException.class,
+                ErrorGroups.Table.TABLE_NOT_FOUND_ERR,
                 "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]",
                 ses,
                 "ALTER TABLE NOT_EXISTS_TABLE DROP COLUMN VAL1"
@@ -182,6 +182,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE DROP COLUMN VAL1");
         checkError(
                 ColumnNotFoundException.class,
+                ErrorGroups.Table.COLUMN_NOT_FOUND_ERR,
                 "Column does not exist [tableName=\"PUBLIC\".\"TEST\", columnName=\"VAL1\"]",
                 ses,
                 "ALTER TABLE TEST DROP COLUMN VAL1"
@@ -193,6 +194,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         checkDdl(true, ses, "DROP TABLE TEST");
         checkError(
                 TableNotFoundException.class,
+                ErrorGroups.Table.TABLE_NOT_FOUND_ERR,
                 "The table does not exist [name=\"PUBLIC\".\"TEST\"]",
                 ses,
                 "DROP TABLE TEST"
@@ -202,6 +204,7 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
 
         checkError(
                 IndexNotFoundException.class,
+                Index.INDEX_NOT_FOUND_ERR,
                 "Index does not exist [name=\"PUBLIC\".\"TEST_IDX\"]", ses,
                 "DROP INDEX TEST_IDX"
         );
@@ -269,31 +272,32 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         }
 
         // Parse error.
-        checkError(SqlException.class, Sql.STMT_PARSE_ERR, "Failed to parse query", ses, "SELECT ID FROM");
+        checkSqlError(Sql.STMT_PARSE_ERR, "Failed to parse query", ses, "SELECT ID FROM");
 
         // Validation errors.
-        checkError(SqlException.class, Sql.STMT_VALIDATION_ERR, "Column 'VAL0' does not allow NULLs", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Column 'VAL0' does not allow NULLs", ses,
                 "INSERT INTO TEST VALUES (2, NULL)");
 
-        checkError(SqlException.class, Sql.STMT_VALIDATION_ERR, "Object 'NOT_EXISTING_TABLE' not found", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Object 'NOT_EXISTING_TABLE' not found", ses,
                 "SELECT * FROM NOT_EXISTING_TABLE");
 
-        checkError(SqlException.class, Sql.STMT_VALIDATION_ERR, "Column 'NOT_EXISTING_COLUMN' not found", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Column 'NOT_EXISTING_COLUMN' not found", ses,
                 "SELECT NOT_EXISTING_COLUMN FROM TEST");
 
-        checkError(SqlException.class, Sql.STMT_VALIDATION_ERR, "Multiple statements are not allowed", ses, "SELECT 1; SELECT 2");
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Multiple statements are not allowed", ses, "SELECT 1; SELECT 2");
 
-        checkError(SqlException.class, Sql.STMT_VALIDATION_ERR, "Table without PRIMARY KEY is not supported", ses,
+        checkSqlError(Sql.STMT_VALIDATION_ERR, "Table without PRIMARY KEY is not supported", ses,
                 "CREATE TABLE TEST2 (VAL INT)");
 
         // Execute error.
-        checkError(SqlException.class, Sql.RUNTIME_ERR, "/ by zero", ses, "SELECT 1 / ?", 0);
-        checkError(SqlException.class, Sql.RUNTIME_ERR, "negative substring length not allowed", ses, "SELECT SUBSTRING('foo', 1, -3)");
+        checkSqlError(Sql.RUNTIME_ERR, "/ by zero", ses, "SELECT 1 / ?", 0);
+        checkSqlError(Sql.RUNTIME_ERR, "/ by zero", ses, "UPDATE TEST SET val0 = val0/(val0 - ?) + " + ROW_COUNT, 0);
+        checkSqlError(Sql.RUNTIME_ERR, "negative substring length not allowed", ses, "SELECT SUBSTRING('foo', 1, -3)");
 
         // No result set error.
         {
             ResultSet rs = ses.execute(null, "CREATE TABLE TEST3 (ID INT PRIMARY KEY)");
-            assertThrowsPublicException(rs::next, NoRowSetExpectedException.class, Sql.QUERY_NO_RESULT_SET_ERR, "Query has no result set");
+            assertThrowsSqlException(NoRowSetExpectedException.class, Sql.QUERY_NO_RESULT_SET_ERR, "Query has no result set", rs::next);
         }
 
         // Cursor closed error.
@@ -301,8 +305,8 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
             ResultSet rs = ses.execute(null, "SELECT * FROM TEST");
             Thread.sleep(300); // ResultSetImpl fetches next page in background, wait to it to complete to avoid flakiness.
             rs.close();
-            assertThrowsPublicException(() -> rs.forEachRemaining(Object::hashCode),
-                    CursorClosedException.class, Sql.CURSOR_CLOSED_ERR, null);
+            assertThrowsSqlException(CursorClosedException.class, Sql.CURSOR_CLOSED_ERR, "Cursor is closed",
+                    () -> rs.forEachRemaining(Object::hashCode));
         }
     }
 
@@ -344,6 +348,34 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         assertEquals(0, ((IgniteImpl) CLUSTER_NODES.get(0)).txManager().pending());
     }
 
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-20342")
+    @Test
+    public void runtimeErrorInTransaction() {
+        int size = 100;
+        sql("CREATE TABLE tst(id INT PRIMARY KEY, val INT)");
+        for (int i = 0; i < size; i++) {
+            sql("INSERT INTO tst VALUES (?,?)", i, i);
+        }
+
+        Session ses = igniteSql().createSession();
+        Transaction tx = igniteTx().begin();
+
+        try {
+            String sqlText = "UPDATE tst SET val = val/(val - ?) + " + size;
+
+            for (int i = 0; i < size; i++) {
+                int param = i;
+                assertThrowsSqlException(
+                        Sql.RUNTIME_ERR,
+                        "/ by zero",
+                        () -> ses.execute(tx, sqlText, param));
+
+            }
+        } finally {
+            tx.commit();
+        }
+    }
+
     @Test
     public void batch() {
         sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)");
@@ -368,14 +400,14 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         // Check invalid query type
         SqlException ex = assertThrowsSqlException(
                 Sql.STMT_VALIDATION_ERR,
-                "Invalid SQL statement type in the batch",
+                "Invalid SQL statement type",
                 () -> ses.executeBatch(null, "SELECT * FROM TEST", args)
         );
         MatcherAssert.assertThat(ex, instanceOf(SqlBatchException.class));
 
         ex = assertThrowsSqlException(
                 Sql.STMT_VALIDATION_ERR,
-                "Invalid SQL statement type in the batch",
+                "Invalid SQL statement type",
                 () -> ses.executeBatch(null, "CREATE TABLE TEST1(ID INT PRIMARY KEY, VAL0 INT)", args)
         );
         MatcherAssert.assertThat(ex, instanceOf(SqlBatchException.class));
@@ -460,19 +492,19 @@ public class ItSqlSynchronousApiTest extends ClusterPerClassIntegrationTest {
         res.close();
     }
 
-    private static <T extends IgniteException> T checkError(Class<T> expCls, String msg, Session ses, String sql, Object... args) {
-        return checkError(expCls, null, msg, ses, sql, args);
+    private static <T extends IgniteException> T checkError(Class<T> expCls, Integer code, String msg, Session ses, String sql,
+            Object... args) {
+        return assertThrowsPublicException(() -> ses.execute(null, sql, args), expCls, code, msg);
     }
 
-    private static <T extends IgniteException> T checkError(
-            Class<T> expCls,
-            @Nullable Integer code,
-            @Nullable String msg,
+    private static SqlException checkSqlError(
+            int code,
+            String msg,
             Session ses,
             String sql,
             Object... args
     ) {
-        return assertThrowsPublicException(() -> ses.execute(null, sql, args), expCls, code, msg);
+        return assertThrowsSqlException(code, msg, () -> ses.execute(null, sql, args));
     }
 
     static void checkDml(int expectedAffectedRows, Session ses, String sql, Object... args) {
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java
index fbea36d88e..ac90aa2ad4 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java
@@ -89,6 +89,7 @@ public class ItCreateTableDdlTest extends ClusterPerClassIntegrationTest {
     public void tableWithInvalidColumns() {
         assertThrowsSqlException(
                 Sql.STMT_PARSE_ERR,
+                "Failed to parse query: Encountered \")\"",
                 () -> sql("CREATE TABLE T0()")
         );
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java
index 949b48d8a1..f8fdff77af 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java
@@ -126,7 +126,10 @@ public class ItDataTypesTest extends ClusterPerClassIntegrationTest {
 
         assertEquals(Set.of(101), rows.stream().map(r -> r.get(0)).collect(Collectors.toSet()));
 
-        assertThrowsSqlException(Sql.STMT_VALIDATION_ERR, () -> sql("INSERT INTO tbl(c1, c2) VALUES (2, NULL)"));
+        assertThrowsSqlException(
+                Sql.STMT_VALIDATION_ERR,
+                "Column 'C2' does not allow NULLs",
+                () -> sql("INSERT INTO tbl(c1, c2) VALUES (2, NULL)"));
     }
 
     /**
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
index f207dcef06..df7c2dc76b 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
@@ -120,6 +120,7 @@ public class ItDmlTest extends ClusterPerClassIntegrationTest {
 
         var ex = assertThrowsSqlException(
                 Sql.CONSTRAINT_VIOLATION_ERR,
+                "PK unique constraint is violated",
                 () -> sql("INSERT INTO test VALUES (0, 0), (1, 1), (2, 2)")
         );
 
@@ -170,6 +171,7 @@ public class ItDmlTest extends ClusterPerClassIntegrationTest {
 
         SqlException ex = assertThrowsSqlException(
                 Sql.CONSTRAINT_VIOLATION_ERR,
+                "PK unique constraint is violated",
                 () -> sql(insertStatement)
         );
 
@@ -397,7 +399,10 @@ public class ItDmlTest extends ClusterPerClassIntegrationTest {
 
         sql("CREATE TABLE test2 (k int PRIMARY KEY, a int, b int)");
 
-        SqlException ex = assertThrowsSqlException(Sql.CONSTRAINT_VIOLATION_ERR, () -> sql(
+        SqlException ex = assertThrowsSqlException(
+                Sql.CONSTRAINT_VIOLATION_ERR,
+                "PK unique constraint is violated",
+                () -> sql(
                         "MERGE INTO test2 USING test1 ON test1.a = test2.a "
                                 + "WHEN MATCHED THEN UPDATE SET b = test1.b + 1 "
                                 + "WHEN NOT MATCHED THEN INSERT (k, a, b) VALUES (0, a, b)"));
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java
index a35303bb06..17cd8db9c4 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java
@@ -157,8 +157,13 @@ public class ItDynamicParameterTest extends ClusterPerClassIntegrationTest {
      */
     @Test
     public void testWithDifferentParametersTypesMismatch() {
-        assertThrowsSqlException(Sql.STMT_VALIDATION_ERR, () -> assertQuery("SELECT COALESCE(12.2, ?)").withParams("b").check());
-        assertThrowsSqlException(Sql.STMT_VALIDATION_ERR, () -> assertQuery("SELECT COALESCE(?, ?)").withParams(12.2, "b").check());
+        assertThrowsSqlException(
+                Sql.STMT_VALIDATION_ERR,
+                "Illegal mixing of types in CASE or COALESCE statement",
+                () -> assertQuery("SELECT COALESCE(12.2, ?)").withParams("b").check());
+        assertThrowsSqlException(Sql.STMT_VALIDATION_ERR,
+                "Illegal mixing of types in CASE or COALESCE statement",
+                () -> assertQuery("SELECT COALESCE(?, ?)").withParams(12.2, "b").check());
     }
 
     @Test
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java
index eecebe80d1..11a7a1eb14 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java
@@ -61,12 +61,15 @@ public class ItLimitOffsetTest extends ClusterPerClassIntegrationTest {
                 () -> session.execute(null, "SELECT * FROM test LIMIT " + bigInt));
 
         assertThrowsSqlException(Sql.STMT_PARSE_ERR,
+                "Failed to parse query: Encountered \"-\"",
                 () -> session.execute(null, "SELECT * FROM test OFFSET -1 ROWS FETCH FIRST -1 ROWS ONLY"));
 
         assertThrowsSqlException(Sql.STMT_PARSE_ERR,
+                "Failed to parse query: Encountered \"-\"",
                 () -> session.execute(null, "SELECT * FROM test OFFSET -1 ROWS"));
 
         assertThrowsSqlException(Sql.STMT_PARSE_ERR,
+                "Failed to parse query: Encountered \"+\"",
                 () -> session.execute(null, "SELECT * FROM test OFFSET 2+1 ROWS"));
 
         // Check with parameters
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryCancelledException.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryCancelledException.java
index f3fa66370a..427af0a840 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryCancelledException.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryCancelledException.java
@@ -35,4 +35,13 @@ public class QueryCancelledException extends IgniteException {
     public QueryCancelledException() {
         super(EXECUTION_CANCELLED_ERR, ERR_MSG);
     }
+
+    /**
+     * Constructor.
+     *
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public QueryCancelledException(Throwable cause) {
+        super(EXECUTION_CANCELLED_ERR, ERR_MSG, cause);
+    }
 }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryContext.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryContext.java
index fec3794cfd..0780c21307 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryContext.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryContext.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.sql.engine;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Set;
 import org.apache.calcite.plan.Context;
@@ -44,7 +45,8 @@ public class QueryContext implements Context {
      */
     private QueryContext(Set<SqlQueryType> allowedQueries, Object[] params) {
         this.params = params;
-        this.allowedQueries = allowedQueries;
+        //use EnumSet to have the same order always
+        this.allowedQueries = EnumSet.copyOf(allowedQueries);
     }
 
     /** Returns a set of {@link SqlQueryType allowed query types}. **/
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java
index 8c7bc44509..1528871187 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java
@@ -62,7 +62,6 @@ import org.apache.ignite.internal.sql.engine.exec.LifecycleAware;
 import org.apache.ignite.internal.sql.engine.exec.MailboxRegistryImpl;
 import org.apache.ignite.internal.sql.engine.exec.QueryTaskExecutor;
 import org.apache.ignite.internal.sql.engine.exec.QueryTaskExecutorImpl;
-import org.apache.ignite.internal.sql.engine.exec.QueryValidationException;
 import org.apache.ignite.internal.sql.engine.exec.SqlRowHandler;
 import org.apache.ignite.internal.sql.engine.exec.ddl.DdlCommandHandlerWrapper;
 import org.apache.ignite.internal.sql.engine.message.MessageServiceImpl;
@@ -500,7 +499,7 @@ public class SqlQueryProcessor implements QueryProcessor {
                 queryType,
                 plan.metadata(),
                 txWrapper,
-                new AsyncCursor<List<Object>>() {
+                new AsyncCursor<>() {
                     @Override
                     public CompletableFuture<BatchedResult<List<Object>>> requestNextAsync(int rows) {
                         session.touch();
@@ -669,9 +668,9 @@ public class SqlQueryProcessor implements QueryProcessor {
         SqlQueryType queryType = parsedResult.queryType();
 
         if (!allowedTypes.contains(queryType)) {
-            String message = format("Invalid SQL statement type in the batch. Expected {} but got {}.", allowedTypes, queryType);
+            String message = format("Invalid SQL statement type. Expected {} but got {}", allowedTypes, queryType);
 
-            throw new QueryValidationException(message);
+            throw new SqlException(STMT_VALIDATION_ERR, message);
         }
 
         if (parsedResult.dynamicParamsCount() != params.length) {
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExchangeServiceImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExchangeServiceImpl.java
index b601cb329b..3cdca59192 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExchangeServiceImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExchangeServiceImpl.java
@@ -27,6 +27,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.function.Consumer;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.sql.engine.exec.rel.Inbox;
 import org.apache.ignite.internal.sql.engine.exec.rel.Outbox;
 import org.apache.ignite.internal.sql.engine.message.MessageService;
@@ -115,7 +116,7 @@ public class ExchangeServiceImpl implements ExchangeService {
             traceableErr = error = new IgniteInternalException(INTERNAL_ERR, error);
         }
 
-        if (!(traceableErr instanceof ExecutionCancelledException)) {
+        if (!(traceableErr instanceof QueryCancelledException)) {
             LOG.info(format("Failed to execute query fragment: queryId={}, fragmentId={}", queryId, fragmentId), error);
         } else if (LOG.isDebugEnabled()) {
             LOG.debug(format("Failed to execute query fragment: queryId={}, fragmentId={}", queryId, fragmentId), error);
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionCancelledException.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionCancelledException.java
deleted file mode 100644
index eabb7128bc..0000000000
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionCancelledException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.sql.engine.exec;
-
-import static org.apache.ignite.lang.ErrorGroups.Sql.EXECUTION_CANCELLED_ERR;
-
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-
-/**
- * ExecutionCancelledException.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
- */
-public class ExecutionCancelledException extends IgniteInternalCheckedException {
-    /** Serial version UID. */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Constructs a new exception with null as its detail message.
-     */
-    public ExecutionCancelledException() {
-        super(EXECUTION_CANCELLED_ERR);
-    }
-}
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java
index d424080ffd..af98fae7a3 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.sql.engine.exec;
 import static org.apache.ignite.internal.sql.engine.externalize.RelJsonReader.fromJson;
 import static org.apache.ignite.internal.sql.engine.util.Commons.FRAMEWORK_CONFIG;
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
-import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
 import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
 import static org.apache.ignite.lang.IgniteStringFormatter.format;
 
@@ -33,7 +32,6 @@ import java.util.Map;
 import java.util.Queue;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Executor;
@@ -48,6 +46,7 @@ import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.replicator.TablePartitionId;
 import org.apache.ignite.internal.sql.engine.NodeLeftException;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.sql.engine.SqlQueryType;
 import org.apache.ignite.internal.sql.engine.exec.ddl.DdlCommandHandler;
 import org.apache.ignite.internal.sql.engine.exec.rel.AbstractNode;
@@ -307,9 +306,7 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
     }
 
     private static RuntimeException convertDdlException(Throwable e) {
-        if (e instanceof CompletionException) {
-            e = e.getCause();
-        }
+        e = ExceptionUtils.unwrapCause(e);
 
         if (e instanceof ConfigurationChangeException) {
             assert e.getCause() != null;
@@ -374,6 +371,10 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
                     msg.code(),
                     msg.message()
             );
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Query remote fragment execution failed [nodeName={}, queryId={}, fragmentId={}, originalMessage={}]",
+                        nodeName, e.queryId(), e.fragmentId(), e.getMessage());
+            }
 
             dqm.onError(e);
         }
@@ -866,7 +867,7 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
 
                         var finalStepFut = cancelResult.whenComplete((r, e) -> {
                             if (e != null) {
-                                Throwable ex = unwrapCause(e);
+                                Throwable ex = ExceptionUtils.unwrapCause(e);
 
                                 LOG.warn("Fragment closing processed with errors: [queryId={}]", ex, ctx.queryId());
                             }
@@ -892,7 +893,7 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
         }
 
         private CompletableFuture<Void> closeLocalFragments() {
-            ExecutionCancelledException ex = new ExecutionCancelledException();
+            QueryCancelledException ex = new QueryCancelledException();
 
             List<CompletableFuture<?>> localFragmentCompletions = new ArrayList<>();
             for (AbstractNode<?> node : localFragments) {
@@ -950,13 +951,13 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
         /**
          * Synchronously closes the tree's execution iterator.
          *
-         * @param cancel Forces execution to terminate with {@link ExecutionCancelledException}.
+         * @param cancel Forces execution to terminate with {@link QueryCancelledException}.
          * @return Completable future that should run asynchronously.
          */
         private CompletableFuture<Void> closeExecNode(boolean cancel) {
             CompletableFuture<Void> start = new CompletableFuture<>();
 
-            if (!root.completeExceptionally(new ExecutionCancelledException()) && !root.isCompletedExceptionally()) {
+            if (!root.completeExceptionally(new QueryCancelledException()) && !root.isCompletedExceptionally()) {
                 AsyncRootNode<RowT, List<Object>> node = root.getNow(null);
 
                 if (!cancel) {
@@ -965,7 +966,7 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
                     return start.thenCompose(v -> closeFut);
                 }
 
-                node.onError(new ExecutionCancelledException());
+                node.onError(new QueryCancelledException());
             }
 
             return start;
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/QueryValidationException.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/QueryValidationException.java
deleted file mode 100644
index 7683d6f677..0000000000
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/QueryValidationException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.sql.engine.exec;
-
-import static org.apache.ignite.lang.ErrorGroups.Sql.STMT_VALIDATION_ERR;
-
-import org.apache.ignite.lang.IgniteInternalException;
-
-/**
- * QueryValidationException is used during query validation.
- *
- * <p>The exception is used when the expected query type does not match the actual query type obtained after parsing a sql string.
- */
-public class QueryValidationException extends IgniteInternalException {
-    /**
-     * Creates a new exception with the given error message.
-     *
-     * @param msg Error message.
-     */
-    public QueryValidationException(String msg) {
-        super(STMT_VALIDATION_ERR, msg);
-    }
-}
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/ExpressionFactoryImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/ExpressionFactoryImpl.java
index 700e102f68..2a17360ec5 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/ExpressionFactoryImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/ExpressionFactoryImpl.java
@@ -535,14 +535,12 @@ public class ExpressionFactoryImpl<RowT> implements ExpressionFactory<RowT> {
 
         assert nodes.size() == projects.size();
 
-        BlockBuilder tryCatchBlock = new BlockBuilder();
-
         for (int i = 0; i < projects.size(); i++) {
             Expression val = unspecifiedValues.get(i)
                     ? Expressions.field(null, ExpressionFactoryImpl.class, "UNSPECIFIED_VALUE_PLACEHOLDER")
                     : projects.get(i);
 
-            tryCatchBlock.add(
+            builder.add(
                         Expressions.statement(
                                 Expressions.call(hnd,
                                         IgniteMethod.ROW_HANDLER_SET.method(),
@@ -551,8 +549,9 @@ public class ExpressionFactoryImpl<RowT> implements ExpressionFactory<RowT> {
 
         ParameterExpression ex = Expressions.parameter(0, Exception.class, "e");
         Expression sqlException = Expressions.new_(SqlException.class, Expressions.constant(Sql.RUNTIME_ERR), ex);
+        BlockBuilder tryCatchBlock = new BlockBuilder();
 
-        builder.add(Expressions.tryCatch(tryCatchBlock.toBlock(), Expressions.catch_(ex, Expressions.throw_(sqlException))));
+        tryCatchBlock.add(Expressions.tryCatch(builder.toBlock(), Expressions.catch_(ex, Expressions.throw_(sqlException))));
 
         String methodName = biInParams ? IgniteMethod.BI_SCALAR_EXECUTE.method().getName() :
                 IgniteMethod.SCALAR_EXECUTE.method().getName();
@@ -562,7 +561,7 @@ public class ExpressionFactoryImpl<RowT> implements ExpressionFactory<RowT> {
 
         MethodDeclaration decl = Expressions.methodDecl(
                 Modifier.PUBLIC, void.class, methodName,
-                params, builder.toBlock());
+                params, tryCatchBlock.toBlock());
 
         Class<? extends Scalar> clazz = biInParams ? BiScalar.class : SingleScalar.class;
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractNode.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractNode.java
index f4c8f09db0..100e6a3e34 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractNode.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractNode.java
@@ -23,7 +23,7 @@ import static org.apache.ignite.lang.IgniteStringFormatter.format;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.List;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
 import org.apache.ignite.internal.sql.engine.util.Commons;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -156,7 +156,7 @@ public abstract class AbstractNode<RowT> implements Node<RowT> {
 
     protected void checkState() throws Exception {
         if (context().isCancelled() || Thread.interrupted()) {
-            throw new ExecutionCancelledException();
+            throw new QueryCancelledException();
         }
         if (!IgniteUtils.assertionsEnabled()) {
             return;
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AsyncRootNode.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AsyncRootNode.java
index aaddc32f2d..d21c541c79 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AsyncRootNode.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/AsyncRootNode.java
@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.util.AsyncCursor;
 import org.apache.ignite.sql.CursorClosedException;
 
@@ -149,7 +149,7 @@ public class AsyncRootNode<InRowT, OutRowT> implements Downstream<InRowT>, Async
                     Throwable th = ex.get();
 
                     if (th == null) {
-                        th = new ExecutionCancelledException();
+                        th = new QueryCancelledException();
                     }
 
                     Throwable th0 = th;
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/RootNode.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/RootNode.java
index 6f90dedaef..59e7d8c37f 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/RootNode.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/RootNode.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.sql.engine.exec.rel;
 
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
-import static org.apache.ignite.lang.ErrorGroups.Sql.EXECUTION_CANCELLED_ERR;
 
 import java.util.ArrayDeque;
 import java.util.Deque;
@@ -29,11 +28,10 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Function;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
 import org.apache.ignite.internal.sql.engine.util.Commons;
 import org.apache.ignite.internal.util.ExceptionUtils;
-import org.apache.ignite.sql.SqlException;
 
 /**
  * Client iterator.
@@ -110,7 +108,7 @@ public class RootNode<RowT> extends AbstractNode<RowT> implements SingleNode<Row
         lock.lock();
         try {
             if (waiting != -1 || !outBuff.isEmpty()) {
-                ex.compareAndSet(null, new ExecutionCancelledException());
+                ex.compareAndSet(null, new QueryCancelledException());
             }
 
             closed = true; // an exception has to be set first to get right check order
@@ -267,7 +265,7 @@ public class RootNode<RowT> extends AbstractNode<RowT> implements SingleNode<Row
                 cond.await();
             }
         } catch (InterruptedException e) {
-            throw new SqlException(EXECUTION_CANCELLED_ERR, e);
+            throw new QueryCancelledException(e);
         } finally {
             lock.unlock();
         }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/metadata/RemoteFragmentExecutionException.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/metadata/RemoteFragmentExecutionException.java
index 8314e0307e..f14de64544 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/metadata/RemoteFragmentExecutionException.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/metadata/RemoteFragmentExecutionException.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.sql.engine.metadata;
 
-import static org.apache.ignite.lang.IgniteStringFormatter.format;
-
 import java.util.UUID;
 import org.apache.ignite.lang.RemoteException;
 import org.jetbrains.annotations.Nullable;
@@ -54,10 +52,7 @@ public class RemoteFragmentExecutionException extends RemoteException {
             int code,
             @Nullable String message
     ) {
-        super(traceId, code,
-                format("Query remote fragment execution failed: nodeName={}, queryId={}, fragmentId={}, originalMessage={}",
-                        nodeName, queryId, fragmentId, message)
-        );
+        super(traceId, code, message);
 
         this.nodeName = nodeName;
         this.queryId = queryId;
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PrepareServiceImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PrepareServiceImpl.java
index 12ccee6fdd..6f0062424e 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PrepareServiceImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PrepareServiceImpl.java
@@ -188,7 +188,9 @@ public class PrepareServiceImpl implements PrepareService, SchemaUpdateListener
         return result.exceptionally(ex -> {
                     Throwable th = ExceptionUtils.unwrapCause(ex);
                     if (planningContext.timeouted() && th instanceof RelOptPlanner.CannotPlanException) {
-                        throw new SqlException(PLANNING_TIMEOUT_ERR);
+                        throw new SqlException(
+                                PLANNING_TIMEOUT_ERR,
+                                "Planning of a query aborted due to planner timeout threshold is reached");
                     }
 
                     throw new CompletionException(IgniteExceptionMapperUtil.mapToPublicException(th));
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/SqlExceptionMapperProvider.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/SqlExceptionMapperProvider.java
index 6d1a953050..d328d5307f 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/SqlExceptionMapperProvider.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/SqlExceptionMapperProvider.java
@@ -27,8 +27,7 @@ import java.util.Collection;
 import java.util.List;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.ignite.internal.catalog.CatalogValidationException;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionCancelledException;
-import org.apache.ignite.internal.sql.engine.exec.QueryValidationException;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.sql.engine.metadata.RemoteFragmentExecutionException;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteExceptionMapper;
@@ -44,9 +43,7 @@ public class SqlExceptionMapperProvider implements IgniteExceptionMappersProvide
     public Collection<IgniteExceptionMapper<?, ?>> mappers() {
         List<IgniteExceptionMapper<?, ?>> mappers = new ArrayList<>();
 
-        mappers.add(unchecked(ExecutionCancelledException.class, err -> new SqlException(err.traceId(), err.code(), err)));
-
-        mappers.add(unchecked(QueryValidationException.class, err -> new SqlException(err.traceId(), err.code(), err)));
+        mappers.add(unchecked(QueryCancelledException.class, err -> new SqlException(err.traceId(), err.code(), err.getMessage(), err)));
 
         mappers.add(unchecked(RemoteFragmentExecutionException.class, err -> {
             if (err.groupCode() == SQL_ERR_GROUP.groupCode()) {
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java
index f3c9d0e659..e24091aa9e 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/QueryTransactionWrapperSelfTest.java
@@ -53,6 +53,7 @@ public class QueryTransactionWrapperSelfTest extends BaseIgniteAbstractTest {
     public void throwsExceptionForDdlWithExternalTransaction() {
         //noinspection ThrowableNotThrown
         assertThrowsSqlException(ErrorGroups.Sql.STMT_VALIDATION_ERR,
+                "DDL doesn't support transactions",
                 () -> wrapTxOrStartImplicit(SqlQueryType.DDL, transactions, new NoOpTransaction("test")));
         verifyNoInteractions(transactions);
     }
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java
index 98aab74fb0..9330a51782 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java
@@ -60,7 +60,6 @@ import org.apache.ignite.internal.schema.SchemaManager;
 import org.apache.ignite.internal.schema.SchemaRegistry;
 import org.apache.ignite.internal.schema.registry.SchemaRegistryImpl;
 import org.apache.ignite.internal.schema.row.RowAssembler;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionCancelledException;
 import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
 import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestHashIndex;
 import org.apache.ignite.internal.sql.engine.property.PropertiesHelper;
@@ -282,7 +281,7 @@ public class StopCalciteModuleTest extends BaseIgniteAbstractTest {
         await(request.exceptionally(t -> {
             assertInstanceOf(CompletionException.class, t);
             assertInstanceOf(IgniteException.class, t.getCause());
-            assertInstanceOf(ExecutionCancelledException.class, t.getCause().getCause());
+            assertInstanceOf(QueryCancelledException.class, t.getCause().getCause());
 
             return null;
         }));
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java
index a0fcc755c4..836c1093c9 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.metrics.MetricManager;
 import org.apache.ignite.internal.schema.NativeType;
 import org.apache.ignite.internal.schema.NativeTypes;
 import org.apache.ignite.internal.sql.engine.QueryCancel;
+import org.apache.ignite.internal.sql.engine.QueryCancelledException;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionServiceImplTest.TestCluster.TestNode;
 import org.apache.ignite.internal.sql.engine.exec.ddl.DdlCommandHandler;
 import org.apache.ignite.internal.sql.engine.exec.rel.AbstractNode;
@@ -218,7 +219,7 @@ public class ExecutionServiceImplTest extends BaseIgniteAbstractTest {
 
         await(batchFut.exceptionally(ex -> {
             assertInstanceOf(CompletionException.class, ex);
-            assertInstanceOf(ExecutionCancelledException.class, ex.getCause());
+            assertInstanceOf(QueryCancelledException.class, ex.getCause());
 
             return null;
         }));
@@ -258,7 +259,7 @@ public class ExecutionServiceImplTest extends BaseIgniteAbstractTest {
 
         await(batchFut.exceptionally(ex -> {
             assertInstanceOf(CompletionException.class, ex);
-            assertInstanceOf(ExecutionCancelledException.class, ex.getCause());
+            assertInstanceOf(QueryCancelledException.class, ex.getCause());
 
             return null;
         }));
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTimeoutTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTimeoutTest.java
index 56d0ceba56..7df94605f1 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTimeoutTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTimeoutTest.java
@@ -73,6 +73,7 @@ public class PlannerTimeoutTest extends AbstractPlannerTest {
 
             SqlTestUtils.assertThrowsSqlException(
                     PLANNING_TIMEOUT_ERR,
+                    "Planning of a query aborted due to planner timeout threshold is reached",
                     () -> await(prepareService.prepareAsync(parsedResult, ctx)));
         } finally {
             prepareService.stop();
diff --git a/modules/sql-engine/src/testFixtures/java/org/apache/ignite/internal/sql/engine/util/SqlTestUtils.java b/modules/sql-engine/src/testFixtures/java/org/apache/ignite/internal/sql/engine/util/SqlTestUtils.java
index df7a1a13e2..9ae42322e2 100644
--- a/modules/sql-engine/src/testFixtures/java/org/apache/ignite/internal/sql/engine/util/SqlTestUtils.java
+++ b/modules/sql-engine/src/testFixtures/java/org/apache/ignite/internal/sql/engine/util/SqlTestUtils.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.sql.engine.util;
 
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsNot.not;
 import static org.hamcrest.core.StringContains.containsString;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -47,36 +48,41 @@ import org.junit.jupiter.api.function.Executable;
 public class SqlTestUtils {
     private static final ThreadLocalRandom RND = ThreadLocalRandom.current();
 
-
     /**
      * <em>Assert</em> that execution of the supplied {@code executable} throws
-     * an {@link SqlException} with expected error code and return the exception.
+     * an {@link SqlException} with expected error code and message.
      *
-     * @param expectedCode Expected error code of {@link SqlException}
+     * @param expectedCode Expected error code of {@link SqlException}.
+     * @param expectedMessage Expected error message of {@link SqlException}.
      * @param executable Supplier to execute and check thrown exception.
      * @return Thrown the {@link SqlException}.
      */
-    public static SqlException assertThrowsSqlException(int expectedCode, Executable executable) {
-        SqlException ex = assertThrows(SqlException.class, executable);
-        assertEquals(expectedCode, ex.code());
-
-        return ex;
+    public static SqlException assertThrowsSqlException(int expectedCode, String expectedMessage, Executable executable) {
+        return assertThrowsSqlException(SqlException.class, expectedCode, expectedMessage, executable);
     }
 
     /**
      * <em>Assert</em> that execution of the supplied {@code executable} throws
-     * an {@link SqlException} with expected error code and message.
+     * an expected {@link SqlException} with expected error code and message.
      *
+     * @param expectedType Expected exception type.
      * @param expectedCode Expected error code of {@link SqlException}.
      * @param expectedMessage Expected error message of {@link SqlException}.
      * @param executable Supplier to execute and check thrown exception.
      * @return Thrown the {@link SqlException}.
      */
-    public static SqlException assertThrowsSqlException(int expectedCode, String expectedMessage, Executable executable) {
-        SqlException ex = assertThrowsSqlException(expectedCode, executable);
+    public static <T extends SqlException> T assertThrowsSqlException(
+            Class<T> expectedType,
+            int expectedCode,
+            String expectedMessage,
+            Executable executable) {
+        T ex = assertThrows(expectedType, executable);
+        assertEquals(expectedCode, ex.code());
 
         assertThat("Error message", ex.getMessage(), containsString(expectedMessage));
 
+        assertThat("Exception shouldn't be in internal package", ex.getClass().getPackageName(), not(containsString("internal")));
+
         return ex;
     }
 
@@ -157,9 +163,9 @@ public class SqlTestUtils {
             case INT64:
                 return (long) base;
             case FLOAT:
-                return (float) base + ((float) base / 1000);
+                return base + ((float) base / 1000);
             case DOUBLE:
-                return (double) base + ((double) base / 1000);
+                return base + ((double) base / 1000);
             case STRING:
                 return "str_" + base;
             case BYTE_ARRAY:
@@ -167,7 +173,7 @@ public class SqlTestUtils {
             case NULL:
                 return null;
             case DECIMAL:
-                return BigDecimal.valueOf((double) base + ((double) base / 1000));
+                return BigDecimal.valueOf(base + ((double) base / 1000));
             case NUMBER:
                 return BigInteger.valueOf(base);
             case UUID: