IGNITE-21167 Sql. Jdbc. Raise a clearer error message when the tx control statement is used in non-autocommit mode (#3700)

diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcHandlerBase.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcHandlerBase.java
new file mode 100644
index 0000000..cad0258
--- /dev/null
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcHandlerBase.java
@@ -0,0 +1,175 @@
+/*
+ * 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.client.handler;
+
+import static org.apache.ignite.internal.jdbc.proto.IgniteQueryErrorCode.UNSUPPORTED_OPERATION;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletionStage;
+import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
+import org.apache.ignite.internal.jdbc.proto.event.JdbcQuerySingleResult;
+import org.apache.ignite.internal.jdbc.proto.event.Response;
+import org.apache.ignite.internal.lang.IgniteInternalCheckedException;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.sql.engine.AsyncSqlCursor;
+import org.apache.ignite.internal.sql.engine.InternalSqlRow;
+import org.apache.ignite.internal.sql.engine.TxControlInsideExternalTxNotSupportedException;
+import org.apache.ignite.internal.util.AsyncCursor.BatchedResult;
+import org.apache.ignite.internal.util.ExceptionUtils;
+import org.apache.ignite.sql.ColumnMetadata;
+import org.apache.ignite.sql.ColumnType;
+import org.apache.ignite.sql.ResultSetMetadata;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Contains common methods used to process jdbc requests.
+ */
+abstract class JdbcHandlerBase {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Client registry resources. */
+    protected final ClientResourceRegistry resources;
+
+    JdbcHandlerBase(ClientResourceRegistry resources) {
+        this.resources = resources;
+        this.log = Loggers.forClass(this.getClass());
+    }
+
+    /**
+     * Creates jdbc result for the cursor.
+     *
+     * @param cur Sql cursor for query.
+     * @param pageSize Page size.
+     * @return JdbcQuerySingleResult filled with first batch of data.
+     */
+    CompletionStage<JdbcQuerySingleResult> createJdbcResult(AsyncSqlCursor<InternalSqlRow> cur, int pageSize) {
+        return cur.requestNextAsync(pageSize).thenApply(batch -> {
+            boolean hasNext = batch.hasMore();
+
+            long cursorId;
+            try {
+                cursorId = resources.put(new ClientResource(cur, cur::closeAsync));
+            } catch (IgniteInternalCheckedException e) {
+                cur.closeAsync();
+
+                return new JdbcQuerySingleResult(Response.STATUS_FAILED,
+                        "Unable to store query cursor.");
+            }
+
+            switch (cur.queryType()) {
+                case EXPLAIN:
+                case QUERY: {
+                    List<ColumnMetadata> columns = cur.metadata().columns();
+
+                    return buildSingleRequest(batch, columns, cursorId, !hasNext);
+                }
+                case DML: {
+                    if (!validateDmlResult(cur.metadata(), hasNext)) {
+                        return new JdbcQuerySingleResult(Response.STATUS_FAILED, "Unexpected result for DML query");
+                    }
+
+                    long updCount = (long) batch.items().get(0).get(0);
+
+                    return new JdbcQuerySingleResult(cursorId, updCount);
+                }
+                case DDL:
+                case TX_CONTROL:
+                    return new JdbcQuerySingleResult(cursorId, 0);
+                default:
+                    return new JdbcQuerySingleResult(UNSUPPORTED_OPERATION,
+                            "Query type is not supported yet [queryType=" + cur.queryType() + ']');
+            }
+        });
+    }
+
+    private static JdbcQuerySingleResult buildSingleRequest(
+            BatchedResult<InternalSqlRow> batch,
+            List<ColumnMetadata> columns,
+            long cursorId,
+            boolean hasNext
+    ) {
+        List<BinaryTupleReader> rows = new ArrayList<>(batch.items().size());
+        for (InternalSqlRow item : batch.items()) {
+            rows.add(item.asBinaryTuple());
+        }
+
+        int[] decimalScales = new int[columns.size()];
+        List<ColumnType> schema = new ArrayList<>(columns.size());
+
+        int countOfDecimal = 0;
+        for (ColumnMetadata column : columns) {
+            schema.add(column.type());
+            if (column.type() == ColumnType.DECIMAL) {
+                decimalScales[countOfDecimal++] = column.scale();
+            }
+        }
+        decimalScales = Arrays.copyOf(decimalScales, countOfDecimal);
+
+        return new JdbcQuerySingleResult(cursorId, rows, schema, decimalScales, hasNext);
+    }
+
+    JdbcQuerySingleResult createErrorResult(String logMessage, Throwable origin, @Nullable String errMessagePrefix) {
+        Throwable ex = ExceptionUtils.unwrapCause(origin);
+
+        log.info(logMessage, ex);
+
+        String errorMessage;
+
+        if (ex instanceof TxControlInsideExternalTxNotSupportedException) {
+            errorMessage = "Transaction control statements are not supported when autocommit mode is disabled";
+        } else {
+            errorMessage = getErrorMessage(ex);
+        }
+
+        return new JdbcQuerySingleResult(Response.STATUS_FAILED, (errMessagePrefix == null ? "" : errMessagePrefix) + errorMessage);
+    }
+
+    /**
+     * Validate dml result. Check if it stores only one value of Long type.
+     *
+     * @param meta Fetched data from cursor.
+     * @param next  HasNext flag.
+     * @return Boolean value indicates if data is valid or not.
+     */
+    private static boolean validateDmlResult(ResultSetMetadata meta, boolean next) {
+        if (next) {
+            return false;
+        }
+
+        if (meta.columns().size() != 1) {
+            return false;
+        }
+
+        return meta.columns().get(0).type() == ColumnType.INT64;
+    }
+
+    /**
+     * Get a message of given exception for further sending to the client.
+     *
+     * @param t Throwable.
+     * @return String filled with exception message.
+     */
+    @Nullable static String getErrorMessage(Throwable t) {
+        Throwable cause = ExceptionUtils.unwrapCause(t);
+        return cause.getMessage();
+    }
+}
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryCursorHandlerImpl.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryCursorHandlerImpl.java
index b01aaf6..ad893bb 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryCursorHandlerImpl.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryCursorHandlerImpl.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.client.handler;
 
-import static org.apache.ignite.client.handler.JdbcQueryEventHandlerImpl.buildSingleRequest;
-import static org.apache.ignite.internal.jdbc.proto.IgniteQueryErrorCode.UNSUPPORTED_OPERATION;
-
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.nio.ByteBuffer;
@@ -42,24 +39,20 @@
 import org.apache.ignite.internal.lang.IgniteInternalCheckedException;
 import org.apache.ignite.internal.sql.engine.AsyncSqlCursor;
 import org.apache.ignite.internal.sql.engine.InternalSqlRow;
-import org.apache.ignite.internal.sql.engine.SqlQueryType;
 import org.apache.ignite.sql.ColumnMetadata;
 import org.apache.ignite.sql.ResultSetMetadata;
 
 /**
  * Jdbc query event handler implementation.
  */
-public class JdbcQueryCursorHandlerImpl implements JdbcQueryCursorHandler {
-    /** Client registry resources. */
-    private final ClientResourceRegistry resources;
-
+public class JdbcQueryCursorHandlerImpl extends JdbcHandlerBase implements JdbcQueryCursorHandler {
     /**
      * Constructor.
      *
      * @param resources Client resources.
      */
     JdbcQueryCursorHandlerImpl(ClientResourceRegistry resources) {
-        this.resources = resources;
+        super(resources);
     }
 
     /** {@inheritDoc} */
@@ -85,7 +78,7 @@
                 StringWriter sw = getWriterWithStackTrace(t);
 
                 return new JdbcQueryFetchResult(Response.STATUS_FAILED,
-                    "Failed to fetch query results [curId=" + req.cursorId() + "]. Error message:" + sw);
+                    "Failed to fetch query results [curId=" + req.cursorId() + "]. Error message: " + sw);
             }
 
             List<ByteBuffer> rows = new ArrayList<>(batch.items().size());
@@ -115,52 +108,17 @@
         }
 
         return asyncSqlCursor.closeAsync().thenCompose(c -> asyncSqlCursor.nextResult())
-                .thenCompose(cur -> cur.requestNextAsync(req.fetchSize())
-                    .thenApply(batch -> {
-                        try {
-                            SqlQueryType queryType = cur.queryType();
+                .thenCompose(cur -> createJdbcResult(cur, req.fetchSize()))
+                .exceptionally(t -> {
+                    iterateThroughResultsAndCloseThem(asyncSqlCursor);
 
-                            long cursorId = resources.put(new ClientResource(cur, cur::closeAsync));
+                    String msgPrefix = "Failed to fetch query results [curId=" + req.cursorId() + "].";
 
-                            switch (queryType) {
-                                case EXPLAIN:
-                                case QUERY: {
-                                    List<ColumnMetadata> columns = cur.metadata().columns();
-
-                                    return buildSingleRequest(batch, columns, cursorId, !batch.hasMore());
-                                }
-                                case DML: {
-                                    long updCount = (long) batch.items().get(0).get(0);
-
-                                    return new JdbcQuerySingleResult(cursorId, updCount);
-                                }
-                                case DDL:
-                                case TX_CONTROL:
-                                    return new JdbcQuerySingleResult(cursorId, 0);
-                                default:
-                                    return new JdbcQuerySingleResult(UNSUPPORTED_OPERATION,
-                                            "Query type is not supported yet [queryType=" + cur.queryType() + ']');
-                            }
-                        } catch (IgniteInternalCheckedException e) {
-                            return new JdbcQuerySingleResult(Response.STATUS_FAILED,
-                                    "Unable to store query cursor.");
-                        }
-                    })
-                ).handle((res, t) -> {
-                    if (t != null) {
-                        iterateThroughResultsAndCloseThem(asyncSqlCursor);
-
-                        StringWriter sw = getWriterWithStackTrace(t);
-
-                        return new JdbcQuerySingleResult(Response.STATUS_FAILED,
-                                "Failed to fetch query results [curId=" + req.cursorId() + "]. Error message:" + sw);
-                    }
-
-                    return res;
+                    return createErrorResult(msgPrefix, t, msgPrefix + " Error message: ");
                 });
     }
 
-    static void iterateThroughResultsAndCloseThem(AsyncSqlCursor<InternalSqlRow> cursor) {
+    private static void iterateThroughResultsAndCloseThem(AsyncSqlCursor<InternalSqlRow> cursor) {
         Function<AsyncSqlCursor<InternalSqlRow>, CompletableFuture<AsyncSqlCursor<InternalSqlRow>>> traverser = new Function<>() {
             @Override
             public CompletableFuture<AsyncSqlCursor<InternalSqlRow>> apply(AsyncSqlCursor<InternalSqlRow> cur) {
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 c11c4e4..0f74bac 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
@@ -18,7 +18,6 @@
 package org.apache.ignite.client.handler;
 
 import static org.apache.ignite.internal.jdbc.proto.IgniteQueryErrorCode.UNKNOWN;
-import static org.apache.ignite.internal.jdbc.proto.IgniteQueryErrorCode.UNSUPPORTED_OPERATION;
 import static org.apache.ignite.internal.sql.engine.SqlQueryType.DML;
 import static org.apache.ignite.internal.util.ArrayUtils.OBJECT_EMPTY_ARRAY;
 import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
@@ -27,16 +26,11 @@
 import it.unimi.dsi.fastutil.ints.IntArrayList;
 import java.sql.Statement;
 import java.time.ZoneId;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionStage;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.client.handler.requests.jdbc.JdbcMetadataCatalog;
 import org.apache.ignite.client.handler.requests.jdbc.JdbcQueryCursor;
-import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
 import org.apache.ignite.internal.jdbc.proto.JdbcQueryEventHandler;
 import org.apache.ignite.internal.jdbc.proto.JdbcStatementType;
 import org.apache.ignite.internal.jdbc.proto.event.JdbcBatchExecuteRequest;
@@ -57,8 +51,6 @@
 import org.apache.ignite.internal.jdbc.proto.event.Response;
 import org.apache.ignite.internal.lang.IgniteInternalCheckedException;
 import org.apache.ignite.internal.lang.IgniteInternalException;
-import org.apache.ignite.internal.logger.IgniteLogger;
-import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.sql.engine.AsyncSqlCursor;
 import org.apache.ignite.internal.sql.engine.InternalSqlRow;
 import org.apache.ignite.internal.sql.engine.QueryProcessor;
@@ -67,21 +59,13 @@
 import org.apache.ignite.internal.sql.engine.property.SqlProperties;
 import org.apache.ignite.internal.sql.engine.property.SqlPropertiesHelper;
 import org.apache.ignite.internal.tx.InternalTransaction;
-import org.apache.ignite.internal.util.AsyncCursor.BatchedResult;
-import org.apache.ignite.internal.util.ExceptionUtils;
-import org.apache.ignite.sql.ColumnMetadata;
-import org.apache.ignite.sql.ColumnType;
-import org.apache.ignite.sql.ResultSetMetadata;
 import org.apache.ignite.tx.IgniteTransactions;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Jdbc query event handler implementation.
  */
-public class JdbcQueryEventHandlerImpl implements JdbcQueryEventHandler {
-    /** Logger. */
-    private static final IgniteLogger LOG = Loggers.forClass(JdbcQueryEventHandlerImpl.class);
-
+public class JdbcQueryEventHandlerImpl extends JdbcHandlerBase implements JdbcQueryEventHandler {
     /** {@link SqlQueryType}s allowed in JDBC select statements. **/
     private static final Set<SqlQueryType> SELECT_STATEMENT_QUERIES = Set.of(SqlQueryType.QUERY, SqlQueryType.EXPLAIN);
 
@@ -94,9 +78,6 @@
     /** Jdbc metadata info. */
     private final JdbcMetadataCatalog meta;
 
-    /** Current JDBC cursors. */
-    private final ClientResourceRegistry resources;
-
     /** Ignite transactions API. */
     private final IgniteTransactions igniteTransactions;
 
@@ -114,9 +95,10 @@
             ClientResourceRegistry resources,
             IgniteTransactions igniteTransactions
     ) {
+        super(resources);
+
         this.processor = processor;
         this.meta = meta;
-        this.resources = resources;
         this.igniteTransactions = igniteTransactions;
     }
 
@@ -169,14 +151,8 @@
                 req.arguments() == null ? OBJECT_EMPTY_ARRAY : req.arguments()
         );
 
-        return result.thenCompose(cursor -> createJdbcResult(new JdbcQueryCursor<>(req.maxRows(), cursor), req))
-                .exceptionally(t -> {
-                    LOG.info("Exception while executing query [query=" + req.sqlQuery() + "]", ExceptionUtils.unwrapCause(t));
-
-                    String msg = getErrorMessage(t);
-
-                    return new JdbcQuerySingleResult(Response.STATUS_FAILED, msg);
-                });
+        return result.thenCompose(cursor -> createJdbcResult(new JdbcQueryCursor<>(req.maxRows(), cursor), req.pageSize()))
+                .exceptionally(t -> createErrorResult("Exception while executing query [query=" + req.sqlQuery() + "]", t, null));
     }
 
     private static SqlProperties createProperties(JdbcStatementType stmtType, boolean multiStatement, ZoneId timeZoneId) {
@@ -350,110 +326,6 @@
         });
     }
 
-    /**
-     * Get a message of given exception for further sending to the client.
-     *
-     * @param t Throwable.
-     * @return String filled with exception message.
-     */
-    @Nullable private String getErrorMessage(Throwable t) {
-        Throwable cause = ExceptionUtils.unwrapCause(t);
-        return cause.getMessage();
-    }
-
-    /**
-     * Creates jdbc result for the cursor.
-     *
-     * @param cur Sql cursor for query.
-     * @param req Execution request.
-     * @return JdbcQuerySingleResult filled with first batch of data.
-     */
-    private CompletionStage<JdbcQuerySingleResult> createJdbcResult(AsyncSqlCursor<InternalSqlRow> cur, JdbcQueryExecuteRequest req) {
-        return cur.requestNextAsync(req.pageSize()).thenApply(batch -> {
-            boolean hasNext = batch.hasMore();
-
-            long cursorId;
-            try {
-                cursorId = resources.put(new ClientResource(cur, cur::closeAsync));
-            } catch (IgniteInternalCheckedException e) {
-                cur.closeAsync();
-
-                return new JdbcQuerySingleResult(Response.STATUS_FAILED,
-                        "Unable to store query cursor.");
-            }
-
-            switch (cur.queryType()) {
-                case EXPLAIN:
-                case QUERY: {
-                    List<ColumnMetadata> columns = cur.metadata().columns();
-
-                    return buildSingleRequest(batch, columns, cursorId, !hasNext);
-                }
-                case DML: {
-                    if (!validateDmlResult(cur.metadata(), hasNext)) {
-                        return new JdbcQuerySingleResult(Response.STATUS_FAILED,
-                                "Unexpected result for DML [query=" + req.sqlQuery() + ']');
-                    }
-
-                    long updCount = (long) batch.items().get(0).get(0);
-
-                    return new JdbcQuerySingleResult(cursorId, updCount);
-                }
-                case DDL:
-                case TX_CONTROL:
-                    return new JdbcQuerySingleResult(cursorId, 0);
-                default:
-                    return new JdbcQuerySingleResult(UNSUPPORTED_OPERATION,
-                            "Query type is not supported yet [queryType=" + cur.queryType() + ']');
-            }
-        });
-    }
-
-    static JdbcQuerySingleResult buildSingleRequest(
-            BatchedResult<InternalSqlRow> batch,
-            List<ColumnMetadata> columns,
-            long cursorId,
-            boolean hasNext
-    ) {
-        List<BinaryTupleReader> rows = new ArrayList<>(batch.items().size());
-        for (InternalSqlRow item : batch.items()) {
-            rows.add(item.asBinaryTuple());
-        }
-
-        int[] decimalScales = new int[columns.size()];
-        List<ColumnType> schema = new ArrayList<>(columns.size());
-
-        int countOfDecimal = 0;
-        for (ColumnMetadata column : columns) {
-            schema.add(column.type());
-            if (column.type() == ColumnType.DECIMAL) {
-                decimalScales[countOfDecimal++] = column.scale();
-            }
-        }
-        decimalScales = Arrays.copyOf(decimalScales, countOfDecimal);
-
-        return new JdbcQuerySingleResult(cursorId, rows, schema, decimalScales, hasNext);
-    }
-
-    /**
-     * Validate dml result. Check if it stores only one value of Long type.
-     *
-     * @param meta Fetched data from cursor.
-     * @param next  HasNext flag.
-     * @return Boolean value indicates if data is valid or not.
-     */
-    private boolean validateDmlResult(ResultSetMetadata meta, boolean next) {
-        if (next) {
-            return false;
-        }
-
-        if (meta.columns().size() != 1) {
-            return false;
-        }
-
-        return meta.columns().get(0).type() == ColumnType.INT64;
-    }
-
     static class JdbcConnectionContext {
         private final AtomicBoolean closed = new AtomicBoolean();
 
diff --git a/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcMultiStatementSelfTest.java b/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcMultiStatementSelfTest.java
index 96041f1..09c1164 100644
--- a/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcMultiStatementSelfTest.java
+++ b/modules/jdbc/src/integrationTest/java/org/apache/ignite/jdbc/ItJdbcMultiStatementSelfTest.java
@@ -27,9 +27,12 @@
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
+import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
@@ -381,20 +384,6 @@
     }
 
     @Test
-    public void testAutoCommitFalseNonCompleted() throws Exception {
-        String txErrMsg = "Transaction control statement cannot be executed within an external transaction";
-        conn.setAutoCommit(false);
-        assertThrowsSqlException(txErrMsg, () -> stmt.execute("COMMIT"));
-
-        boolean res = stmt.execute("SELECT 1;COMMIT");
-        assertTrue(res);
-        assertNotNull(stmt.getResultSet());
-        assertThrowsSqlException(txErrMsg, () -> stmt.getMoreResults());
-
-        assertThrowsSqlException(txErrMsg, () -> stmt.execute("START TRANSACTION; SELECT 1;"));
-    }
-
-    @Test
     public void testAutoCommitFalse() throws Exception {
         conn.setAutoCommit(false);
 
@@ -422,11 +411,61 @@
     }
 
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-21167")
-    public void testAutoCommitFalseWithEmptyTx() throws Exception {
-        String txErrMsg = "Transaction control statement cannot be executed within an external transaction";
+    @SuppressWarnings("ThrowableNotThrown")
+    public void testAutoCommitFalseTxControlStatementsNotSupported() throws Exception {
+        String txErrMsg = "Transaction control statements are not supported when autocommit mode is disabled";
         conn.setAutoCommit(false);
-        assertThrowsSqlException(txErrMsg, () -> stmt.execute("START TRANSACTION; SELECT 1; COMMIT;"));
+        assertThrowsSqlException(txErrMsg, () -> stmt.execute("START TRANSACTION; SELECT 1; COMMIT"));
+        assertThrowsSqlException(txErrMsg, () -> stmt.execute("COMMIT"));
+        assertThrowsSqlException(txErrMsg, () -> stmt.execute("START TRANSACTION"));
+
+        boolean res = stmt.execute("SELECT 1;COMMIT");
+        assertTrue(res);
+        assertNotNull(stmt.getResultSet());
+        assertThrowsSqlException(txErrMsg, () -> stmt.getMoreResults());
+
+        // TX control statements don't affect a JDBC managed transaction.
+        {
+            long initialRowsCount;
+
+            try (ResultSet rs = stmt.executeQuery("SELECT COUNT(1) FROM TEST_TX")) {
+                assertTrue(rs.next());
+
+                initialRowsCount = rs.getLong(1);
+            }
+
+            stmt.execute("INSERT INTO TEST_TX VALUES (5, 5, '5'); COMMIT; INSERT INTO TEST_TX VALUES (6, 6, '6')");
+            assertEquals(1, stmt.getUpdateCount());
+
+            // Next statement throws the expected exception.
+            assertThrowsSqlException(txErrMsg, () -> stmt.getMoreResults());
+
+            stmt.close();
+
+            // JDBC managed transaction was not rolled back or committed.
+            try (Connection conn0 = DriverManager.getConnection(URL)) {
+                Statement stmt0 = conn0.createStatement();
+
+                try (ResultSet rs = stmt0.executeQuery("SELECT COUNT(1) FROM TEST_TX")) {
+                    assertTrue(rs.next());
+                    assertEquals(initialRowsCount, rs.getLong(1));
+                }
+            }
+
+            // Commit JDBC managed transaction.
+            conn.commit();
+
+            try (Connection conn0 = DriverManager.getConnection(URL)) {
+                Statement stmt0 = conn0.createStatement();
+
+                try (ResultSet rs = stmt0.executeQuery("SELECT COUNT(1) FROM TEST_TX")) {
+                    assertTrue(rs.next());
+
+                    // The first DML statement was successfully inserted.
+                    assertEquals(initialRowsCount + 1, rs.getLong(1));
+                }
+            }
+        }
     }
 
     @Test