You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/10/30 16:52:04 UTC

[08/52] [abbrv] ignite git commit: ignite-1607 Implemented deadlock-free optimistic serializable tx mode

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
index cedb693..2577d93 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -52,6 +52,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
 
 /**
  *
@@ -190,6 +191,11 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
 
             txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, false);
             txOperations(OPTIMISTIC, REPEATABLE_READ, crossCacheTx, true);
+
+            if (writeSync == FULL_SYNC) {
+                txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, false);
+                txOperations(OPTIMISTIC, SERIALIZABLE, crossCacheTx, true);
+            }
         }
         finally {
             ignite.destroyCache(CACHE1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index f4813ff..da54d15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collections;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
@@ -330,14 +331,21 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
      * @throws IgniteCheckedException If failed.
      */
     private void remove(Ignite ignite, IgniteCache<String, Integer> cache, final int cnt,
-        TransactionConcurrency concurrency, TransactionIsolation isolation) throws Exception {
+        TransactionConcurrency concurrency, final TransactionIsolation isolation) throws Exception {
         try {
             info("Removing values form cache [0," + cnt + ')');
 
             CU.inTx(ignite, cache, concurrency, isolation, new CIX1<IgniteCache<String, Integer>>() {
                 @Override public void applyx(IgniteCache<String, Integer> cache) {
-                    for (int i = 0; i < cnt; i++)
-                        cache.remove("key" + i);
+                    for (int i = 0; i < cnt; i++) {
+                        String key = "key" + i;
+
+                        // Use removeAll for serializable tx to avoid version check.
+                        if (isolation == TransactionIsolation.SERIALIZABLE)
+                            cache.removeAll(Collections.singleton(key));
+                        else
+                            cache.remove(key);
+                    }
                 }
             });
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index ec3ea0c..a6b5535 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -3692,7 +3692,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         if (txShouldBeUsed()) {
             try (Transaction tx = transactions().txStart(OPTIMISTIC, READ_COMMITTED)) {
                 // Remove missing key.
-                assertTrue(jcache().remove(UUID.randomUUID().toString()));
+                assertFalse(jcache().remove(UUID.randomUUID().toString()));
 
                 tx.commit();
             }
@@ -3708,7 +3708,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         if (txShouldBeUsed()) {
             try (Transaction tx = transactions().txStart(OPTIMISTIC, READ_COMMITTED)) {
                 // Remove missing key.
-                assertTrue(jcache().remove(UUID.randomUUID().toString()));
+                assertFalse(jcache().remove(UUID.randomUUID().toString()));
 
                 tx.setRollbackOnly();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index 122910e..5044516 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -30,6 +30,8 @@ import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
@@ -37,9 +39,11 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.util.lang.GridTuple;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -48,11 +52,18 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
 import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
 
 /**
  * Tests that removes are not lost when topology changes.
@@ -155,29 +166,54 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
      * @throws Exception If failed.
      */
     public void testPutAndRemove() throws Exception {
-        putAndRemove(DUR, GridTestUtils.TestMemoryMode.HEAP);
+        putAndRemove(DUR, null, null, GridTestUtils.TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAndRemovePessimisticTx() throws Exception {
+        if (atomicityMode() != CacheAtomicityMode.TRANSACTIONAL)
+            return;
+
+        putAndRemove(30_000, PESSIMISTIC, REPEATABLE_READ, GridTestUtils.TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAndRemoveOptimisticSerializableTx() throws Exception {
+        if (atomicityMode() != CacheAtomicityMode.TRANSACTIONAL)
+            return;
+
+        putAndRemove(30_000, OPTIMISTIC, SERIALIZABLE, GridTestUtils.TestMemoryMode.HEAP);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAndRemoveOffheapEvict() throws Exception {
-        putAndRemove(30_000, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
+        putAndRemove(30_000, null, null, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPutAndRemoveOffheapEvictSwap() throws Exception {
-        putAndRemove(30_000, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
+        putAndRemove(30_000, null, null, GridTestUtils.TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
 
     /**
      * @param duration Test duration.
+     * @param txConcurrency Transaction concurrency if test explicit transaction.
+     * @param txIsolation Transaction isolation if test explicit transaction.
      * @param memMode Memory mode.
      * @throws Exception If failed.
      */
-    private void putAndRemove(long duration, GridTestUtils.TestMemoryMode memMode) throws Exception {
+    private void putAndRemove(long duration,
+        final TransactionConcurrency txConcurrency,
+        final TransactionIsolation txIsolation,
+        GridTestUtils.TestMemoryMode memMode) throws Exception {
         assertEquals(testClientNode(), (boolean) grid(0).configuration().isClientMode());
 
         grid(0).destroyCache(null);
@@ -216,6 +252,8 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
 
                 ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
+                IgniteTransactions txs = sndCache0.unwrap(Ignite.class).transactions();
+
                 while (!stop.get()) {
                     for (int i = 0; i < 100; i++) {
                         int key = rnd.nextInt(KEYS_CNT);
@@ -225,14 +263,54 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
                         while (true) {
                             try {
                                 if (put) {
-                                    sndCache0.put(key, i);
+                                    boolean failed = false;
+
+                                    if (txConcurrency != null) {
+                                        try (Transaction tx = txs.txStart(txConcurrency, txIsolation)) {
+                                            sndCache0.put(key, i);
+
+                                            tx.commit();
+                                        }
+                                        catch (CacheException | IgniteException e) {
+                                            if (!X.hasCause(e, ClusterTopologyCheckedException.class)) {
+                                                log.error("Unexpected error: " + e);
+
+                                                throw e;
+                                            }
 
-                                    expVals.put(key, F.t(i));
+                                            failed = true;
+                                        }
+                                    }
+                                    else
+                                        sndCache0.put(key, i);
+
+                                    if (!failed)
+                                        expVals.put(key, F.t(i));
                                 }
                                 else {
-                                    sndCache0.remove(key);
+                                    boolean failed = false;
+
+                                    if (txConcurrency != null) {
+                                        try (Transaction tx = txs.txStart(txConcurrency, txIsolation)) {
+                                            sndCache0.remove(key);
+
+                                            tx.commit();
+                                        }
+                                        catch (CacheException | IgniteException e) {
+                                            if (!X.hasCause(e, ClusterTopologyCheckedException.class)) {
+                                                log.error("Unexpected error: " + e);
+
+                                                throw e;
+                                            }
+
+                                            failed = true;
+                                        }
+                                    }
+                                    else
+                                        sndCache0.remove(key);
 
-                                    expVals.put(key, F.<Integer>t(null));
+                                    if (!failed)
+                                        expVals.put(key, F.<Integer>t(null));
                                 }
 
                                 break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
index 67bc08c..1ef77f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
@@ -162,9 +162,6 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
 
         c.setPeerClassLoadingEnabled(false);
 
-        // Enable tracing.
-//        Logger.getLogger("org.apache.ignite.kernal.processors.cache.GridCacheDgcManager.trace").setLevel(Level.DEBUG);
-
         return c;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
index 084bc75..234f362 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
@@ -88,7 +88,8 @@ public class GridCacheMvccFlagsTest extends GridCommonAbstractTest {
             true,
             true,
             true,
-            true
+            true,
+            null
         );
 
         c.setOwner();
@@ -128,7 +129,8 @@ public class GridCacheMvccFlagsTest extends GridCommonAbstractTest {
             false,
             false,
             false,
-            false
+            false,
+            null
         );
 
         short flags = c.flags();

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
index 0af7183..1b97663 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
@@ -595,6 +595,170 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testSerializableLocks() throws Exception {
+        checkSerializableAdd(false);
+
+        checkSerializableAdd(true);
+
+        checkNonSerializableConflict();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkNonSerializableConflict() throws Exception {
+        GridCacheAdapter<String, String> cache = grid.internalCache();
+
+        UUID nodeId = UUID.randomUUID();
+
+        GridCacheMvcc mvcc = new GridCacheMvcc(cache.context());
+
+        GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1");
+
+        GridCacheMvccCandidate cand1 = mvcc.addLocal(e,
+            nodeId,
+            null,
+            1,
+            version(1),
+            0,
+            null,
+            false,
+            true,
+            false,
+            true
+        );
+
+        assertNotNull(cand1);
+
+        GridCacheMvccCandidate cand2 = mvcc.addLocal(e,
+            nodeId,
+            null,
+            1,
+            version(2),
+            0,
+            new GridCacheVersion(0, 0, 30, 1),
+            false,
+            true,
+            false,
+            true
+        );
+
+        assertNull(cand2);
+    }
+
+    /**
+     * @param incVer If {@code true} lock version is incremented.
+     * @throws Exception If failed.
+     */
+    private void checkSerializableAdd(boolean incVer) throws Exception {
+        GridCacheAdapter<String, String> cache = grid.internalCache();
+
+        UUID nodeId = UUID.randomUUID();
+
+        GridCacheMvcc mvcc = new GridCacheMvcc(cache.context());
+
+        GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1");
+
+        GridCacheVersion serOrder1 = new GridCacheVersion(0, 0, 10, 1);
+        GridCacheVersion serOrder2 = new GridCacheVersion(0, 0, 20, 1);
+        GridCacheVersion serOrder3 = new GridCacheVersion(0, 0, 15, 1);
+        GridCacheVersion serOrder4 = new GridCacheVersion(0, 0, 30, 1);
+
+        GridCacheVersion ver1 = incVer ? version(1) : version(4);
+        GridCacheVersion ver2 = incVer ? version(2) : version(3);
+        GridCacheVersion ver3 = incVer ? version(3) : version(2);
+        GridCacheVersion ver4 = incVer ? version(4) : version(1);
+
+        GridCacheMvccCandidate cand1 = mvcc.addLocal(e,
+            nodeId,
+            null,
+            1,
+            ver1,
+            0,
+            serOrder1,
+            false,
+            true,
+            false,
+            true
+            );
+
+        assertNotNull(cand1);
+
+        GridCacheMvccCandidate cand2 = mvcc.addLocal(e,
+            nodeId,
+            null,
+            2,
+            ver2,
+            0,
+            serOrder2,
+            false,
+            true,
+            false,
+            true
+        );
+
+        assertNotNull(cand2);
+
+        GridCacheMvccCandidate cand3 = mvcc.addLocal(e,
+            nodeId,
+            null,
+            3,
+            ver3,
+            0,
+            serOrder3,
+            false,
+            true,
+            false,
+            true
+        );
+
+        assertNull(cand3);
+
+        GridCacheMvccCandidate cand4 = mvcc.addLocal(e,
+            nodeId,
+            null,
+            4,
+            ver4,
+            0,
+            serOrder4,
+            false,
+            true,
+            false,
+            true
+        );
+
+        assertNotNull(cand4);
+
+        GridCacheMvccCandidate owner = mvcc.recheck();
+
+        assertNull(owner);
+
+        cand2.setReady();
+
+        owner = mvcc.recheck();
+
+        assertNull(owner);
+
+        cand1.setReady();
+
+        owner = mvcc.recheck();
+
+        assertSame(cand1, owner);
+
+        owner = mvcc.recheck();
+
+        assertSame(cand1, owner);
+
+        mvcc.remove(cand1.version());
+
+        owner = mvcc.recheck();
+
+        assertSame(cand2, owner);
+    }
+
+    /**
      * Gets version based on order.
      *
      * @param order Order.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
index cdf8eca..59f9a9d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
@@ -101,7 +101,8 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
             true,
             false,
             false,
-            false
+            false,
+            null
         );
 
         Marshaller marshaller = getTestResources().getMarshaller();

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/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 1fef4d5..abb2767 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
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEnt
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple3;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -441,6 +442,22 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** @inheritDoc */
+    @Nullable @Override public T2<CacheObject, GridCacheVersion> innerGetVersioned(
+        IgniteInternalTx tx,
+        boolean readSwap,
+        boolean unmarshal,
+        boolean updateMetrics,
+        boolean evt,
+        UUID subjId,
+        Object transformClo,
+        String taskName,
+        @Nullable IgniteCacheExpiryPolicy expiryPlc) {
+        assert false;
+
+        return null;
+    }
+
+    /** @inheritDoc */
     @Override public CacheObject innerReload() {
         return val;
     }
@@ -456,9 +473,14 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         boolean evt,
         boolean metrics,
         AffinityTopologyVersion topVer,
-        CacheEntryPredicate[] filter, GridDrType drType,
-        long drExpireTime, @Nullable GridCacheVersion drVer, UUID subjId, String taskName) throws IgniteCheckedException,
-        GridCacheEntryRemovedException {
+        CacheEntryPredicate[] filter,
+        GridDrType drType,
+        long drExpireTime,
+        @Nullable GridCacheVersion drVer,
+        UUID subjId,
+        String taskName,
+        @Nullable GridCacheVersion dhtVer)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
         return new GridCacheUpdateTxResult(true, rawPut(val, ttl));
     }
 
@@ -528,8 +550,9 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         GridDrType drType,
         @Nullable GridCacheVersion drVer,
         UUID subjId,
-        String taskName
-    ) throws IgniteCheckedException, GridCacheEntryRemovedException {
+        String taskName,
+        @Nullable GridCacheVersion dhtVer
+        ) throws IgniteCheckedException, GridCacheEntryRemovedException {
         obsoleteVer = ver;
 
         CacheObject old = val;
@@ -552,7 +575,10 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** @inheritDoc */
-    @Override public boolean tmLock(IgniteInternalTx tx, long timeout) {
+    @Override public boolean tmLock(IgniteInternalTx tx,
+        long timeout,
+        @Nullable GridCacheVersion serOrder,
+        GridCacheVersion serReadVer) {
         assert false; return false;
     }
 
@@ -613,6 +639,13 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** @inheritDoc */
+    @Override public boolean checkSerializableReadVersion(GridCacheVersion ver) {
+        assert false;
+
+        return false;
+    }
+
+    /** @inheritDoc */
     @Override public boolean initialValue(
         CacheObject val,
         GridCacheVersion ver,
@@ -640,8 +673,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** @inheritDoc */
-    @Override public boolean versionedValue(CacheObject val, GridCacheVersion curVer, GridCacheVersion newVer) {
-        assert false; return false;
+    @Override public GridCacheVersion versionedValue(CacheObject val,
+        GridCacheVersion curVer,
+        GridCacheVersion newVer) {
+        assert false;
+
+        return null;
     }
 
     /** @inheritDoc */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
index fcf46cf..dff0344 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
@@ -175,9 +175,7 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
         for (int i = 0; i < iterations(); i++) {
             IgniteCache<Integer, String> cache = jcache(gridIdx);
 
-            Transaction tx = ignite(gridIdx).transactions().txStart(concurrency, isolation, 0, 0);
-
-            try {
+            try (Transaction tx = ignite(gridIdx).transactions().txStart(concurrency, isolation, 0, 0)) {
                 int prevKey = -1;
 
                 for (Integer key : getKeys()) {
@@ -236,46 +234,22 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
                     debug("Committed transaction [i=" + i + ", tx=" + tx + ']');
             }
             catch (TransactionOptimisticException e) {
-                if (concurrency != OPTIMISTIC || isolation != SERIALIZABLE) {
-                    error("Received invalid optimistic failure.", e);
+                if (!(concurrency == OPTIMISTIC && isolation == SERIALIZABLE)) {
+                    log.error("Unexpected error: " + e, e);
 
                     throw e;
                 }
-
-                if (isTestDebug())
-                    info("Optimistic transaction failure (will rollback) [i=" + i + ", msg=" + e.getMessage() +
-                        ", tx=" + tx.xid() + ']');
-
-                try {
-                    tx.rollback();
-                }
-                catch (IgniteException ex) {
-                    error("Failed to rollback optimistic failure: " + tx, ex);
-
-                    throw ex;
-                }
             }
-            catch (Exception e) {
-                error("Transaction failed (will rollback): " + tx, e);
-
-                tx.rollback();
+            catch (Throwable e) {
+                log.error("Unexpected error: " + e, e);
 
                 throw e;
             }
-            catch (Error e) {
-                error("Error when executing transaction (will rollback): " + tx, e);
-
-                tx.rollback();
+        }
 
-                throw e;
-            }
-            finally {
-                Transaction t = ignite(gridIdx).transactions().tx();
+        Transaction tx = ignite(gridIdx).transactions().tx();
 
-                assert t == null : "Thread should not have transaction upon completion ['t==tx'=" + (t == tx) +
-                    ", t=" + t + (t != tx ? "tx=" + tx : "tx=''") + ']';
-            }
-        }
+        assertNull("Thread should not have transaction upon completion", tx);
 
         if (printMemoryStats()) {
             if (cntr.getAndIncrement() % 100 == 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
index 9e14d30..f13ba8c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
@@ -219,78 +219,98 @@ public abstract class IgniteTxMultiThreadedAbstractTest extends IgniteTxAbstract
      * @throws Exception If failed.
      */
     public void testOptimisticSerializableConsistency() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-582");
-
         final IgniteCache<Integer, Long> cache = grid(0).cache(null);
 
-        final int THREADS = 2;
+        final int THREADS = 3;
 
         final int ITERATIONS = 100;
 
-        final int key = 0;
+        for (int key0 = 100_000; key0 < 100_000 + 20; key0++) {
+            final int key = key0;
 
-        cache.put(key, 0L);
+            cache.put(key, 0L);
 
-        List<IgniteInternalFuture<Collection<Long>>> futs = new ArrayList<>(THREADS);
+            List<IgniteInternalFuture<Collection<Long>>> futs = new ArrayList<>(THREADS);
 
-        for (int i = 0; i < THREADS; i++) {
-            futs.add(GridTestUtils.runAsync(new Callable<Collection<Long>>() {
-                @Override public Collection<Long> call() throws Exception {
-                    Collection<Long> res = new ArrayList<>();
+            for (int i = 0; i < THREADS; i++) {
+                futs.add(GridTestUtils.runAsync(new Callable<Collection<Long>>() {
+                    @Override public Collection<Long> call() throws Exception {
+                        Collection<Long> res = new ArrayList<>();
 
-                    for (int i = 0; i < ITERATIONS; i++) {
-                        while (true) {
-                            try (Transaction tx = grid(0).transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
-                                long val = cache.get(key);
+                        for (int i = 0; i < ITERATIONS; i++) {
+                            while (true) {
+                                try (Transaction tx = grid(0).transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                                    long val = cache.get(key);
 
-                                cache.put(key, val + 1);
+                                    cache.put(key, val + 1);
 
-                                tx.commit();
+                                    tx.commit();
 
-                                assertTrue(res.add(val + 1));
+                                    assertTrue(res.add(val + 1));
 
-                                break;
-                            }
-                            catch(TransactionOptimisticException e) {
-                                log.info("Got error, will retry: " + e);
+                                    break;
+                                }
+                                catch (TransactionOptimisticException e) {
+                                    // Retry.
+                                }
                             }
                         }
+
+                        return res;
                     }
+                }));
+            }
 
-                    return res;
-                }
-            }));
-        }
+            long total = 0;
 
-        List<Collection<Long>> cols = new ArrayList<>(THREADS);
+            List<Collection<Long>> cols = new ArrayList<>(THREADS);
 
-        for (IgniteInternalFuture<Collection<Long>> fut : futs) {
-            Collection<Long> col = fut.get();
+            for (IgniteInternalFuture<Collection<Long>> fut : futs) {
+                Collection<Long> col = fut.get();
 
-            assertEquals(ITERATIONS, col.size());
+                assertEquals(ITERATIONS, col.size());
 
-            cols.add(col);
-        }
+                total += col.size();
+
+                cols.add(col);
+            }
+
+            log.info("Cache value: " + cache.get(key));
 
-        Set<Long> duplicates = new HashSet<>();
+            Set<Long> duplicates = new HashSet<>();
 
-        for (Collection<Long> col1 : cols) {
-            for (Long val1 : col1) {
-                for (Collection<Long> col2 : cols) {
-                    if (col1 == col2)
-                        continue;
+            for (Collection<Long> col1 : cols) {
+                for (Long val1 : col1) {
+                    for (Collection<Long> col2 : cols) {
+                        if (col1 == col2)
+                            continue;
 
-                    for (Long val2 : col2) {
-                        if (val1.equals(val2)) {
-                            duplicates.add(val2);
+                        for (Long val2 : col2) {
+                            if (val1.equals(val2)) {
+                                duplicates.add(val2);
 
-                            break;
+                                break;
+                            }
                         }
                     }
                 }
             }
-        }
 
-        assertTrue("Found duplicated values: " + duplicates, duplicates.isEmpty());
+            assertTrue("Found duplicated values: " + duplicates, duplicates.isEmpty());
+
+            assertEquals((long)THREADS * ITERATIONS, total);
+
+            // Try to update one more time to make sure cache is in consistent state.
+            try (Transaction tx = grid(0).transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                long val = cache.get(key);
+
+                cache.put(key, val);
+
+                tx.commit();
+            }
+
+            for (int i = 0; i < gridCount(); i++)
+                assertEquals(total, grid(i).cache(null).get(key));
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.java
new file mode 100644
index 0000000..ab9dc76
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxMultiThreadedSelfTest;
+
+/**
+ *
+ */
+public class GridCachePartitionedNearDisabledTxMultiThreadedSelfTest
+    extends GridCachePartitionedTxMultiThreadedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean nearEnabled() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index b6e86f2..cb83798 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -80,6 +80,7 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
 import org.eclipse.jetty.util.ConcurrentHashSet;
 import org.jetbrains.annotations.Nullable;
 
@@ -93,6 +94,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
 
 /**
  *
@@ -803,6 +805,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
         TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
 
+        for (int i = 0; i < 100; i++)
+            primaryCache(i, null).put(i, -1);
+
         final Map<Integer, Integer> map = new HashMap<>();
 
         for (int i = 0; i < 100; i++)
@@ -866,6 +871,150 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
     /**
      * @throws Exception If failed.
      */
+    public void testOptimisticSerializableTx() throws Exception {
+        optimisticSerializableTx(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOptimisticSerializableTxNearEnabled() throws Exception {
+        optimisticSerializableTx(new NearCacheConfiguration());
+    }
+
+    /**
+     * @param nearCfg Near cache configuration.
+     * @throws Exception If failed.
+     */
+    private void optimisticSerializableTx(NearCacheConfiguration nearCfg) throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setNearConfiguration(nearCfg);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        client = true;
+
+        final Ignite ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i);
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        spi.blockMessages(GridNearTxPrepareRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearTxPrepareRequest.class, ignite1.localNode().id());
+
+        spi.record(GridNearTxPrepareRequest.class);
+
+        final IgniteCache<Integer, Integer> cache = ignite2.cache(null);
+
+        IgniteInternalFuture<?> putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                try (Transaction tx = ignite2.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                    cache.putAll(map);
+
+                    tx.commit();
+                }
+
+                return null;
+            }
+        });
+
+        assertFalse(putFut.isDone());
+
+        client = false;
+
+        IgniteEx ignite3 = startGrid(3);
+
+        log.info("Stop block1.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        spi.record(null);
+
+        checkData(map, null, cache, 4);
+
+        List<Object> msgs = spi.recordedMessages();
+
+        for (Object msg : msgs)
+            assertTrue(((GridNearTxPrepareRequest)msg).firstClientRequest());
+
+        assertEquals(5, msgs.size());
+
+        ignite3.close();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i + 1);
+
+        spi.blockMessages(GridNearTxPrepareRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearTxPrepareRequest.class, ignite1.localNode().id());
+
+        spi.record(GridNearTxPrepareRequest.class);
+
+        putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                try (Transaction tx = ignite2.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                    for (Map.Entry<Integer, Integer> e : map.entrySet())
+                        cache.put(e.getKey(), e.getValue());
+
+                    tx.commit();
+                }
+
+                return null;
+            }
+        });
+
+        ignite3 = startGrid(3);
+
+        log.info("Stop block2.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        spi.record(null);
+
+        msgs = spi.recordedMessages();
+
+        for (Object msg : msgs)
+            assertTrue(((GridNearTxPrepareRequest)msg).firstClientRequest());
+
+        assertEquals(5, msgs.size());
+
+        checkData(map, null, cache, 4);
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i + 2);
+
+        try (Transaction tx = ignite2.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+            cache.putAll(map);
+
+            tx.commit();
+        }
+
+        checkData(map, null, cache, 4);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testLock() throws Exception {
         lock(null);
     }
@@ -1428,6 +1577,13 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
     /**
      * @throws Exception If failed.
      */
+    public void testOptimisticSerializableTxPutAllMultinode() throws Exception {
+        multinode(null, TestType.OPTIMISTIC_SERIALIZABLE_TX);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testPessimisticTxPutAllMultinode() throws Exception {
         multinode(null, TestType.PESSIMISTIC_TX);
     }
@@ -1497,7 +1653,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
                     IgniteCache<Integer, Integer> cache = ignite.cache(null);
 
-                    boolean useTx = testType == TestType.OPTIMISTIC_TX || testType == TestType.PESSIMISTIC_TX;
+                    boolean useTx = testType == TestType.OPTIMISTIC_TX ||
+                        testType == TestType.OPTIMISTIC_SERIALIZABLE_TX ||
+                        testType == TestType.PESSIMISTIC_TX;
 
                     if (useTx || testType == TestType.LOCK) {
                         assertEquals(TRANSACTIONAL,
@@ -1532,7 +1690,10 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
                                     TransactionConcurrency concurrency =
                                         testType == TestType.PESSIMISTIC_TX ? PESSIMISTIC : OPTIMISTIC;
 
-                                    try (Transaction tx = txs.txStart(concurrency, REPEATABLE_READ)) {
+                                    TransactionIsolation isolation = testType == TestType.OPTIMISTIC_SERIALIZABLE_TX ?
+                                        SERIALIZABLE : REPEATABLE_READ;
+
+                                    try (Transaction tx = txs.txStart(concurrency, isolation)) {
                                         cache.putAll(map);
 
                                         tx.commit();
@@ -1822,6 +1983,8 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
                     super.sendMessage(msg.get1(), msg.get2());
                 }
+
+                blockedMsgs.clear();
             }
         }
     }
@@ -1837,6 +2000,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
         OPTIMISTIC_TX,
 
         /** */
+        OPTIMISTIC_SERIALIZABLE_TX,
+
+        /** */
         PESSIMISTIC_TX,
 
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
index 8500e97..7fe0138 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -58,6 +59,7 @@ import static org.apache.ignite.testframework.GridTestUtils.setMemoryMode;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
 
 /**
  *
@@ -175,6 +177,13 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
     /**
      * @throws Exception If failed.
      */
+    public void testCrossCacheOptimisticSerializableTxFailover() throws Exception {
+        crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, SERIALIZABLE, TestMemoryMode.HEAP);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testCrossCacheOptimisticTxFailoverOffheapSwap() throws Exception {
         crossCacheTxFailover(PARTITIONED, true, OPTIMISTIC, REPEATABLE_READ, TestMemoryMode.OFFHEAP_EVICT_SWAP);
     }
@@ -424,6 +433,11 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
         @Override public int hashCode() {
             return (int)(key ^ (key >>> 32));
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestKey.class, this);
+        }
     }
 
     /**
@@ -446,6 +460,11 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
         public long value() {
             return val;
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheLockFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheLockFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheLockFailoverSelfTest.java
index 8645497..8a4609b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheLockFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheLockFailoverSelfTest.java
@@ -21,6 +21,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
@@ -28,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFutureTimeoutException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 
 /**
@@ -41,6 +43,15 @@ public class IgniteCacheLockFailoverSelfTest extends GridCacheAbstractSelfTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index 3913957..74d2d09 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -98,7 +98,11 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
 
         cfg.setCacheConfiguration(ccfg);
 
-        cfg.setCommunicationSpi(new DelayCommunicationSpi());
+        DelayCommunicationSpi spi = new DelayCommunicationSpi();
+
+        spi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(spi);
 
         if (testClientNode() && getTestGridName(0).equals(gridName))
             cfg.setClientMode(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
index 9546d61..02aa824 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxExceptionSelfTest.java
@@ -31,6 +31,7 @@ public class GridCacheNearTxExceptionSelfTest extends IgniteTxExceptionAbstractS
         return PARTITIONED;
     }
 
+    /** {@inheritDoc} */
     @Override public void testTransformBackup(){
         fail("https://issues.apache.org/jira/browse/IGNITE-1601");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
index 6ed25eb..20ee904 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxMultiThreadedSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.IgniteTxMultiThreadedAbstractTest;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -40,11 +41,6 @@ public class GridCachePartitionedTxMultiThreadedSelfTest extends IgniteTxMultiTh
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
-    @Override public void testOptimisticSerializableCommitMultithreaded() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-806");
-    }
-
-    /** {@inheritDoc} */
     @SuppressWarnings({"ConstantConditions"})
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
@@ -60,6 +56,8 @@ public class GridCachePartitionedTxMultiThreadedSelfTest extends IgniteTxMultiTh
 
         cc.setWriteSynchronizationMode(FULL_SYNC);
 
+        cc.setNearConfiguration(nearEnabled() ? new NearCacheConfiguration() : null);
+
         c.setCacheConfiguration(cc);
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
@@ -74,6 +72,13 @@ public class GridCachePartitionedTxMultiThreadedSelfTest extends IgniteTxMultiTh
         return c;
     }
 
+    /**
+     * @return {@code True} if near cache is enabled.
+     */
+    protected boolean nearEnabled() {
+        return true;
+    }
+
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 3;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateAfterLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateAfterLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateAfterLoadTest.java
new file mode 100644
index 0000000..32e204b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateAfterLoadTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.datastreamer;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class DataStreamerUpdateAfterLoadTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** */
+    private static final int NODES = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(NODES - 1);
+
+        client = true;
+
+        startGrid(NODES - 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUpdateAfterLoad() throws Exception {
+        Ignite ignite0 = ignite(0);
+
+        for (CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
+            int key = 0;
+
+            try (IgniteCache<Integer, Integer> cache = ignite0.createCache(ccfg)) {
+                key = testLoadAndUpdate(cache.getName(), key, false);
+
+                testLoadAndUpdate(cache.getName(), key, true);
+
+                ignite0.destroyCache(cache.getName());
+            }
+        }
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param key Key.
+     * @param allowOverwrite Streamer flag.
+     * @return Next key.
+     * @throws Exception If failed.
+     */
+    private int testLoadAndUpdate(String cacheName, int key, boolean allowOverwrite) throws Exception {
+        for (int loadNode = 0; loadNode < NODES; loadNode++) {
+            Ignite loadIgnite = ignite(loadNode);
+
+            for (int updateNode = 0; updateNode < NODES; updateNode++) {
+                try (IgniteDataStreamer<Integer, Integer> streamer = loadIgnite.dataStreamer(cacheName)) {
+                    streamer.allowOverwrite(allowOverwrite);
+
+                    streamer.addData(key, key);
+                }
+
+                Ignite updateIgnite = ignite(updateNode);
+
+                IgniteCache<Integer, Integer> cache = updateIgnite.cache(cacheName);
+
+                if (allowOverwrite)
+                    atomicClockModeDelay(cache);
+
+                updateIgnite.cache(cacheName).put(key, key + 1);
+
+                checkValue(key, key + 1, cacheName);
+
+                key++;
+            }
+        }
+
+        return key;
+    }
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param cacheName Cache name.
+     */
+    private void checkValue(Integer key, Integer val, String cacheName) {
+        for (int i = 0; i < NODES; i++) {
+            IgniteCache<Integer, Integer> cache = ignite(i).cache(cacheName);
+
+            assertEquals("Unexpected value " + i, val, cache.get(key));
+        }
+    }
+
+    /**
+     * @return Cache configurations to test.
+     */
+    private List<CacheConfiguration<Integer, Integer>> cacheConfigurations() {
+        List<CacheConfiguration<Integer, Integer>> ccfgs = new ArrayList<>();
+
+        ccfgs.add(cacheConfiguration(CacheAtomicityMode.ATOMIC, PRIMARY, 1, "cache-" + ccfgs.size()));
+        ccfgs.add(cacheConfiguration(CacheAtomicityMode.ATOMIC, PRIMARY, 0, "cache-" + ccfgs.size()));
+        ccfgs.add(cacheConfiguration(CacheAtomicityMode.ATOMIC, CLOCK, 1, "cache-" + ccfgs.size()));
+        ccfgs.add(cacheConfiguration(CacheAtomicityMode.TRANSACTIONAL, null, 1, "cache-" + ccfgs.size()));
+        ccfgs.add(cacheConfiguration(CacheAtomicityMode.TRANSACTIONAL, null, 0, "cache-" + ccfgs.size()));
+
+        return ccfgs;
+    }
+
+    /**
+     * @param atomicityMode Cache atomicity mode.
+     * @param writeOrderMode Cache write order mode.
+     * @param backups Number of backups.
+     * @param name Cache name.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(CacheAtomicityMode atomicityMode,
+        CacheAtomicWriteOrderMode writeOrderMode,
+        int backups,
+        String name) {
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setName(name);
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setBackups(backups);
+        ccfg.setAtomicWriteOrderMode(writeOrderMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
index cf2ff41..a6dfc9c 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Tests hashmap load.
@@ -81,7 +82,10 @@ public class GridHashMapLoadTest extends GridCommonAbstractTest {
 
             map.put(key, new GridCacheMapEntry(ctx, ctx.toCacheKeyObject(key),
                 key.hashCode(), ctx.toCacheObject(val), null, 1) {
-                @Override public boolean tmLock(IgniteInternalTx tx, long timeout) {
+                @Override public boolean tmLock(IgniteInternalTx tx,
+                    long timeout,
+                    @Nullable GridCacheVersion serOrder,
+                    GridCacheVersion serReadVer) {
                     return false;
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
index cc5226f..f50a732 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
@@ -180,6 +180,14 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceS
     }
 
     /**
+     *
+     */
+    public void clearAll() {
+        for (Space space : spaces.values())
+            space.clear();
+    }
+
+    /**
      * Gets space, creates if does not exist.
      *
      * @param spaceName Space name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index e4c2129..28d5c73 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -77,6 +77,8 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.junits.GridAbstractTest;
 import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionRollbackException;
 import org.jetbrains.annotations.Nullable;
 
@@ -1026,8 +1028,23 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @throws Exception If failed.
      */
     protected <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
+        return doInTransaction(ignite, PESSIMISTIC, REPEATABLE_READ, clo);
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @param clo Closure.
+     * @return Result of closure execution.
+     * @throws Exception If failed.
+     */
+    protected <T> T doInTransaction(Ignite ignite,
+        TransactionConcurrency concurrency,
+        TransactionIsolation isolation,
+        Callable<T> clo) throws Exception {
         while (true) {
-            try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            try (Transaction tx = ignite.transactions().txStart(concurrency, isolation)) {
                 T res = clo.call();
 
                 tx.commit();

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index d2904e87..c62a131 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -122,6 +122,7 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSel
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImplSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerMultinodeCreateCacheTest;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamerUpdateAfterLoadTest;
 import org.apache.ignite.testframework.GridTestUtils;
 
 /**
@@ -213,7 +214,8 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheAffinityApiSelfTest.class);
         suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
         GridTestUtils.addTestIfNeeded(suite, DataStreamProcessorSelfTest.class, ignoredTests);
-        suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
+        GridTestUtils.addTestIfNeeded(suite, DataStreamerUpdateAfterLoadTest.class, ignoredTests);
+            suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
         suite.addTestSuite(DataStreamerMultinodeCreateCacheTest.class);
         suite.addTestSuite(DataStreamerImplSelfTest.class);
         GridTestUtils.addTestIfNeeded(suite, GridCacheEntryMemorySizeSelfTest.class, ignoredTests);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 93bd26c..5138dac 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdate
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionNotLoadedEventSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedNearDisabledTxMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTransformEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheClientNodeChangingTopologyTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheClientNodePartitionsExchangeTest;
@@ -190,6 +191,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheDhtInternalEntrySelfTest.class));
         suite.addTest(new TestSuite(GridCacheDhtMappingSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedTxMultiThreadedSelfTest.class));
+        suite.addTest(new TestSuite(GridCachePartitionedNearDisabledTxMultiThreadedSelfTest.class));
         suite.addTest(new TestSuite(GridCacheDhtPreloadSelfTest.class));
         suite.addTest(new TestSuite(GridCacheDhtPreloadOffHeapSelfTest.class));
         suite.addTest(new TestSuite(GridCacheDhtPreloadBigDataSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
new file mode 100644
index 0000000..0f86c4c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest;
+import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest;
+
+/**
+ * Test suite.
+ */
+public class IgniteCacheTestSuite5 extends TestSuite {
+    /**
+     * @return IgniteCache test suite.
+     * @throws Exception Thrown in case of the failure.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("IgniteCache Test Suite part 5");
+
+        suite.addTestSuite(CacheSerializableTransactionsTest.class);
+        suite.addTestSuite(CacheNearReaderUpdateTest.class);
+
+        return suite;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index 82bf766..82fc3f8 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -86,5 +86,8 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxOffHeapValuesBenchmark -sn IgniteNode -ds ${ver}tx-put-offheap-val-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-offheap-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-join-offheap-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-put-offheap-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-put-offheap-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 100 -dn IgnitePutAllBenchmark -sn IgniteNode -ds ${ver}atomic-putAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 100 -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup\
 "

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java
new file mode 100644
index 0000000..32e7653
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java
@@ -0,0 +1,81 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionOptimisticException;
+import org.apache.ignite.yardstick.cache.model.Account;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
+
+/**
+ *
+ */
+public class IgniteAccountSerializableTxBenchmark extends IgniteAccountTxAbstractBenchmark {
+    /** */
+    private static final int ACCOUNT_NUMBER = 3;
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        Set<Integer> accountIds = new HashSet<>();
+
+        while (accountIds.size() < ACCOUNT_NUMBER)
+            accountIds.add(nextRandom(args.range()));
+
+        while (true) {
+            try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
+                Map<Integer, Account> accounts = (Map)cache.getAll(accountIds);
+
+                if (accounts.size() != ACCOUNT_NUMBER)
+                    throw new Exception("Failed to find accounts: " + accountIds);
+
+                Integer fromId = accountIds.iterator().next();
+
+                int fromBalance = accounts.get(fromId).balance();
+
+                for (Integer id : accountIds) {
+                    if (id.equals(fromId))
+                        continue;
+
+                    Account account = accounts.get(id);
+
+                    if (fromBalance > 0) {
+                        fromBalance--;
+
+                        cache.put(id, new Account(account.balance() + 1));
+                    }
+                }
+
+                cache.put(fromId, new Account(fromBalance));
+
+                tx.commit();
+            }
+            catch (TransactionOptimisticException e) {
+                continue;
+            }
+
+            break;
+        }
+
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxAbstractBenchmark.java
new file mode 100644
index 0000000..0266a3c
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxAbstractBenchmark.java
@@ -0,0 +1,61 @@
+/*
+ * 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.yardstick.cache;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.yardstick.cache.model.Account;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ *
+ */
+public abstract class IgniteAccountTxAbstractBenchmark extends IgniteCacheAbstractBenchmark {
+    /** */
+    protected IgniteTransactions txs;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        txs = ignite().transactions();
+
+        println(cfg, "Populating data...");
+
+        long start = System.nanoTime();
+
+        try (IgniteDataStreamer<Integer, Account> dataLdr = ignite().dataStreamer(cache.getName())) {
+            for (int i = 0; i < args.range() && !Thread.currentThread().isInterrupted(); i++) {
+                dataLdr.addData(i, new Account(100_000));
+
+                if (i % 100000 == 0)
+                    println(cfg, "Populated accounts: " + i);
+            }
+        }
+
+        println(cfg, "Finished populating data in " + ((System.nanoTime() - start) / 1_000_000) + " ms.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java
new file mode 100644
index 0000000..78e675e
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.yardstick.cache.model.Account;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ *
+ */
+public class IgniteAccountTxBenchmark extends IgniteAccountTxAbstractBenchmark {
+    /** */
+    private static final int ACCOUNT_NUMBER = 3;
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        Set<Integer> accountIds = new TreeSet<>();
+
+        while (accountIds.size() < ACCOUNT_NUMBER)
+            accountIds.add(nextRandom(args.range()));
+
+        try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            Map<Integer, Account> accounts = (Map)cache.getAll(accountIds);
+
+            if (accounts.size() != ACCOUNT_NUMBER)
+                throw new Exception("Failed to find accounts: " + accountIds);
+
+            Integer fromId = accountIds.iterator().next();
+
+            int fromBalance = accounts.get(fromId).balance();
+
+            for (Integer id : accountIds) {
+                if (id.equals(fromId))
+                    continue;
+
+                Account account = accounts.get(id);
+
+                if (fromBalance > 0) {
+                    fromBalance--;
+
+                    cache.put(id, new Account(account.balance() + 1));
+                }
+            }
+
+            cache.put(fromId, new Account(fromBalance));
+
+            tx.commit();
+        }
+
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9d7543a8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
index 1260f9c..22a9eac 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
@@ -88,6 +88,9 @@ public abstract class IgniteCacheAbstractBenchmark extends IgniteAbstractBenchma
         /** */
         final int max;
 
+        /** */
+        final ThreadLocalRandom rnd;
+
         /**
          * @param min Min.
          * @param max Max.
@@ -95,13 +98,15 @@ public abstract class IgniteCacheAbstractBenchmark extends IgniteAbstractBenchma
         private ThreadRange(int min, int max) {
             this.min = min;
             this.max = max;
+
+            rnd = ThreadLocalRandom.current();
         }
 
         /**
          * @return Next random key.
          */
         int nextRandom() {
-            return ThreadLocalRandom.current().nextInt(min, max);
+            return rnd.nextInt(min, max);
         }
     }
 }
\ No newline at end of file