You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2018/10/30 06:09:08 UTC

[17/28] ignite git commit: IGNITE-9982: MVCC: Do not throw "MVCC disabled" exception for COMMIT and ROLLBACK commands. This closes #5086.

IGNITE-9982: MVCC: Do not throw "MVCC disabled" exception for COMMIT and ROLLBACK commands. This closes #5086.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7000e6e3
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7000e6e3
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7000e6e3

Branch: refs/heads/ignite-627
Commit: 7000e6e350719fd003cce43c13ec035658143005
Parents: 1fe02df
Author: ipavlukhin <vo...@gmail.com>
Authored: Mon Oct 29 12:16:44 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 29 12:18:06 2018 +0300

----------------------------------------------------------------------
 .../suite/IgniteJdbcDriverMvccTestSuite.java    |  20 +--
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   | 138 +++++++------------
 .../processors/query/h2/IgniteH2Indexing.java   |   8 +-
 ...sactionCommandsWithMvccDisabledSelfTest.java |  74 ++++++++++
 .../index/SqlTransactionsComandsSelfTest.java   |  83 -----------
 ...sactionsComandsWithMvccDisabledSelfTest.java |  83 -----------
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +-
 7 files changed, 139 insertions(+), 271 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7000e6e3/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java
index 606c32d..df8054f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverMvccTestSuite.java
@@ -26,23 +26,23 @@ import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerAutoCommitComplexSe
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsServerNoAutoCommitComplexSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsWithMvccEnabledSelfTest;
 
+/** */
 public class IgniteJdbcDriverMvccTestSuite extends TestSuite {
     /**
      * @return JDBC Driver Test Suite.
-     * @throws Exception In case of error.
      */
-    public static TestSuite suite() throws Exception {
+    public static TestSuite suite() {
         TestSuite suite = new TestSuite("Ignite JDBC Driver Test Suite");
 
-        suite.addTest(new TestSuite(JdbcThinConnectionMvccEnabledSelfTest.class));
-        suite.addTest(new TestSuite(JdbcVersionMismatchSelfTest.class));
-        
+        suite.addTestSuite(JdbcThinConnectionMvccEnabledSelfTest.class);
+        suite.addTestSuite(JdbcVersionMismatchSelfTest.class);
+
         // Transactions
-        suite.addTest(new TestSuite(JdbcThinTransactionsWithMvccEnabledSelfTest.class));
-        suite.addTest(new TestSuite(JdbcThinTransactionsClientAutoCommitComplexSelfTest.class));
-        suite.addTest(new TestSuite(JdbcThinTransactionsServerAutoCommitComplexSelfTest.class));
-        suite.addTest(new TestSuite(JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.class));
-        suite.addTest(new TestSuite(JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.class));
+        suite.addTestSuite(JdbcThinTransactionsWithMvccEnabledSelfTest.class);
+        suite.addTestSuite(JdbcThinTransactionsClientAutoCommitComplexSelfTest.class);
+        suite.addTestSuite(JdbcThinTransactionsServerAutoCommitComplexSelfTest.class);
+        suite.addTestSuite(JdbcThinTransactionsClientNoAutoCommitComplexSelfTest.class);
+        suite.addTestSuite(JdbcThinTransactionsServerNoAutoCommitComplexSelfTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7000e6e3/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
index 80397e6..5e12bae 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
@@ -66,6 +66,7 @@ import static java.sql.ResultSet.TYPE_FORWARD_ONLY;
 import static java.sql.Statement.NO_GENERATED_KEYS;
 import static java.sql.Statement.RETURN_GENERATED_KEYS;
 import static org.apache.ignite.configuration.ClientConnectorConfiguration.DFLT_PORT;
+import static org.apache.ignite.internal.processors.odbc.SqlStateCode.TRANSACTION_STATE_EXCEPTION;
 
 /**
  * Connection test.
@@ -978,29 +979,20 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
      */
     public void testGetSetAutoCommit() throws Exception {
         try (Connection conn = DriverManager.getConnection(URL)) {
-            assertTrue(conn.getAutoCommit());
-
-            // Cannot disable autocommit when MVCC is disabled.
-            GridTestUtils.assertThrows(log,
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        conn.setAutoCommit(false);
+            boolean ac0 = conn.getAutoCommit();
 
-                        return null;
-                    }
-                },
-                SQLException.class,
-                "MVCC must be enabled in order to invoke transactional operation: COMMIT"
-            );
+            conn.setAutoCommit(!ac0);
+            // assert no exception
 
-            assertTrue(conn.getAutoCommit());
+            conn.setAutoCommit(ac0);
+            // assert no exception
 
             conn.close();
 
             // Exception when called on closed connection
             checkConnectionClosed(new RunnableX() {
                 @Override public void run() throws Exception {
-                    conn.setAutoCommit(true);
+                    conn.setAutoCommit(ac0);
                 }
             });
         }
@@ -1024,19 +1016,6 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
                 "Transaction cannot be committed explicitly in auto-commit mode"
             );
 
-            // Cannot disable autocommit when MVCC is disabled.
-            GridTestUtils.assertThrows(log,
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        conn.setAutoCommit(false);
-
-                        return null;
-                    }
-                },
-                SQLException.class,
-                "MVCC must be enabled in order to invoke transactional operation: COMMIT"
-            );
-
             assertTrue(conn.getAutoCommit());
 
             // Should not be called in auto-commit mode
@@ -1081,21 +1060,6 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
                 "Transaction cannot be rolled back explicitly in auto-commit mode."
             );
 
-            // Cannot disable autocommit when MVCC is disabled.
-            GridTestUtils.assertThrows(log,
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        conn.setAutoCommit(false);
-
-                        return null;
-                    }
-                },
-                SQLException.class,
-                "MVCC must be enabled in order to invoke transactional operation: COMMIT"
-            );
-
-            assertTrue(conn.getAutoCommit());
-
             conn.close();
 
             // Exception when called on closed connection
@@ -1108,6 +1072,47 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * @throws Exception if failed.
+     */
+    public void testBeginFailsWhenMvccIsDisabled() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            conn.createStatement().execute("BEGIN");
+
+            fail("Exception is expected");
+        }
+        catch (SQLException e) {
+            assertEquals(TRANSACTION_STATE_EXCEPTION, e.getSQLState());
+        }
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testCommitIgnoredWhenMvccIsDisabled() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            conn.setAutoCommit(false);
+            conn.createStatement().execute("COMMIT");
+
+            conn.commit();
+        }
+        // assert no exception
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRollbackIgnoredWhenMvccIsDisabled() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            conn.setAutoCommit(false);
+
+            conn.createStatement().execute("ROLLBACK");
+
+            conn.rollback();
+        }
+        // assert no exception
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testGetMetaData() throws Exception {
@@ -1392,21 +1397,6 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
                 "Savepoint cannot be set in auto-commit mode"
             );
 
-            // Cannot disable autocommit when MVCC is disabled.
-            GridTestUtils.assertThrows(log,
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        conn.setAutoCommit(false);
-
-                        return null;
-                    }
-                },
-                SQLException.class,
-                "MVCC must be enabled in order to invoke transactional operation: COMMIT"
-            );
-
-            assertTrue(conn.getAutoCommit());
-
             conn.close();
 
             checkConnectionClosed(new RunnableX() {
@@ -1452,21 +1442,6 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
                 "Savepoint cannot be set in auto-commit mode"
             );
 
-            // Cannot disable autocommit when MVCC is disabled.
-            GridTestUtils.assertThrows(log,
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        conn.setAutoCommit(false);
-
-                        return null;
-                    }
-                },
-                SQLException.class,
-                "MVCC must be enabled in order to invoke transactional operation: COMMIT"
-            );
-
-            assertTrue(conn.getAutoCommit());
-
             conn.close();
 
             checkConnectionClosed(new RunnableX() {
@@ -1512,21 +1487,6 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
                 "Auto-commit mode"
             );
 
-            // Cannot disable autocommit when MVCC is disabled.
-            GridTestUtils.assertThrows(log,
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        conn.setAutoCommit(false);
-
-                        return null;
-                    }
-                },
-                SQLException.class,
-                "MVCC must be enabled in order to invoke transactional operation: COMMIT"
-            );
-
-            assertTrue(conn.getAutoCommit());
-
             conn.close();
 
             checkConnectionClosed(new RunnableX() {
@@ -2091,4 +2051,4 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
             }
         };
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7000e6e3/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 06868c4..3194eed 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -2012,16 +2012,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @throws IgniteCheckedException if failed.
      */
     private void processTxCommand(SqlCommand cmd, SqlFieldsQuery qry) throws IgniteCheckedException {
-        if (!mvccEnabled(ctx))
-            throw new IgniteSQLException("MVCC must be enabled in order to invoke transactional operation: " +
-                qry.getSql(), IgniteQueryErrorCode.MVCC_DISABLED);
-
         NestedTxMode nestedTxMode = qry instanceof SqlFieldsQueryEx ? ((SqlFieldsQueryEx)qry).getNestedTxMode() :
             NestedTxMode.DEFAULT;
 
         GridNearTxLocal tx = tx(ctx);
 
         if (cmd instanceof SqlBeginTransactionCommand) {
+            if (!mvccEnabled(ctx))
+                throw new IgniteSQLException("MVCC must be enabled in order to start transaction.",
+                    IgniteQueryErrorCode.MVCC_DISABLED);
+
             if (tx != null) {
                 if (nestedTxMode == null)
                     nestedTxMode = NestedTxMode.DEFAULT;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7000e6e3/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionCommandsWithMvccDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionCommandsWithMvccDisabledSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionCommandsWithMvccDisabledSelfTest.java
new file mode 100644
index 0000000..b8b9360
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionCommandsWithMvccDisabledSelfTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.processors.cache.index;
+
+import java.util.concurrent.Callable;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+public class SqlTransactionCommandsWithMvccDisabledSelfTest extends AbstractSchemaSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(commonConfiguration(0));
+
+        super.execute(grid(0), "CREATE TABLE INTS(k int primary key, v int) WITH \"wrap_value=false,cache_name=ints," +
+            "atomicity=transactional\"");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testBeginWithMvccDisabled() throws Exception {
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                execute(grid(0), "BEGIN");
+
+                return null;
+            }
+        }, IgniteSQLException.class, "MVCC must be enabled in order to start transaction.");
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testCommitWithMvccDisabled() throws Exception {
+        execute(grid(0), "COMMIT");
+        // assert no exception
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testRollbackWithMvccDisabled() throws Exception {
+        execute(grid(0), "ROLLBACK");
+        // assert no exception
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7000e6e3/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java
deleted file mode 100644
index 8b3fbe3..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsSelfTest.java
+++ /dev/null
@@ -1,83 +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.processors.cache.index;
-
-import java.util.concurrent.Callable;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.testframework.GridTestUtils;
-
-/**
- *
- */
-public class SqlTransactionsComandsSelfTest extends AbstractSchemaSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGrid(commonConfiguration(0));
-
-        super.execute(grid(0), "CREATE TABLE INTS(k int primary key, v int) WITH \"wrap_value=false,cache_name=ints," +
-            "atomicity=transactional\"");
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-
-        super.afterTestsStopped();
-    }
-
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testBeginWithMvccDisabledThrows() throws Exception {
-        checkMvccDisabledBehavior("BEGIN");
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testCommitWithMvccDisabledThrows() throws Exception {
-        checkMvccDisabledBehavior("COMMIT");
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testRollbackWithMvccDisabledThrows() throws Exception {
-        checkMvccDisabledBehavior("rollback");
-    }
-
-    /**
-     * @param sql Operation to test.
-     * @throws Exception if failed.
-     */
-    private void checkMvccDisabledBehavior(String sql) throws Exception {
-        try (IgniteEx node = startGrid(commonConfiguration(1))) {
-            GridTestUtils.assertThrows(null, new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    execute(node, sql);
-
-                    return null;
-                }
-            }, IgniteSQLException.class, "MVCC must be enabled in order to invoke transactional operation: " + sql);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7000e6e3/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java
deleted file mode 100644
index d2931ba..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlTransactionsComandsWithMvccDisabledSelfTest.java
+++ /dev/null
@@ -1,83 +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.processors.cache.index;
-
-import java.util.concurrent.Callable;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.testframework.GridTestUtils;
-
-/**
- *
- */
-public class SqlTransactionsComandsWithMvccDisabledSelfTest extends AbstractSchemaSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGrid(commonConfiguration(0));
-
-        super.execute(grid(0), "CREATE TABLE INTS(k int primary key, v int) WITH \"wrap_value=false,cache_name=ints," +
-            "atomicity=transactional\"");
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-
-        super.afterTestsStopped();
-    }
-
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testBeginWithMvccDisabledThrows() throws Exception {
-        checkMvccDisabledBehavior("BEGIN");
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testCommitWithMvccDisabledThrows() throws Exception {
-        checkMvccDisabledBehavior("COMMIT");
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testRollbackWithMvccDisabledThrows() throws Exception {
-        checkMvccDisabledBehavior("rollback");
-    }
-
-    /**
-     * @param sql Operation to test.
-     * @throws Exception if failed.
-     */
-    private void checkMvccDisabledBehavior(String sql) throws Exception {
-        try (IgniteEx node = startGrid(commonConfiguration(1))) {
-            GridTestUtils.assertThrows(null, new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    execute(node, sql);
-
-                    return null;
-                }
-            }, IgniteSQLException.class, "MVCC must be enabled in order to invoke transactional operation: " + sql);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7000e6e3/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index f29a79b..fd71794 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -137,7 +137,7 @@ import org.apache.ignite.internal.processors.cache.index.IgniteDecimalSelfTest;
 import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest;
 import org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest;
 import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest;
-import org.apache.ignite.internal.processors.cache.index.SqlTransactionsComandsSelfTest;
+import org.apache.ignite.internal.processors.cache.index.SqlTransactionCommandsWithMvccDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest;
@@ -456,7 +456,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
 
         suite.addTestSuite(GridIndexRebuildSelfTest.class);
 
-        suite.addTestSuite(SqlTransactionsComandsSelfTest.class);
+        suite.addTestSuite(SqlTransactionCommandsWithMvccDisabledSelfTest.class);
 
         suite.addTestSuite(IgniteSqlDefaultValueTest.class);
         suite.addTestSuite(IgniteDecimalSelfTest.class);