Repository: ignite
Updated Branches:
  refs/heads/master 4d736fc20 -> 254a65292


http://git-wip-us.apache.org/repos/asf/ignite/blob/254a6529/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
index 57cee61..1a6cb55 100644
--- 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
@@ -18,44 +18,30 @@
 package org.apache.ignite.internal.processors.query.h2;
 
 import java.io.File;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Semaphore;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import 
org.apache.ignite.internal.processors.cache.index.DynamicIndexAbstractSelfTest;
-import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import 
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
-import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import 
org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
-import 
org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl;
 import org.apache.ignite.internal.util.lang.GridCursor;
-import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
 
 /**
  * Index rebuild after node restart test.
  */
 public class GridIndexRebuildSelfTest extends DynamicIndexAbstractSelfTest {
     /** Data size. */
-    private final static int AMOUNT = 10;
+    protected static final int AMOUNT = 300;
 
     /** Data size. */
-    private final static String CACHE_NAME = "T";
+    protected static final String CACHE_NAME = "T";
 
     /** Test instance to allow interaction with static context. */
     private static GridIndexRebuildSelfTest INSTANCE;
@@ -63,14 +49,13 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
     /** Latch to signal that rebuild may start. */
     private final CountDownLatch rebuildLatch = new CountDownLatch(1);
 
-    /** Latch to signal that concurrent put may start. */
-    private final Semaphore rebuildSemaphore = new Semaphore(1, true);
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration commonConfiguration(int idx) 
throws Exception {
         IgniteConfiguration cfg =  super.commonConfiguration(idx);
 
-        
cfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration().setPersistenceEnabled(true);
+        cfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration()
+            .setMaxSize(300*1024L*1024L)
+            .setPersistenceEnabled(true);
 
         return cfg;
     }
@@ -85,24 +70,25 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
         INSTANCE = this;
     }
 
-    /**
-     * Do test with MVCC enabled.
-     */
-    public void testMvccEnabled() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-7259";);
-        doTest(true);
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
     }
 
-    /**
-     * Do test with MVCC disabled.
-     */
-    public void testMvccDisabled() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-7259";);
-        doTest(false);
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        cleanPersistenceDir();
     }
 
     /**
-     * Do test.<p>
+     * Do test.
+     * <p>
      * Steps are as follows:
      * <ul>
      *     <li>Put some data;</li>
@@ -120,12 +106,11 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
      *             index rebuild for them has happened after put.</li>
      *         </ul>
      *     </li>
-     * </ul>
-     * @param mvccEnabled MVCC flag.
+     * </ul></p>
      * @throws Exception if failed.
      */
-    private void doTest(boolean mvccEnabled) throws Exception {
-        IgniteEx srv = startServer(mvccEnabled);
+    public void testIndexRebuild() throws Exception {
+        IgniteEx srv = startServer();
 
         execute(srv, "CREATE TABLE T(k int primary key, v int) WITH 
\"cache_name=T,wrap_value=false," +
             "atomicity=transactional\"");
@@ -136,12 +121,9 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
 
         assertNotNull(cc);
 
-        if (mvccEnabled)
-            lockVersion(srv);
-
         putData(srv, false);
 
-        checkDataState(srv, mvccEnabled, false);
+        checkDataState(srv, false);
 
         File cacheWorkDir = 
((FilePageStoreManager)cc.context().shared().pageStore()).cacheWorkDir(cc.configuration());
 
@@ -151,22 +133,22 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
 
         assertTrue(U.delete(idxPath));
 
-        srv = startServer(mvccEnabled);
+        srv = startServer();
 
         putData(srv, true);
 
-        checkDataState(srv, mvccEnabled, true);
+        checkDataState(srv, true);
     }
 
     /**
      * Check versions presence in index tree.
+     *
      * @param srv Node.
-     * @param mvccEnabled MVCC flag.
      * @param afterRebuild Whether index rebuild has occurred.
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
-    private void checkDataState(IgniteEx srv, boolean mvccEnabled, boolean 
afterRebuild) throws IgniteCheckedException {
+    protected void checkDataState(IgniteEx srv, boolean afterRebuild) throws 
IgniteCheckedException {
         IgniteInternalCache icache = srv.cachex(CACHE_NAME);
 
         IgniteCache cache = srv.cache(CACHE_NAME);
@@ -179,47 +161,23 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
             while (cur.next()) {
                 CacheDataRow row = cur.get();
 
-                int key  = 
row.key().value(icache.context().cacheObjectContext(), false);
-
-                if (mvccEnabled) {
-                    List<IgniteBiTuple<Object, MvccVersion>> vers = 
store.mvccFindAllVersions(icache.context(), row.key());
-
-                    if (!afterRebuild || key <= AMOUNT / 2)
-                        assertEquals(key, vers.size());
-                    else {
-                        // For keys affected by concurrent put there are two 
versions -
-                        // -1 (concurrent put mark) and newest restored value 
as long as put cleans obsolete versions.
-                        assertEquals(2, vers.size());
-
-                        assertEquals(-1, vers.get(0).getKey());
-                        assertEquals(key, vers.get(1).getKey());
-                    }
-                }
-                else {
-                    if (!afterRebuild || key <= AMOUNT / 2)
-                        assertEquals(key, cache.get(key));
-                    else
-                        assertEquals(-1, cache.get(key));
-                }
+                int key = 
row.key().value(icache.context().cacheObjectContext(), false);
+
+                if (!afterRebuild || key <= AMOUNT / 2)
+                    assertEquals(key, cache.get(key));
+                else
+                    assertEquals(-1, cache.get(key));
             }
         }
     }
 
     /**
-     * Lock coordinator version in order to keep MVCC versions in place.
-     * @param node Node.
-     * @throws IgniteCheckedException if failed.
-     */
-    private static void lockVersion(IgniteEx node) throws 
IgniteCheckedException {
-        node.context().coordinators().requestSnapshotAsync().get();
-    }
-
-    /**
      * Put data to cache.
+     *
      * @param node Node.
      * @throws Exception if failed.
      */
-    private void putData(Ignite node, final boolean forConcurrentPut) throws 
Exception {
+    protected void putData(Ignite node, final boolean forConcurrentPut) throws 
Exception {
         final IgniteCache<Integer, Integer> cache = node.cache(CACHE_NAME);
 
         assertNotNull(cache);
@@ -230,13 +188,9 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
                 if (i <= AMOUNT / 2)
                     continue;
 
-                rebuildSemaphore.acquire();
-
                 cache.put(i, -1);
 
                 rebuildLatch.countDown();
-
-                rebuildSemaphore.release();
             }
             else {
                 // Data streamer is not used intentionally in order to 
preserve all versions.
@@ -248,16 +202,16 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
 
     /**
      * Start server node.
-     * @param mvccEnabled MVCC flag.
+     *
      * @return Started node.
      * @throws Exception if failed.
      */
-    private IgniteEx startServer(boolean mvccEnabled) throws Exception {
+    protected IgniteEx startServer() throws Exception {
         // Have to do this for each starting node - see GridQueryProcessor 
ctor, it nulls
         // idxCls static field on each call.
         GridQueryProcessor.idxCls = BlockingIndexing.class;
 
-        IgniteConfiguration cfg = 
serverConfiguration(0).setMvccEnabled(mvccEnabled);
+        IgniteConfiguration cfg = serverConfiguration(0);
 
         IgniteEx res = startGrid(cfg);
 
@@ -266,21 +220,6 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
         return res;
     }
 
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-
-        stopAllGrids();
-
-        cleanPersistenceDir();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        cleanPersistenceDir();
-    }
-
     /**
      * Blocking indexing processor.
      */
@@ -295,56 +234,7 @@ public class GridIndexRebuildSelfTest extends 
DynamicIndexAbstractSelfTest {
             else
                 firstRbld = false;
 
-            int cacheId = CU.cacheId(cacheName);
-
-            GridCacheContext cctx = 
ctx.cache().context().cacheContext(cacheId);
-
-            final GridCacheQueryManager qryMgr = cctx.queries();
-
-            SchemaIndexCacheVisitor visitor = new 
SchemaIndexCacheVisitorImpl(cctx);
-
-            visitor.visit(new TestRebuildClosure(qryMgr, cctx.mvccEnabled()));
-
-            for (H2TableDescriptor tblDesc : tables(cacheName))
-                tblDesc.table().markRebuildFromHashInProgress(false);
-        }
-    }
-
-    /**
-     * Test closure.
-     */
-    private final static class TestRebuildClosure extends 
RebuildIndexFromHashClosure {
-        /** Seen keys set to track moment when concurrent put may start. */
-        private final Set<KeyCacheObject> keys =
-            Collections.newSetFromMap(new ConcurrentHashMap<KeyCacheObject, 
Boolean>());
-
-        /**
-         * @param qryMgr      Query manager.
-         * @param mvccEnabled MVCC status flag.
-         */
-        TestRebuildClosure(GridCacheQueryManager qryMgr, boolean mvccEnabled) {
-            super(qryMgr, mvccEnabled);
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized void apply(CacheDataRow row) throws 
IgniteCheckedException {
-            // For half of the keys, we want to do rebuild
-            // after corresponding key had been put from a concurrent thread.
-            boolean keyFirstMet = keys.add(row.key()) && keys.size() > AMOUNT 
/ 2;
-
-            if (keyFirstMet) {
-                try {
-                    INSTANCE.rebuildSemaphore.acquire();
-                }
-                catch (InterruptedException e) {
-                    throw new IgniteCheckedException(e);
-                }
-            }
-
-            super.apply(row);
-
-            if (keyFirstMet)
-                INSTANCE.rebuildSemaphore.release();
+            super.rebuildIndexesFromHash(cacheName);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/254a6529/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
new file mode 100644
index 0000000..3d65db8
--- /dev/null
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildWithMvccEnabledSelfTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.query.h2;
+
+import java.io.File;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import 
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/**
+ * Index rebuild after node restart test.
+ */
+public class GridIndexRebuildWithMvccEnabledSelfTest extends 
GridIndexRebuildSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration serverConfiguration(int idx, 
boolean filter) throws Exception {
+        return super.serverConfiguration(idx, filter)
+            .setMvccVacuumTimeInterval(Integer.MAX_VALUE)
+            .setMvccEnabled(true);
+    }
+
+    /** {@inheritDoc} */
+    public void testIndexRebuild() throws Exception {
+        IgniteEx srv = startServer();
+
+        execute(srv, "CREATE TABLE T(k int primary key, v int) WITH 
\"cache_name=T,wrap_value=false," +
+            "atomicity=transactional\"");
+
+        execute(srv, "CREATE INDEX IDX ON T(v)");
+
+        IgniteInternalCache cc = srv.cachex(CACHE_NAME);
+
+        assertNotNull(cc);
+
+        lockVersion(srv);
+
+        putData(srv, false);
+
+        checkDataState(srv, false);
+
+        File cacheWorkDir = 
((FilePageStoreManager)cc.context().shared().pageStore()).cacheWorkDir(cc.configuration());
+
+        File idxPath = cacheWorkDir.toPath().resolve("index.bin").toFile();
+
+        stopAllGrids();
+
+        assertTrue(U.delete(idxPath));
+
+        srv = startServer();
+
+        putData(srv, true);
+
+        checkDataState(srv, true);
+    }
+
+    /**
+     * Lock coordinator version in order to keep MVCC versions in place.
+     *
+     * @param node Node.
+     * @throws IgniteCheckedException if failed.
+     */
+    private static void lockVersion(IgniteEx node) throws 
IgniteCheckedException {
+        node.context().coordinators().requestSnapshotAsync().get();
+    }
+
+    /** {@inheritDoc} */
+    protected void checkDataState(IgniteEx srv, boolean afterRebuild) throws 
IgniteCheckedException {
+        IgniteInternalCache icache = srv.cachex(CACHE_NAME);
+
+        assertNotNull(icache);
+
+        CacheObjectContext coCtx = icache.context().cacheObjectContext();
+
+        for (IgniteCacheOffheapManager.CacheDataStore store : 
icache.context().offheap().cacheDataStores()) {
+            GridCursor<? extends CacheDataRow> cur = store.cursor();
+
+            while (cur.next()) {
+                CacheDataRow row = cur.get();
+
+                int key = row.key().value(coCtx, false);
+
+                List<IgniteBiTuple<Object, MvccVersion>> vers = 
store.mvccFindAllVersions(icache.context(), row.key());
+
+                if (!afterRebuild || key <= AMOUNT / 2)
+                    assertEquals(key, vers.size());
+                else {
+                    // For keys affected by concurrent put there are two 
versions -
+                    // -1 (concurrent put mark) and newest restored value as 
long as put cleans obsolete versions.
+                    assertEquals(2, vers.size());
+
+                    Object val0 = 
((CacheObject)vers.get(0).getKey()).value(coCtx, false);
+                    Object val1 = 
((CacheObject)vers.get(1).getKey()).value(coCtx, false);
+
+                    assertEquals(-1, val0);
+                    assertEquals(key, val1);
+                }
+
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/254a6529/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
 
b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
index a8087da..ab9d2e6 100644
--- 
a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccSqlTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import 
org.apache.ignite.internal.processors.cache.index.SqlTransactionsCommandsWithMvccEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccBulkLoadTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccDmlSimpleTest;
 import 
org.apache.ignite.internal.processors.cache.mvcc.CacheMvccIteratorWithConcurrentJdbcTransactionTest;
@@ -37,6 +38,7 @@ import 
org.apache.ignite.internal.processors.cache.mvcc.CacheMvccReplicatedSqlTx
 import 
org.apache.ignite.internal.processors.cache.mvcc.CacheMvccScanQueryWithConcurrentJdbcTransactionTest;
 import 
org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentJdbcTransactionTest;
 import 
org.apache.ignite.internal.processors.cache.mvcc.CacheMvccStreamingInsertTest;
+import 
org.apache.ignite.internal.processors.query.h2.GridIndexRebuildWithMvccEnabledSelfTest;
 
 /**
  *
@@ -48,25 +50,39 @@ public class IgniteCacheMvccSqlTestSuite extends TestSuite {
     public static TestSuite suite() {
         TestSuite suite = new TestSuite("IgniteCache SQL MVCC Test Suite");
 
+        // Simle tests.
+        suite.addTestSuite(CacheMvccDmlSimpleTest.class);
+        
suite.addTestSuite(SqlTransactionsCommandsWithMvccEnabledSelfTest.class);
+
+        suite.addTestSuite(GridIndexRebuildWithMvccEnabledSelfTest.class);
+
+        // JDBC tests.
         
suite.addTestSuite(CacheMvccSizeWithConcurrentJdbcTransactionTest.class);
         
suite.addTestSuite(CacheMvccScanQueryWithConcurrentJdbcTransactionTest.class);
         
suite.addTestSuite(CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest.class);
         
suite.addTestSuite(CacheMvccIteratorWithConcurrentJdbcTransactionTest.class);
+
+        // Load tests.
+        suite.addTestSuite(CacheMvccBulkLoadTest.class);
+        suite.addTestSuite(CacheMvccStreamingInsertTest.class);
+
         suite.addTestSuite(CacheMvccPartitionedSqlQueriesTest.class);
         suite.addTestSuite(CacheMvccReplicatedSqlQueriesTest.class);
         suite.addTestSuite(CacheMvccPartitionedSqlTxQueriesTest.class);
         suite.addTestSuite(CacheMvccReplicatedSqlTxQueriesTest.class);
+
         
suite.addTestSuite(CacheMvccPartitionedSqlTxQueriesWithReducerTest.class);
         
suite.addTestSuite(CacheMvccReplicatedSqlTxQueriesWithReducerTest.class);
         suite.addTestSuite(CacheMvccPartitionedSelectForUpdateQueryTest.class);
         suite.addTestSuite(CacheMvccReplicatedSelectForUpdateQueryTest.class);
+
+        // Failover tests.
         suite.addTestSuite(CacheMvccPartitionedBackupsTest.class);
         suite.addTestSuite(CacheMvccReplicatedBackupsTest.class);
+
         
suite.addTestSuite(CacheMvccPartitionedSqlCoordinatorFailoverTest.class);
         
suite.addTestSuite(CacheMvccReplicatedSqlCoordinatorFailoverTest.class);
-        suite.addTestSuite(CacheMvccBulkLoadTest.class);
-        suite.addTestSuite(CacheMvccStreamingInsertTest.class);
-        suite.addTestSuite(CacheMvccDmlSimpleTest.class);
+
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/254a6529/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 b0670e8..671db13 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
@@ -110,7 +110,6 @@ import 
org.apache.ignite.internal.processors.cache.index.DynamicIndexServerCoord
 import 
org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFIlterBasicSelfTest;
 import 
org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFilterCoordinatorBasicSelfTest;
 import 
org.apache.ignite.internal.processors.cache.index.H2ConnectionLeaksSelfTest;
-import org.apache.ignite.internal.processors.cache.index.IgniteDecimalSelfTest;
 import 
org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsClientBasicSelfTest;
 import 
org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerBasicSelfTest;
 import 
org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerCoordinatorBasicSelfTest;
@@ -131,11 +130,11 @@ import 
org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComple
 import 
org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest;
 import 
org.apache.ignite.internal.processors.cache.index.H2RowCachePageEvictionTest;
 import org.apache.ignite.internal.processors.cache.index.H2RowCacheSelfTest;
+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.SqlTransactionsComandsWithMvccDisabledSelfTest;
-import 
org.apache.ignite.internal.processors.cache.index.SqlTransactionsSelfTest;
+import 
org.apache.ignite.internal.processors.cache.index.SqlTransactionsComandsSelfTest;
 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;
@@ -165,18 +164,18 @@ import 
org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
 import org.apache.ignite.internal.processors.query.LazyQuerySelfTest;
 import 
org.apache.ignite.internal.processors.query.MultipleStatementsSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.query.SqlIllegalSchemaSelfTest;
-import org.apache.ignite.internal.processors.query.SqlSystemViewsSelfTest;
 import org.apache.ignite.internal.processors.query.SqlPushDownFunctionTest;
 import org.apache.ignite.internal.processors.query.SqlSchemaSelfTest;
+import org.apache.ignite.internal.processors.query.SqlSystemViewsSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildSelfTest;
-import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.H2ResultSetIteratorNullifyOnEndSelfTest;
+import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest;
-import 
org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.PreparedStatementExSelfTest;
+import 
org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
 import 
org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest;
@@ -447,8 +446,7 @@ public class IgniteCacheQuerySelfTestSuite extends 
TestSuite {
 
         suite.addTestSuite(GridIndexRebuildSelfTest.class);
 
-        suite.addTestSuite(SqlTransactionsSelfTest.class);
-        
suite.addTestSuite(SqlTransactionsComandsWithMvccDisabledSelfTest.class);
+        suite.addTestSuite(SqlTransactionsComandsSelfTest.class);
 
         suite.addTestSuite(IgniteSqlDefaultValueTest.class);
         suite.addTestSuite(IgniteDecimalSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/254a6529/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
index 1fd8e72..1f600dd 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
@@ -80,7 +80,10 @@ namespace Apache.Ignite.Core.Tests.ApiParity
             "TimeServerPortBase",
             "TimeServerPortRange",
             "IncludeProperties",
-            "isAutoActivationEnabled"  // IGNITE-7301
+            "isAutoActivationEnabled",  // IGNITE-7301
+            "isMvccEnabled", //TODO: IGNITE-9390: Remove when Mvcc support 
will be added.
+            "MvccVacuumTimeInterval", //TODO: IGNITE-9390: Remove when Mvcc 
support will be added.
+            "MvccVacuumThreadCnt" //TODO: IGNITE-9390: Remove when Mvcc 
support will be added.
         };
 
         /// <summary>

Reply via email to