You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2018/10/15 14:42:53 UTC

[1/5] ignite git commit: IGNITE-7953: MVCC: Continuous queries support. This closes #4767.

Repository: ignite
Updated Branches:
  refs/heads/master 7e547b139 -> 51a202a4c


http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
index 08511d9..dbcea2b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
@@ -22,30 +22,25 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinu
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchForceServerModeAckTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryAsyncFilterListenerTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryConcurrentPartitionUpdateTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedAtomicTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedTxTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedAtomicTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedTxTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEventBufferTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryExecuteInPrimaryTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFactoryAsyncFilterRandomOperationTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFactoryFilterRandomOperationTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicNearEnabledSelfSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryLostPartitionTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryOperationFromCallbackTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryOperationP2PTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryOrderingEventTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTwoNodesTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerFailoverTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerClientSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerFailoverTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerPartitionedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerRandomOperationsTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationNearEnabledTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationStoreEnabledTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.ClientReconnectContinuousQueryTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryMarshallerTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryPeerClassLoadingTest;
@@ -75,7 +70,6 @@ import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheC
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryImmutableEntryTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryNoUnsubscribeTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryReconnectTest;
 
 /**
  * Test suite for cache queries.
@@ -101,16 +95,20 @@ public class IgniteCacheQuerySelfTestSuite3 extends TestSuite {
         suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
+
         suite.addTestSuite(GridCacheContinuousQueryReplicatedTxOneNodeTest.class);
         suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicOneNodeTest.class);
         suite.addTestSuite(GridCacheContinuousQueryPartitionTxOneNodeTest.class);
         suite.addTestSuite(GridCacheContinuousQueryPartitionAtomicOneNodeTest.class);
+
         suite.addTestSuite(IgniteCacheContinuousQueryClientTest.class);
         suite.addTestSuite(IgniteCacheContinuousQueryClientReconnectTest.class);
         suite.addTestSuite(IgniteCacheContinuousQueryClientTxReconnectTest.class);
+
         suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
         suite.addTestSuite(CacheContinuousQueryRandomOperationsTwoNodesTest.class);
         suite.addTestSuite(GridCacheContinuousQueryConcurrentTest.class);
+
         suite.addTestSuite(CacheContinuousQueryAsyncFilterListenerTest.class);
         suite.addTestSuite(CacheContinuousQueryFactoryFilterRandomOperationTest.class);
         suite.addTestSuite(CacheContinuousQueryFactoryAsyncFilterRandomOperationTest.class);
@@ -132,7 +130,6 @@ public class IgniteCacheQuerySelfTestSuite3 extends TestSuite {
         suite.addTestSuite(IgniteCacheContinuousQueryNoUnsubscribeTest.class);
         suite.addTestSuite(ClientReconnectContinuousQueryTest.class);
         suite.addTestSuite(ContinuousQueryPeerClassLoadingTest.class);
-        suite.addTestSuite(ClientReconnectContinuousQueryTest.class);
         suite.addTestSuite(ContinuousQueryMarshallerTest.class);
 
         suite.addTestSuite(CacheContinuousQueryConcurrentPartitionUpdateTest.class);


[2/5] ignite git commit: IGNITE-7953: MVCC: Continuous queries support. This closes #4767.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationTest.java
index ce45570..842b6f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationTest.java
@@ -40,6 +40,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
@@ -104,6 +105,18 @@ public class CacheKeepBinaryIterationTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testMvccTxOnHeap() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,1, TRANSACTIONAL_SNAPSHOT);
+
+        doTestScanQuery(ccfg, true, true);
+        doTestScanQuery(ccfg, true, false);
+        doTestScanQuery(ccfg, false, true);
+        doTestScanQuery(ccfg, false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testAtomicOnHeapLocalEntries() throws Exception {
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC);
 
@@ -126,6 +139,19 @@ public class CacheKeepBinaryIterationTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxOnHeapLocalEntries() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL_SNAPSHOT);
+
+        doTestLocalEntries(ccfg, true, true);
+        doTestLocalEntries(ccfg, true, false);
+        doTestLocalEntries(ccfg, false, true);
+        doTestLocalEntries(ccfg, false, false);
+    }
+
+
+    /**
      * @param ccfg Cache configuration.
      */
     private void doTestScanQuery(final CacheConfiguration<Object, Object> ccfg, boolean keepBinary,

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
index 9b531c6..9ec25d3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java
@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit;
 import javax.cache.event.CacheEntryListenerException;
 import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -77,6 +78,12 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
         else {
             CacheConfiguration ccfg = defaultCacheConfiguration();
 
+            ccfg.setAtomicityMode(atomicityMode());
+
+            // TODO IGNITE-9530 Remove this clause.
+            if (atomicityMode() == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT)
+                ccfg.setNearConfiguration(null);
+
             cfg.setCacheConfiguration(ccfg);
         }
 
@@ -84,6 +91,13 @@ public class ClientReconnectContinuousQueryTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @return Transaction snapshot.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /**
      * Test client reconnect to alive grid.
      *
      * @throws Exception If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 0ace0ba..707ef4a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -78,6 +78,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -112,13 +113,18 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
 
             cacheCfg.setCacheMode(cacheMode());
             cacheCfg.setAtomicityMode(atomicityMode());
-            cacheCfg.setNearConfiguration(nearConfiguration());
+            cacheCfg.setLoadPreviousValue(true);
             cacheCfg.setRebalanceMode(ASYNC);
             cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-            cacheCfg.setCacheStoreFactory(new StoreFactory());
-            cacheCfg.setReadThrough(true);
-            cacheCfg.setWriteThrough(true);
-            cacheCfg.setLoadPreviousValue(true);
+            cacheCfg.setNearConfiguration(nearConfiguration());
+
+            if (atomicityMode() != TRANSACTIONAL_SNAPSHOT) {
+                cacheCfg.setCacheStoreFactory(new StoreFactory()); // TODO IGNITE-8582 enable for tx snapshot.
+                cacheCfg.setReadThrough(true); // TODO IGNITE-8582 enable for tx snapshot.
+                cacheCfg.setWriteThrough(true); // TODO IGNITE-8582 enable for tx snapshot.
+            }
+            else
+                cacheCfg.setIndexedTypes(Integer.class, Integer.class);
 
             cfg.setCacheConfiguration(cacheCfg);
         }
@@ -240,6 +246,23 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
     protected abstract int gridCount();
 
     /**
+     * @param cache Cache.
+     * @param key Key.
+     * @param val Value.
+     */
+    protected void cachePut(IgniteCache cache, Integer key, Integer val) {
+        cache.put(key, val);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     */
+    protected void cacheRemove(IgniteCache cache, Integer key) {
+        cache.remove(key);
+    }
+
+    /**
      * @throws Exception If failed.
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
@@ -314,13 +337,13 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         });
 
         try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
-            cache.put(1, 1);
-            cache.put(2, 2);
-            cache.put(3, 3);
+            cachePut(cache,1, 1);
+            cachePut(cache,2, 2);
+            cachePut(cache,3, 3);
 
-            cache.remove(2);
+            cacheRemove(cache, 2);
 
-            cache.put(1, 10);
+            cachePut(cache, 1, 10);
 
             assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
 
@@ -370,7 +393,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
 
         try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
             for (int i = 0; i < 100; i++)
-                cache.put(i, i);
+                cachePut(cache, i, i);
         }
     }
 
@@ -409,13 +432,13 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             for (int i = 0; i < gridCount(); i++) {
                 IgniteCache<Object, Object> cache0 = grid(i).cache(DEFAULT_CACHE_NAME);
 
-                cache0.put(1, 1);
-                cache0.put(2, 2);
-                cache0.put(3, 3);
+                cachePut(cache0, 1, 1);
+                cachePut(cache0, 2, 2);
+                cachePut(cache0, 3, 3);
 
-                cache0.remove(1);
-                cache0.remove(2);
-                cache0.remove(3);
+                cacheRemove(cache0, 1);
+                cacheRemove(cache0, 2);
+                cacheRemove(cache0, 3);
 
                 final int iter = i + 1;
 
@@ -467,7 +490,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         final int keyCnt = parts * 2;
 
         for (int i = 0; i < parts / 2; i++)
-            cache.put(i, i);
+            cachePut(cache, i, i);
 
         for (int i = 0; i < 10; i++) {
             if (i % 2 == 0) {
@@ -486,7 +509,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
                 QueryCursor<Cache.Entry<Integer, Integer>> qryCur = cache.query(qry);
 
                 for (int key = 0; key < keyCnt; key++)
-                    cache.put(key, key);
+                    cachePut(cache, key, key);
 
                 try {
                     assert GridTestUtils.waitForCondition(new PA() {
@@ -501,7 +524,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             }
             else {
                 for (int key = 0; key < keyCnt; key++)
-                    cache.put(key, key);
+                    cachePut(cache, key, key);
             }
         }
     }
@@ -544,16 +567,16 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         });
 
         try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
-            cache.put(1, 1);
-            cache.put(2, 2);
-            cache.put(3, 3);
-            cache.put(4, 4);
+            cachePut(cache, 1, 1);
+            cachePut(cache, 2, 2);
+            cachePut(cache, 3, 3);
+            cachePut(cache, 4, 4);
 
-            cache.remove(2);
-            cache.remove(3);
+            cacheRemove(cache, 2);
+            cacheRemove(cache, 3);
 
-            cache.put(1, 10);
-            cache.put(4, 40);
+            cachePut(cache, 1, 10);
+            cachePut(cache, 4, 40);
 
             assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
 
@@ -631,8 +654,8 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
                     break;
             }
 
-            cache.put(locKey, 1);
-            cache.put(rmtKey, 2);
+            cachePut(cache, locKey, 1);
+            cachePut(cache, rmtKey, 2);
 
             assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
 
@@ -706,12 +729,12 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             Iterator<Integer> it = keys.iterator();
 
             for (int i = 0; i < 4; i++)
-                cache.put(it.next(), 0);
+                cachePut(cache, it.next(), 0);
 
             assert !latch.await(2, SECONDS);
 
             for (int i = 0; i < 2; i++)
-                cache.put(it.next(), 0);
+                cachePut(cache, it.next(), 0);
 
             assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
 
@@ -790,7 +813,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             }
 
             for (Integer k : keys)
-                cache.put(k, 0);
+                cachePut(cache, k, 0);
 
             assert !latch.await(2, SECONDS);
             assert latch.await(1000 + LATCH_TIMEOUT, MILLISECONDS);
@@ -832,7 +855,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         });
 
         for (int i = 0; i < 10; i++)
-            cache.put(i, i);
+            cachePut(cache, i, i);
 
         try (QueryCursor<Cache.Entry<Integer, Integer>> cur = cache.query(qry)) {
             List<Cache.Entry<Integer, Integer>> res = cur.getAll();
@@ -884,7 +907,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         });
 
         for (int i = 0; i < 10; i++)
-            cache.put(i, i);
+            cachePut(cache, i, i);
 
         try (QueryCursor<Cache.Entry<Integer, Integer>> cur = cache.query(qry)) {
             List<Cache.Entry<Integer, Integer>> res = cur.getAll();
@@ -906,8 +929,8 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
                 exp++;
             }
 
-            cache.put(10, 10);
-            cache.put(11, 11);
+            cachePut(cache, 10, 10);
+            cachePut(cache, 11, 11);
 
             assert latch.await(LATCH_TIMEOUT, MILLISECONDS) : latch.getCount();
 
@@ -978,8 +1001,8 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         try (QueryCursor<Cache.Entry<Object, Object>> ignored = cache.query(qry)) {
             cache.put(new GridCacheInternalKeyImpl("test", "test"), 1);
 
-            cache.put(1, 1);
-            cache.put(2, 2);
+            cachePut(cache, 1, 1);
+            cachePut(cache, 2, 2);
 
             assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
 
@@ -1014,7 +1037,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
                 log.info("Started node without cache: " + ignite);
             }
 
-            cache.put(1, 1);
+            cachePut(cache, 1, 1);
 
             assertTrue(latch.await(5000, MILLISECONDS));
         }
@@ -1102,7 +1125,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
 
             try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
                 for (int i = 0; i < 100; i++)
-                    cache.put(i, i);
+                    cachePut(cache, i, i);
 
                 assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
                 assert execLatch.await(LATCH_TIMEOUT, MILLISECONDS);
@@ -1147,8 +1170,8 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         });
 
         try (QueryCursor<Cache.Entry<Object, Object>> ignored = cache.query(qry)) {
-            cache.put(1, 1);
-            cache.put(2, 2);
+            cachePut(cache, 1, 1);
+            cachePut(cache, 2, 2);
 
             // Wait for expiration.
             Thread.sleep(2000);

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java
index 0241a69..9dca5ea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryConcurrentTest.java
@@ -40,6 +40,7 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
@@ -106,6 +107,13 @@ public class GridCacheContinuousQueryConcurrentTest extends GridCommonAbstractTe
     /**
      * @throws Exception If failed.
      */
+    public void testReplicatedMvccTx() throws Exception {
+        testRegistration(cacheConfiguration(CacheMode.REPLICATED, CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, 1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testRestartReplicated() throws Exception {
         testRestartRegistration(cacheConfiguration(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, 2));
     }
@@ -127,6 +135,13 @@ public class GridCacheContinuousQueryConcurrentTest extends GridCommonAbstractTe
     /**
      * @throws Exception If failed.
      */
+    public void testRestartPartitionMvccTx() throws Exception {
+        testRestartRegistration(cacheConfiguration(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, 2));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testReplicatedAtomic() throws Exception {
         testRegistration(cacheConfiguration(CacheMode.REPLICATED, CacheAtomicityMode.ATOMIC, 2));
     }
@@ -141,6 +156,13 @@ public class GridCacheContinuousQueryConcurrentTest extends GridCommonAbstractTe
     /**
      * @throws Exception If failed.
      */
+    public void testPartitionMvccTx() throws Exception {
+        testRegistration(cacheConfiguration(CacheMode.PARTITIONED, CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT, 2));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testPartitionAtomic() throws Exception {
         testRegistration(cacheConfiguration(CacheMode.PARTITIONED, CacheAtomicityMode.ATOMIC, 2));
     }
@@ -342,17 +364,44 @@ public class GridCacheContinuousQueryConcurrentTest extends GridCommonAbstractTe
         // were busy setting up the cache listener.
         // Check asynchronously.
         // Complete the promise if the key was inserted concurrently.
-        cache.getAsync(key).listen(new IgniteInClosure<IgniteFuture<String>>() {
-            @Override public void apply(IgniteFuture<String> f) {
-                String val = f.get();
+        if (!((IgniteCacheProxy)cache).context().mvccEnabled()) {
+            cache.getAsync(key).listen(new IgniteInClosure<IgniteFuture<String>>() {
+                @Override public void apply(IgniteFuture<String> f) {
+                    String val = f.get();
 
-                if (val != null) {
-                    log.info("Completed by get: " + id);
+                    if (val != null) {
+                        log.info("Completed by get: " + id);
 
-                    (((GridFutureAdapter)((IgniteFutureImpl)promise).internalFuture())).onDone("by get");
+                        (((GridFutureAdapter)((IgniteFutureImpl)promise).internalFuture())).onDone("by async get");
+                    }
                 }
-            }
-        });
+            });
+        }
+        else {
+            // For MVCC caches we need to wait until updated value becomes visible for consequent readers.
+            // When MVCC transaction completes, it's updates are not visible immediately for the new transactions.
+            // This is caused by the lag between transaction completes on the node and mvcc coordinator
+            // removes this transaction from the active list.
+            GridTestUtils.runAsync(new Runnable() {
+                @Override public void run() {
+                    String v;
+
+                    while (!Thread.currentThread().isInterrupted()) {
+                        v = cache.get(key);
+
+                        if (v == null)
+                            doSleep(100);
+                        else {
+                            log.info("Completed by async mvcc get: " + id);
+
+                            (((GridFutureAdapter)((IgniteFutureImpl)promise).internalFuture())).onDone("by get");
+
+                            break;
+                        }
+                    }
+                }
+            });
+        }
 
         return promise;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java
index b316042..0eb2e87 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java
@@ -40,6 +40,7 @@ import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cluster.ClusterNode;
@@ -318,11 +319,18 @@ public class GridCacheContinuousQueryMultiNodesFilteringTest extends GridCommonA
         return new CacheConfiguration("test-cache-cq")
             .setBackups(1)
             .setNodeFilter(filter)
-            .setAtomicityMode(ATOMIC)
+            .setAtomicityMode(atomicityMode())
             .setWriteSynchronizationMode(FULL_SYNC)
             .setCacheMode(PARTITIONED);
     }
 
+    /**
+     * @return Atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
     /** */
     private final static class ListenerConfiguration extends MutableCacheEntryListenerConfiguration {
         /** Operation. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java
index 6474df5..d02c6ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryReplicatedTxOneNodeTest.java
@@ -17,9 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache.query.continuous;
 
+import java.util.Iterator;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
 import javax.cache.event.CacheEntryEvent;
 import javax.cache.event.CacheEntryListenerException;
 import javax.cache.event.CacheEntryUpdatedListener;
@@ -55,6 +57,10 @@ public class GridCacheContinuousQueryReplicatedTxOneNodeTest extends GridCommonA
         cacheCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
+        // TODO IGNITE-9530 Remove this clause.
+        if (atomicMode() == CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT)
+            cacheCfg.setNearConfiguration(null);
+
         cfg.setCacheConfiguration(cacheCfg);
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
@@ -164,7 +170,14 @@ public class GridCacheContinuousQueryReplicatedTxOneNodeTest extends GridCommonA
             for (int i = 0; i < 10; i++)
                 cache.put("key" + i, i);
 
-            cache.clear();
+            if (atomicMode() != CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT)
+                cache.clear();
+            else { // TODO IGNITE-7952. Remove "else" clause - do cache.clear() instead of iteration.
+                for (Iterator it = cache.iterator(); it.hasNext();) {
+                    it.next();
+                    it.remove();
+                }
+            }
 
             qry.setLocalListener(new CacheEntryUpdatedListener<String, Integer>() {
                 @Override public void onUpdated(Iterable<CacheEntryEvent<? extends String, ? extends Integer>> evts)

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java
index 5baa3a7..a8f0d71 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java
@@ -30,6 +30,7 @@ import javax.cache.event.CacheEntryEvent;
 import javax.cache.event.CacheEntryEventFilter;
 import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -98,6 +99,13 @@ public class IgniteCacheContinuousQueryBackupQueueTest extends GridCommonAbstrac
         return cfg;
     }
 
+    /**
+     * @return Atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientTest.java
index 1e40170..ab0cb50 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -61,7 +62,7 @@ public class IgniteCacheContinuousQueryClientTest extends GridCommonAbstractTest
         CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
 
         ccfg.setCacheMode(PARTITIONED);
-        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setAtomicityMode(atomicityMode());
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
         cfg.setCacheConfiguration(ccfg);
@@ -71,6 +72,13 @@ public class IgniteCacheContinuousQueryClientTest extends GridCommonAbstractTest
         return cfg;
     }
 
+    /**
+     * @return Atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
index 81a7515..4e50cb9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java
@@ -25,6 +25,7 @@ import javax.cache.event.CacheEntryEvent;
 import javax.cache.event.CacheEntryEventFilter;
 import javax.cache.event.CacheEntryUpdatedListener;
 import javax.cache.event.EventType;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -66,7 +67,7 @@ public class IgniteCacheContinuousQueryImmutableEntryTest extends GridCommonAbst
 
         CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
         ccfg.setCacheMode(PARTITIONED);
-        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setAtomicityMode(atomicityMode());
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
         cfg.setCacheConfiguration(ccfg);
@@ -76,6 +77,13 @@ public class IgniteCacheContinuousQueryImmutableEntryTest extends GridCommonAbst
         return cfg;
     }
 
+    /**
+     * @return Atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java
index d9804bf..74f80df 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListImplSelfTest.java
@@ -480,11 +480,6 @@ public class CacheFreeListImplSelfTest extends GridCommonAbstractTest {
         @Override public byte newMvccTxState() {
             return 0;
         }
-
-        /** {@inheritDoc} */
-        @Override public boolean isKeyAbsentBefore() {
-            return false;
-        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 344a1cc..c8cf9aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -81,6 +81,7 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
                 new GridCacheSharedTtlCleanupManager(),
                 new PartitionsEvictManager(),
                 new CacheNoopJtaManager(),
+                null,
                 null
             ),
             defaultCacheConfiguration(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index 6246aa5..7fb896f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -127,11 +127,6 @@ public abstract class GridH2Row extends GridH2SearchRowAdapter implements CacheD
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isKeyAbsentBefore() {
-        return row.isKeyAbsentBefore();
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean indexSearchRow() {
         return false;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractContinuousQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractContinuousQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractContinuousQuerySelfTest.java
new file mode 100644
index 0000000..1418d47
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractContinuousQuerySelfTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+
+/**
+ *
+ */
+public abstract class CacheMvccAbstractContinuousQuerySelfTest extends GridCacheContinuousQueryAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testInternalKey() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    public void testExpired() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7311");
+    }
+
+    /** {@inheritDoc} */
+    public void testLoadCache() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7954");
+    }
+
+    /** {@inheritDoc} */
+    public void testEvents() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9321");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlContinuousQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlContinuousQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlContinuousQuerySelfTest.java
new file mode 100644
index 0000000..96fdf06
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractSqlContinuousQuerySelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+
+/**
+ * Base class for MVCC continuous queries.
+ */
+public abstract class CacheMvccAbstractSqlContinuousQuerySelfTest extends CacheMvccAbstractContinuousQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected void cachePut(IgniteCache cache, Integer key, Integer val) {
+        cache.query(new SqlFieldsQuery("MERGE INTO Integer (_key, _val) values (" + key + ',' + val + ')')).getAll();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void cacheRemove(IgniteCache  cache, Integer key) {
+        cache.query(new SqlFieldsQuery("DELETE FROM Integer WHERE _key=" + key)).getAll();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java
new file mode 100644
index 0000000..ed97b1b
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccBasicContinuousQueryTest.java
@@ -0,0 +1,230 @@
+/*
+ * 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.mvcc;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryUpdatedListener;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
+import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.PA;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.transactions.Transaction;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.processors.cache.mvcc.MvccCachingManager.TX_SIZE_THRESHOLD;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Basic continuous queries test with enabled mvcc.
+ */
+public class CacheMvccBasicContinuousQueryTest extends CacheMvccAbstractTest  {
+    /** */
+    private static final long LATCH_TIMEOUT = 5000;
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        // Wait for all routines are unregistered
+        GridTestUtils.waitForCondition(new PA() {
+            @Override public boolean apply() {
+                for (Ignite node : G.allGrids()) {
+                    GridContinuousProcessor proc = ((IgniteEx)node).context().continuous();
+
+                    if(((Map)U.field(proc, "rmtInfos")).size() > 0)
+                        return false;
+                }
+
+                return true;
+            }
+        }, 3000);
+
+        for (Ignite node : G.allGrids()) {
+            GridContinuousProcessor proc = ((IgniteEx)node).context().continuous();
+
+            assertEquals(1, ((Map)U.field(proc, "locInfos")).size());
+            assertEquals(0, ((Map)U.field(proc, "rmtInfos")).size());
+            assertEquals(0, ((Map)U.field(proc, "startFuts")).size());
+            assertEquals(0, ((Map)U.field(proc, "stopFuts")).size());
+            assertEquals(0, ((Map)U.field(proc, "bufCheckThreads")).size());
+
+            CacheContinuousQueryManager mgr = ((IgniteEx)node).context().cache().internalCache(DEFAULT_CACHE_NAME).context().continuousQueries();
+
+            assertEquals(0, ((Map)U.field(mgr, "lsnrs")).size());
+
+            MvccCachingManager cachingMgr = ((IgniteEx)node).context().cache().context().mvccCaching();
+
+            assertEquals(0, ((Map)U.field(cachingMgr, "enlistCache")).size());
+            assertEquals(0, ((Map)U.field(cachingMgr, "cntrs")).size());
+        }
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAllEntries() throws Exception {
+        Ignite node = startGrids(3);
+
+        final IgniteCache cache = node.createCache(
+            cacheConfiguration(cacheMode(), FULL_SYNC, 1, 2)
+                .setCacheMode(CacheMode.REPLICATED)
+                .setIndexedTypes(Integer.class, Integer.class));
+
+        ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+        final Map<Integer, List<Integer>> map = new HashMap<>();
+        final CountDownLatch latch = new CountDownLatch(5);
+
+        qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                for (CacheEntryEvent<? extends Integer, ? extends Integer> e : evts) {
+                    synchronized (map) {
+                        List<Integer> vals = map.get(e.getKey());
+
+                        if (vals == null) {
+                            vals = new ArrayList<>();
+
+                            map.put(e.getKey(), vals);
+                        }
+
+                        vals.add(e.getValue());
+                    }
+
+                    latch.countDown();
+                }
+            }
+        });
+
+        try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
+
+            try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                String dml = "INSERT INTO Integer (_key, _val) values (1,1),(2,2)";
+
+                cache.query(new SqlFieldsQuery(dml)).getAll();
+
+                tx.commit();
+            }
+
+            try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                String dml1 = "MERGE INTO Integer (_key, _val) values (3,3)";
+
+                cache.query(new SqlFieldsQuery(dml1)).getAll();
+
+                String dml2 = "DELETE FROM Integer WHERE _key = 2";
+
+                cache.query(new SqlFieldsQuery(dml2)).getAll();
+
+                String dml3 = "UPDATE Integer SET _val = 10 WHERE _key = 1";
+
+                cache.query(new SqlFieldsQuery(dml3)).getAll();
+
+                tx.commit();
+            }
+
+            try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                String dml = "INSERT INTO Integer (_key, _val) values (4,4),(5,5)";
+
+                cache.query(new SqlFieldsQuery(dml)).getAll();
+
+                tx.rollback();
+            }
+
+            assert latch.await(LATCH_TIMEOUT, MILLISECONDS);
+
+            assertEquals(3, map.size());
+
+            List<Integer> vals = map.get(1);
+
+            assertNotNull(vals);
+            assertEquals(2, vals.size());
+            assertEquals(1, (int)vals.get(0));
+            assertEquals(10, (int)vals.get(1));
+
+            vals = map.get(2);
+
+            assertNotNull(vals);
+            assertEquals(2, vals.size());
+            assertEquals(2, (int)vals.get(0));
+            assertEquals(2, (int)vals.get(1));
+
+            vals = map.get(3);
+
+            assertNotNull(vals);
+            assertEquals(1, vals.size());
+            assertEquals(3, (int)vals.get(0));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCachingMaxSize() throws Exception {
+        Ignite node = startGrids(1);
+
+        final IgniteCache cache = node.createCache(
+            cacheConfiguration(cacheMode(), FULL_SYNC, 1, 2)
+                .setCacheMode(CacheMode.PARTITIONED)
+                .setIndexedTypes(Integer.class, Integer.class));
+
+        ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
+
+        qry.setLocalListener(new CacheEntryUpdatedListener<Integer, Integer>() {
+            @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends Integer>> evts) {
+                // No-op.
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = cache.query(qry)) {
+                    try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                        for (int i = 0; i < TX_SIZE_THRESHOLD + 1; i++)
+                            cache.query(new SqlFieldsQuery("INSERT INTO Integer (_key, _val) values (" + i + ", 1)")).getAll();
+
+                        tx.commit();
+                    }
+                }
+
+                return null;
+            }
+        },  CacheException.class, "Failed to run update. Transaction is too large. Consider reducing transaction size");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClientReconnectContinuousQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClientReconnectContinuousQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClientReconnectContinuousQueryTest.java
new file mode 100644
index 0000000..33e0960
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClientReconnectContinuousQueryTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.ClientReconnectContinuousQueryTest;
+
+/**
+ *
+ */
+public class CacheMvccClientReconnectContinuousQueryTest extends ClientReconnectContinuousQueryTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryBackupQueueTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryBackupQueueTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryBackupQueueTest.java
new file mode 100644
index 0000000..3a598a2
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryBackupQueueTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryBackupQueueTest;
+
+/**
+ *
+ */
+public class CacheMvccContinuousQueryBackupQueueTest extends IgniteCacheContinuousQueryBackupQueueTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientReconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientReconnectTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientReconnectTest.java
new file mode 100644
index 0000000..4c4c95b
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientReconnectTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+
+/**
+ * Mvcc CQ client reconnect test.
+ */
+public class CacheMvccContinuousQueryClientReconnectTest  extends IgniteCacheContinuousQueryClientReconnectTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicMode() {
+        return TRANSACTIONAL_SNAPSHOT;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientTest.java
new file mode 100644
index 0000000..5c6c7a8
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryClientTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
+
+/**
+ * Mvcc CQ client test.
+ */
+public class CacheMvccContinuousQueryClientTest extends IgniteCacheContinuousQueryClientTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryImmutableEntryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryImmutableEntryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryImmutableEntryTest.java
new file mode 100644
index 0000000..bef9c70
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryImmutableEntryTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryImmutableEntryTest;
+
+/**
+ *
+ */
+public class CacheMvccContinuousQueryImmutableEntryTest extends IgniteCacheContinuousQueryImmutableEntryTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryMultiNodesFilteringTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryMultiNodesFilteringTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryMultiNodesFilteringTest.java
new file mode 100644
index 0000000..714e834
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryMultiNodesFilteringTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryMultiNodesFilteringTest;
+
+/**
+ *
+ */
+public class CacheMvccContinuousQueryMultiNodesFilteringTest extends GridCacheContinuousQueryMultiNodesFilteringTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedSelfTest.java
new file mode 100644
index 0000000..80b039d
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Mvcc continuous query test for partitioned cache.
+ */
+public class CacheMvccContinuousQueryPartitionedSelfTest extends CacheMvccAbstractContinuousQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedTxOneNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedTxOneNodeTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedTxOneNodeTest.java
new file mode 100644
index 0000000..795932e
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryPartitionedTxOneNodeTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedTxOneNodeTest;
+
+/**
+ * Mvcc continuous query test for one node.
+ */
+public class CacheMvccContinuousQueryPartitionedTxOneNodeTest extends GridCacheContinuousQueryReplicatedTxOneNodeTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedSelfTest.java
new file mode 100644
index 0000000..c9adbf9
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Mvcc continuous query test for replicated cache.
+ */
+public class CacheMvccContinuousQueryReplicatedSelfTest extends CacheMvccAbstractContinuousQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedTxOneNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedTxOneNodeTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedTxOneNodeTest.java
new file mode 100644
index 0000000..d522ee9
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousQueryReplicatedTxOneNodeTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedTxOneNodeTest;
+
+/**
+ * Mvcc continuous query test for one node.
+ */
+public class CacheMvccContinuousQueryReplicatedTxOneNodeTest extends GridCacheContinuousQueryReplicatedTxOneNodeTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerClientSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerClientSelfTest.java
new file mode 100644
index 0000000..a3ea0e8
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerClientSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerClientSelfTest;
+
+/**
+ *
+ */
+public class CacheMvccContinuousWithTransformerClientSelfTest extends CacheContinuousWithTransformerClientSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testExpired() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7311");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerPartitionedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerPartitionedSelfTest.java
new file mode 100644
index 0000000..d029143
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerPartitionedSelfTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerReplicatedSelfTest;
+
+/**
+ *
+ */
+public class CacheMvccContinuousWithTransformerPartitionedSelfTest extends CacheContinuousWithTransformerReplicatedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testExpired() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7311");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerReplicatedSelfTest.java
new file mode 100644
index 0000000..a294e17
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccContinuousWithTransformerReplicatedSelfTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousWithTransformerReplicatedSelfTest;
+
+/**
+ *
+ */
+public class CacheMvccContinuousWithTransformerReplicatedSelfTest
+    extends CacheContinuousWithTransformerReplicatedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testExpired() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7311");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryPartitionedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryPartitionedSelfTest.java
new file mode 100644
index 0000000..cef553e
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryPartitionedSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Mvcc continuous query test for partitioned SQL cache.
+ */
+public class CacheMvccSqlContinuousQueryPartitionedSelfTest extends CacheMvccAbstractSqlContinuousQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryReplicatedSelfTest.java
new file mode 100644
index 0000000..948e6e1
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlContinuousQueryReplicatedSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.cache.CacheMode;
+
+/**
+ * Mvcc continuous query test for replicated SQL cache.
+ */
+public class CacheMvccSqlContinuousQueryReplicatedSelfTest extends CacheMvccAbstractSqlContinuousQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/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 b5cb3e0..ce2a130 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
@@ -20,7 +20,21 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.index.MvccEmptyTransactionSelfTest;
 import org.apache.ignite.internal.processors.cache.index.SqlTransactionsCommandsWithMvccEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccBasicContinuousQueryTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccBulkLoadTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccClientReconnectContinuousQueryTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryBackupQueueTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryClientReconnectTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryClientTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryImmutableEntryTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryMultiNodesFilteringTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryPartitionedTxOneNodeTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousQueryReplicatedTxOneNodeTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousWithTransformerClientSelfTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousWithTransformerPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccContinuousWithTransformerReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccDmlSimpleTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccIteratorWithConcurrentJdbcTransactionTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccLocalEntriesWithConcurrentJdbcTransactionTest;
@@ -40,8 +54,10 @@ import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccScanQueryWithCo
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSizeWithConcurrentJdbcTransactionTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlConfigurationValidationTest;
-import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlUpdateCountersTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlContinuousQueryPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlContinuousQueryReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlLockTimeoutTest;
+import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccSqlUpdateCountersTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccStreamingInsertTest;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheMvccTxNodeMappingTest;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccRepeatableReadBulkOpsTest;
@@ -102,6 +118,28 @@ public class IgniteCacheMvccSqlTestSuite extends TestSuite {
         suite.addTestSuite(CacheMvccPartitionedSqlCoordinatorFailoverTest.class);
         suite.addTestSuite(CacheMvccReplicatedSqlCoordinatorFailoverTest.class);
 
+        // Continuous queries.
+        suite.addTestSuite(CacheMvccBasicContinuousQueryTest.class);
+        suite.addTestSuite(CacheMvccContinuousQueryPartitionedSelfTest.class);
+        suite.addTestSuite(CacheMvccContinuousQueryReplicatedSelfTest.class);
+        suite.addTestSuite(CacheMvccSqlContinuousQueryPartitionedSelfTest.class);
+        suite.addTestSuite(CacheMvccSqlContinuousQueryReplicatedSelfTest.class);
+
+        suite.addTestSuite(CacheMvccContinuousQueryPartitionedTxOneNodeTest.class);
+        suite.addTestSuite(CacheMvccContinuousQueryReplicatedTxOneNodeTest.class);
+
+        suite.addTestSuite(CacheMvccContinuousQueryClientReconnectTest.class);
+        suite.addTestSuite(CacheMvccContinuousQueryClientTest.class);
+
+        suite.addTestSuite(CacheMvccContinuousQueryMultiNodesFilteringTest.class);
+        suite.addTestSuite(CacheMvccContinuousQueryBackupQueueTest.class);
+        suite.addTestSuite(CacheMvccContinuousQueryImmutableEntryTest.class);
+        suite.addTestSuite(CacheMvccClientReconnectContinuousQueryTest.class);
+
+        suite.addTestSuite(CacheMvccContinuousWithTransformerClientSelfTest.class);
+        suite.addTestSuite(CacheMvccContinuousWithTransformerPartitionedSelfTest.class);
+        suite.addTestSuite(CacheMvccContinuousWithTransformerReplicatedSelfTest.class);
+
         return suite;
     }
 }


[5/5] ignite git commit: IGNITE-7953: MVCC: Continuous queries support. This closes #4767.

Posted by vo...@apache.org.
IGNITE-7953: MVCC: Continuous queries support. This closes #4767.


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

Branch: refs/heads/master
Commit: 51a202a4c48220fa919f47147bd4889033cd35a8
Parents: 7e547b1
Author: rkondakov <ko...@mail.ru>
Authored: Mon Oct 15 17:42:38 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 15 17:42:39 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   6 +-
 .../processors/cache/CacheGroupContext.java     |   8 +
 .../processors/cache/GridCacheEntryEx.java      |  25 +-
 .../processors/cache/GridCacheMapEntry.java     | 202 ++++++-----
 .../cache/GridCacheMvccEntryInfo.java           |   5 -
 .../processors/cache/GridCacheProcessor.java    |  14 +-
 .../cache/GridCacheSharedContext.java           |  24 +-
 .../cache/GridCacheUpdateTxResult.java          |  51 +++
 .../cache/IgniteCacheOffheapManager.java        |  20 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  11 +
 .../GridDistributedTxRemoteAdapter.java         |  16 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   3 +-
 .../dht/GridDhtTxAbstractEnlistFuture.java      |   8 +
 .../cache/distributed/dht/GridDhtTxRemote.java  | 149 --------
 .../near/GridNearTxAbstractEnlistFuture.java    |   7 +-
 .../near/GridNearTxEnlistFuture.java            |   7 +-
 .../cache/distributed/near/GridNearTxLocal.java |   2 +-
 .../near/GridNearTxQueryEnlistFuture.java       |   4 +
 .../GridNearTxQueryResultsEnlistFuture.java     |   7 +-
 .../processors/cache/dr/GridCacheDrManager.java |  31 --
 .../cache/dr/GridOsCacheDrManager.java          |  13 -
 .../cache/mvcc/MvccCachingManager.java          | 341 +++++++++++++++++++
 .../cache/mvcc/MvccProcessorImpl.java           |   8 +-
 .../processors/cache/mvcc/MvccTxEntry.java      | 203 +++++++++++
 .../cache/mvcc/MvccUpdateVersionAware.java      |   5 -
 .../processors/cache/mvcc/MvccUtils.java        |  11 +
 .../cache/persistence/CacheDataRowAdapter.java  |   5 -
 .../persistence/GridCacheOffheapManager.java    |  15 +-
 .../cache/persistence/tree/io/DataPageIO.java   |  15 +-
 .../wal/reader/IgniteWalIteratorFactory.java    |   2 +-
 .../continuous/CacheContinuousQueryManager.java |  18 +-
 .../cache/transactions/IgniteTxAdapter.java     |  27 --
 .../cache/transactions/IgniteTxHandler.java     | 171 +++++++++-
 .../transactions/IgniteTxLocalAdapter.java      |  10 +-
 .../cache/tree/mvcc/data/MvccDataRow.java       |  64 ++--
 .../cache/tree/mvcc/data/MvccUpdateDataRow.java |  98 ++++--
 .../cache/tree/mvcc/data/MvccUpdateResult.java  |  16 +-
 .../continuous/GridContinuousProcessor.java     |  10 +-
 .../processors/cache/GridCacheTestEntryEx.java  |  11 +-
 .../wal/IgniteWalIteratorSwitchSegmentTest.java |   3 +-
 .../pagemem/BPlusTreePageMemoryImplTest.java    |   1 +
 .../BPlusTreeReuseListPageMemoryImplTest.java   |   1 +
 .../pagemem/IndexStoragePageMemoryImplTest.java |   1 +
 .../pagemem/PageMemoryImplNoLoadTest.java       |   1 +
 .../persistence/pagemem/PageMemoryImplTest.java |   1 +
 .../continuous/CacheContinuousBatchAckTest.java |  51 +++
 ...eContinuousQueryAsyncFilterListenerTest.java | 165 +++++++--
 ...nuousQueryConcurrentPartitionUpdateTest.java | 110 +++++-
 ...acheContinuousQueryExecuteInPrimaryTest.java |  33 ++
 ...usQueryFactoryFilterRandomOperationTest.java |  58 ++--
 .../CacheContinuousQueryLostPartitionTest.java  |  24 +-
 ...ontinuousQueryOperationFromCallbackTest.java | 205 ++++++++---
 .../CacheContinuousQueryOperationP2PTest.java   |  48 +++
 .../CacheContinuousQueryOrderingEventTest.java  | 137 ++++++--
 ...acheContinuousQueryRandomOperationsTest.java | 271 +++++++++++++--
 ...inuousWithTransformerReplicatedSelfTest.java |   9 +
 ...CacheKeepBinaryIterationNearEnabledTest.java |   9 +
 ...acheKeepBinaryIterationStoreEnabledTest.java |  10 +
 .../CacheKeepBinaryIterationTest.java           |  26 ++
 .../ClientReconnectContinuousQueryTest.java     |  14 +
 ...ridCacheContinuousQueryAbstractSelfTest.java | 109 +++---
 .../GridCacheContinuousQueryConcurrentTest.java |  65 +++-
 ...eContinuousQueryMultiNodesFilteringTest.java |  10 +-
 ...eContinuousQueryReplicatedTxOneNodeTest.java |  15 +-
 ...niteCacheContinuousQueryBackupQueueTest.java |   8 +
 .../IgniteCacheContinuousQueryClientTest.java   |  10 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |  10 +-
 .../database/CacheFreeListImplSelfTest.java     |   5 -
 .../loadtests/hashmap/GridCacheTestContext.java |   1 +
 .../processors/query/h2/opt/GridH2Row.java      |   5 -
 ...acheMvccAbstractContinuousQuerySelfTest.java |  63 ++++
 ...eMvccAbstractSqlContinuousQuerySelfTest.java |  40 +++
 .../mvcc/CacheMvccBasicContinuousQueryTest.java | 230 +++++++++++++
 ...eMvccClientReconnectContinuousQueryTest.java |  30 ++
 ...CacheMvccContinuousQueryBackupQueueTest.java |  30 ++
 ...eMvccContinuousQueryClientReconnectTest.java |  32 ++
 .../CacheMvccContinuousQueryClientTest.java     |  30 ++
 ...heMvccContinuousQueryImmutableEntryTest.java |  30 ++
 ...cContinuousQueryMultiNodesFilteringTest.java |  30 ++
 ...eMvccContinuousQueryPartitionedSelfTest.java |  29 ++
 ...ContinuousQueryPartitionedTxOneNodeTest.java |  36 ++
 ...heMvccContinuousQueryReplicatedSelfTest.java |  29 ++
 ...cContinuousQueryReplicatedTxOneNodeTest.java |  37 ++
 ...ContinuousWithTransformerClientSelfTest.java |  35 ++
 ...nuousWithTransformerPartitionedSelfTest.java |  42 +++
 ...inuousWithTransformerReplicatedSelfTest.java |  36 ++
 ...ccSqlContinuousQueryPartitionedSelfTest.java |  30 ++
 ...vccSqlContinuousQueryReplicatedSelfTest.java |  29 ++
 .../testsuites/IgniteCacheMvccSqlTestSuite.java |  40 ++-
 .../IgniteCacheQuerySelfTestSuite3.java         |  15 +-
 90 files changed, 3260 insertions(+), 672 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index d05bdb0..81b767f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -31,7 +31,6 @@ import org.apache.ignite.internal.client.GridClient;
 import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
 import org.apache.ignite.internal.util.GridLogThrottle;
-import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.stream.StreamTransformer;
 import org.jetbrains.annotations.Nullable;
 
@@ -999,6 +998,11 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT = "IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT";
 
     /**
+     * Maximum number for cached MVCC transaction updates. This caching is used for continuous query with MVCC caches.
+     */
+    public static final String IGNITE_MVCC_TX_SIZE_CACHING_THRESHOLD = "IGNITE_MVCC_TX_SIZE_CACHING_THRESHOLD";
+
+    /**
      * Try reuse memory on deactivation. Useful in case of huge page memory region size.
      */
     public static final String IGNITE_REUSE_MEMORY_ON_DEACTIVATE = "IGNITE_REUSE_MEMORY_ON_DEACTIVATE";

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
index e1b307a..a546a36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
@@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseL
 import org.apache.ignite.internal.processors.cache.query.continuous.CounterSkipContext;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.typedef.CI1;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -884,6 +885,13 @@ public class CacheGroupContext {
     }
 
     /**
+     * @return {@code True} if there is at least one cache with registered CQ exists in this group.
+     */
+    public boolean hasContinuousQueryCaches() {
+        return !F.isEmpty(contQryCaches);
+    }
+
+    /**
      * @throws IgniteCheckedException If failed.
      */
     public void start() throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index eb49c79..cfd70ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -28,9 +28,9 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.eviction.EvictableEntry;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
 import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState;
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFilter;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.util.lang.GridTuple3;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -358,8 +359,9 @@ public interface GridCacheEntryEx {
      * @param topVer Topology version.
      * @param mvccVer Mvcc version.
      * @param op Cache operation.
-     * @param needHistory Whether to collect rows created or affected by the current tx.
+     * @param needHist Whether to collect rows created or affected by the current tx.
      * @param noCreate Entry should not be created when enabled, e.g. SQL INSERT.
+     * @param needOldVal Flag if it is need to return the old value (value before current tx has been started).
      * @param filter Filter.
      * @param retVal Previous value return flag.
      * @return Tuple containing success flag and old value. If success is {@code false},
@@ -377,8 +379,9 @@ public interface GridCacheEntryEx {
         AffinityTopologyVersion topVer,
         MvccSnapshot mvccVer,
         GridCacheOperation op,
-        boolean needHistory,
+        boolean needHist,
         boolean noCreate,
+        boolean needOldVal,
         @Nullable CacheEntryPredicate filter,
         boolean retVal) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
@@ -387,7 +390,8 @@ public interface GridCacheEntryEx {
      * @param affNodeId Partitioned node iD.
      * @param topVer Topology version.
      * @param mvccVer Mvcc version.
-     * @param needHistory Whether to collect rows created or affected by the current tx.
+     * @param needHist Whether to collect rows created or affected by the current tx.
+     * @param needOldValue Flag if it is need to return the old value (value before current tx has been started).
      * @param filter Filter.
      * @param retVal Previous value return flag.
      * @return Tuple containing success flag and old value. If success is {@code false},
@@ -400,7 +404,8 @@ public interface GridCacheEntryEx {
         UUID affNodeId,
         AffinityTopologyVersion topVer,
         MvccSnapshot mvccVer,
-        boolean needHistory,
+        boolean needHist,
+        boolean needOldValue,
         @Nullable CacheEntryPredicate filter,
         boolean retVal) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
@@ -1171,8 +1176,12 @@ public interface GridCacheEntryEx {
      * @param tx Transaction.
      * @param affNodeId Affinity node id.
      * @param topVer Topology version.
+     * @param entries Entries.
      * @param op Cache operation.
-     * @param mvccVer Mvcc version.  @return Update result.
+     * @param mvccVer Mvcc version.
+     * @param futId Future id.
+     * @param batchNum Batch number.
+     * @return Update result.
      * @throws IgniteCheckedException, If failed.
      * @throws GridCacheEntryRemovedException, If entry has been removed.
      */
@@ -1182,7 +1191,9 @@ public interface GridCacheEntryEx {
         AffinityTopologyVersion topVer,
         List<GridCacheEntryInfo> entries,
         GridCacheOperation op,
-        MvccSnapshot mvccVer)
+        MvccSnapshot mvccVer,
+        IgniteUuid futId,
+        int batchNum)
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index d417bd4..9bb8aec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -45,9 +45,9 @@ import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult.UpdateOutcome;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
 import org.apache.ignite.internal.processors.cache.extras.GridCacheEntryExtras;
 import org.apache.ignite.internal.processors.cache.extras.GridCacheMvccEntryExtras;
@@ -67,7 +67,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter;
 import org.apache.ignite.internal.processors.cache.transactions.TxCounters;
-import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateDataRow;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccUpdateResult;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.ResultType;
 import org.apache.ignite.internal.processors.cache.version.GridCacheLazyPlainVersionedEntry;
@@ -99,6 +98,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
@@ -115,13 +115,13 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRA
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE;
 import static org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult.UpdateOutcome.INVOKE_NO_OP;
 import static org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult.UpdateOutcome.REMOVE_NO_VAL;
+import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.compareIgnoreOpCounter;
 import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.NO_KEY;
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.CONCURRENT_UPDATE;
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.DUPLICATE_KEY;
 import static org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode.TRANSACTION_SERIALIZATION_ERROR;
 import static org.apache.ignite.internal.processors.dr.GridDrType.DR_BACKUP;
 import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE;
-import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRIMARY;
 
 /**
  * Adapter for cache entry.
@@ -1056,6 +1056,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         GridCacheOperation op,
         boolean needHistory,
         boolean noCreate,
+        boolean needOldVal,
         CacheEntryPredicate filter,
         boolean retVal) throws IgniteCheckedException, GridCacheEntryRemovedException {
         assert tx != null;
@@ -1094,13 +1095,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             assert ttl >= 0 : ttl;
             assert expireTime >= 0 : expireTime;
 
+
             // Detach value before index update.
             val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx);
 
             assert val != null || invoke;
 
-            res = cctx.offheap().mvccUpdate(this, val, newVer, expireTime, mvccVer, tx.local(), needHistory, noCreate,
-                filter, retVal, entryProc, invokeArgs);
+            res = cctx.offheap().mvccUpdate(this, val, newVer, expireTime, mvccVer, tx.local(), needHistory,
+                noCreate, needOldVal, filter, retVal, entryProc, invokeArgs);
 
             assert res != null;
 
@@ -1121,6 +1123,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if(invoke) // No-op invoke happened.
                     updRes.invokeResult(res.invokeResult());
 
+                updRes.filtered(true);
+
                 return updRes;
             }
             else if(noCreate && !invoke && res.resultType() == ResultType.PREV_NULL)
@@ -1135,7 +1139,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 IgniteInternalFuture<?> lockFut = cctx.kernalContext().coordinators().waitFor(cctx, lockVer);
 
                 lockFut.listen(new MvccUpdateLockListener(tx, this, affNodeId, topVer, val, ttl0, mvccVer,
-                    op, needHistory, noCreate, filter, retVal, resFut, entryProc, invokeArgs));
+                    op, needHistory, noCreate, resFut, needOldVal, filter, retVal, entryProc, invokeArgs));
 
                 return new GridCacheUpdateTxResult(false, resFut);
             }
@@ -1149,7 +1153,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (res.resultType() == ResultType.PREV_NULL) {
                 TxCounters counters = tx.txCounters(true);
 
-                if (res.isOwnValueOverridden()) {
+                if (compareIgnoreOpCounter(res.resultVersion(), mvccVer) == 0) {
                     if (res.isKeyAbsentBefore())
                         counters.incrementUpdateCounter(cctx.cacheId(), partition());
                 }
@@ -1158,7 +1162,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 counters.accumulateSizeDelta(cctx.cacheId(), partition(), 1);
             }
-            else if (res.resultType() == ResultType.PREV_NOT_NULL && !res.isOwnValueOverridden()) {
+            else if (res.resultType() == ResultType.PREV_NOT_NULL && compareIgnoreOpCounter(res.resultVersion(), mvccVer) != 0) {
                 TxCounters counters = tx.txCounters(true);
 
                 counters.incrementUpdateCounter(cctx.cacheId(), partition());
@@ -1166,7 +1170,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             else if (res.resultType() == ResultType.REMOVED_NOT_NULL) {
                 TxCounters counters = tx.txCounters(true);
 
-                if (res.isOwnValueOverridden()) {
+                if (compareIgnoreOpCounter(res.resultVersion(), mvccVer) == 0) {
                     if (res.isKeyAbsentBefore()) // Do not count own update removal.
                         counters.decrementUpdateCounter(cctx.cacheId(), partition());
                 }
@@ -1192,8 +1196,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             update(val, expireTime, ttl, newVer, true);
 
-            mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, val, newVer, topVer, mvccVer);
-
             recordNodeId(affNodeId, topVer);
         }
         finally {
@@ -1209,13 +1211,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         GridCacheUpdateTxResult updRes = valid ? new GridCacheUpdateTxResult(true, 0L, logPtr) :
             new GridCacheUpdateTxResult(false, logPtr);
 
-        CacheDataRow oldRow = ((MvccUpdateDataRow)res).oldRow();
+        if (retVal && (res.resultType() == ResultType.PREV_NOT_NULL || res.resultType() == ResultType.VERSION_FOUND))
+            updRes.prevValue(res.oldValue());
 
-        if(retVal && (res.resultType() == ResultType.PREV_NOT_NULL || res.resultType() == ResultType.VERSION_FOUND)) {
-            assert oldRow != null;
+        if (needOldVal && compareIgnoreOpCounter(res.resultVersion(), mvccVer) != 0 && (
+            res.resultType() == ResultType.PREV_NOT_NULL || res.resultType() == ResultType.REMOVED_NOT_NULL))
+            updRes.oldValue(res.oldValue());
 
-            updRes.prevValue(oldRow.value());
-        }
+        updRes.newValue(res.newValue());
 
         if(invoke) {
             assert res.invokeResult() != null;
@@ -1235,6 +1238,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         AffinityTopologyVersion topVer,
         MvccSnapshot mvccVer,
         boolean needHistory,
+        boolean needOldVal,
         @Nullable CacheEntryPredicate filter,
         boolean retVal) throws IgniteCheckedException, GridCacheEntryRemovedException {
         assert tx != null;
@@ -1257,14 +1261,21 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             assert newVer != null : "Failed to get write version for tx: " + tx;
 
-            res = cctx.offheap().mvccRemove(this, mvccVer, tx.local(), needHistory, filter, retVal);
+            res = cctx.offheap().mvccRemove(this, mvccVer, tx.local(), needHistory, needOldVal, filter, retVal);
 
             assert res != null;
 
             if (res.resultType() == ResultType.VERSION_MISMATCH)
                 throw serializationError();
-            else if (res.resultType() == ResultType.PREV_NULL || res.resultType() == ResultType.FILTERED)
+            else if (res.resultType() == ResultType.PREV_NULL)
                 return new GridCacheUpdateTxResult(false);
+            else if (res.resultType() == ResultType.FILTERED) {
+                GridCacheUpdateTxResult updRes = new GridCacheUpdateTxResult(false);
+
+                updRes.filtered(true);
+
+                return updRes;
+            }
             else if (res.resultType() == ResultType.LOCKED) {
                 unlockEntry();
 
@@ -1275,7 +1286,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 IgniteInternalFuture<?> lockFut = cctx.kernalContext().coordinators().waitFor(cctx, lockVer);
 
                 lockFut.listen(new MvccRemoveLockListener(tx, this, affNodeId, topVer, mvccVer, needHistory,
-                    resFut, retVal, filter));
+                    resFut, needOldVal, retVal, filter));
 
                 return new GridCacheUpdateTxResult(false, resFut);
             }
@@ -1286,7 +1297,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (res.resultType() == ResultType.PREV_NOT_NULL) {
                 TxCounters counters = tx.txCounters(true);
 
-                if (res.isOwnValueOverridden()) {
+                if (compareIgnoreOpCounter(res.resultVersion(), mvccVer) == 0) {
                     if (res.isKeyAbsentBefore()) // Do not count own update removal.
                         counters.decrementUpdateCounter(cctx.cacheId(), partition());
                 }
@@ -1301,8 +1312,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             update(null, 0, 0, newVer, true);
 
-            mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, null, newVer, topVer, mvccVer);
-
             recordNodeId(affNodeId, topVer);
         }
         finally {
@@ -1318,13 +1327,12 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         GridCacheUpdateTxResult updRes = valid ? new GridCacheUpdateTxResult(true, 0L, logPtr) :
             new GridCacheUpdateTxResult(false, logPtr);
 
-        CacheDataRow oldRow = ((MvccUpdateDataRow)res).oldRow();
-
-        if(retVal && (res.resultType() == ResultType.PREV_NOT_NULL || res.resultType() == ResultType.VERSION_FOUND)) {
-            assert oldRow != null;
+        if(retVal && (res.resultType() == ResultType.PREV_NOT_NULL || res.resultType() == ResultType.VERSION_FOUND))
+            updRes.prevValue(res.oldValue());
 
-            updRes.prevValue(oldRow.value());
-        }
+        if (needOldVal && compareIgnoreOpCounter(res.resultVersion(), mvccVer) != 0 &&
+            (res.resultType() == ResultType.PREV_NOT_NULL  || res.resultType() == ResultType.REMOVED_NOT_NULL))
+            updRes.oldValue(res.oldValue());
 
         updRes.mvccHistory(res.history());
 
@@ -1387,24 +1395,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         return new GridCacheUpdateTxResult(valid, logPtr);
     }
 
-    /**
-     * Enlist for DR if needed.
-     *
-     * @param drType DR type.
-     * @param val Value.
-     * @param ver Version.
-     * @param topVer Topology version.
-     * @param mvccVer MVCC snapshot.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    private void mvccDrReplicate(GridDrType drType, CacheObject val, GridCacheVersion ver,
-        AffinityTopologyVersion topVer,
-        MvccSnapshot mvccVer) throws IgniteCheckedException {
-
-        if (cctx.isDrEnabled() && drType != DR_NONE && !isInternal())
-            cctx.dr().mvccReplicate(key, val, rawTtl(), rawExpireTime(), ver.conflictVersion(), drType, topVer, mvccVer);
-    }
-
     /** {@inheritDoc} */
     @Override public final GridCacheUpdateTxResult innerSet(
         @Nullable IgniteInternalTx tx,
@@ -1475,7 +1465,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             boolean internal = isInternal() || !context().userCache();
 
             Map<UUID, CacheContinuousQueryListener> lsnrCol =
-                notifyContinuousQueries(tx) ? cctx.continuousQueries().updateListeners(internal, false) : null;
+                notifyContinuousQueries(tx) ?
+                    cctx.continuousQueries().updateListeners(internal, false) : null;
 
             if (startVer && (retval || intercept || lsnrCol != null))
                 unswap(retval);
@@ -1707,7 +1698,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             boolean internal = isInternal() || !context().userCache();
 
             Map<UUID, CacheContinuousQueryListener> lsnrCol =
-                notifyContinuousQueries(tx) ? cctx.continuousQueries().updateListeners(internal, false) : null;
+                notifyContinuousQueries(tx) ?
+                    cctx.continuousQueries().updateListeners(internal, false) : null;
 
             if (startVer && (retval || intercept || lsnrCol != null))
                 unswap();
@@ -1859,16 +1851,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             return new GridCacheUpdateTxResult(false, logPtr);
     }
 
-    /**
-     * @param tx Transaction.
-     * @return {@code True} if should notify continuous query manager.
-     */
-    private boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) {
-        return cctx.isLocal() ||
-            cctx.isReplicated() ||
-            (!isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local()));
-    }
-
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public GridTuple3<Boolean, Object, EntryProcessorResult<Object>> innerUpdateLocal(
@@ -2975,6 +2957,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /**
+     * @param tx Transaction.
+     * @return {@code True} if should notify continuous query manager.
+     */
+    private boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) {
+        return cctx.isLocal() ||
+            cctx.isReplicated() ||
+            (!isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local()));
+    }
+
+    /**
      * Update TTL if it is changed.
      *
      * @param expiryPlc Expiry policy.
@@ -5012,6 +5004,26 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         private GridCacheMapEntry entry;
 
         /** */
+        private IgniteUuid futId;
+
+        /** */
+        private int batchNum;
+
+        /** Flag if it is need to return the old value (value before current tx has been started). */
+        private final boolean needOldVal;
+
+
+        /**
+         *
+         * @param tx Transaction.
+         * @param entry Entry.
+         * @param affNodeId Node id.
+         * @param topVer Topology version.
+         * @param mvccVer Mvcc version.
+         * @param needHistory Need history flag.
+         * @param resFut Result future.
+         * @param needOldVal Flag if it is need to return the old value (value before current tx has been started).
+         */
         MvccRemoveLockListener(IgniteInternalTx tx,
             GridCacheMapEntry entry,
             UUID affNodeId,
@@ -5019,6 +5031,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             MvccSnapshot mvccVer,
             boolean needHistory,
             GridFutureAdapter<GridCacheUpdateTxResult> resFut,
+            boolean needOldVal,
             boolean retVal,
             @Nullable CacheEntryPredicate filter) {
             this.tx = tx;
@@ -5028,6 +5041,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             this.mvccVer = mvccVer;
             this.needHistory = needHistory;
             this.resFut = resFut;
+            this.needOldVal = needOldVal;
             this.needVal = retVal;
             this.filter = filter;
         }
@@ -5058,12 +5072,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 valid = entry.valid(tx.topologyVersion());
 
+                boolean needOldVal = cctx.shared().mvccCaching().continuousQueryListeners(cctx, tx, entry.key()) != null;
+
                 cctx.shared().database().checkpointReadLock();
 
                 try {
-                    res = cctx.offheap().mvccRemove(entry, mvccVer, tx.local(), needHistory, filter, needVal);
-                }
-                finally {
+                    res = cctx.offheap().mvccRemove(entry, mvccVer, tx.local(), needHistory, needOldVal, filter, needVal);
+                } finally {
                     cctx.shared().database().checkpointReadUnlock();
                 }
 
@@ -5074,11 +5089,20 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                     return;
                 }
-                else if (res.resultType() == ResultType.PREV_NULL || res.resultType() == ResultType.FILTERED) {
+                else if (res.resultType() == ResultType.PREV_NULL) {
                     resFut.onDone(new GridCacheUpdateTxResult(false));
 
                     return;
                 }
+                else if (res.resultType() == ResultType.FILTERED) {
+                    GridCacheUpdateTxResult updRes = new GridCacheUpdateTxResult(false);
+
+                    updRes.filtered(true);
+
+                    resFut.onDone(updRes);
+
+                    return;
+                }
                 else if (res.resultType() == ResultType.LOCKED) {
                     entry.unlockEntry();
 
@@ -5095,7 +5119,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if (res.resultType() == ResultType.PREV_NOT_NULL) {
                     TxCounters counters = tx.txCounters(true);
 
-                    if (res.isOwnValueOverridden()) {
+                    if (compareIgnoreOpCounter(res.resultVersion(), mvccVer) == 0) {
                         if (res.isKeyAbsentBefore()) // Do not count own update removal.
                             counters.decrementUpdateCounter(cctx.cacheId(), entry.partition());
                     }
@@ -5119,8 +5143,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 entry.update(null, 0, 0, newVer, true);
 
-                entry.mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, null, newVer, topVer, mvccVer);
-
                 entry.recordNodeId(affNodeId, topVer);
             }
             catch (IgniteCheckedException e) {
@@ -5143,6 +5165,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             updRes.mvccHistory(res.history());
 
+            if (needOldVal  && compareIgnoreOpCounter(res.resultVersion(), mvccVer) != 0 &&
+                (res.resultType() == ResultType.PREV_NOT_NULL || res.resultType() == ResultType.REMOVED_NOT_NULL))
+                updRes.oldValue(res.oldValue());
+
             resFut.onDone(updRes);
         }
     }
@@ -5294,6 +5320,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         /** Need previous value flag.*/
         private final boolean needVal;
 
+        /** Flag if it is need to return the old value (value before current tx has been started). */
+        private boolean needOldVal;
+
         /** */
         MvccUpdateLockListener(IgniteInternalTx tx,
             GridCacheMapEntry entry,
@@ -5305,9 +5334,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             GridCacheOperation op,
             boolean needHistory,
             boolean noCreate,
+            GridFutureAdapter<GridCacheUpdateTxResult> resFut,
+            boolean needOldVal,
             CacheEntryPredicate filter,
             boolean needVal,
-            GridFutureAdapter<GridCacheUpdateTxResult> resFut,
             EntryProcessor entryProc,
             Object[] invokeArgs) {
             this.tx = tx;
@@ -5323,6 +5353,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             this.filter = filter;
             this.needVal = needVal;
             this.resFut = resFut;
+            this.needOldVal = needOldVal;
             this.entryProc = entryProc;
             this.invokeArgs = invokeArgs;
         }
@@ -5374,9 +5405,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 try {
                     res = cctx.offheap().mvccUpdate(entry, val, newVer, expireTime, mvccVer, tx.local(), needHistory,
-                        noCreate, filter, needVal, entryProc, invokeArgs);
-                }
-                finally {
+                        noCreate, needOldVal, filter, needVal, entryProc, invokeArgs);
+                } finally {
                     cctx.shared().database().checkpointReadUnlock();
                 }
 
@@ -5403,6 +5433,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         updRes.invokeResult(res.invokeResult());
                     }
 
+                    updRes.filtered(true);
+
                     resFut.onDone(updRes);
 
                     return;
@@ -5421,7 +5453,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if (res.resultType() == ResultType.PREV_NULL) {
                     TxCounters counters = tx.txCounters(true);
 
-                    if (res.isOwnValueOverridden()) {
+                    if (compareIgnoreOpCounter(res.resultVersion(), mvccVer) == 0) {
                         if (res.isKeyAbsentBefore())
                             counters.incrementUpdateCounter(cctx.cacheId(), entry.partition());
                     }
@@ -5430,7 +5462,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                     counters.accumulateSizeDelta(cctx.cacheId(), entry.partition(), 1);
                 }
-                else if (res.resultType() == ResultType.PREV_NOT_NULL && !res.isOwnValueOverridden()) {
+                else if (res.resultType() == ResultType.PREV_NOT_NULL && compareIgnoreOpCounter(res.resultVersion(), mvccVer) != 0) {
                     TxCounters counters = tx.txCounters(true);
 
                     counters.incrementUpdateCounter(cctx.cacheId(), entry.partition());
@@ -5438,7 +5470,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 else if (res.resultType() == ResultType.REMOVED_NOT_NULL) {
                     TxCounters counters = tx.txCounters(true);
 
-                    if (res.isOwnValueOverridden()) {
+                    if (compareIgnoreOpCounter(res.resultVersion(), mvccVer) == 0) {
                         if (res.isKeyAbsentBefore()) // Do not count own update removal.
                             counters.decrementUpdateCounter(cctx.cacheId(), entry.partition());
                     }
@@ -5463,8 +5495,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 entry.update(val, expireTime, ttl, newVer, true);
 
-                entry.mvccDrReplicate(tx.local() ? DR_PRIMARY : DR_BACKUP, val, newVer, topVer, mvccVer);
-
                 entry.recordNodeId(affNodeId, topVer);
             }
             catch (IgniteCheckedException e) {
@@ -5491,6 +5521,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 updRes.invokeResult(res.invokeResult());
             }
 
+            if (needOldVal && compareIgnoreOpCounter(res.resultVersion(), mvccVer) != 0 &&
+                (res.resultType() == ResultType.PREV_NOT_NULL || res.resultType() == ResultType.REMOVED_NOT_NULL))
+                updRes.oldValue(res.oldValue());
+
             updRes.mvccHistory(res.history());
 
             resFut.onDone(updRes);
@@ -6586,7 +6620,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         AffinityTopologyVersion topVer,
         List<GridCacheEntryInfo> entries,
         GridCacheOperation op,
-        MvccSnapshot mvccVer)
+        MvccSnapshot mvccVer,
+        IgniteUuid futId,
+        int batchNum)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
 
         WALPointer logPtr = null;
@@ -6619,6 +6655,20 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     info.newMvccTxState());
             }
 
+            CacheObject oldVal = null;
+
+            // Assume list contains  only previous committed row and rows changed by the current tx.
+            if (!entries.isEmpty()) {
+                // Assume the oldest value is the last one.
+                GridCacheMvccEntryInfo last = (GridCacheMvccEntryInfo)entries.get(entries.size() - 1);
+
+                if (MvccUtils.compare(last.mvccVersion(),
+                    mvccVer.coordinatorVersion(),
+                    mvccVer.counter(),
+                    mvccVer.operationCounter()) != 0)
+                    oldVal = last.value();
+            }
+
             if (cctx.deferredDelete() && deletedUnlocked() && !detached())
                 deletedUnlocked(false);
 
@@ -6641,8 +6691,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
             update(val, expireTime, ttl, ver, true);
 
-            mvccDrReplicate(DR_BACKUP, val, ver, topVer, mvccVer);
-
             recordNodeId(affNodeId, topVer);
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java
index bd4146e..027305d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccEntryInfo.java
@@ -122,11 +122,6 @@ public class GridCacheMvccEntryInfo extends GridCacheEntryInfo implements MvccVe
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isKeyAbsentBefore() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index a6de1e5..68698ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -63,7 +63,6 @@ import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.configuration.WALMode;
-import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
@@ -85,18 +84,19 @@ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProce
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.StopCachesOnClientReconnectExchangeTask;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearAtomicCache;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTransactionalCache;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrManager;
 import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
 import org.apache.ignite.internal.processors.cache.local.GridLocalCache;
 import org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccCachingManager;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
@@ -168,6 +168,7 @@ import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -2677,6 +2678,8 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
 
         CacheJtaManagerAdapter jta = JTA.createOptional();
 
+        MvccCachingManager mvccCachingMgr = new MvccCachingManager();
+
         return new GridCacheSharedContext(
             kernalCtx,
             tm,
@@ -2694,7 +2697,8 @@ public class GridCacheProcessor extends GridProcessorAdapter implements Metastor
             ttl,
             evict,
             jta,
-            storeSesLsnrs
+            storeSesLsnrs,
+            mvccCachingMgr
         );
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index caa3d20..1f2f636 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.topology.Grid
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccCachingManager;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccProcessor;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
@@ -129,6 +130,9 @@ public class GridCacheSharedContext<K, V> {
     /** */
     private PartitionsEvictManager evictMgr;
 
+    /** Mvcc caching manager. */
+    private MvccCachingManager mvccCachingMgr;
+
     /** Cache contexts map. */
     private ConcurrentHashMap<Integer, GridCacheContext<K, V>> ctxMap;
 
@@ -207,7 +211,8 @@ public class GridCacheSharedContext<K, V> {
         GridCacheSharedTtlCleanupManager ttlMgr,
         PartitionsEvictManager evictMgr,
         CacheJtaManagerAdapter jtaMgr,
-        Collection<CacheStoreSessionListener> storeSesLsnrs
+        Collection<CacheStoreSessionListener> storeSesLsnrs,
+        MvccCachingManager mvccCachingMgr
     ) {
         this.kernalCtx = kernalCtx;
 
@@ -227,7 +232,8 @@ public class GridCacheSharedContext<K, V> {
             affMgr,
             ioMgr,
             ttlMgr,
-            evictMgr
+            evictMgr,
+            mvccCachingMgr
         );
 
         this.storeSesLsnrs = storeSesLsnrs;
@@ -393,7 +399,8 @@ public class GridCacheSharedContext<K, V> {
             affMgr,
             ioMgr,
             ttlMgr,
-            evictMgr
+            evictMgr,
+            mvccCachingMgr
         );
 
         this.mgrs = mgrs;
@@ -452,7 +459,8 @@ public class GridCacheSharedContext<K, V> {
         CacheAffinitySharedManager affMgr,
         GridCacheIoManager ioMgr,
         GridCacheSharedTtlCleanupManager ttlMgr,
-        PartitionsEvictManager evictMgr
+        PartitionsEvictManager evictMgr,
+        MvccCachingManager mvccCachingMgr
     ) {
         this.mvccMgr = add(mgrs, mvccMgr);
         this.verMgr = add(mgrs, verMgr);
@@ -469,6 +477,7 @@ public class GridCacheSharedContext<K, V> {
         this.ioMgr = add(mgrs, ioMgr);
         this.ttlMgr = add(mgrs, ttlMgr);
         this.evictMgr = add(mgrs, evictMgr);
+        this.mvccCachingMgr = add(mgrs, mvccCachingMgr);
     }
 
     /**
@@ -815,6 +824,13 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @return Mvcc transaction enlist caching manager.
+     */
+    public MvccCachingManager mvccCaching() {
+        return mvccCachingMgr;
+    }
+
+    /**
      * @return Node ID.
      */
     public UUID localNodeId() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java
index d2a2870..8a68100 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateTxResult.java
@@ -54,6 +54,15 @@ public class GridCacheUpdateTxResult {
     /** Invoke result. */
     private CacheInvokeResult invokeRes;
 
+    /** New value. */
+    private CacheObject newVal;
+
+    /** Value before the current tx. */
+    private CacheObject oldVal;
+
+    /** Filtered flag. */
+    private boolean filtered;
+
     /**
      * Constructor.
      *
@@ -190,6 +199,48 @@ public class GridCacheUpdateTxResult {
         return invokeRes;
     }
 
+    /**
+     * @return New value.
+     */
+    public CacheObject newValue() {
+        return newVal;
+    }
+
+    /**
+     * @return Old value.
+     */
+    public CacheObject oldValue() {
+        return oldVal;
+    }
+
+    /**
+     * @param newVal New value.
+     */
+    public void newValue(CacheObject newVal) {
+        this.newVal = newVal;
+    }
+
+    /**
+     * @param oldVal Old value.
+     */
+    public void oldValue(CacheObject oldVal) {
+        this.oldVal = oldVal;
+    }
+
+    /**
+     * @return Filtered flag.
+     */
+    public boolean filtered() {
+        return filtered;
+    }
+
+    /**
+     * @param filtered Filtered flag.
+     */
+    public void filtered(boolean filtered) {
+        this.filtered = filtered;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheUpdateTxResult.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
index c9c2430..044830c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
@@ -275,8 +275,9 @@ public interface IgniteCacheOffheapManager {
      * @param expireTime Expire time.
      * @param mvccSnapshot MVCC snapshot.
      * @param primary {@code True} if on primary node.
-     * @param needHistory Flag to collect history.
+     * @param needHist Flag to collect history.
      * @param noCreate Flag indicating that row should not be created if absent.
+     * @param needOldVal {@code True} if need old value.
      * @param filter Filter.
      * @param retVal Flag to return previous value.
      * @param entryProc Entry processor.
@@ -291,8 +292,9 @@ public interface IgniteCacheOffheapManager {
         long expireTime,
         MvccSnapshot mvccSnapshot,
         boolean primary,
-        boolean needHistory,
+        boolean needHist,
         boolean noCreate,
+        boolean needOldVal,
         @Nullable CacheEntryPredicate filter,
         boolean retVal,
         EntryProcessor entryProc,
@@ -302,7 +304,8 @@ public interface IgniteCacheOffheapManager {
      * @param entry Entry.
      * @param mvccSnapshot MVCC snapshot.
      * @param primary {@code True} if on primary node.
-     * @param needHistory Flag to collect history.
+     * @param needHist Flag to collect history.
+     * @param needOldVal {@code True} if need old value.
      * @param filter Filter.
      * @param retVal Flag to return previous value.
      * @return Update result.
@@ -312,7 +315,8 @@ public interface IgniteCacheOffheapManager {
         GridCacheMapEntry entry,
         MvccSnapshot mvccSnapshot,
         boolean primary,
-        boolean needHistory,
+        boolean needHist,
+        boolean needOldVal,
         @Nullable CacheEntryPredicate filter,
         boolean retVal) throws IgniteCheckedException;
 
@@ -805,8 +809,9 @@ public interface IgniteCacheOffheapManager {
          * @param entryProc Entry processor.
          * @param invokeArgs Entry processor invoke arguments.
          * @param primary {@code True} if update is executed on primary node.
-         * @param needHistory Flag to collect history.
+         * @param needHist Flag to collect history.
          * @param noCreate Flag indicating that row should not be created if absent.
+         * @param needOldVal {@code True} if need old value.
          * @param retVal Flag to return previous value.
          * @return Update result.
          * @throws IgniteCheckedException If failed.
@@ -822,8 +827,9 @@ public interface IgniteCacheOffheapManager {
             EntryProcessor entryProc,
             Object[] invokeArgs,
             boolean primary,
-            boolean needHistory,
+            boolean needHist,
             boolean noCreate,
+            boolean needOldVal,
             boolean retVal) throws IgniteCheckedException;
 
         /**
@@ -833,6 +839,7 @@ public interface IgniteCacheOffheapManager {
          * @param filter Filter.
          * @param primary {@code True} if update is executed on primary node.
          * @param needHistory Flag to collect history.
+         * @param needOldVal {@code True} if need old value.
          * @param retVal Flag to return previous value.
          * @return List of transactions to wait for.
          * @throws IgniteCheckedException If failed.
@@ -844,6 +851,7 @@ public interface IgniteCacheOffheapManager {
             @Nullable CacheEntryPredicate filter,
             boolean primary,
             boolean needHistory,
+            boolean needOldVal,
             boolean retVal) throws IgniteCheckedException;
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 0182c8a..e40cc53 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -517,6 +517,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         boolean primary,
         boolean needHistory,
         boolean noCreate,
+        boolean needOldVal,
         @Nullable CacheEntryPredicate filter,
         boolean retVal,
         EntryProcessor entryProc,
@@ -538,6 +539,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             primary,
             needHistory,
             noCreate,
+            needOldVal,
             retVal);
     }
 
@@ -547,6 +549,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         MvccSnapshot mvccSnapshot,
         boolean primary,
         boolean needHistory,
+        boolean needOldVal,
         @Nullable CacheEntryPredicate filter,
         boolean retVal) throws IgniteCheckedException {
         if (entry.detached() || entry.isNear())
@@ -560,6 +563,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             filter,
             primary,
             needHistory,
+            needOldVal,
             retVal);
     }
 
@@ -1868,6 +1872,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             boolean primary,
             boolean needHistory,
             boolean noCreate,
+            boolean needOldVal,
             boolean retVal) throws IgniteCheckedException {
             assert mvccSnapshot != null;
             assert primary || !needHistory;
@@ -1901,6 +1906,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
                     needHistory,
                     // we follow fast update visit flow here if row cannot be created by current operation
                     noCreate,
+                    needOldVal,
                     retVal || entryProc != null);
 
                 assert cctx.shared().database().checkpointLockIsHeldByThread();
@@ -2046,6 +2052,8 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
                 }
             }
             catch (Exception e) {
+                log.error("Exception was thrown during entry processing.", e);
+
                 err = e;
             }
 
@@ -2064,6 +2072,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             @Nullable CacheEntryPredicate filter,
             boolean primary,
             boolean needHistory,
+            boolean needOldVal,
             boolean retVal) throws IgniteCheckedException {
             assert mvccSnapshot != null;
             assert primary || mvccSnapshot.activeTransactions().size() == 0 : mvccSnapshot;
@@ -2094,6 +2103,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
                     false,
                     needHistory,
                     true,
+                    needOldVal,
                     retVal);
 
                 assert cctx.shared().database().checkpointLockIsHeldByThread();
@@ -2160,6 +2170,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
                     true,
                     false,
                     false,
+                    false,
                     false);
 
                 assert cctx.shared().database().checkpointLockIsHeldByThread();

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index b8803a3..bd13fc3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -29,6 +29,7 @@ import java.util.UUID;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -50,9 +51,9 @@ import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWra
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
 import org.apache.ignite.internal.processors.cache.persistence.StorageException;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
@@ -818,6 +819,8 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                             if (txCntrs != null)
                                 applyPartitionsUpdatesCounters(txCntrs.updateCounters());
 
+                            cctx.mvccCaching().onTxFinished(this, true);
+
                             if (!near() && !F.isEmpty(dataEntries) && cctx.wal() != null) {
                                 // Set new update counters for data entries received from persisted tx entries.
                                 List<DataEntry> entriesWithCounters = dataEntries.stream()
@@ -840,8 +843,6 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                     finally {
                         cctx.database().checkpointReadUnlock();
 
-                        notifyDrManager(state() == COMMITTING && err == null);
-
                         if (wrapper != null)
                             wrapper.initialize(ret);
                     }
@@ -933,8 +934,6 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
     /** {@inheritDoc} */
     @Override public final void rollbackRemoteTx() {
         try {
-            notifyDrManager(false);
-
             // Note that we don't evict near entries here -
             // they will be deleted by their corresponding transactions.
             if (state(ROLLING_BACK) || state() == UNKNOWN) {
@@ -946,6 +945,13 @@ public abstract class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                     applyPartitionsUpdatesCounters(counters.updateCounters());
 
                 state(ROLLED_BACK);
+
+                try {
+                    cctx.mvccCaching().onTxFinished(this, false);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
             }
         }
         catch (RuntimeException | Error e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 52638c0..edd28b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -2302,7 +2302,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             MvccSnapshot snapshot = new MvccSnapshotWithoutTxs(s0.coordinatorVersion(), s0.counter(),
                 req.operationCounter(), s0.cleanupVersion());
 
-            tx.mvccEnlistBatch(ctx, req.op(), req.keys(), req.values(), snapshot);
+            ctx.tm().txHandler().mvccEnlistBatch(tx, ctx, req.op(), req.keys(), req.values(), snapshot,
+                req.dhtFutureId(), req.batchId());
 
             GridDhtTxQueryEnlistResponse res = new GridDhtTxQueryEnlistResponse(req.cacheId(),
                 req.dhtFutureId(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
index 484bc41..e2c8237 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxAbstractEnlistFuture.java
@@ -428,6 +428,8 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
 
                     tx.markQueryEnlisted(mvccSnapshot);
 
+                    boolean needOldVal = cctx.shared().mvccCaching().continuousQueryListeners(cctx, tx, key) != null;
+
                     GridCacheUpdateTxResult res;
 
                     while (true) {
@@ -442,6 +444,7 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
                                         topVer,
                                         mvccSnapshot,
                                         isMoving(key.partition()),
+                                        needOldVal,
                                         filter,
                                         needResult());
 
@@ -463,6 +466,7 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
                                         op.cacheOperation(),
                                         isMoving(key.partition()),
                                         op.noCreate(),
+                                        needOldVal,
                                         filter,
                                         needResult());
 
@@ -632,6 +636,10 @@ public abstract class GridDhtTxAbstractEnlistFuture<T> extends GridCacheFutureAd
         if (!updRes.success())
             return;
 
+        if (!updRes.filtered())
+            cctx.shared().mvccCaching().addEnlisted(entry.key(), updRes.newValue(), 0, 0, lockVer,
+                updRes.oldValue(), tx.local(), tx.topologyVersion(), mvccSnapshot, cctx.cacheId(), tx, null, -1);
+
         if (op != EnlistOperation.LOCK)
             addToBatch(entry.key(), val, updRes.mvccHistory(), entry.context().cacheId());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
index 1f5f5a9..259c311 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
@@ -21,40 +21,27 @@ import java.io.Externalizable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cluster.ClusterTopologyException;
-import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
-import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteSingleStateImpl;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxRemoteStateImpl;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.query.EnlistOperation;
-import org.apache.ignite.internal.processors.query.IgniteSQLException;
-import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.tostring.GridToStringBuilder;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
@@ -382,142 +369,6 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
         txState.addWriteEntry(key, txEntry);
     }
 
-    /**
-     *
-     * @param ctx Cache context.
-     * @param op Operation.
-     * @param keys Keys.
-     * @param vals Values.
-     * @param snapshot Mvcc snapshot.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void mvccEnlistBatch(GridCacheContext ctx, EnlistOperation op, List<KeyCacheObject> keys,
-        List<Message> vals, MvccSnapshot snapshot) throws IgniteCheckedException {
-        assert keys != null && (vals == null || vals.size() == keys.size());
-
-        WALPointer ptr = null;
-
-        GridDhtCacheAdapter dht = ctx.dht();
-
-        addActiveCache(ctx, false);
-
-        for (int i = 0; i < keys.size(); i++) {
-            KeyCacheObject key = keys.get(i);
-
-            assert key != null;
-
-            int part = ctx.affinity().partition(key);
-
-            GridDhtLocalPartition locPart = ctx.topology().localPartition(part, topologyVersion(), false);
-
-            if (locPart == null || !locPart.reserve())
-                throw new ClusterTopologyException("Can not reserve partition. Please retry on stable topology.");
-
-            try {
-                CacheObject val = null;
-                EntryProcessor entryProc = null;
-                Object[] invokeArgs = null;
-
-                Message val0 = vals != null ? vals.get(i) : null;
-
-                CacheEntryInfoCollection entries =
-                    val0 instanceof CacheEntryInfoCollection ? (CacheEntryInfoCollection)val0 : null;
-
-                if (entries == null && !op.isDeleteOrLock() && !op.isInvoke())
-                    val = (val0 instanceof CacheObject) ? (CacheObject)val0 : null;
-
-                if(entries == null && op.isInvoke()) {
-                    assert val0 instanceof GridInvokeValue;
-
-                    GridInvokeValue invokeVal = (GridInvokeValue)val0;
-
-                    entryProc = invokeVal.entryProcessor();
-                    invokeArgs = invokeVal.invokeArgs();
-                }
-
-                assert entryProc != null || !op.isInvoke();
-
-                GridDhtCacheEntry entry = dht.entryExx(key, topologyVersion());
-
-                GridCacheUpdateTxResult updRes;
-
-                while (true) {
-                    ctx.shared().database().checkpointReadLock();
-
-                    try {
-                        if (entries == null) {
-                            switch (op) {
-                                case DELETE:
-                                    updRes = entry.mvccRemove(
-                                        this,
-                                        ctx.localNodeId(),
-                                        topologyVersion(),
-                                        snapshot,
-                                        false,
-                                        null,
-                                        false);
-
-                                    break;
-
-                                case INSERT:
-                                case TRANSFORM:
-                                case UPSERT:
-                                case UPDATE:
-                                    updRes = entry.mvccSet(
-                                        this,
-                                        ctx.localNodeId(),
-                                        val,
-                                        entryProc,
-                                        invokeArgs,
-                                        0,
-                                        topologyVersion(),
-                                        snapshot,
-                                        op.cacheOperation(),
-                                        false,
-                                        false,
-                                        null,
-                                        false);
-
-                                    break;
-
-                                default:
-                                    throw new IgniteSQLException("Cannot acquire lock for operation [op= "
-                                        + op + "]" + "Operation is unsupported at the moment ",
-                                        IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
-                            }
-                        }
-                        else {
-                            updRes = entry.mvccUpdateRowsWithPreloadInfo(this,
-                                ctx.localNodeId(),
-                                topologyVersion(),
-                                entries.infos(),
-                                op.cacheOperation(),
-                                snapshot);
-                        }
-
-                        break;
-                    }
-                    catch (GridCacheEntryRemovedException ignore) {
-                        entry = dht.entryExx(key);
-                    }
-                    finally {
-                        ctx.shared().database().checkpointReadUnlock();
-                    }
-                }
-
-                assert updRes.updateFuture() == null : "Entry should not be locked on the backup";
-
-                ptr = updRes.loggedPointer();
-            }
-            finally {
-                locPart.release();
-            }
-        }
-
-        if (ptr != null && !ctx.tm().logTxRecords())
-            ctx.shared().wal().flush(ptr, true);
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(GridDhtTxRemote.class, this, "super", super.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java
index a3ba31f..9691936 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxAbstractEnlistFuture.java
@@ -345,7 +345,8 @@ public abstract class GridNearTxAbstractEnlistFuture<T> extends GridCacheCompoun
             }
         }
         finally {
-            cctx.topology().readUnlock();
+            if(cctx.topology().holdsLock())
+                cctx.topology().readUnlock();
         }
     }
 
@@ -362,6 +363,10 @@ public abstract class GridNearTxAbstractEnlistFuture<T> extends GridCacheCompoun
         if (!DONE_UPD.compareAndSet(this, 0, 1))
             return false;
 
+        // Need to unlock topology to avoid deadlock with binary descriptors registration.
+        if(cctx.topology().holdsLock())
+            cctx.topology().readUnlock();
+
         cctx.tm().txContext(tx);
 
         Throwable ex0 = ex;

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
index 3325f6f..ed75c36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxEnlistFuture.java
@@ -158,6 +158,10 @@ public class GridNearTxEnlistFuture extends GridNearTxAbstractEnlistFuture<GridC
 
             boolean first = (nodeId != null);
 
+            // Need to unlock topology to avoid deadlock with binary descriptors registration.
+            if(!topLocked && cctx.topology().holdsLock())
+                cctx.topology().readUnlock();
+
             for (Batch batch : next) {
                 ClusterNode node = batch.node();
 
@@ -380,7 +384,8 @@ public class GridNearTxEnlistFuture extends GridNearTxAbstractEnlistFuture<GridC
                 }
             }
 
-            dhtTx.mvccEnlistBatch(cctx, it.operation(), keys, vals, mvccSnapshot.withoutActiveTransactions());
+            cctx.tm().txHandler().mvccEnlistBatch(dhtTx, cctx, it.operation(), keys, vals,
+                mvccSnapshot.withoutActiveTransactions(), null, -1);
         }
         catch (IgniteCheckedException e) {
             onDone(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index e321779..46698fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -769,7 +769,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
         try {
             Set<?> keys = map != null ? map.keySet() : invokeMap.keySet();
 
-            final Map<KeyCacheObject, Message> enlisted = new HashMap<>(keys.size());
+            final Map<KeyCacheObject, Message> enlisted = new LinkedHashMap<>(keys.size());
 
             for (Object key : keys) {
                 if (isRollbackOnly())

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
index 4da4e3f..8528345 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryEnlistFuture.java
@@ -129,6 +129,10 @@ public class GridNearTxQueryEnlistFuture extends GridNearTxQueryAbstractEnlistFu
             boolean first = true;
             boolean clientFirst = false;
 
+            // Need to unlock topology to avoid deadlock with binary descriptors registration.
+            if(!topLocked && cctx.topology().holdsLock())
+                cctx.topology().readUnlock();
+
             for (ClusterNode node : F.view(primary, F.remoteNodes(cctx.localNodeId()))) {
                 add(mini = new MiniFuture(node));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
index 0faa6da..f375405 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxQueryResultsEnlistFuture.java
@@ -150,6 +150,10 @@ public class GridNearTxQueryResultsEnlistFuture extends GridNearTxQueryAbstractE
 
             boolean first = (nodeId != null);
 
+            // Need to unlock topology to avoid deadlock with binary descriptors registration.
+            if(!topLocked && cctx.topology().holdsLock())
+                cctx.topology().readUnlock();
+
             for (Batch batch : next) {
                 ClusterNode node = batch.node();
 
@@ -360,7 +364,8 @@ public class GridNearTxQueryResultsEnlistFuture extends GridNearTxQueryAbstractE
                 }
             }
 
-            dhtTx.mvccEnlistBatch(cctx, it.operation(), keys, vals, mvccSnapshot.withoutActiveTransactions());
+            cctx.tm().txHandler().mvccEnlistBatch(dhtTx, cctx, it.operation(), keys, vals,
+                mvccSnapshot.withoutActiveTransactions(), null, -1);
         }
         catch (IgniteCheckedException e) {
             onDone(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
index f362374..f2a4b30 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheManager;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.jetbrains.annotations.Nullable;
@@ -57,36 +56,6 @@ public interface GridCacheDrManager extends GridCacheManager {
         AffinityTopologyVersion topVer)throws IgniteCheckedException;
 
     /**
-     * Enlist for DR.
-     *
-     * @param key Key.
-     * @param val Value.
-     * @param ttl TTL.
-     * @param expireTime Expire time.
-     * @param ver Version.
-     * @param drType Replication type.
-     * @param topVer Topology version.
-     * @param mvccVer Tx mvcc version.
-     * @throws IgniteCheckedException If failed.
-     */
-    void mvccReplicate(KeyCacheObject key,
-        @Nullable CacheObject val,
-        long ttl,
-        long expireTime,
-        GridCacheVersion ver,
-        GridDrType drType,
-        AffinityTopologyVersion topVer,
-        MvccVersion mvccVer) throws IgniteCheckedException;
-
-    /**
-     * @param mvccVer Tx mvcc version.
-     * @param commit {@code True} if tx committed, {@code False} otherwise.
-     * @param topVer Tx snapshot affinity version.
-     * @throws IgniteCheckedException If failed.
-     */
-    void onTxFinished(MvccVersion mvccVer, boolean commit, AffinityTopologyVersion topVer);
-
-    /**
      * Process partitions exchange event.
      *
      * @param topVer Topology version.


[3/5] ignite git commit: IGNITE-7953: MVCC: Continuous queries support. This closes #4767.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
index e66e7f1..328b023 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
@@ -57,6 +57,7 @@ import org.apache.ignite.transactions.Transaction;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -127,6 +128,20 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
     /**
      * @throws Exception If failed.
      */
+    public void testNonDeadLockInListenerMvccTx() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), true, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInListenerMvccTxJCacheApi() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), true, true, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testNonDeadLockInListenerAtomic() throws Exception {
         testNonDeadLockInListener(cacheConfiguration(PARTITIONED, 2, ATOMIC), true, true, false);
     }
@@ -194,6 +209,27 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
         testNonDeadLockInListener(cacheConfiguration(REPLICATED, 2, TRANSACTIONAL), true, true, true);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInListenerMvcc() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), true, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInListenerReplicatedMvcc() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT), true, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInListenerReplicatedJCacheApiMvcc() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT), true, true, true);
+    }
+
     ///
     /// ASYNC FILTER AND LISTENER. TEST FILTER.
     ///
@@ -215,6 +251,20 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
     /**
      * @throws Exception If failed.
      */
+    public void testNonDeadLockInFilterMvccTx() throws Exception {
+        testNonDeadLockInFilter(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), true, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInFilterMvccTxJCacheApi() throws Exception {
+        testNonDeadLockInFilter(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), true, true, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testNonDeadLockInFilterAtomic() throws Exception {
         testNonDeadLockInFilter(cacheConfiguration(PARTITIONED, 2, ATOMIC), true, true, false);
     }
@@ -261,6 +311,27 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
         testNonDeadLockInFilter(cacheConfiguration(REPLICATED, 2, TRANSACTIONAL), true, true, false);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInFilterMvcc() throws Exception {
+        testNonDeadLockInFilter(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), true, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInFilterReplicatedMvcc() throws Exception {
+        testNonDeadLockInFilter(cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT), true, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInFilterReplicatedJCacheApiMvcc() throws Exception {
+        testNonDeadLockInFilter(cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT), true, true, false);
+    }
+
     ///
     /// ASYNC LISTENER. TEST LISTENER.
     ///
@@ -275,6 +346,13 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
     /**
      * @throws Exception If failed.
      */
+    public void testNonDeadLockInFilterMvccTxSyncFilter() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), false, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testNonDeadLockInFilterAtomicSyncFilter() throws Exception {
         testNonDeadLockInListener(cacheConfiguration(PARTITIONED, 2, ATOMIC), false, true, false);
     }
@@ -308,6 +386,20 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInFilterSyncFilterMvcc() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT), false, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNonDeadLockInFilterReplicatedSyncFilterMvcc() throws Exception {
+        testNonDeadLockInListener(cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT), false, true, false);
+    }
+
+    /**
      * @param ccfg Cache configuration.
      * @param asyncFltr Async filter.
      * @param asyncLsnr Async listener.
@@ -371,18 +463,41 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
                             else if (!val.equals(val0))
                                 return;
 
-                            Transaction tx = null;
+                            // For MVCC mode we need to wait until updated value becomes visible. Usually this is
+                            // several ms to wait - mvcc coordinator need some time to register tx as finished.
+                            if (ccfg.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT) {
+                                Object v = null;
 
-                            try {
-                                if (cache0.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL)
-                                    tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+                                while (v == null && !Thread.currentThread().isInterrupted()) {
+                                    v = cache0.get(key);
 
+                                    if (v == null)
+                                        doSleep(50);
+                                }
+                            }
+
+                            try {
                                 assertEquals(val, val0);
 
-                                cache0.put(key, newVal);
+                                if (cache0.getConfiguration(CacheConfiguration.class).getAtomicityMode() != ATOMIC) {
+                                    boolean committed = false;
 
-                                if (tx != null)
-                                    tx.commit();
+                                    while (!committed && !Thread.currentThread().isInterrupted()) {
+                                        try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                                            cache0.put(key, newVal);
+
+                                            tx.commit();
+
+                                            committed =true;
+                                        }
+                                        catch (Exception ex) {
+                                            assertTrue(ex.toString(),
+                                                ex.getMessage() != null && ex.getMessage().contains("Cannot serialize transaction due to write conflict"));
+                                        }
+                                    }
+                                }
+                                else
+                                    cache0.put(key, newVal);
 
                                 latch.countDown();
                             }
@@ -391,10 +506,6 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
 
                                 throw new IgniteException(exp);
                             }
-                            finally {
-                                if (tx != null)
-                                    tx.close();
-                            }
                         }
                     };
 
@@ -521,19 +632,29 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
                             else if (!val.equals(val0))
                                 return;
 
-                            Transaction tx = null;
-
                             try {
-                                if (cache0.getConfiguration(CacheConfiguration.class)
-                                    .getAtomicityMode() == TRANSACTIONAL)
-                                    tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
-
                                 assertEquals(val, val0);
 
-                                cache0.put(key, newVal);
+                                if (cache0.getConfiguration(CacheConfiguration.class).getAtomicityMode() != ATOMIC) {
+                                    boolean committed = false;
 
-                                if (tx != null)
-                                    tx.commit();
+                                    while (!committed && !Thread.currentThread().isInterrupted()) {
+                                        try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+
+                                            cache0.put(key, newVal);
+
+                                            tx.commit();
+
+                                            committed =true;
+                                        }
+                                        catch (Exception ex) {
+                                            assertTrue(ex.toString(),
+                                                ex.getMessage() != null && ex.getMessage().contains("Cannot serialize transaction due to write conflict"));
+                                        }
+                                    }
+                                }
+                                else
+                                    cache0.put(key, newVal);
 
                                 latch.countDown();
                             }
@@ -542,10 +663,6 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
 
                                 throw new IgniteException(exp);
                             }
-                            finally {
-                                if (tx != null)
-                                    tx.close();
-                            }
                         }
                     };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java
index 6c0db43..53a1715 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryConcurrentPartitionUpdateTest.java
@@ -23,10 +23,12 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.CacheException;
 import javax.cache.event.CacheEntryEvent;
 import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.query.ContinuousQuery;
@@ -42,10 +44,14 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  *
@@ -92,6 +98,13 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo
     /**
      * @throws Exception If failed.
      */
+    public void testConcurrentUpdatePartitionMvccTx() throws Exception {
+        concurrentUpdatePartition(TRANSACTIONAL_SNAPSHOT, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testConcurrentUpdatePartitionAtomicCacheGroup() throws Exception {
         concurrentUpdatePartition(ATOMIC, true);
     }
@@ -104,6 +117,13 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentUpdatePartitionMvccTxCacheGroup() throws Exception {
+        concurrentUpdatePartition(TRANSACTIONAL_SNAPSHOT, true);
+    }
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @param cacheGrp {@code True} if test cache multiple caches in the same group.
      * @throws Exception If failed.
@@ -179,8 +199,30 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo
                     ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
                     for (int i = 0; i < UPDATES; i++) {
-                        for (int c = 0; c < srvCaches.size(); c++)
-                            srvCaches.get(c).put(keys.get(rnd.nextInt(KEYS)), i);
+                        for (int c = 0; c < srvCaches.size(); c++) {
+                            if (atomicityMode == ATOMIC)
+                                srvCaches.get(c).put(keys.get(rnd.nextInt(KEYS)), i);
+                            else  {
+                                IgniteCache<Object, Object> cache0 = srvCaches.get(c);
+                                IgniteTransactions txs = cache0.unwrap(Ignite.class).transactions();
+
+                                boolean committed = false;
+
+                                while (!committed) {
+                                    try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                                        cache0.put(keys.get(rnd.nextInt(KEYS)), i);
+
+                                        tx.commit();
+
+                                        committed = true;
+                                    }
+                                    catch (CacheException e) {
+                                        assertTrue(e.getMessage() != null &&
+                                            e.getMessage().contains("Cannot serialize transaction due to write conflict"));
+                                    }
+                                }
+                            }
+                        }
                     }
 
                     return null;
@@ -246,6 +288,13 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo
     /**
      * @throws Exception If failed.
      */
+    public void testConcurrentUpdatesAndQueryStartMvccTx() throws Exception {
+        concurrentUpdatesAndQueryStart(TRANSACTIONAL_SNAPSHOT, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testConcurrentUpdatesAndQueryStartAtomicCacheGroup() throws Exception {
         concurrentUpdatesAndQueryStart(ATOMIC, true);
     }
@@ -258,6 +307,13 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentUpdatesAndQueryStartMvccTxCacheGroup() throws Exception {
+        concurrentUpdatesAndQueryStart(TRANSACTIONAL_SNAPSHOT, true);
+    }
+
+    /**
      * @param atomicityMode Cache atomicity mode.
      * @param cacheGrp {@code True} if test cache multiple caches in the same group.
      * @throws Exception If failed.
@@ -333,8 +389,29 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo
                         ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
                         while (!stop.get()) {
-                            for (IgniteCache<Object, Object> srvCache : srvCaches)
-                                srvCache.put(keys.get(rnd.nextInt(KEYS)), rnd.nextInt(100) - 200);
+                            for (IgniteCache<Object, Object> srvCache : srvCaches)  {
+                                if (atomicityMode == ATOMIC)
+                                    srvCache.put(keys.get(rnd.nextInt(KEYS)), rnd.nextInt(100) - 200);
+                                else  {
+                                    IgniteTransactions txs = srvCache.unwrap(Ignite.class).transactions();
+
+                                    boolean committed = false;
+
+                                    while (!committed) {
+                                        try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                                            srvCache.put(keys.get(rnd.nextInt(KEYS)), rnd.nextInt(100) - 200);
+
+                                            tx.commit();
+
+                                            committed = true;
+                                        }
+                                        catch (CacheException e) {
+                                            assertTrue(e.getMessage() != null &&
+                                                e.getMessage().contains("Cannot serialize transaction due to write conflict"));
+                                        }
+                                    }
+                                }
+                            }
                         }
 
                         return null;
@@ -361,8 +438,29 @@ public class CacheContinuousQueryConcurrentPartitionUpdateTest extends GridCommo
                     ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
                     for (int i = 0; i < UPDATES; i++) {
-                        for (IgniteCache<Object, Object> srvCache : srvCaches)
-                            srvCache.put(keys.get(rnd.nextInt(KEYS)), i);
+                        for (IgniteCache<Object, Object> srvCache : srvCaches) {
+                            if (atomicityMode == ATOMIC)
+                                srvCache.put(keys.get(rnd.nextInt(KEYS)), i);
+                            else  {
+                                IgniteTransactions txs = srvCache.unwrap(Ignite.class).transactions();
+
+                                boolean committed = false;
+
+                                while (!committed) {
+                                    try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                                        srvCache.put(keys.get(rnd.nextInt(KEYS)), i);
+
+                                        tx.commit();
+
+                                        committed = true;
+                                    }
+                                    catch (CacheException e) {
+                                        assertTrue(e.getMessage() != null &&
+                                            e.getMessage().contains("Cannot serialize transaction due to write conflict"));
+                                    }
+                                }
+                            }
+                        }
                     }
 
                     return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryExecuteInPrimaryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryExecuteInPrimaryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryExecuteInPrimaryTest.java
index ffc7fd9..789504c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryExecuteInPrimaryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryExecuteInPrimaryTest.java
@@ -50,6 +50,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -159,6 +160,38 @@ public class CacheContinuousQueryExecuteInPrimaryTest extends GridCommonAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testMvccTransactionLocalCache() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-9530");
+
+        CacheConfiguration<Integer, String> ccfg = cacheConfiguration(TRANSACTIONAL_SNAPSHOT, LOCAL);
+
+        doTestWithoutEventsEntries(ccfg);
+        doTestWithEventsEntries(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTransactionReplicatedCache() throws Exception {
+        CacheConfiguration<Integer, String> ccfg = cacheConfiguration(TRANSACTIONAL_SNAPSHOT, REPLICATED);
+
+        doTestWithoutEventsEntries(ccfg);
+        doTestWithEventsEntries(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTransactionPartitionedCache() throws Exception {
+        CacheConfiguration<Integer, String> ccfg = cacheConfiguration(TRANSACTIONAL_SNAPSHOT, PARTITIONED);
+
+        doTestWithoutEventsEntries(ccfg);
+        doTestWithEventsEntries(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     private void doTestWithoutEventsEntries(CacheConfiguration<Integer, String> ccfg) throws Exception {
 
         try (IgniteCache<Integer, String> cache = grid(0).createCache(ccfg)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFactoryFilterRandomOperationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFactoryFilterRandomOperationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFactoryFilterRandomOperationTest.java
index 993ef79..d216d4f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFactoryFilterRandomOperationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFactoryFilterRandomOperationTest.java
@@ -46,6 +46,7 @@ import javax.cache.event.CacheEntryRemovedListener;
 import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.query.CacheQueryEntryEvent;
@@ -61,11 +62,12 @@ import org.jetbrains.annotations.NotNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFactoryFilterRandomOperationTest.NonSerializableFilter.isAccepted;
 import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.CLIENT;
 import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.SERVER;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
@@ -274,8 +276,16 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
 
         Transaction tx = null;
 
-        if (cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL && rnd.nextBoolean())
-            tx = ignite.transactions().txStart(txRandomConcurrency(rnd), txRandomIsolation(rnd));
+        CacheAtomicityMode atomicityMode = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode();
+
+        boolean mvccEnabled = atomicityMode == TRANSACTIONAL_SNAPSHOT;
+
+        if (atomicityMode != ATOMIC && rnd.nextBoolean()) {
+            TransactionConcurrency concurrency = mvccEnabled ? PESSIMISTIC : txRandomConcurrency(rnd);
+            TransactionIsolation isolation = mvccEnabled ? REPEATABLE_READ : txRandomIsolation(rnd);
+
+            tx = ignite.transactions().txStart(concurrency, isolation);
+        }
 
         try {
             // log.info("Random operation [key=" + key + ", op=" + op + ']');
@@ -287,7 +297,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr);
+                    updatePartitionCounter(cache, key, partCntr, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
@@ -302,7 +312,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr);
+                    updatePartitionCounter(cache, key, partCntr, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
@@ -312,12 +322,13 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                 }
 
                 case 2: {
-                    cache.remove(key);
+                    boolean res = cache.remove(key);
 
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr);
+                    // We don't update part counter if nothing was removed when MVCC enabled.
+                    updatePartitionCounter(cache, key, partCntr, mvccEnabled && !res);
 
                     waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, oldVal, oldVal);
 
@@ -327,12 +338,13 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                 }
 
                 case 3: {
-                    cache.getAndRemove(key);
+                    Object res = cache.getAndRemove(key);
 
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr);
+                    // We don't update part counter if nothing was removed when MVCC enabled.
+                    updatePartitionCounter(cache, key, partCntr, mvccEnabled && res == null);
 
                     waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, oldVal, oldVal);
 
@@ -347,7 +359,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr);
+                    updatePartitionCounter(cache, key, partCntr, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
@@ -357,12 +369,15 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                 }
 
                 case 5: {
-                    cache.invoke(key, new EntrySetValueProcessor(null, rnd.nextBoolean()));
+                    EntrySetValueProcessor proc = new EntrySetValueProcessor(null, rnd.nextBoolean());
+
+                    cache.invoke(key, proc);
 
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr);
+                    // We don't update part counter if nothing was removed when MVCC enabled.
+                    updatePartitionCounter(cache, key, partCntr, mvccEnabled && proc.getOldVal() == null);
 
                     waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, oldVal, oldVal);
 
@@ -378,7 +393,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                         tx.commit();
 
                     if (oldVal == null) {
-                        updatePartitionCounter(cache, key, partCntr);
+                        updatePartitionCounter(cache, key, partCntr, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, null);
 
@@ -397,7 +412,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                         tx.commit();
 
                     if (oldVal == null) {
-                        updatePartitionCounter(cache, key, partCntr);
+                        updatePartitionCounter(cache, key, partCntr, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, null);
 
@@ -416,7 +431,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                         tx.commit();
 
                     if (oldVal != null) {
-                        updatePartitionCounter(cache, key, partCntr);
+                        updatePartitionCounter(cache, key, partCntr, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
@@ -435,7 +450,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                         tx.commit();
 
                     if (oldVal != null) {
-                        updatePartitionCounter(cache, key, partCntr);
+                        updatePartitionCounter(cache, key, partCntr, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
@@ -459,7 +474,7 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
                             if (tx != null)
                                 tx.commit();
 
-                            updatePartitionCounter(cache, key, partCntr);
+                            updatePartitionCounter(cache, key, partCntr, false);
 
                             waitAndCheckEvent(evtsQueues, partCntr, affinity(cache), key, newVal, oldVal);
 
@@ -523,8 +538,10 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
      * @param cache Cache.
      * @param key Key
      * @param cntrs Partition counters.
+     * @param skipUpdCntr Skip update counter flag.
      */
-    private void updatePartitionCounter(IgniteCache<Object, Object> cache, Object key, Map<Integer, Long> cntrs) {
+    private void updatePartitionCounter(IgniteCache<Object, Object> cache, Object key, Map<Integer, Long> cntrs,
+        boolean skipUpdCntr) {
         Affinity<Object> aff = cache.unwrap(Ignite.class).affinity(cache.getName());
 
         int part = aff.partition(key);
@@ -534,7 +551,10 @@ public class CacheContinuousQueryFactoryFilterRandomOperationTest extends CacheC
         if (partCntr == null)
             partCntr = 0L;
 
-        cntrs.put(part, ++partCntr);
+        if (!skipUpdCntr)
+            partCntr++;
+
+        cntrs.put(part, partCntr);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
index bcbf1e0..150d94d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryLostPartitionTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import static javax.cache.configuration.FactoryBuilder.factoryOf;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
@@ -55,6 +56,9 @@ public class CacheContinuousQueryLostPartitionTest extends GridCommonAbstractTes
     /** Cache name. */
     public static final String TX_CACHE_NAME = "tx_test_cache";
 
+    /** Cache name. */
+    public static final String MVCC_TX_CACHE_NAME = "mvcc_tx_test_cache";
+
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
@@ -83,6 +87,13 @@ public class CacheContinuousQueryLostPartitionTest extends GridCommonAbstractTes
     /**
      * @throws Exception If failed.
      */
+    public void testMvccTxEvent() throws Exception {
+        testEvent(MVCC_TX_CACHE_NAME, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testAtomicEvent() throws Exception {
         testEvent(CACHE_NAME, false);
     }
@@ -97,6 +108,13 @@ public class CacheContinuousQueryLostPartitionTest extends GridCommonAbstractTes
     /**
      * @throws Exception If failed.
      */
+    public void testMvccTxClientEvent() throws Exception {
+        testEvent(MVCC_TX_CACHE_NAME, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testAtomicClientEvent() throws Exception {
         testEvent(CACHE_NAME, true);
     }
@@ -202,7 +220,7 @@ public class CacheContinuousQueryLostPartitionTest extends GridCommonAbstractTes
         spi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(spi);
-        cfg.setCacheConfiguration(cache(TX_CACHE_NAME), cache(CACHE_NAME));
+        cfg.setCacheConfiguration(cache(TX_CACHE_NAME), cache(CACHE_NAME), cache(MVCC_TX_CACHE_NAME));
 
         if (igniteInstanceName.endsWith("3"))
             cfg.setClientMode(true);
@@ -221,8 +239,10 @@ public class CacheContinuousQueryLostPartitionTest extends GridCommonAbstractTes
 
         if (cacheName.equals(CACHE_NAME))
             cfg.setAtomicityMode(ATOMIC);
-        else
+        else if (cacheName.equals(TX_CACHE_NAME))
             cfg.setAtomicityMode(TRANSACTIONAL);
+        else
+            cfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
 
         cfg.setRebalanceMode(SYNC);
         cfg.setWriteSynchronizationMode(PRIMARY_SYNC);

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
index 0540b43..3cb13bf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationFromCallbackTest.java
@@ -63,10 +63,13 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  *
@@ -197,6 +200,60 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxTwoBackupsFilter() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
+
+        doTest(ccfg, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxTwoBackupsFilterPrimary() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, PRIMARY_SYNC);
+
+        doTest(ccfg, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicatedFilter() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED, 0, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
+
+        doTest(ccfg, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxTwoBackup() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
+
+        doTest(ccfg, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicated() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
+
+        doTest(ccfg, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicatedPrimary() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED, 2, TRANSACTIONAL_SNAPSHOT, PRIMARY_SYNC);
+
+        doTest(ccfg, true);
+    }
+
+    /**
      * @param ccfg Cache configuration.
      * @throws Exception If failed.
      */
@@ -252,33 +309,50 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
 
                         QueryTestKey key = new QueryTestKey(rnd.nextInt(KEYS));
 
-                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() ==
-                            TRANSACTIONAL && rnd.nextBoolean();
+                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() !=
+                            ATOMIC && rnd.nextBoolean();
 
                         Transaction tx = null;
 
-                        if (startTx)
-                            tx = cache.unwrap(Ignite.class).transactions().txStart();
+                        boolean committed = false;
 
-                        try {
-                            if ((cache.get(key) == null) || rnd.nextBoolean())
-                                cache.invoke(key, new IncrementTestEntryProcessor());
-                            else {
-                                QueryTestValue val;
-                                QueryTestValue newVal;
+                        while (!committed && !Thread.currentThread().isInterrupted()) {
+                            try {
+                                if (startTx)
+                                    tx = cache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
 
-                                do {
-                                    val = cache.get(key);
+                                if ((cache.get(key) == null) || rnd.nextBoolean())
+                                    cache.invoke(key, new IncrementTestEntryProcessor());
+                                else {
+                                    QueryTestValue val;
+                                    QueryTestValue newVal;
 
-                                    newVal = val == null ?
-                                        new QueryTestValue(0) : new QueryTestValue(val.val1 + 1);
+                                    do {
+                                        val = cache.get(key);
+
+                                        newVal = val == null ?
+                                            new QueryTestValue(0) : new QueryTestValue(val.val1 + 1);
+                                    }
+                                    while (!cache.replace(key, val, newVal));
                                 }
-                                while (!cache.replace(key, val, newVal));
+
+                                if (tx != null)
+                                    tx.commit();
+
+                                committed = true;
+                            }
+                            catch (Exception e) {
+                                assertTrue(e.getMessage(), e.getMessage() != null &&
+                                    (e.getMessage().contains("Transaction has been rolled back") ||
+                                        e.getMessage().contains("Cannot serialize transaction due to write conflict")));
+
+                                // Wait MVCC updates become visible.
+                                doSleep(50);
+                            }
+                            finally {
+                                if (tx != null)
+                                    tx.close();
                             }
-                        }
-                        finally {
-                            if (tx != null)
-                                tx.commit();
                         }
                     }
                 }
@@ -302,7 +376,7 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
                     @Override public boolean apply() {
                         return cbCntr.get() >= expCnt;
                     }
-                }, TimeUnit.SECONDS.toMillis(60));
+                }, TimeUnit.SECONDS.toMillis(120));
 
                 assertTrue("Failed to wait events [exp=" + expCnt + ", act=" + cbCntr.get() + "]", res);
 
@@ -319,7 +393,7 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
                     @Override public boolean apply() {
                         return filterCbCntr.get() >= expInvkCnt;
                     }
-                }, TimeUnit.SECONDS.toMillis(60));
+                }, TimeUnit.SECONDS.toMillis(120));
 
                 assertEquals(expInvkCnt, filterCbCntr.get());
 
@@ -407,17 +481,45 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
             if (e.getKey().compareTo(new QueryTestKey(KEYS)) < 0) {
                 IgniteCache<QueryTestKey, QueryTestValue> cache = ignite.cache(cacheName);
 
-                if (ThreadLocalRandom.current().nextBoolean()) {
-                    Set<QueryTestKey> keys = new LinkedHashSet<>();
+                boolean committed = false;
+                Transaction tx = null;
+                boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL_SNAPSHOT;
+
+                while (!committed && !Thread.currentThread().isInterrupted()) {
+                    try {
+                        if (startTx)
+                            tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
 
-                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                        keys.add(new QueryTestKey(key));
+                        if (ThreadLocalRandom.current().nextBoolean()) {
+                            Set<QueryTestKey> keys = new LinkedHashSet<>();
 
-                    cache.invokeAll(keys, new IncrementTestEntryProcessor());
-                }
-                else {
-                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                        cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
+                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                                keys.add(new QueryTestKey(key));
+
+                            cache.invokeAll(keys, new IncrementTestEntryProcessor());
+                        }
+                        else {
+                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                                cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
+                        }
+
+                        if (tx != null)
+                            tx.commit();
+
+                        committed = true;
+                    }
+                    catch (Exception ex) {
+                        assertTrue(ex.getMessage(), ex.getMessage() != null &&
+                            (ex.getMessage().contains("Transaction has been rolled back") ||
+                                ex.getMessage().contains("Cannot serialize transaction due to write conflict")));
+
+                        // Wait MVCC updates become visible.
+                        doSleep(50);
+                    }
+                    finally {
+                        if (tx != null)
+                            tx.close();
+                    }
                 }
 
                 filterCbCntr.incrementAndGet();
@@ -477,17 +579,42 @@ public class CacheContinuousQueryOperationFromCallbackTest extends GridCommonAbs
                     cntr.incrementAndGet();
 
                     if (cache != null) {
-                        if (ThreadLocalRandom.current().nextBoolean()) {
-                            Set<QueryTestKey> keys = new LinkedHashSet<>();
+                        boolean committed = false;
+                        Transaction tx = null;
+                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL_SNAPSHOT;
 
-                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                                keys.add(new QueryTestKey(key));
+                        while (!committed && !Thread.currentThread().isInterrupted()) {
+                            try {
+                                if (startTx)
+                                    tx = cache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
 
-                            cache.invokeAll(keys, new IncrementTestEntryProcessor());
-                        }
-                        else {
-                            for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
-                                cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
+                                if (ThreadLocalRandom.current().nextBoolean()) {
+                                    Set<QueryTestKey> keys = new LinkedHashSet<>();
+
+                                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                                        keys.add(new QueryTestKey(key));
+
+                                    cache.invokeAll(keys, new IncrementTestEntryProcessor());
+                                }
+                                else {
+                                    for (int key = KEYS; key < KEYS + KEYS_FROM_CALLBACK; key++)
+                                        cache.invoke(new QueryTestKey(key), new IncrementTestEntryProcessor());
+                                }
+
+                                if (tx != null)
+                                    tx.commit();
+
+                                committed = true;
+                            }
+                            catch (Exception ex) {
+                                assertTrue(ex.getMessage(), ex.getMessage() != null &&
+                                    (ex.getMessage().contains("Transaction has been rolled back") ||
+                                        ex.getMessage().contains("Cannot serialize transaction due to write conflict")));
+                            }
+                            finally {
+                                if (tx != null)
+                                    tx.close();
+                            }
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationP2PTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationP2PTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationP2PTest.java
index eb53fa3..f2889df 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationP2PTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOperationP2PTest.java
@@ -42,6 +42,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -185,6 +186,53 @@ public class CacheContinuousQueryOperationP2PTest extends GridCommonAbstractTest
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTx() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT
+        );
+
+        testContinuousQuery(ccfg, false);
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT
+        );
+
+        testContinuousQuery(ccfg, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicated() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL_SNAPSHOT
+        );
+
+        testContinuousQuery(ccfg, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicatedClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL_SNAPSHOT
+        );
+
+        testContinuousQuery(ccfg, true);
+    }
+
+    /**
      * @param ccfg Cache configuration.
      * @param isClient Client.
      * @throws Exception If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java
index 7442e24..1ad45dd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryOrderingEventTest.java
@@ -62,10 +62,13 @@ import org.apache.ignite.transactions.Transaction;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  *
@@ -163,6 +166,33 @@ public class CacheContinuousQueryOrderingEventTest extends GridCommonAbstractTes
         doOrderingTest(ccfg, false);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxOnheapTwoBackup() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
+
+        doOrderingTest(ccfg, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxOnheapWithoutBackup() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL_SNAPSHOT, PRIMARY_SYNC);
+
+        doOrderingTest(ccfg, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxOnheapWithoutBackupFullSync() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
+
+        doOrderingTest(ccfg, false);
+    }
+
     // ASYNC
 
     /**
@@ -238,6 +268,33 @@ public class CacheContinuousQueryOrderingEventTest extends GridCommonAbstractTes
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxOnheapTwoBackupAsync() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 2, TRANSACTIONAL_SNAPSHOT, PRIMARY_SYNC);
+
+        doOrderingTest(ccfg, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxOnheapAsync() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL_SNAPSHOT, PRIMARY_SYNC);
+
+        doOrderingTest(ccfg, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxOnheapAsyncFullSync() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL_SNAPSHOT, FULL_SYNC);
+
+        doOrderingTest(ccfg, true);
+    }
+
+    /**
      * @param ccfg Cache configuration.
      * @param async Async filter.
      * @throws Exception If failed.
@@ -298,46 +355,60 @@ public class CacheContinuousQueryOrderingEventTest extends GridCommonAbstractTes
 
                         QueryTestKey key = new QueryTestKey(rnd.nextInt(KEYS));
 
-                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() ==
-                            TRANSACTIONAL && rnd.nextBoolean();
+                        boolean startTx = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() !=
+                            ATOMIC && rnd.nextBoolean();
 
                         Transaction tx = null;
 
-                        if (startTx)
-                            tx = cache.unwrap(Ignite.class).transactions().txStart();
-
-                        try {
-                            if ((cache.get(key) == null) || rnd.nextBoolean()) {
-                                cache.invoke(key, new CacheEntryProcessor<QueryTestKey, QueryTestValue, Object>() {
-                                    @Override public Object process(
-                                        MutableEntry<QueryTestKey, QueryTestValue> entry,
-                                        Object... arguments)
-                                        throws EntryProcessorException {
-                                        if (entry.exists())
-                                            entry.setValue(new QueryTestValue(entry.getValue().val1 + 1));
-                                        else
-                                            entry.setValue(new QueryTestValue(0));
-
-                                        return null;
-                                    }
-                                });
-                            }
-                            else {
-                                QueryTestValue val;
-                                QueryTestValue newVal;
+                        boolean committed = false;
+
+                        while (!committed && !Thread.currentThread().isInterrupted()) {
+                            try {
+                                if (startTx)
+                                    tx = cache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+
+                                if ((cache.get(key) == null) || rnd.nextBoolean()) {
+                                    cache.invoke(key, new CacheEntryProcessor<QueryTestKey, QueryTestValue, Object>() {
+                                        @Override public Object process(
+                                            MutableEntry<QueryTestKey, QueryTestValue> entry,
+                                            Object... arguments)
+                                            throws EntryProcessorException {
+                                            if (entry.exists())
+                                                entry.setValue(new QueryTestValue(entry.getValue().val1 + 1));
+                                            else
+                                                entry.setValue(new QueryTestValue(0));
+
+                                            return null;
+                                        }
+                                    });
+                                }
+                                else {
+                                    QueryTestValue val;
+                                    QueryTestValue newVal;
 
-                                do {
-                                    val = cache.get(key);
+                                    do {
+                                        val = cache.get(key);
 
-                                    newVal = val == null ?
-                                        new QueryTestValue(0) : new QueryTestValue(val.val1 + 1);
+                                        newVal = val == null ?
+                                            new QueryTestValue(0) : new QueryTestValue(val.val1 + 1);
+                                    }
+                                    while (!cache.replace(key, val, newVal));
                                 }
-                                while (!cache.replace(key, val, newVal));
+
+                                if (tx != null)
+                                    tx.commit();
+
+                                committed = true;
+                            }
+                            catch (Exception e) {
+                                assertTrue(e.getMessage(), e.getMessage() != null &&
+                                    (e.getMessage().contains("Transaction has been rolled back") ||
+                                        e.getMessage().contains("Cannot serialize transaction due to write conflict")));
+                            }
+                            finally {
+                                if (tx != null)
+                                    tx.close();
                             }
-                        }
-                        finally {
-                            if (tx != null)
-                                tx.commit();
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
index 6d15df2..af92c31 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryRandomOperationsTest.java
@@ -84,12 +84,14 @@ import static javax.cache.event.EventType.REMOVED;
 import static javax.cache.event.EventType.UPDATED;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.ALL;
 import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.CLIENT;
 import static org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest.ContinuousDeploy.SERVER;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
@@ -330,6 +332,42 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testMvccTx() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testDoubleRemoveAtomicWithoutBackup() throws Exception {
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             0,
@@ -346,7 +384,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             0,
             ATOMIC,
-            false);
+            true);
 
         doTestNotModifyOperation(ccfg);
     }
@@ -394,7 +432,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
             1,
             TRANSACTIONAL,
-            false);
+            true);
 
         doTestNotModifyOperation(ccfg);
     }
@@ -418,6 +456,18 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
             0,
             TRANSACTIONAL,
+            true);
+
+        doTestNotModifyOperation(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleRemoveMvccTx() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT,
             false);
 
         doTestNotModifyOperation(ccfg);
@@ -426,6 +476,46 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testDoubleRemoveMvccTxWithStore() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8582");
+
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT,
+            true);
+
+        doTestNotModifyOperation(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleRemoveReplicatedMvccTx() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestNotModifyOperation(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleRemoveReplicatedMvccTxWithStore() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8582");
+
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            true);
+
+        doTestNotModifyOperation(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testDoubleRemoveReplicatedAtomic() throws Exception {
         CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
             0,
@@ -827,6 +917,102 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, CLIENT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxClientExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            1,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, CLIENT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicated() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxReplicatedClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(REPLICATED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, CLIENT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxNoBackups() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxNoBackupsAllNodes() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxNoBackupsExplicit() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, SERVER);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMvccTxNoBackupsClient() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED,
+            0,
+            TRANSACTIONAL_SNAPSHOT,
+            false);
+
+        doTestContinuousQuery(ccfg, CLIENT);
+    }
+
+    /**
      * @param ccfg Cache configuration.
      * @param deploy The place where continuous query will be started.
      * @throws Exception If failed.
@@ -988,11 +1174,19 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
 
         Transaction tx = null;
 
-        if (cache.getConfiguration(CacheConfiguration.class).getAtomicityMode() == TRANSACTIONAL && rnd.nextBoolean())
-            tx = ignite.transactions().txStart(txRandomConcurrency(rnd), txRandomIsolation(rnd));
+        CacheAtomicityMode atomicityMode = cache.getConfiguration(CacheConfiguration.class).getAtomicityMode();
+
+        boolean mvccEnabled = atomicityMode == TRANSACTIONAL_SNAPSHOT;
+
+        if (atomicityMode != ATOMIC && rnd.nextBoolean()) {
+            TransactionConcurrency concurrency = mvccEnabled ? PESSIMISTIC : txRandomConcurrency(rnd);
+            TransactionIsolation isolation = mvccEnabled ? REPEATABLE_READ : txRandomIsolation(rnd);
+
+            tx = ignite.transactions().txStart(concurrency, isolation);
+        }
 
         try {
-            // log.info("Random operation [key=" + key + ", op=" + op + ']');
+            log.info("Random operation [key=" + key + ", op=" + op + ']');
 
             switch (op) {
                 case 0: {
@@ -1001,7 +1195,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, oldVal);
 
@@ -1016,7 +1210,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, oldVal);
 
@@ -1026,12 +1220,13 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                 }
 
                 case 2: {
-                    cache.remove(key);
+                    boolean res = cache.remove(key);
 
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                    // We don't update part counter if nothing was removed when MVCC enabled.
+                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs, mvccEnabled && !res);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, oldVal, oldVal);
 
@@ -1041,12 +1236,13 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                 }
 
                 case 3: {
-                    cache.getAndRemove(key);
+                    Object res = cache.getAndRemove(key);
 
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                    // We don't update part counter if nothing was removed when MVCC enabled.
+                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs, mvccEnabled && res == null);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, oldVal, oldVal);
 
@@ -1061,7 +1257,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, oldVal);
 
@@ -1071,12 +1267,15 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                 }
 
                 case 5: {
-                    cache.invoke(key, new EntrySetValueProcessor(null, rnd.nextBoolean()));
+                    EntrySetValueProcessor proc = new EntrySetValueProcessor(null, rnd.nextBoolean());
+
+                    cache.invoke(key, proc);
 
                     if (tx != null)
                         tx.commit();
 
-                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                    // We don't update part counter if nothing was removed when MVCC enabled.
+                    updatePartitionCounter(cache, key, partCntr, expEvtCntrs,mvccEnabled && proc.getOldVal() == null);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, oldVal, oldVal);
 
@@ -1092,7 +1291,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                         tx.commit();
 
                     if (oldVal == null) {
-                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, null);
 
@@ -1111,7 +1310,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                         tx.commit();
 
                     if (oldVal == null) {
-                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, null);
 
@@ -1130,7 +1329,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                         tx.commit();
 
                     if (oldVal != null) {
-                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, oldVal);
 
@@ -1149,7 +1348,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                         tx.commit();
 
                     if (oldVal != null) {
-                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                        updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                         waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, oldVal);
 
@@ -1173,7 +1372,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                             if (tx != null)
                                 tx.commit();
 
-                            updatePartitionCounter(cache, key, partCntr, expEvtCntrs);
+                            updatePartitionCounter(cache, key, partCntr, expEvtCntrs, false);
 
                             waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), key, newVal, oldVal);
 
@@ -1212,7 +1411,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                         tx.commit();
 
                     for (Map.Entry<Object, Object> e : vals.entrySet())
-                        updatePartitionCounter(cache, e.getKey(), partCntr, expEvtCntrs);
+                        updatePartitionCounter(cache, e.getKey(), partCntr, expEvtCntrs, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), vals, expData);
 
@@ -1233,7 +1432,7 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
                         tx.commit();
 
                     for (Map.Entry<Object, Object> e : vals.entrySet())
-                        updatePartitionCounter(cache, e.getKey(), partCntr, expEvtCntrs);
+                        updatePartitionCounter(cache, e.getKey(), partCntr, expEvtCntrs, false);
 
                     waitAndCheckEvent(evtsQueues, partCntr, expEvtCntrs, affinity(cache), vals, expData);
 
@@ -1330,16 +1529,18 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
      * @return {@link TransactionConcurrency}.
      */
     private TransactionConcurrency txRandomConcurrency(Random rnd) {
-        return rnd.nextBoolean() ? TransactionConcurrency.OPTIMISTIC : TransactionConcurrency.PESSIMISTIC;
+        return rnd.nextBoolean() ? TransactionConcurrency.OPTIMISTIC : PESSIMISTIC;
     }
 
     /**
      * @param cache Cache.
      * @param key Key
      * @param cntrs Partition counters.
+     * @param evtCntrs Event counters.
+     * @param skipUpdCntr Skip update counter flag.
      */
     private void updatePartitionCounter(IgniteCache<Object, Object> cache, Object key, Map<Integer, Long> cntrs,
-        Map<Object, Long> evtCntrs) {
+        Map<Object, Long> evtCntrs, boolean skipUpdCntr) {
         Affinity<Object> aff = cache.unwrap(Ignite.class).affinity(cache.getName());
 
         int part = aff.partition(key);
@@ -1349,7 +1550,10 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
         if (partCntr == null)
             partCntr = 0L;
 
-        cntrs.put(part, ++partCntr);
+        if (!skipUpdCntr)
+            partCntr++;
+
+        cntrs.put(part, partCntr);
         evtCntrs.put(key, partCntr);
     }
 
@@ -1577,6 +1781,12 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
      *
      */
     protected static class EntrySetValueProcessor implements EntryProcessor<Object, Object, Object> {
+        /**
+         * Static variable: we need to obtain a previous value from another node.
+         * Assume this is a single threaded execution.
+         */
+        private static Object oldVal;
+
         /** */
         private Object val;
 
@@ -1607,6 +1817,8 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             if (skipModify)
                 return null;
 
+            oldVal = e.getValue();
+
             Object old = retOld ? e.getValue() : null;
 
             if (val != null)
@@ -1617,6 +1829,17 @@ public class CacheContinuousQueryRandomOperationsTest extends GridCommonAbstract
             return old;
         }
 
+        /**
+         * @return Old value.
+         */
+        Object getOldVal() {
+            Object oldVal0 = oldVal;
+
+            oldVal = null; // Clean value.
+
+            return oldVal0;
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(EntrySetValueProcessor.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
index 7aa91a2..60e2f80 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousWithTransformerReplicatedSelfTest.java
@@ -32,6 +32,7 @@ import javax.cache.expiry.Duration;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.query.ContinuousQueryWithTransformer;
@@ -100,6 +101,7 @@ public class CacheContinuousWithTransformerReplicatedSelfTest extends GridCommon
             CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
 
             ccfg.setCacheMode(cacheMode());
+            ccfg.setAtomicityMode(atomicityMode());
 
             cfg.setCacheConfiguration(ccfg);
         }
@@ -107,6 +109,13 @@ public class CacheContinuousWithTransformerReplicatedSelfTest extends GridCommon
         return cfg;
     }
 
+    /**
+     * @return Cache atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         gridToRunQuery().cache(DEFAULT_CACHE_NAME).removeAll();

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationNearEnabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationNearEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationNearEnabledTest.java
index 9eb56dc..0684d1f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationNearEnabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationNearEnabledTest.java
@@ -39,4 +39,13 @@ public class CacheKeepBinaryIterationNearEnabledTest extends CacheKeepBinaryIter
         return ccfg;
     }
 
+    /** {@inheritDoc} */
+    @Override public void testMvccTxOnHeap() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMvccTxOnHeapLocalEntries() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-7187");
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationStoreEnabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationStoreEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationStoreEnabledTest.java
index f98229d..910ce4a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationStoreEnabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheKeepBinaryIterationStoreEnabledTest.java
@@ -50,6 +50,16 @@ public class CacheKeepBinaryIterationStoreEnabledTest extends CacheKeepBinaryIte
         return ccfg;
     }
 
+    /** {@inheritDoc} */
+    @Override public void testMvccTxOnHeap() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8582");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMvccTxOnHeapLocalEntries() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-8582");
+    }
+
     /**
      *
      */


[4/5] ignite git commit: IGNITE-7953: MVCC: Continuous queries support. This closes #4767.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
index 8d7e4d8..f3c1b23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.lang.IgniteFuture;
@@ -79,18 +78,6 @@ public class GridOsCacheDrManager implements GridCacheDrManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void mvccReplicate(KeyCacheObject key, @Nullable CacheObject val, long ttl, long expireTime,
-        GridCacheVersion ver, GridDrType drType, AffinityTopologyVersion topVer,
-        MvccVersion mvccVer) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onTxFinished(MvccVersion mvccVer, boolean commit, AffinityTopologyVersion topVer) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void onExchange(AffinityTopologyVersion topVer, boolean left) throws IgniteCheckedException {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java
new file mode 100644
index 0000000..259f69b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccCachingManager.java
@@ -0,0 +1,341 @@
+/*
+ * 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.mvcc;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage;
+import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxKey;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryListener;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.cache.transactions.TxCounters;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_MVCC_TX_SIZE_CACHING_THRESHOLD;
+import static org.apache.ignite.internal.processors.dr.GridDrType.DR_BACKUP;
+import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRIMARY;
+
+/**
+ * Manager for caching MVCC transaction updates.
+ * This updates can be used further in CQ, DR and other places.
+ */
+public class MvccCachingManager extends GridCacheSharedManagerAdapter {
+    /** Maximum possible transaction size when caching is enabled. */
+    public static final int TX_SIZE_THRESHOLD = IgniteSystemProperties.getInteger(IGNITE_MVCC_TX_SIZE_CACHING_THRESHOLD,
+        20_000);
+
+    /** Cached enlist values*/
+    private final Map<GridCacheVersion, EnlistBuffer> enlistCache = new ConcurrentHashMap<>();
+
+    /** Counters map. Used for OOM prevention caused by the big transactions. */
+    private final Map<TxKey, AtomicInteger> cntrs = new ConcurrentHashMap<>();
+
+    /**
+     * Adds enlisted tx entry to cache.
+     * @param key Key.
+     * @param val Value.
+     * @param ttl Time to live.
+     * @param expireTime Expire time.
+     * @param ver Version.
+     * @param oldVal Old value.
+     * @param primary Flag whether this is a primary node.
+     * @param topVer Topology version.
+     * @param mvccVer Mvcc version.
+     * @param cacheId Cache id.
+     * @param tx Transaction.
+     * @param futId Dht future id.
+     * @param batchNum Batch number (for batches reordering prevention).
+     * @throws IgniteCheckedException If failed.
+     */
+    public void addEnlisted(KeyCacheObject key,
+        @Nullable CacheObject val,
+        long ttl,
+        long expireTime,
+        GridCacheVersion ver,
+        CacheObject oldVal,
+        boolean primary,
+        AffinityTopologyVersion topVer,
+        MvccVersion mvccVer,
+        int cacheId,
+        IgniteInternalTx tx,
+        IgniteUuid futId,
+        int batchNum) throws IgniteCheckedException {
+        assert key != null;
+        assert mvccVer != null;
+        assert tx != null;
+
+        if (log.isDebugEnabled()) {
+            log.debug("Added entry to mvcc cache: [key=" + key + ", val=" + val + ", oldVal=" + oldVal +
+                ", primary=" + primary + ", mvccVer=" + mvccVer + ", cacheId=" + cacheId + ", ver=" + ver +']');
+        }
+
+        GridCacheContext ctx0 = cctx.cacheContext(cacheId);
+
+        // Do not cache updates if there is no DR or CQ enabled.
+        if (!needDrReplicate(ctx0, key) &&
+            F.isEmpty(continuousQueryListeners(ctx0, tx, key)) &&
+            !ctx0.group().hasContinuousQueryCaches())
+            return;
+
+        AtomicInteger cntr = cntrs.computeIfAbsent(new TxKey(mvccVer.coordinatorVersion(), mvccVer.counter()),
+            v -> new AtomicInteger());
+
+        if (cntr.incrementAndGet() > TX_SIZE_THRESHOLD)
+            throw new IgniteCheckedException("Transaction is too large. Consider reducing transaction size or " +
+                "turning off continuous queries and datacenter replication [size=" + cntr.get() + ", txXid=" + ver + ']');
+
+        MvccTxEntry e = new MvccTxEntry(key, val, ttl, expireTime, ver, oldVal, primary, topVer, mvccVer, cacheId);
+
+        EnlistBuffer cached = enlistCache.computeIfAbsent(ver, v -> new EnlistBuffer());
+
+        cached.add(primary ? null : futId, primary ? -1 : batchNum, key, e);
+    }
+
+    /**
+     *
+     * @param tx Transaction.
+     * @param commit {@code True} if commit.
+     */
+    public void onTxFinished(IgniteInternalTx tx, boolean commit) throws IgniteCheckedException {
+        if (log.isDebugEnabled())
+            log.debug("Transaction finished: [commit=" + commit + ", tx=" + tx  + ']');
+
+        if (tx.system() || tx.internal() || tx.mvccSnapshot() == null)
+            return;
+
+        cntrs.remove(new TxKey(tx.mvccSnapshot().coordinatorVersion(), tx.mvccSnapshot().counter()));
+
+        EnlistBuffer buf = enlistCache.remove(tx.xidVersion());
+
+        if (buf == null)
+            return;
+
+        Map<KeyCacheObject, MvccTxEntry> cached = buf.getCached();
+
+        if (F.isEmpty(cached) || !commit)
+            return;
+
+        TxCounters txCntrs = tx.txCounters(false);
+
+        assert txCntrs != null;
+
+        Collection<PartitionUpdateCountersMessage> cntrsColl =  txCntrs.updateCounters();
+
+        assert  !F.isEmpty(cntrsColl) : cntrsColl;
+
+        // cacheId -> partId -> initCntr -> cntr + delta.
+        Map<Integer, Map<Integer, T2<AtomicLong, Long>>> cntrsMap = new HashMap<>();
+
+        for (PartitionUpdateCountersMessage msg : cntrsColl) {
+            for (int i = 0; i < msg.size(); i++) {
+                Map<Integer, T2<AtomicLong, Long>> cntrPerPart =
+                    cntrsMap.computeIfAbsent(msg.cacheId(), k -> new HashMap<>());
+
+                T2 prev = cntrPerPart.put(msg.partition(i),
+                    new T2<>(new AtomicLong(msg.initialCounter(i)), msg.initialCounter(i) + msg.updatesCount(i)));
+
+                assert prev == null;
+            }
+        }
+
+        // Feed CQ & DR with entries.
+        for (Map.Entry<KeyCacheObject, MvccTxEntry> entry : cached.entrySet()) {
+            MvccTxEntry e = entry.getValue();
+
+            assert e.key().partition() != -1;
+
+            Map<Integer, T2<AtomicLong, Long>> cntrPerCache = cntrsMap.get(e.cacheId());
+
+            GridCacheContext ctx0 = cctx.cacheContext(e.cacheId());
+
+            assert ctx0 != null && cntrPerCache != null;
+
+            T2<AtomicLong, Long> cntr = cntrPerCache.get(e.key().partition());
+
+            long resCntr = cntr.getKey().incrementAndGet();
+
+            assert resCntr <= cntr.getValue();
+
+            e.updateCounter(resCntr);
+
+            if (ctx0.group().sharedGroup()) {
+                ctx0.group().onPartitionCounterUpdate(ctx0.cacheId(), e.key().partition(), resCntr,
+                    tx.topologyVersion(), tx.local());
+            }
+
+            if (log.isDebugEnabled())
+                log.debug("Process cached entry:" + e);
+
+            // DR
+            if (ctx0.isDrEnabled()) {
+                ctx0.dr().replicate(e.key(), e.value(), e.ttl(), e.expireTime(), e.version(),
+                    tx.local() ? DR_PRIMARY : DR_BACKUP, e.topologyVersion());
+            }
+
+            // CQ
+            CacheContinuousQueryManager contQryMgr = ctx0.continuousQueries();
+
+            if (ctx0.continuousQueries().notifyContinuousQueries(tx)) {
+                contQryMgr.getListenerReadLock().lock();
+
+                try {
+                    Map<UUID, CacheContinuousQueryListener> lsnrCol = continuousQueryListeners(ctx0, tx, e.key());
+
+                    if (!F.isEmpty(lsnrCol)) {
+                        contQryMgr.onEntryUpdated(
+                            lsnrCol,
+                            e.key(),
+                            e.value(),
+                            e.oldValue(),
+                            false,
+                            e.key().partition(),
+                            tx.local(),
+                            false,
+                            e.updateCounter(),
+                            null,
+                            e.topologyVersion());
+                    }
+                }
+                finally {
+                    contQryMgr.getListenerReadLock().unlock();
+                }
+            }
+        }
+    }
+
+    /**
+     * @param ctx0 Cache context.
+     * @param key Key.
+     * @return {@code True} if need to replicate this value.
+     */
+    public boolean needDrReplicate(GridCacheContext ctx0, KeyCacheObject key) {
+        return ctx0.isDrEnabled() && !key.internal();
+    }
+
+    /**
+     * @param ctx0 Cache context.
+     * @param tx Transaction.
+     * @param key Key.
+     * @return Map of listeners to be notified by this update.
+     */
+    public Map<UUID, CacheContinuousQueryListener> continuousQueryListeners(GridCacheContext ctx0, @Nullable IgniteInternalTx tx, KeyCacheObject key) {
+        boolean internal = key.internal() || !ctx0.userCache();
+
+        return ctx0.continuousQueries().notifyContinuousQueries(tx) ?
+            ctx0.continuousQueries().updateListeners(internal, false) : null;
+    }
+
+    /**
+     * Buffer for collecting enlisted entries. The main goal of this buffer is to fix reordering of dht enlist requests
+     * on backups.
+     */
+    private static class EnlistBuffer {
+        /** Last DHT future id. */
+        private IgniteUuid lastFutId;
+
+        /** Main buffer for entries. */
+        private Map<KeyCacheObject, MvccTxEntry> cached = new LinkedHashMap<>();
+
+        /** Pending entries. */
+        private SortedMap<Integer, Map<KeyCacheObject, MvccTxEntry>> pending;
+
+        /**
+         * Adds entry to caching buffer.
+         *
+         * @param futId Future id.
+         * @param batchNum Batch number.
+         * @param key Key.
+         * @param e Entry.
+         */
+        synchronized void add(IgniteUuid futId, int batchNum, KeyCacheObject key, MvccTxEntry e) {
+            if (batchNum >= 0) {
+                /*
+                 * Assume that batches within one future may be reordered. But batches between futures cannot be
+                 * reordered. This means that if batches from the new DHT future has arrived, all batches from the
+                 * previous one has already been collected.
+                 */
+                if (lastFutId != null && !lastFutId.equals(futId)) { // Request from new DHT future arrived.
+                    lastFutId = futId;
+
+                    // Flush pending for previous future.
+                    flushPending();
+                }
+
+                if (pending == null)
+                    pending = new TreeMap<>() ;
+
+                MvccTxEntry prev = pending.computeIfAbsent(batchNum, k -> new LinkedHashMap<>()).put(key, e);
+
+                if (prev != null && prev.oldValue() != null)
+                    e.oldValue(prev.oldValue());
+            }
+            else { // batchNum == -1 means no reordering (e.g. this is a primary node).
+                assert batchNum == -1;
+
+                MvccTxEntry prev = cached.put(key, e);
+
+                if (prev != null && prev.oldValue() != null)
+                    e.oldValue(prev.oldValue());
+            }
+        }
+
+        /**
+         * @return Cached entries map.
+         */
+        synchronized Map<KeyCacheObject, MvccTxEntry> getCached() {
+            flushPending();
+
+            return cached;
+        }
+
+        /**
+         * Flush pending updates to cached map.
+         */
+        private void flushPending() {
+            if (F.isEmpty(pending))
+                return;
+
+            for (Map.Entry<Integer, Map<KeyCacheObject, MvccTxEntry>> entry : pending.entrySet()) {
+                Map<KeyCacheObject, MvccTxEntry> vals = entry.getValue();
+
+                cached.putAll(vals);
+            }
+
+            pending.clear();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
index 40d4ac9..e377b0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccProcessorImpl.java
@@ -287,7 +287,8 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
             startVacuumWorkers();
 
-            log.info("Mvcc processor started.");
+            if (log.isInfoEnabled())
+                log.info("Mvcc processor started.");
         }
     }
 
@@ -380,8 +381,9 @@ public class MvccProcessorImpl extends GridProcessorAdapter implements MvccProce
 
             crdVer = crd.coordinatorVersion();
 
-            log.info("Initialize local node as mvcc coordinator [node=" + ctx.localNodeId() +
-                ", crdVer=" + crdVer + ']');
+            if (log.isInfoEnabled())
+                log.info("Initialize local node as mvcc coordinator [node=" + ctx.localNodeId() +
+                    ", crdVer=" + crdVer + ']');
 
             prevCrdQueries.init(activeQueries, F.view(discoCache.allNodes(), this::supportsMvcc), ctx.discovery());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccTxEntry.java
new file mode 100644
index 0000000..28864f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccTxEntry.java
@@ -0,0 +1,203 @@
+/*
+ * 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.mvcc;
+
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Holder for the enlisted entries data.
+ */
+public class MvccTxEntry {
+    /** */
+    private KeyCacheObject key;
+
+    /** */
+    private CacheObject val;
+
+    /** */
+    private int cacheId;
+
+    /** */
+    private GridCacheVersion ver;
+
+    /** */
+    private CacheObject oldVal;
+
+    /** */
+    private boolean primary;
+
+    /** */
+    private AffinityTopologyVersion topVer;
+
+    /** */
+    private MvccVersion mvccVer;
+
+    /** */
+    private long ttl;
+
+    /** */
+    private long expireTime;
+
+    /** */
+    private long updCntr;
+
+    /**
+     * @param key Key.
+     * @param val New value.
+     * @param ttl Time to live.
+     * @param expireTime Expire time.
+     * @param ver Tx grig cache version.
+     * @param oldVal Old value.
+     * @param primary {@code True} if this is a primary node.
+     * @param topVer Topology version.
+     * @param mvccVer Mvcc version.
+     * @param cacheId Cache id.
+     */
+    public MvccTxEntry(KeyCacheObject key,
+        @Nullable CacheObject val,
+        long ttl,
+        long expireTime,
+        GridCacheVersion ver,
+        CacheObject oldVal,
+        boolean primary,
+        AffinityTopologyVersion topVer,
+        MvccVersion mvccVer,
+        int cacheId) {
+        assert key != null;
+        assert mvccVer != null;
+
+        this.key = key;
+        this.val = val;
+        this.ttl = ttl;
+        this.expireTime = expireTime;
+        this.ver = ver;
+        this.oldVal = oldVal;
+        this.primary = primary;
+        this.topVer = topVer;
+        this.mvccVer = mvccVer;
+        this.cacheId = cacheId;
+    }
+
+    /**
+     * @return Versioned entry (for DR).
+     */
+    public GridCacheRawVersionedEntry versionedEntry() {
+        return new GridCacheRawVersionedEntry(key, val, ttl, expireTime, ver);
+    }
+
+    /**
+     * @return Key.
+     */
+    public KeyCacheObject key() {
+        return key;
+    }
+
+    /**
+     * @return Value.
+     */
+    public CacheObject value() {
+        return val;
+    }
+
+    /**
+     * @return Time to live.
+     */
+    public long ttl() {
+        return ttl;
+    }
+
+    /**
+     * @return Expire time.
+     */
+    public long expireTime() {
+        return expireTime;
+    }
+
+    /**
+     * @return Version.
+     */
+    public GridCacheVersion version() {
+        return ver;
+    }
+
+    /**
+     * @return Old value.
+     */
+    public CacheObject oldValue() {
+        return oldVal;
+    }
+
+    /**
+     * @param oldVal Old value.
+     */
+    public void oldValue(CacheObject oldVal) {
+        this.oldVal = oldVal;
+    }
+
+    /**
+     * @return {@code True} if this entry is created on a primary node.
+     */
+    public boolean isPrimary() {
+        return primary;
+    }
+
+    /**
+     * @return Topology version.
+     */
+    public AffinityTopologyVersion topologyVersion() {
+        return topVer;
+    }
+
+    /**
+     * @return Mvcc version.
+     */
+    public MvccVersion mvccVersion() {
+        return mvccVer;
+    }
+
+    /**
+     * @return Cache id.
+     */
+    public int cacheId() {
+        return cacheId;
+    }
+
+    /**
+     * @return Update counter.
+     */
+    public long updateCounter() {
+        return updCntr;
+    }
+
+    /**
+     * @param updCntr Update counter.
+     */
+    public void updateCounter(long updCntr) {
+        this.updCntr = updCntr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MvccTxEntry.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java
index fee928e..17804c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUpdateVersionAware.java
@@ -73,9 +73,4 @@ public interface MvccUpdateVersionAware {
     public default MvccVersion newMvccVersion() {
         return new MvccVersionImpl(newMvccCoordinatorVersion(), newMvccCounter(), newMvccOperationCounter());
     }
-
-    /**
-     * @return {@code True} if this key was inserted in the cache with this row in the same transaction.
-     */
-    public boolean isKeyAbsentBefore();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
index a659245..9441c17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
@@ -465,6 +465,17 @@ public class MvccUtils {
     }
 
     /**
+     * Compares left version (xid_min) with the given version ignoring operation counter.
+     *
+     * @param left Version.
+     * @param right Version.
+     * @return Comparison result, see {@link Comparable}.
+     */
+    public static int compareIgnoreOpCounter(MvccVersion left, MvccVersion right) {
+        return compare(left.coordinatorVersion(), left.counter(), 0, right.coordinatorVersion(), right.counter(), 0);
+    }
+
+    /**
      * Compares new row version (xid_max) with the given counter and coordinator versions.
      *
      * @param row Row.

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
index 8b44ff6..e6360df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
@@ -661,11 +661,6 @@ public class CacheDataRowAdapter implements CacheDataRow {
         return TxState.NA;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean isKeyAbsentBefore() {
-        return false;
-    }
-
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index c7a6e97..cb682f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -58,10 +58,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheTtlManager;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteHistoricalIterator;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl;
@@ -1226,11 +1226,6 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         @Override public byte newMvccTxState() {
             return 0; // TODO IGNITE-7384
         }
-
-        /** {@inheritDoc} */
-        @Override public boolean isKeyAbsentBefore() {
-            return false;
-        }
     }
 
     /**
@@ -1825,11 +1820,12 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
             boolean primary,
             boolean needHistory,
             boolean noCreate,
+            boolean needOldVal,
             boolean retVal) throws IgniteCheckedException {
             CacheDataStore delegate = init0(false);
 
             return delegate.mvccUpdate(cctx, key, val, ver, expireTime, mvccVer, filter, entryProc, invokeArgs, primary,
-                needHistory, noCreate, retVal);
+                needHistory, noCreate, needOldVal, retVal);
         }
 
         /** {@inheritDoc} */
@@ -1840,10 +1836,11 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
             CacheEntryPredicate filter,
             boolean primary,
             boolean needHistory,
+            boolean needOldVal,
             boolean retVal) throws IgniteCheckedException {
             CacheDataStore delegate = init0(false);
 
-            return delegate.mvccRemove(cctx, key, mvccVer, filter, primary, needHistory, retVal);
+            return delegate.mvccRemove(cctx, key, mvccVer,filter,  primary, needHistory, needOldVal, retVal);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
index 95a46eb..f9e6bcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/DataPageIO.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
 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.tree.mvcc.data.MvccUpdateResult;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.GridStringBuilder;
 
@@ -71,11 +72,14 @@ public class DataPageIO extends AbstractDataPageIO<CacheDataRow> {
             if (mvccInfoSize > 0) {
                 assert MvccUtils.mvccVersionIsValid(row.mvccCoordinatorVersion(), row.mvccCounter(), row.mvccOperationCounter());
 
+                byte keyAbsentBeforeFlag = (byte)((row instanceof MvccUpdateResult) &&
+                    ((MvccUpdateResult)row).isKeyAbsentBefore() ? 1 : 0);
+
                 // xid_min.
                 PageUtils.putLong(addr, 0, row.mvccCoordinatorVersion());
                 PageUtils.putLong(addr, 8, row.mvccCounter());
                 PageUtils.putInt(addr, 16, row.mvccOperationCounter() | (row.mvccTxState() << MVCC_HINTS_BIT_OFF) |
-                    ((row.isKeyAbsentBefore() ? 1 : 0) << MVCC_KEY_ABSENT_BEFORE_OFF));
+                    (keyAbsentBeforeFlag << MVCC_KEY_ABSENT_BEFORE_OFF));
 
                 assert row.newMvccCoordinatorVersion() == 0
                     || MvccUtils.mvccVersionIsValid(row.newMvccCoordinatorVersion(), row.newMvccCounter(), row.newMvccOperationCounter());
@@ -84,7 +88,7 @@ public class DataPageIO extends AbstractDataPageIO<CacheDataRow> {
                 PageUtils.putLong(addr, 20, row.newMvccCoordinatorVersion());
                 PageUtils.putLong(addr, 28, row.newMvccCounter());
                 PageUtils.putInt(addr, 36, row.newMvccOperationCounter() | (row.newMvccTxState() << MVCC_HINTS_BIT_OFF) |
-                    ((row.isKeyAbsentBefore() ? 1 : 0) << MVCC_KEY_ABSENT_BEFORE_OFF));
+                    (keyAbsentBeforeFlag << MVCC_KEY_ABSENT_BEFORE_OFF));
 
                 addr += mvccInfoSize;
             }
@@ -208,6 +212,9 @@ public class DataPageIO extends AbstractDataPageIO<CacheDataRow> {
 
         final int len = Math.min(curLen - rowOff, payloadSize);
 
+        byte keyAbsentBeforeFlag = (byte)((row instanceof MvccUpdateResult) &&
+            ((MvccUpdateResult)row).isKeyAbsentBefore() ? 1 : 0);
+
         if (type == EXPIRE_TIME)
             writeExpireTimeFragment(buf, row.expireTime(), rowOff, len, prevLen);
         else if (type == CACHE_ID)
@@ -217,11 +224,11 @@ public class DataPageIO extends AbstractDataPageIO<CacheDataRow> {
                 row.mvccCoordinatorVersion(),
                 row.mvccCounter(),
                 row.mvccOperationCounter() | (row.mvccTxState() << MVCC_HINTS_BIT_OFF) |
-                    ((row.isKeyAbsentBefore() ? 1 : 0) << MVCC_KEY_ABSENT_BEFORE_OFF),
+                    (keyAbsentBeforeFlag << MVCC_KEY_ABSENT_BEFORE_OFF),
                 row.newMvccCoordinatorVersion(),
                 row.newMvccCounter(),
                 row.newMvccOperationCounter() | (row.newMvccTxState() << MVCC_HINTS_BIT_OFF) |
-                    ((row.isKeyAbsentBefore() ? 1 : 0) << MVCC_KEY_ABSENT_BEFORE_OFF),
+                    (keyAbsentBeforeFlag << MVCC_KEY_ABSENT_BEFORE_OFF),
                 len);
         else if (type != VERSION) {
             // Write key or value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
index f9388ea..4ea1828 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
@@ -367,7 +367,7 @@ public class IgniteWalIteratorFactory {
             kernalCtx, null, null, null,
             null, null, null, dbMgr, null,
             null, null, null, null,
-            null, null,null, null
+            null, null,null, null, null
         );
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 03a9749..c41e1a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -64,9 +64,10 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
-import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
@@ -210,6 +211,16 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param tx Transaction.
+     * @return {@code True} if should notify continuous query manager.
+     */
+    public boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) {
+        return cctx.isLocal() ||
+            cctx.isReplicated() ||
+            (!cctx.isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local()));
+    }
+
+    /**
      * @param lsnrs Listeners to notify.
      * @param key Entry key.
      * @param partId Partition id.
@@ -496,11 +507,6 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         final boolean keepBinary,
         final boolean includeExpired) throws IgniteCheckedException
     {
-        //TODO IGNITE-7953
-        if (cctx.transactionalSnapshot())
-            throw new UnsupportedOperationException("Continuous queries are not supported for transactional caches " +
-                "when MVCC is enabled.");
-
         IgniteOutClosure<CacheContinuousQueryHandler> clsr;
 
         if (rmtTransFactory != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index cb52771..4dbc354 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -77,7 +77,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEnt
 import org.apache.ignite.internal.processors.cluster.BaselineTopology;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
-import org.apache.ignite.internal.util.GridIntIterator;
 import org.apache.ignite.internal.util.GridSetWrapper;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
@@ -1859,32 +1858,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
     }
 
     /**
-     * Notify Dr on tx finished.
-     *
-     * @param commit {@code True} if commited, {@code False} otherwise.
-     */
-    protected void notifyDrManager(boolean commit) {
-        if (system() || internal())
-            return;
-
-        IgniteTxState txState = txState();
-
-        if (mvccSnapshot == null || txState.cacheIds().isEmpty())
-            return;
-
-        GridIntIterator iter = txState.cacheIds().iterator();
-
-        while (iter.hasNext()) {
-            int cacheId = iter.next();
-
-            GridCacheContext ctx0 = cctx.cacheContext(cacheId);
-
-            if (ctx0.isDrEnabled())
-                ctx0.dr().onTxFinished(mvccSnapshot, commit, topologyVersionSnapshot());
-        }
-    }
-
-    /**
      * @param e Transaction entry.
      * @param primaryOnly Flag to include backups into check or not.
      * @return {@code True} if entry is locally mapped as a primary or back up node.

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 314bb52..4c7b65d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -20,15 +20,19 @@ package org.apache.ignite.internal.processors.cache.transactions;
 import java.util.Collection;
 import java.util.List;
 import java.util.UUID;
+import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -37,14 +41,14 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedExceptio
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWrapper;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
@@ -55,6 +59,10 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrep
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemote;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridInvokeValue;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest;
@@ -64,7 +72,11 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPr
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxRemote;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.query.EnlistOperation;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
@@ -79,6 +91,8 @@ import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.transactions.TransactionState;
 import org.jetbrains.annotations.Nullable;
@@ -1799,6 +1813,157 @@ public class IgniteTxHandler {
     }
 
     /**
+     * Writes updated values on the backup node.
+     *
+     * @param tx Transaction.
+     * @param ctx Cache context.
+     * @param op Operation.
+     * @param keys Keys.
+     * @param vals Values sent from the primary node.
+     * @param snapshot Mvcc snapshot.
+     * @param batchNum Batch number.
+     * @param futId Future id.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void mvccEnlistBatch(GridDhtTxRemote tx, GridCacheContext ctx, EnlistOperation op, List<KeyCacheObject> keys,
+        List<Message> vals, MvccSnapshot snapshot, IgniteUuid futId, int batchNum) throws IgniteCheckedException {
+        assert keys != null && (vals == null || vals.size() == keys.size());
+        assert tx != null;
+
+        WALPointer ptr = null;
+
+        GridDhtCacheAdapter dht = ctx.dht();
+
+        tx.addActiveCache(ctx, false);
+
+        for (int i = 0; i < keys.size(); i++) {
+            KeyCacheObject key = keys.get(i);
+
+            assert key != null;
+
+            int part = ctx.affinity().partition(key);
+
+            GridDhtLocalPartition locPart = ctx.topology().localPartition(part, tx.topologyVersion(), false);
+
+            if (locPart == null || !locPart.reserve())
+                throw new ClusterTopologyException("Can not reserve partition. Please retry on stable topology.");
+
+            try {
+                CacheObject val = null;
+                EntryProcessor entryProc = null;
+                Object[] invokeArgs = null;
+
+                boolean needOldVal = ctx.shared().mvccCaching().continuousQueryListeners(ctx, tx, key) != null;
+
+                Message val0 = vals != null ? vals.get(i) : null;
+
+                CacheEntryInfoCollection entries =
+                    val0 instanceof CacheEntryInfoCollection ? (CacheEntryInfoCollection)val0 : null;
+
+                if (entries == null && !op.isDeleteOrLock() && !op.isInvoke())
+                    val = (val0 instanceof CacheObject) ? (CacheObject)val0 : null;
+
+                if(entries == null && op.isInvoke()) {
+                    assert val0 instanceof GridInvokeValue;
+
+                    GridInvokeValue invokeVal = (GridInvokeValue)val0;
+
+                    entryProc = invokeVal.entryProcessor();
+                    invokeArgs = invokeVal.invokeArgs();
+                }
+
+                assert entryProc != null || !op.isInvoke();
+
+                GridDhtCacheEntry entry = dht.entryExx(key, tx.topologyVersion());
+
+                GridCacheUpdateTxResult updRes;
+
+                while (true) {
+                    ctx.shared().database().checkpointReadLock();
+
+                    try {
+                        if (entries == null) {
+                            switch (op) {
+                                case DELETE:
+                                    updRes = entry.mvccRemove(
+                                        tx,
+                                        ctx.localNodeId(),
+                                        tx.topologyVersion(),
+                                        snapshot,
+                                        false,
+                                        needOldVal,
+                                        null,
+                                        false);
+
+                                    break;
+
+                                case INSERT:
+                                case TRANSFORM:
+                                case UPSERT:
+                                case UPDATE:
+                                    updRes = entry.mvccSet(
+                                        tx,
+                                        ctx.localNodeId(),
+                                        val,
+                                        entryProc,
+                                        invokeArgs,
+                                        0,
+                                        tx.topologyVersion(),
+                                        snapshot,
+                                        op.cacheOperation(),
+                                        false,
+                                        false,
+                                        needOldVal,
+                                        null,
+                                        false);
+
+                                    break;
+
+                                default:
+                                    throw new IgniteSQLException("Cannot acquire lock for operation [op= "
+                                        + op + "]" + "Operation is unsupported at the moment ",
+                                        IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+                            }
+                        }
+                        else {
+                            updRes = entry.mvccUpdateRowsWithPreloadInfo(tx,
+                                ctx.localNodeId(),
+                                tx.topologyVersion(),
+                                entries.infos(),
+                                op.cacheOperation(),
+                                snapshot,
+                                futId,
+                                batchNum);
+                        }
+
+                        break;
+                    }
+                    catch (GridCacheEntryRemovedException ignore) {
+                        entry = dht.entryExx(key);
+                    }
+                    finally {
+                        ctx.shared().database().checkpointReadUnlock();
+                    }
+                }
+
+                if (!updRes.filtered())
+                    ctx.shared().mvccCaching().addEnlisted(key, updRes.newValue(), 0, 0, tx.xidVersion(),
+                        updRes.oldValue(), tx.local(), tx.topologyVersion(), snapshot, ctx.cacheId(), tx, futId, batchNum);
+
+                assert updRes.updateFuture() == null : "Entry should not be locked on the backup";
+
+                ptr = updRes.loggedPointer();
+            }
+            finally {
+                locPart.release();
+            }
+        }
+
+        if (ptr != null && !ctx.tm().logTxRecords())
+            ctx.shared().wal().flush(ptr, true);
+    }
+
+    /**
      * @param cacheCtx Context.
      * @param key Key
      * @param ver Version.

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index b7624db..775b61c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -983,6 +983,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                 // Apply cache sizes only for primary nodes. Update counters were applied on prepare state.
                 applyTxSizes();
 
+                cctx.mvccCaching().onTxFinished(this, true);
+
                 if (ptr != null && !cctx.tm().logTxRecords())
                     cctx.wal().flush(ptr, false);
             }
@@ -995,8 +997,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
             finally {
                 cctx.database().checkpointReadUnlock();
 
-                notifyDrManager(state() == COMMITTING && err == null);
-
                 cctx.tm().resetContext();
             }
         }
@@ -1092,6 +1092,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                 assert !needsCompletedVersions || committedVers != null : "Missing committed versions for transaction: " + this;
                 assert !needsCompletedVersions || rolledbackVers != null : "Missing rolledback versions for transaction: " + this;
             }
+
+            cctx.mvccCaching().onTxFinished(this, commit);
         }
     }
 
@@ -1130,8 +1132,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
     @Override public void userRollback(boolean clearThreadMap) throws IgniteCheckedException {
         TransactionState state = state();
 
-        notifyDrManager(false);
-
         if (state != ROLLING_BACK && state != ROLLED_BACK) {
             setRollbackOnly();
 
@@ -1149,6 +1149,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {
             cctx.tm().rollbackTx(this, clearThreadMap, forceSkipCompletedVers);
 
+            cctx.mvccCaching().onTxFinished(this, false);
+
             if (!internal()) {
                 Collection<CacheStoreManager> stores = txState.stores(cctx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java
index 9ded71f..bdd3166 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccDataRow.java
@@ -34,9 +34,9 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_HINTS_BIT_OFF;
+import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_HINTS_MASK;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_KEY_ABSENT_BEFORE_MASK;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_KEY_ABSENT_BEFORE_OFF;
-import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_MASK;
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA;
 import static org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO.MVCC_INFO_SIZE;
 
@@ -46,38 +46,35 @@ import static org.apache.ignite.internal.processors.cache.persistence.tree.io.Da
 public class MvccDataRow extends DataRow {
     /** Mvcc coordinator version. */
     @GridToStringInclude
-    protected long mvccCrd;
+    private long mvccCrd;
 
     /** Mvcc counter. */
     @GridToStringInclude
-    protected long mvccCntr;
+    private long mvccCntr;
 
     /** Mvcc operation counter. */
     @GridToStringInclude
-    protected int mvccOpCntr;
+    private int mvccOpCntr;
 
     /** Mvcc tx state. */
     @GridToStringInclude
-    protected byte mvccTxState;
+    private byte mvccTxState;
 
     /** New mvcc coordinator version. */
     @GridToStringInclude
-    protected long newMvccCrd;
+    private long newMvccCrd;
 
     /** New mvcc counter. */
     @GridToStringInclude
-    protected long newMvccCntr;
+    private long newMvccCntr;
 
     /** New mvcc operation counter. */
     @GridToStringInclude
-    protected int newMvccOpCntr;
+    private int newMvccOpCntr;
 
     /** New mvcc tx state. */
     @GridToStringInclude
-    protected byte newMvccTxState;
-
-    /** Flag, whether this key was absent in cache before this transaction. */
-    protected boolean keyAbsentBefore;
+    private byte newMvccTxState;
 
     /**
      * @param link Link.
@@ -109,9 +106,9 @@ public class MvccDataRow extends DataRow {
         assert MvccUtils.mvccVersionIsValid(crdVer, mvccCntr, mvccOpCntr);
 
         assert rowData == RowData.LINK_ONLY
-            || this.mvccCrd == crdVer && this.mvccCntr == mvccCntr && this.mvccOpCntr == mvccOpCntr :
+            || mvccCoordinatorVersion() == crdVer && mvccCounter() == mvccCntr && mvccOperationCounter() == mvccOpCntr :
         "mvccVer=" + new MvccVersionImpl(crdVer, mvccCntr, mvccOpCntr) +
-            ", dataMvccVer=" + new MvccVersionImpl(this.mvccCrd, this.mvccCntr, this.mvccOpCntr) ;
+            ", dataMvccVer=" + new MvccVersionImpl(mvccCoordinatorVersion(), mvccCounter(), mvccOperationCounter()) ;
 
         if (rowData == RowData.LINK_ONLY) {
             this.mvccCrd = crdVer;
@@ -158,9 +155,8 @@ public class MvccDataRow extends DataRow {
 
         int withHint = PageUtils.getInt(addr, off + 16);
 
-        mvccOpCntr = withHint & ~MVCC_OP_COUNTER_MASK;
+        mvccOpCntr = withHint & ~MVCC_HINTS_MASK;
         mvccTxState = (byte)(withHint >>> MVCC_HINTS_BIT_OFF);
-        keyAbsentBefore = ((withHint & MVCC_KEY_ABSENT_BEFORE_MASK) >>> MVCC_KEY_ABSENT_BEFORE_OFF) == 1;
 
         assert MvccUtils.mvccVersionIsValid(mvccCrd, mvccCntr, mvccOpCntr);
 
@@ -170,12 +166,9 @@ public class MvccDataRow extends DataRow {
 
         withHint = PageUtils.getInt(addr, off + 36);
 
-        newMvccOpCntr = withHint & ~MVCC_OP_COUNTER_MASK;
+        newMvccOpCntr = withHint & ~MVCC_HINTS_MASK;
         newMvccTxState = (byte)(withHint >>> MVCC_HINTS_BIT_OFF);
 
-        if (newMvccCrd != MVCC_CRD_COUNTER_NA)
-            keyAbsentBefore = ((withHint & MVCC_KEY_ABSENT_BEFORE_MASK) >>> MVCC_KEY_ABSENT_BEFORE_OFF) == 1;
-
         assert newMvccCrd == MVCC_CRD_COUNTER_NA || MvccUtils.mvccVersionIsValid(newMvccCrd, newMvccCntr, newMvccOpCntr);
 
         return MVCC_INFO_SIZE;
@@ -193,7 +186,7 @@ public class MvccDataRow extends DataRow {
 
     /** {@inheritDoc} */
     @Override public int mvccOperationCounter() {
-        return mvccOpCntr;
+        return mvccOpCntr & ~MVCC_KEY_ABSENT_BEFORE_MASK;
     }
 
     /** {@inheritDoc} */
@@ -213,7 +206,7 @@ public class MvccDataRow extends DataRow {
 
     /** {@inheritDoc} */
     @Override public int newMvccOperationCounter() {
-        return newMvccOpCntr;
+        return newMvccOpCntr & ~MVCC_KEY_ABSENT_BEFORE_MASK;
     }
 
     /** {@inheritDoc} */
@@ -255,9 +248,30 @@ public class MvccDataRow extends DataRow {
         this.newMvccTxState = newMvccTxState;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean isKeyAbsentBefore() {
-        return keyAbsentBefore;
+    /**
+     * @return {@code True} if key absent before.
+     */
+    protected boolean keyAbsentBeforeFlag() {
+        long withHint = newMvccCrd == MVCC_CRD_COUNTER_NA ? mvccOpCntr : newMvccOpCntr;
+
+        return ((withHint & MVCC_KEY_ABSENT_BEFORE_MASK) >>> MVCC_KEY_ABSENT_BEFORE_OFF) == 1;
+    }
+
+    /**
+     * @param flag {@code True} if key is absent before.
+     */
+    protected void keyAbsentBeforeFlag(boolean flag) {
+        if (flag) {
+            if (mvccCrd != MVCC_CRD_COUNTER_NA)
+                mvccOpCntr |= MVCC_KEY_ABSENT_BEFORE_MASK;
+
+            if (newMvccCrd != MVCC_CRD_COUNTER_NA)
+                newMvccOpCntr |= MVCC_KEY_ABSENT_BEFORE_MASK;
+        }
+        else {
+            mvccOpCntr &= ~MVCC_KEY_ABSENT_BEFORE_MASK;
+            newMvccOpCntr &= ~MVCC_KEY_ABSENT_BEFORE_MASK;
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java
index 23711a5..fd1de50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateDataRow.java
@@ -98,11 +98,14 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
     /** */
     private static final int DELETED = FAST_MISMATCH << 1;
 
-    /** Whether tx has overridden it's own update. */
-    private static final int OWN_VALUE_OVERRIDDEN = DELETED << 1;
+    /** Force read full entry instead of header only. Old value == value before tx started. */
+    private static final int NEED_OLD_VALUE = DELETED << 1;
 
-    /** Force read full entry instead of header only.  */
-    private static final int NEED_PREV_VALUE = OWN_VALUE_OVERRIDDEN << 1;
+    /**
+     * Force read full entry instead of header only. Prev value == value on previous tx step or old value
+     * if it is a first tx step.
+     */
+    private static final int NEED_PREV_VALUE = NEED_OLD_VALUE << 1;
 
     /** */
     @GridToStringExclude
@@ -133,7 +136,7 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
     private long resCntr;
 
     /** */
-    private List<MvccLinkAwareSearchRow> historyRows;
+    private List<MvccLinkAwareSearchRow> histRows;
 
     /** */
     @GridToStringExclude
@@ -153,8 +156,9 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
      * @param newVer Update version.
      * @param primary Primary node flag.
      * @param lockOnly Whether no actual update should be done and the only thing to do is to acquire lock.
-     * @param needHistory Whether to collect rows created or affected by the current tx.
+     * @param needHist Whether to collect rows created or affected by the current tx.
      * @param fastUpdate Fast update visit mode.
+     * @param needOldVal {@code True} if need old value.
      */
     public MvccUpdateDataRow(
         GridCacheContext cctx,
@@ -168,8 +172,9 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
         @Nullable CacheEntryPredicate filter,
         boolean primary,
         boolean lockOnly,
-        boolean needHistory,
+        boolean needHist,
         boolean fastUpdate,
+        boolean needOldVal,
         boolean needPrevValue) {
         super(key,
             val,
@@ -183,7 +188,6 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
         this.mvccSnapshot = mvccSnapshot;
         this.cctx = cctx;
         this.filter = filter;
-        this.keyAbsentBefore = primary; // True for primary and false for backup (backups do not use this flag).
 
         assert !lockOnly || val == null;
 
@@ -195,16 +199,21 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
         if (primary && (lockOnly || val == null))
             flags |= CAN_WRITE | REMOVE_OR_LOCK;
 
-        if (needHistory)
+        if (needHist)
             flags |= NEED_HISTORY;
 
         if (fastUpdate)
             flags |= FAST_UPDATE;
 
+        if (needOldVal)
+            flags |= NEED_OLD_VALUE;
+
         if(needPrevValue)
             flags |= NEED_PREV_VALUE;
 
         setFlags(flags);
+
+        keyAbsentBeforeFlag(primary); // True for primary and false for backup (backups do not use this flag).
     }
 
     /** {@inheritDoc} */
@@ -224,7 +233,7 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
             long lockCntr = rowIo.getMvccLockCounter(pageAddr, idx);
 
             // We cannot continue while entry is locked by another transaction.
-            if ((lockCrd != mvccCrd || lockCntr != mvccCntr)
+            if ((lockCrd != mvccCoordinatorVersion() || lockCntr != mvccCounter())
                 && isActive(cctx, lockCrd, lockCntr, mvccSnapshot)) {
                 resCrd = lockCrd;
                 resCntr = lockCntr;
@@ -256,7 +265,7 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
             }
 
             if (compare(mvccSnapshot, rowCrd, rowCntr) == 0) {
-                res = mvccOpCntr == rowOpCntr ? ResultType.VERSION_FOUND :
+                res = mvccOperationCounter() == rowOpCntr ? ResultType.VERSION_FOUND :
                     removed ? ResultType.PREV_NULL : ResultType.PREV_NOT_NULL;
 
                 if (removed)
@@ -274,11 +283,11 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
                 if(filter != null && !applyFilter(res == ResultType.PREV_NOT_NULL ? oldRow.value() : null))
                     res = FILTERED;
 
-                setFlags(LAST_COMMITTED_FOUND | OWN_VALUE_OVERRIDDEN);
+                setFlags(LAST_COMMITTED_FOUND);
 
                 // Copy new key flag from the previous row version if it was created by the current tx.
                 if (isFlagsSet(PRIMARY))
-                    keyAbsentBefore = row.isKeyAbsentBefore();
+                    keyAbsentBeforeFlag(row.keyAbsentBeforeFlag());
             }
         }
 
@@ -327,12 +336,15 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
                     else {
                         res = ResultType.PREV_NOT_NULL;
 
-                        keyAbsentBefore = false;
+                        keyAbsentBeforeFlag(false);
 
                         // Actually, full row can be omitted for replace(k,newval) and putIfAbsent, but
                         // operation context is not available here and full row required if filter is set.
-                        if( (isFlagsSet(NEED_PREV_VALUE) || filter != null))
-                            oldRow = tree.getRow(io, pageAddr, idx, RowData.FULL);
+                        if((isFlagsSet(NEED_PREV_VALUE) || isFlagsSet(NEED_OLD_VALUE) || filter != null)) {
+                            oldRow = tree.getRow(io, pageAddr, idx, RowData.NO_KEY);
+
+                            oldRow.key(key);
+                        }
                         else
                             oldRow = row;
                     }
@@ -383,8 +395,8 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
                     // Lock entry for primary partition if needed.
                     // If invisible row is found for FAST_UPDATE case we should not lock row.
                     if (!isFlagsSet(DELETED) && isFlagsSet(PRIMARY | REMOVE_OR_LOCK) && !isFlagsSet(FAST_MISMATCH)) {
-                        rowIo.setMvccLockCoordinatorVersion(pageAddr, idx, mvccCrd);
-                        rowIo.setMvccLockCounter(pageAddr, idx, mvccCntr);
+                        rowIo.setMvccLockCoordinatorVersion(pageAddr, idx, mvccCoordinatorVersion());
+                        rowIo.setMvccLockCounter(pageAddr, idx, mvccCounter());
 
                         // TODO Delta record IGNITE-7991
 
@@ -432,7 +444,7 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
 
             // We can cleanup previous row only if it was deleted by another
             // transaction and delete version is less or equal to cleanup one
-            if (rowNewCrd < mvccCrd || Long.compare(cleanupVer, rowNewCntr) >= 0)
+            if (rowNewCrd < mvccCoordinatorVersion() || Long.compare(cleanupVer, rowNewCntr) >= 0)
                 setFlags(CAN_CLEANUP);
         }
 
@@ -447,18 +459,18 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
             // Row obsoleted by current operation, all rows created or updated with current tx.
             if (isFlagsSet(NEED_HISTORY)
                 && (row == oldRow
-                    || (rowCrd == mvccCrd && rowCntr == mvccCntr)
-                    || (rowNewCrd == mvccCrd && rowNewCntr == mvccCntr))) {
-                if (historyRows == null)
-                    historyRows = new ArrayList<>();
+                    || (rowCrd == mvccCoordinatorVersion() && rowCntr == mvccCounter())
+                    || (rowNewCrd == mvccCoordinatorVersion() && rowNewCntr == mvccCounter()))) {
+                if (histRows == null)
+                    histRows = new ArrayList<>();
 
-                historyRows.add(new MvccLinkAwareSearchRow(cacheId, key, rowCrd, rowCntr, rowOpCntr & ~MVCC_OP_COUNTER_MASK, rowLink));
+                histRows.add(new MvccLinkAwareSearchRow(cacheId, key, rowCrd, rowCntr, rowOpCntr & ~MVCC_OP_COUNTER_MASK, rowLink));
             }
 
             if (cleanupVer > MVCC_OP_COUNTER_NA // Do not clean if cleanup version is not assigned.
                 && !isFlagsSet(CAN_CLEANUP)
                 && isFlagsSet(LAST_COMMITTED_FOUND)
-                && (rowCrd < mvccCrd || Long.compare(cleanupVer, rowCntr) >= 0))
+                && (rowCrd < mvccCoordinatorVersion() || Long.compare(cleanupVer, rowCntr) >= 0))
                 // all further versions are guaranteed to be less than cleanup version
                 setFlags(CAN_CLEANUP);
         }
@@ -531,34 +543,52 @@ public class MvccUpdateDataRow extends MvccDataRow implements MvccUpdateResult,
         switch (resultType()) {
             case VERSION_FOUND:
             case PREV_NULL:
+                return new MvccVersionImpl(mvccCoordinatorVersion(), mvccCounter(), mvccOperationCounter());
 
-                return new MvccVersionImpl(mvccCrd, mvccCntr, mvccOpCntr);
             case PREV_NOT_NULL:
-
+            case REMOVED_NOT_NULL:
                 return new MvccVersionImpl(oldRow.mvccCoordinatorVersion(), oldRow.mvccCounter(), oldRow.mvccOperationCounter());
+
             case LOCKED:
             case VERSION_MISMATCH:
-
                 assert resCrd != MVCC_CRD_COUNTER_NA && resCntr != MVCC_COUNTER_NA;
 
                 return new MvccVersionImpl(resCrd, resCntr, MVCC_OP_COUNTER_NA);
-            default:
 
+            case FILTERED:
+                if (oldRow != null)
+                    return new MvccVersionImpl(oldRow.mvccCoordinatorVersion(), oldRow.mvccCounter(), oldRow.mvccOperationCounter());
+                else
+                    return new MvccVersionImpl(mvccCoordinatorVersion(), mvccCounter(), mvccOperationCounter());
+
+            default:
                 throw new IllegalStateException("Unexpected result type: " + resultType());
         }
     }
 
     /** {@inheritDoc} */
     @Override public List<MvccLinkAwareSearchRow> history() {
-        if (isFlagsSet(NEED_HISTORY) && historyRows == null)
-            historyRows = new ArrayList<>();
+        if (isFlagsSet(NEED_HISTORY) && histRows == null)
+            histRows = new ArrayList<>();
+
+        return histRows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject newValue() {
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject oldValue() {
+        assert oldRow != null;
 
-        return historyRows;
+        return oldRow.value();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isOwnValueOverridden() {
-        return isFlagsSet(OWN_VALUE_OVERRIDDEN);
+    @Override public boolean isKeyAbsentBefore() {
+        return keyAbsentBeforeFlag();
     }
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java
index a8f5bb9..f0d33fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/mvcc/data/MvccUpdateResult.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.tree.mvcc.data;
 
 import java.util.List;
 import org.apache.ignite.internal.processors.cache.CacheInvokeResult;
+import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccVersion;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccLinkAwareSearchRow;
 
@@ -37,23 +38,26 @@ public interface MvccUpdateResult {
     public MvccVersion resultVersion();
 
     /**
-     *
      * @return Collection of row created or affected by the current tx.
      */
     public List<MvccLinkAwareSearchRow> history();
 
     /**
-     * @return {@code True} if this key was inserted in the cache with this row in the same transaction.
+     * @return New value of updated entry.
      */
-    public boolean isKeyAbsentBefore();
+    public CacheObject newValue();
 
     /**
-     * @return Flag whether tx has overridden it's own update.
+     * @return Old value.
      */
-    public boolean isOwnValueOverridden();
+    public CacheObject oldValue();
+
+    /**
+     * @return {@code True} if this key was inserted in the cache with this row in the same transaction.
+     */
+    public boolean isKeyAbsentBefore();
 
     /**
-     *
      * @return Entry processor invoke result.
      */
     CacheInvokeResult invokeResult();

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 4cbff6d..f95740e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -875,11 +875,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                         true);
                 }
 
-            ctx.discovery().sendCustomEvent(msg);
-        }
-        catch (IgniteCheckedException e) {
-            startFuts.remove(routineId);
-            locInfos.remove(routineId);
+                ctx.discovery().sendCustomEvent(msg);
+            }
+            catch (IgniteCheckedException e) {
+                startFuts.remove(routineId);
+                locInfos.remove(routineId);
 
                 unregisterHandler(routineId, hnd, true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 26d1b94..cc634fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -42,6 +42,7 @@ import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFilter
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple3;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -488,8 +489,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
 
     /** {@inheritDoc} */
     @Override public GridCacheUpdateTxResult mvccSet(@Nullable IgniteInternalTx tx, UUID affNodeId, CacheObject val,
-        EntryProcessor entryProc, Object[] invokeArgs, long ttl0, AffinityTopologyVersion topVer, MvccSnapshot mvccVer,
-        GridCacheOperation op, boolean needHistory, boolean noCreate, CacheEntryPredicate filter, boolean retVal)
+        EntryProcessor entryProc, Object[] invokeArgs, long ttl0, AffinityTopologyVersion topVer, MvccSnapshot mvccVer, GridCacheOperation op, boolean needHistory,
+        boolean noCreate, boolean needOldVal, CacheEntryPredicate filter, boolean retVal)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         rawPut(val, ttl);
 
@@ -498,7 +499,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
 
     /** {@inheritDoc} */
     @Override public GridCacheUpdateTxResult mvccRemove(@Nullable IgniteInternalTx tx, UUID affNodeId,
-        AffinityTopologyVersion topVer, MvccSnapshot mvccVer, boolean needHistory,
+        AffinityTopologyVersion topVer, MvccSnapshot mvccVer, boolean needHistory, boolean needOldVal,
         CacheEntryPredicate filter, boolean retVal)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         obsoleteVer = ver;
@@ -952,7 +953,9 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         AffinityTopologyVersion topVer,
         List<GridCacheEntryInfo> entries,
         GridCacheOperation op,
-        MvccSnapshot mvccVer) throws IgniteCheckedException, GridCacheEntryRemovedException {
+        MvccSnapshot mvccVer,
+        IgniteUuid futId,
+        int batchNum) throws IgniteCheckedException, GridCacheEntryRemovedException {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java
index 74db28f..fd78347 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java
@@ -153,7 +153,7 @@ public class IgniteWalIteratorSwitchSegmentTest extends GridCommonAbstractTest {
                 null,
                 null,
                 null,
-
+                null,
                 null)
         ).createSerializer(serVer);
 
@@ -494,6 +494,7 @@ public class IgniteWalIteratorSwitchSegmentTest extends GridCommonAbstractTest {
             null,
             null,
             null,
+            null,
             null
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
index bb0c416..f51056f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
@@ -79,6 +79,7 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest {
             null,
             null,
             null,
+            null,
             null
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
index 71e308b..9a7d63b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
@@ -79,6 +79,7 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest
             null,
             null,
             null,
+            null,
             null
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java
index d3530b4..cbf9dea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java
@@ -94,6 +94,7 @@ public class IndexStoragePageMemoryImplTest extends IndexStorageSelfTest {
             null,
             null,
             null,
+            null,
             null
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
index 83e1894..36a64f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
@@ -84,6 +84,7 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest {
             null,
             null,
             null,
+            null,
             null
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
index fe79105..cfd9543 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
@@ -305,6 +305,7 @@ public class PageMemoryImplTest extends GridCommonAbstractTest {
             null,
             null,
             null,
+            null,
             null
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51a202a4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java
index 3ee6a20..400ae28 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousBatchAckTest.java
@@ -44,6 +44,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -192,6 +193,56 @@ public class CacheContinuousBatchAckTest extends GridCommonAbstractTest implemen
         checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, TRANSACTIONAL, true));
     }
 
+    // MVCC tests.
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionMvccTx() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL_SNAPSHOT, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionMvccTxWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 1, TRANSACTIONAL_SNAPSHOT, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionMvccTxNoBackup() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL_SNAPSHOT, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionMvccTxNoBackupWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(PARTITIONED, 0, TRANSACTIONAL_SNAPSHOT, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedMvccTx() throws Exception {
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, TRANSACTIONAL_SNAPSHOT, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedMvccTxWithFilter() throws Exception {
+        filterOn.set(true);
+
+        checkBackupAcknowledgeMessage(cacheConfiguration(REPLICATED, 1, TRANSACTIONAL_SNAPSHOT, true));
+    }
+
     /**
      * @param ccfg Cache configuration.
      * @throws Exception If failed.