You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by av...@apache.org on 2016/02/02 16:24:06 UTC

[1/6] ignite git commit: 2224 (cherry picked from commit 0113506)

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5.7 e5fe26395 -> 1842861f5


http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/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 0d9c541..532efa5 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
@@ -53,6 +53,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
@@ -556,6 +557,30 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception In case of error.
      */
+    public void testGetEntry() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+
+        CacheEntry<String, Integer> key1e =  cache.getEntry("key1");
+        CacheEntry<String, Integer> key2e =  cache.getEntry("key2");
+        CacheEntry<String, Integer> wrongKeye =  cache.getEntry("wrongKey");
+
+        assert key1e.getValue() == 1;
+        assert key1e.getKey().equals("key1");
+        assert key1e.version() != null;
+
+        assert key2e.getValue() == 2;
+        assert key2e.getKey().equals("key2");
+        assert key2e.version() != null;
+
+        assert wrongKeye == null;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
     public void testGetAsync() throws Exception {
         IgniteCache<String, Integer> cache = jcache();
 
@@ -664,6 +689,122 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception In case of error.
      */
+    public void testGetEntries() throws Exception {
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        final IgniteCache<String, Integer> cache = jcache();
+
+        try {
+            cache.put("key1", 1);
+            cache.put("key2", 2);
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getEntries(null).isEmpty();
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        assert cache.getEntries(Collections.<String>emptySet()).isEmpty();
+
+        Collection<CacheEntry<String, Integer>> c1 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+        info("Retrieved c1: " + c1);
+
+        assert 2 == c1.size() : "Invalid collection: " + c1;
+
+        boolean b1 = false;
+        boolean b2 = false;
+
+        for (CacheEntry<String, Integer> e: c1){
+            if (e.getKey().equals("key1") && e.getValue().equals(1))
+                b1 = true;
+
+            if (e.getKey().equals("key2") && e.getValue().equals(2))
+                b2 = true;
+        }
+
+        assertTrue(b1 && b2);
+
+        Collection<CacheEntry<String, Integer>> c2 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+        info("Retrieved c2: " + c2);
+
+        assert 2 == c2.size() : "Invalid collection: " + c2;
+
+        b1 = false;
+        b2 = false;
+
+        for (CacheEntry<String, Integer> e: c2){
+            if (e.getKey().equals("key1") && e.getValue().equals(1))
+                b1 = true;
+
+            if (e.getKey().equals("key2") && e.getValue().equals(2))
+                b2 = true;
+        }
+
+        assertTrue(b1 && b2);
+
+        // Now do the same checks but within transaction.
+        if (txShouldBeUsed()) {
+            try (Transaction tx0 = transactions().txStart()) {
+                assert cache.getEntries(Collections.<String>emptySet()).isEmpty();
+
+                c1 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+                info("Retrieved c1: " + c1);
+
+                assert 2 == c1.size() : "Invalid collection: " + c1;
+
+                b1 = false;
+                b2 = false;
+
+                for (CacheEntry<String, Integer> e : c1) {
+                    if (e.getKey().equals("key1") && e.getValue().equals(1))
+                        b1 = true;
+
+                    if (e.getKey().equals("key2") && e.getValue().equals(2))
+                        b2 = true;
+                }
+
+                assertTrue(b1 && b2);
+
+                c2 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+                info("Retrieved c2: " + c2);
+
+                assert 2 == c2.size() : "Invalid collection: " + c2;
+
+                b1 = false;
+                b2 = false;
+
+                for (CacheEntry<String, Integer> e : c2) {
+                    if (e.getKey().equals("key1") && e.getValue().equals(1))
+                        b1 = true;
+
+                    if (e.getKey().equals("key2") && e.getValue().equals(2))
+                        b2 = true;
+                }
+
+                assertTrue(b1 && b2);
+
+                tx0.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
     public void testGetAllWithNulls() throws Exception {
         final IgniteCache<String, Integer> cache = jcache();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
index c57869f..f50a3e0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -29,6 +30,7 @@ import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.Affinity;
@@ -152,19 +154,32 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
      * @throws Exception If failed.
      */
     public void testGet() throws Exception {
-        testGet(primaryKey(0));
+        testGet(primaryKey(0), false);
 
         afterTest();
 
         if (cacheMode() != LOCAL)
-            testGet(backupKey(0));
+            testGet(backupKey(0), false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetEntry() throws Exception {
+        testGet(primaryKey(0), true);
+
+        afterTest();
+
+        if (cacheMode() != LOCAL)
+            testGet(backupKey(0), true);
     }
 
     /**
      * @param key Key.
+     * @param needVer Need version.
      * @throws Exception If failed.
      */
-    private void testGet(String key) throws Exception {
+    private void testGet(String key, boolean needVer) throws Exception {
         // Try when value is not in cache.
 
         interceptor.retInterceptor = new NullGetInterceptor();
@@ -173,7 +188,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         IgniteCache<String, Integer> cache = jcache(0);
 
-        assertEquals(null, cache.get(key));
+        assertEquals(null, needVer ? cache.getEntry(key) : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -185,7 +200,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         log.info("Get 2.");
 
-        assertEquals((Integer)1, cache.get(key));
+        assertEquals((Integer)1, needVer ? cache.getEntry(key).getValue() : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -207,7 +222,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         log.info("Get 3.");
 
-        assertEquals(null, cache.get(key));
+        assertEquals(null, needVer ? cache.getEntry(key) : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -223,7 +238,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         log.info("Get 4.");
 
-        assertEquals((Integer)101, cache.get(key));
+        assertEquals((Integer)101, needVer ? cache.getEntry(key).getValue() : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -241,9 +256,16 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         IgniteCache<String, Integer> cacheAsync = cache.withAsync();
 
-        cacheAsync.get(key);
+        if (needVer) {
+            cacheAsync.getEntry(key);
 
-        assertEquals((Integer)101, cacheAsync.<Integer>future().get());
+            assertEquals((Integer)101, cacheAsync.<CacheEntry<String, Integer>>future().get().getValue());
+        }
+        else {
+            cacheAsync.get(key);
+
+            assertEquals((Integer)101, cacheAsync.<Integer>future().get());
+        }
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -258,6 +280,20 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
      * @throws Exception If failed.
      */
     public void testGetAll() throws Exception {
+        testGetAll(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetEntries() throws Exception {
+        testGetAll(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void testGetAll(boolean needVer) throws Exception {
         Set<String> keys = new LinkedHashSet<>();
 
         for (int i = 0; i < 1000; i++)
@@ -269,10 +305,19 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         IgniteCache<String, Integer> cacheAsync = cache.withAsync();
 
-        Map<String, Integer> map = cache.getAll(keys);
+        Collection<CacheEntry<String, Integer>> c;
+        Map<String, Integer> map;
+
+        if (needVer){
+            c = cache.getEntries(keys);
 
-        for (String key : keys)
-            assertEquals(null, map.get(key));
+            assertTrue(c.isEmpty());
+        }else {
+            map = cache.getAll(keys);
+
+            for (String key : keys)
+                assertEquals(null, map.get(key));
+        }
 
         assertEquals(1000, interceptor.invokeCnt.get());
 
@@ -280,15 +325,28 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         interceptor.retInterceptor = new GetAllInterceptor1();
 
-        map = cache.getAll(keys);
+        if (needVer) {
+            c = cache.getEntries(keys);
 
-        for (String key : keys) {
-            int k = Integer.valueOf(key);
+            assertEquals(500, c.size());
 
-            if (k % 2 == 0)
-                assertEquals(null, map.get(key));
-            else
-                assertEquals((Integer)(k * 2), map.get(key));
+            for (CacheEntry<String, Integer> e : c) {
+                int k = Integer.valueOf(e.getKey());
+
+                assertEquals((Integer)(k * 2), e.getValue());
+            }
+        }
+        else {
+            map = cache.getAll(keys);
+
+            for (String key : keys) {
+                int k = Integer.valueOf(key);
+
+                if (k % 2 == 0)
+                    assertEquals(null, map.get(key));
+                else
+                    assertEquals((Integer)(k * 2), map.get(key));
+            }
         }
 
         assertEquals(1000, interceptor.invokeCnt.get());
@@ -307,40 +365,72 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
             interceptor.retInterceptor = new GetAllInterceptor2();
 
-            if (j == 0)
-                map = cache.getAll(keys);
-            else {
-                cacheAsync.getAll(keys);
+            if (needVer) {
+                if (j == 0)
+                    c = cache.getEntries(keys);
+                else {
+                    cacheAsync.getEntries(keys);
+
+                    c = cacheAsync.<Collection<CacheEntry<String, Integer>>>future().get();
+                }
+
+                for (CacheEntry<String, Integer> e : c) {
+                    int k = Integer.valueOf(e.getKey());
+
+                    switch (k % 3) {
+                        case 1:
+                            Integer exp = k < 500 ? k : null;
 
-                map = cacheAsync.<Map<String, Integer>>future().get();
+                            assertEquals(exp, e.getValue());
+
+                            break;
+
+                        case 2:
+                            assertEquals((Integer)(k * 3), e.getValue());
+
+                            break;
+
+                        default:
+                            fail();
+                    }
+                }
             }
+            else {
+                if (j == 0)
+                    map = cache.getAll(keys);
+                else {
+                    cacheAsync.getAll(keys);
 
-            int i = 0;
+                    map = cacheAsync.<Map<String, Integer>>future().get();
+                }
 
-            for (String key : keys) {
-                switch (i % 3) {
-                    case 0:
-                        assertEquals(null, map.get(key));
+                int i = 0;
 
-                        break;
+                for (String key : keys) {
+                    switch (i % 3) {
+                        case 0:
+                            assertEquals(null, map.get(key));
 
-                    case 1:
-                        Integer exp = i < 500 ? i : null;
+                            break;
 
-                        assertEquals(exp, map.get(key));
+                        case 1:
+                            Integer exp = i < 500 ? i : null;
 
-                        break;
+                            assertEquals(exp, map.get(key));
 
-                    case 2:
-                        assertEquals((Integer)(i * 3), map.get(key));
+                            break;
 
-                        break;
+                        case 2:
+                            assertEquals((Integer)(i * 3), map.get(key));
 
-                    default:
-                        fail();
-                }
+                            break;
 
-                i++;
+                        default:
+                            fail();
+                    }
+
+                    i++;
+                }
             }
 
             assertEquals(1000, interceptor.invokeCnt.get());

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
index 6a7416e..293ba1e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
@@ -263,7 +263,7 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
             waitForLocalEvent(grid(primary).events(), nodeEvent(primary.id()), EVT_CACHE_ENTRY_EVICTED);
 
         // Get value on other node, it should be loaded to near cache.
-        assertEquals(val, nearOther.get(key, true));
+        assertEquals(val, nearOther.get(key, true, false));
 
         entryPrimary = (GridDhtCacheEntry)dhtPrimary.peekEx(key);
         entryBackup = (GridDhtCacheEntry)dhtBackup.peekEx(key);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index 035f1b0..7286fb3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -34,6 +34,7 @@ import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CachePeekMode;
@@ -129,7 +130,7 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) 
+    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args)
         throws CacheException {
         throw new UnsupportedOperationException("Method should be supported.");
     }
@@ -224,11 +225,21 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public CacheEntry<K, V> getEntry(K key) {
+        return compute.call(new GetEntryTask<K, V>(cacheName, isAsync, key));
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<K, V> getAll(Set<? extends K> keys) {
         return compute.call(new GetAllTask<K, V>(cacheName, isAsync, keys));
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<CacheEntry<K, V>> getEntries(Set<? extends K> keys) {
+        return compute.call(new GetEntriesTask<K, V>(cacheName, isAsync, keys));
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) {
         return compute.call(new GetAllOutTxTask<K, V>(cacheName, isAsync, keys));
     }
@@ -710,6 +721,29 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     /**
      *
      */
+    private static class GetEntryTask<K, V> extends CacheTaskAdapter<K, V, CacheEntry<K, V>> {
+        /** Key. */
+        private final K key;
+
+        /**
+         * @param cacheName Cache name.
+         * @param async Async.
+         * @param key Key.
+         */
+        public GetEntryTask(String cacheName, boolean async, K key) {
+            super(cacheName, async);
+            this.key = key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public CacheEntry<K, V> call() throws Exception {
+            return cache().getEntry(key);
+        }
+    }
+
+    /**
+     *
+     */
     private static class RemoveAllTask<K, V> extends CacheTaskAdapter<K, V, Void> {
         /**
          * @param cacheName Cache name.
@@ -973,6 +1007,29 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     /**
      *
      */
+    private static class GetEntriesTask<K, V> extends CacheTaskAdapter<K, V, Collection<CacheEntry<K, V>> > {
+        /** Keys. */
+        private final Set<? extends K> keys;
+
+        /**
+         * @param cacheName Cache name.
+         * @param async Async.
+         * @param keys Keys.
+         */
+        public GetEntriesTask(String cacheName, boolean async, Set<? extends K> keys) {
+            super(cacheName, async);
+            this.keys = keys;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<CacheEntry<K, V>>  call() throws Exception {
+            return cache().getEntries(keys);
+        }
+    }
+
+    /**
+     *
+     */
     private static class GetAllOutTxTask<K, V> extends CacheTaskAdapter<K, V, Map<K, V>> {
         /** Keys. */
         private final Set<? extends K> keys;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 68e52df..45679dd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -19,6 +19,12 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListenerSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticReadCommittedSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticRepeatableReadSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticSerializableSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticReadCommittedSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticRepeatableReadSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticSerializableSeltTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
 import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest;
@@ -241,6 +247,12 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteSystemCacheOnClientTest.class);
 
         suite.addTestSuite(CacheRemoveAllSelfTest.class);
+        suite.addTestSuite(CacheGetEntryOptimisticReadCommittedSeltTest.class);
+        suite.addTestSuite(CacheGetEntryOptimisticRepeatableReadSeltTest.class);
+        suite.addTestSuite(CacheGetEntryOptimisticSerializableSeltTest.class);
+        suite.addTestSuite(CacheGetEntryPessimisticReadCommittedSeltTest.class);
+        suite.addTestSuite(CacheGetEntryPessimisticRepeatableReadSeltTest.class);
+        suite.addTestSuite(CacheGetEntryPessimisticSerializableSeltTest.class);
 
         suite.addTestSuite(CacheStopAndDestroySelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index c638d94..0428c73 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -89,10 +89,14 @@ j=10
 CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutBenchmark -sn IgniteNode -ds ${ver}atomic-put-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutGetBenchmark -sn IgniteNode -ds ${ver}atomic-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutGetEntryBenchmark -sn IgniteNode -ds ${ver}atomic-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutTxImplicitBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-put-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-optim-repRead-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -dn IgnitePutGetEntryTxBenchmark -sn IgniteNode -ds ${ver}tx-optim-repRead-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc PESSIMISTIC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-pessim-repRead-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc PESSIMISTIC -dn IgnitePutGetEntryTxBenchmark -sn IgniteNode -ds ${ver}tx-pessim-repRead-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -txi SERIALIZABLE -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serial-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -txi SERIALIZABLE -dn IgnitePutGetEntryTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serial-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds ${ver}sql-query-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds ${ver}sql-query-join-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds ${ver}sql-query-put-1-backup,\
@@ -114,6 +118,9 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetEntriesPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-getEntriesPutAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-pessimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetEntriesPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-pessimistic-getEntriesPutAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC -txi SERIALIZABLE -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serializable-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC -txi SERIALIZABLE -dn IgniteGetEntriesPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serializable-getEntriesPutAll-1-backup,\
 "

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java
new file mode 100644
index 0000000..501e12d
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * 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.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
+
+/**
+ * Ignite benchmark that performs transactional putAll operations.
+ */
+public class IgniteGetEntriesPutAllTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Integer> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        txs = ignite().transactions();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        final ThreadRange r = threadRange();
+
+        doInTransaction(txs, args.txConcurrency(), args.txIsolation(), new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                SortedMap<Integer, Integer> vals = new TreeMap<>();
+
+                for (int i = 0; i < args.batch(); i++) {
+                    int key = r.nextRandom();
+
+                    vals.put(key, key);
+                }
+
+                cache.getEntries(vals.keySet());
+
+                cache.putAll(vals);
+
+                return null;
+            }
+        });
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Integer> cache() {
+        return ignite().cache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java
new file mode 100644
index 0000000..1289fa1
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java
@@ -0,0 +1,47 @@
+/*
+ * 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 org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntry;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+
+/**
+ * Ignite benchmark that performs put and get operations.
+ */
+public class IgnitePutGetEntryBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        CacheEntry<Integer, Object> val = cache.getEntry(key);
+
+        if (val != null)
+            key = nextRandom(args.range());
+
+        cache.put(key, new SampleValue(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java
new file mode 100644
index 0000000..6e58b41
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * 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.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cache.CacheEntry;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
+
+/**
+ * Ignite benchmark that performs transactional put and get operations.
+ */
+public class IgnitePutGetEntryTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** */
+    private Callable<Void> clo;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        txs = ignite().transactions();
+
+        clo = new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                int key = nextRandom(0, args.range() / 2);
+
+                CacheEntry<Integer, Object> val = cache.getEntry(key);
+
+                if (val != null)
+                    key = nextRandom(args.range() / 2, args.range());
+
+                cache.put(key, new SampleValue(key));
+
+                return null;
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        doInTransaction(txs, args.txConcurrency(), args.txIsolation(), clo);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("tx");
+    }
+}


[3/6] ignite git commit: 2224 (cherry picked from commit 0113506)

Posted by av...@apache.org.
2224
(cherry picked from commit 0113506)


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

Branch: refs/heads/ignite-1.5.7
Commit: f527656da0289dd45f7f8ae0c8d3410949cd54dd
Parents: b146c0f
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Feb 2 15:25:05 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Feb 2 18:12:57 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  45 +-
 .../processors/cache/CacheEntryImplEx.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 297 +++++--
 .../processors/cache/GridCacheContext.java      |  33 +-
 .../processors/cache/GridCacheMapEntry.java     |   2 +-
 .../processors/cache/GridCacheProxyImpl.java    |  51 ++
 .../processors/cache/IgniteCacheProxy.java      |  51 ++
 .../processors/cache/IgniteInternalCache.java   |  85 ++
 .../dht/CacheDistributedGetFutureAdapter.java   |  15 -
 .../distributed/dht/GridDhtCacheAdapter.java    |   7 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |   6 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   4 +-
 .../dht/GridPartitionedGetFuture.java           |  38 +-
 .../dht/GridPartitionedSingleGetFuture.java     |  17 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  82 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  42 +-
 .../distributed/near/GridNearAtomicCache.java   |   6 +-
 .../distributed/near/GridNearCacheAdapter.java  |   6 +-
 .../distributed/near/GridNearCacheEntry.java    |   3 +-
 .../distributed/near/GridNearGetFuture.java     |  45 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   2 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   4 +
 .../GridNearPessimisticTxPrepareFuture.java     |   2 +
 .../near/GridNearTransactionalCache.java        |   9 +-
 .../local/atomic/GridLocalAtomicCache.java      |  97 ++-
 .../cache/transactions/IgniteTxEntry.java       |  36 +-
 .../transactions/IgniteTxLocalAdapter.java      | 196 +++--
 .../cache/transactions/IgniteTxLocalEx.java     |   3 +-
 .../cache/transactions/IgniteTxManager.java     |   2 +-
 .../cache/CacheGetEntryAbstractTest.java        | 803 +++++++++++++++++++
 ...GetEntryOptimisticReadCommittedSeltTest.java |  36 +
 ...etEntryOptimisticRepeatableReadSeltTest.java |  36 +
 ...eGetEntryOptimisticSerializableSeltTest.java |  36 +
 ...etEntryPessimisticReadCommittedSeltTest.java |  36 +
 ...tEntryPessimisticRepeatableReadSeltTest.java |  36 +
 ...GetEntryPessimisticSerializableSeltTest.java |  36 +
 .../cache/CacheReadThroughRestartSelfTest.java  |  43 +-
 .../CacheSerializableTransactionsTest.java      | 142 +++-
 .../cache/GridCacheAbstractFullApiSelfTest.java | 141 ++++
 .../GridCacheInterceptorAbstractSelfTest.java   | 172 +++-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   2 +-
 .../multijvm/IgniteCacheProcessProxy.java       |  59 +-
 .../testsuites/IgniteCacheTestSuite4.java       |  12 +
 .../config/benchmark-multicast.properties       |   7 +
 .../IgniteGetEntriesPutAllTxBenchmark.java      |  73 ++
 .../cache/IgnitePutGetEntryBenchmark.java       |  47 ++
 .../cache/IgnitePutGetEntryTxBenchmark.java     |  73 ++
 47 files changed, 2648 insertions(+), 342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index 886dca6..a791e38 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -31,10 +31,12 @@ import javax.cache.CacheException;
 import javax.cache.configuration.Configuration;
 import javax.cache.event.CacheEntryRemovedListener;
 import javax.cache.expiry.ExpiryPolicy;
+import javax.cache.integration.CacheLoader;
 import javax.cache.integration.CacheWriter;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.EntryProcessorResult;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CacheMode;
@@ -390,18 +392,59 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * <code>null</code> value for a key.
      */
     @IgniteAsyncSupported
-    <T> Map<K, EntryProcessorResult<T>> invokeAll(Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
+    public <T> Map<K, EntryProcessorResult<T>> invokeAll(Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args);
 
     /** {@inheritDoc} */
     @IgniteAsyncSupported
     @Override public V get(K key);
 
+    /**
+     * Gets an entry from the cache.
+     * <p>
+     * If the cache is configured to use read-through, and get would return null
+     * because the entry is missing from the cache, the Cache's {@link CacheLoader}
+     * is called in an attempt to load the entry.
+     *
+     * @param key the key whose associated value is to be returned
+     * @return the element, or null, if it does not exist.
+     * @throws IllegalStateException if the cache is {@link #isClosed()}
+     * @throws NullPointerException  if the key is null
+     * @throws CacheException        if there is a problem fetching the value
+     * @throws ClassCastException    if the implementation is configured to perform
+     * runtime-type-checking, and the key or value types are incompatible with those that have been
+     * configured for the {@link Cache}
+     */
+    @IgniteAsyncSupported
+    public CacheEntry<K, V> getEntry(K key);
+
     /** {@inheritDoc} */
     @IgniteAsyncSupported
     @Override public Map<K, V> getAll(Set<? extends K> keys);
 
     /**
+     * Gets a collection of entries from the {@link Cache}.
+     * <p>
+     * If the cache is configured read-through, and a get for a key would
+     * return null because an entry is missing from the cache, the Cache's
+     * {@link CacheLoader} is called in an attempt to load the entry. If an
+     * entry cannot be loaded for a given key, the key will not be present in
+     * the returned Collection.
+     *
+     * @param keys The keys whose associated values are to be returned.
+     * @return A collection of entries that were found for the given keys. Entries not found
+     *         in the cache are not in the returned collection.
+     * @throws NullPointerException  if keys is null or if keys contains a null
+     * @throws IllegalStateException if the cache is {@link #isClosed()}
+     * @throws CacheException        if there is a problem fetching the values
+     * @throws ClassCastException    if the implementation is configured to perform
+     * runtime-type-checking, and the key or value types are incompatible with those that have been
+     * configured for the {@link Cache}
+     */
+    @IgniteAsyncSupported
+    public Collection<CacheEntry<K, V>> getEntries(Set<? extends K> keys);
+
+    /**
      * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
      * and will not lock any keys if pessimistic transaction is started by thread.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
index 1c7111a..af926c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
@@ -21,9 +21,13 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
+import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.GET_ENTRY_INVALID_VER_AFTER_GET;
+import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.GET_ENTRY_INVALID_VER_UPDATED;
+
 /**
  *
  */
@@ -54,6 +58,14 @@ public class CacheEntryImplEx<K, V> extends CacheEntryImpl<K, V> implements Cach
 
     /** {@inheritDoc} */
     public GridCacheVersion version() {
+        if (ver == GET_ENTRY_INVALID_VER_AFTER_GET) {
+            throw new IgniteException("Impossible to get entry version after " +
+                "get() inside OPTIMISTIC REPEATABLE_READ transaction. Use only getEntry() or getEntries() inside " +
+                "OPTIMISTIC REPEATABLE_READ transaction to solve this problem.");
+        }
+        else if (ver == GET_ENTRY_INVALID_VER_UPDATED)
+            throw new IgniteException("Impossible to get version for entry updated in transaction.");
+
         return ver;
     }
 
@@ -81,7 +93,7 @@ public class CacheEntryImplEx<K, V> extends CacheEntryImpl<K, V> implements Cach
         String res = "CacheEntry [key=" + getKey() +
             ", val=" + getValue();
 
-        if (ver != null) {
+        if (ver != null && ver != GET_ENTRY_INVALID_VER_AFTER_GET && ver != GET_ENTRY_INVALID_VER_UPDATED) {
             res += ", topVer=" + ver.topologyVersion() +
                 ", nodeOrder=" + ver.nodeOrder() +
                 ", order=" + ver.order() +

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index c76aaa6..6949d90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -52,6 +52,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CachePeekMode;
@@ -610,7 +611,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             /*task name*/null,
             /*deserialize binary*/false,
             /*skip values*/true,
-            /*can remap*/true
+            /*can remap*/true,
+            false
         );
     }
 
@@ -636,7 +638,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             /*task name*/null,
             /*deserialize binary*/false,
             /*skip values*/true,
-            /*can remap*/true
+            /*can remap*/true,
+            false
         ).chain(new CX1<IgniteInternalFuture<Map<K, V>>, Boolean>() {
             @Override public Boolean applyx(IgniteInternalFuture<Map<K, V>> fut) throws IgniteCheckedException {
                 Map<K, V> kvMap = fut.get();
@@ -1299,7 +1302,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             /*deserialize cache objects*/true,
             /*skip values*/false,
-            /*can remap*/true
+            /*can remap*/true,
+            false
         ).get().get(key);
     }
 
@@ -1315,7 +1319,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             true,
             false,
-            /*can remap*/true
+            /*can remap*/true,
+            false
         ).chain(new CX1<IgniteInternalFuture<Map<K, V>>, V>() {
             @Override public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
                 return e.get().get(key);
@@ -1335,7 +1340,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             /*deserialize cache objects*/true,
             /*skip values*/false,
-            /*can remap*/false
+            /*can remap*/false,
+            false
         ).get().get(key);
     }
 
@@ -1355,7 +1361,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             !ctx.keepBinary(),
             /*skip values*/false,
-            /*can remap*/true);
+            /*can remap*/true,
+            false);
     }
 
     /**
@@ -1375,7 +1382,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        V val = get(key, !ctx.keepBinary());
+        V val = get(key, !ctx.keepBinary(), false);
 
         if (ctx.config().getInterceptor() != null)
             val = (V)ctx.config().getInterceptor().onGet(key, val);
@@ -1387,6 +1394,30 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public CacheEntry<K, V> getEntry(K key) throws IgniteCheckedException {
+        A.notNull(key, "key");
+
+        boolean statsEnabled = ctx.config().isStatisticsEnabled();
+
+        long start = statsEnabled ? System.nanoTime() : 0L;
+
+        T2<V, GridCacheVersion> t = (T2<V, GridCacheVersion>)get(key, !ctx.keepBinary(), true);
+
+        CacheEntry<K, V> val = t != null ? new CacheEntryImplEx<>(key, t.get1(), t.get2()): null;
+
+        if (ctx.config().getInterceptor() != null) {
+            V val0 = (V)ctx.config().getInterceptor().onGet(key, t != null ? val.getValue() : null);
+
+            val = (val0 != null) ? new CacheEntryImplEx<>(key, val0, t != null ? t.get2() : null) : null;
+        }
+
+        if (statsEnabled)
+            metrics0().addGetTimeNanos(System.nanoTime() - start);
+
+        return val;
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<V> getAsync(final K key) {
         A.notNull(key, "key");
 
@@ -1394,7 +1425,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        IgniteInternalFuture<V> fut = getAsync(key, !ctx.keepBinary());
+        IgniteInternalFuture<V> fut = getAsync(key, !ctx.keepBinary(), false);
 
         if (ctx.config().getInterceptor() != null)
             fut =  fut.chain(new CX1<IgniteInternalFuture<V>, V>() {
@@ -1410,6 +1441,42 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<CacheEntry<K, V>> getEntryAsync(final K key) {
+        A.notNull(key, "key");
+
+        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
+
+        final long start = statsEnabled ? System.nanoTime() : 0L;
+
+        IgniteInternalFuture<T2<V, GridCacheVersion>> fut =
+            (IgniteInternalFuture<T2<V, GridCacheVersion>>)getAsync(key, !ctx.keepBinary(), true);
+
+        final boolean intercept = ctx.config().getInterceptor() != null;
+
+        IgniteInternalFuture<CacheEntry<K, V>> fr = fut.chain(
+            new CX1<IgniteInternalFuture<T2<V, GridCacheVersion>>, CacheEntry<K, V>>() {
+            @Override public CacheEntry<K, V> applyx(IgniteInternalFuture<T2<V, GridCacheVersion>> f)
+                throws IgniteCheckedException {
+                T2<V, GridCacheVersion> t = f.get();
+
+                CacheEntry<K, V> val = t != null ? new CacheEntryImplEx<>(key, t.get1(), t.get2()) : null;
+                if (intercept) {
+                    V val0 = (V)ctx.config().getInterceptor().onGet(key, t != null ? val.getValue() : null);
+
+                    return new CacheEntryImplEx<>(key, val0, t != null ? t.get2() : null);
+                }
+                else
+                    return val;
+            }
+        });
+
+        if (statsEnabled)
+            fut.listen(new UpdateGetTimeStatClosure<T2<V, GridCacheVersion>>(metrics0(), start));
+
+        return fr;
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<K, V> getAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
         A.notNull(keys, "keys");
 
@@ -1417,7 +1484,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        Map<K, V> map = getAll(keys, !ctx.keepBinary());
+        Map<K, V> map = getAll(keys, !ctx.keepBinary(), false);
 
         if (ctx.config().getInterceptor() != null)
             map = interceptGet(keys, map);
@@ -1429,6 +1496,32 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<CacheEntry<K, V>> getEntries(@Nullable Collection<? extends K> keys)
+        throws IgniteCheckedException {
+        A.notNull(keys, "keys");
+
+        boolean statsEnabled = ctx.config().isStatisticsEnabled();
+
+        long start = statsEnabled ? System.nanoTime() : 0L;
+
+        Map<K, T2<V, GridCacheVersion>> map = (Map<K, T2<V, GridCacheVersion>>)getAll(keys, !ctx.keepBinary(), true);
+
+        Collection<CacheEntry<K, V>> res = new HashSet<>();
+
+        if (ctx.config().getInterceptor() != null)
+            res = interceptGetEntries(keys, map);
+        else
+            for (Map.Entry<K, T2<V, GridCacheVersion>> e : map.entrySet())
+                res.add(new CacheEntryImplEx<>(e.getKey(), e.getValue().get1(), e.getValue().get2()));
+
+        if (statsEnabled)
+            metrics0().addGetTimeNanos(System.nanoTime() - start);
+
+        return res;
+    }
+
+
+    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable final Collection<? extends K> keys) {
         A.notNull(keys, "keys");
 
@@ -1436,7 +1529,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        IgniteInternalFuture<Map<K, V>> fut = getAllAsync(keys, !ctx.keepBinary());
+        IgniteInternalFuture<Map<K, V>> fut = getAllAsync(keys, !ctx.keepBinary(), false);
 
         if (ctx.config().getInterceptor() != null)
             return fut.chain(new CX1<IgniteInternalFuture<Map<K, V>>, Map<K, V>>() {
@@ -1451,6 +1544,45 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return fut;
     }
 
+    /** {@inheritDoc} */
+    @Override  public IgniteInternalFuture<Collection<CacheEntry<K, V>>> getEntriesAsync(
+        @Nullable final Collection<? extends K> keys) {
+        A.notNull(keys, "keys");
+
+        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
+
+        final long start = statsEnabled ? System.nanoTime() : 0L;
+
+        IgniteInternalFuture<Map<K, T2<V, GridCacheVersion>>> fut =
+            (IgniteInternalFuture<Map<K, T2<V, GridCacheVersion>>>)
+                ((IgniteInternalFuture)getAllAsync(keys, !ctx.keepBinary(), true));
+
+        final boolean intercept = ctx.config().getInterceptor() != null;
+
+        IgniteInternalFuture<Collection<CacheEntry<K, V>>> rf =
+            fut.chain(new CX1<IgniteInternalFuture<Map<K, T2<V, GridCacheVersion>>>, Collection<CacheEntry<K, V>>>() {
+                @Override public Collection<CacheEntry<K, V>> applyx(
+                    IgniteInternalFuture<Map<K, T2<V, GridCacheVersion>>> f) throws IgniteCheckedException {
+                    if (intercept)
+                        return interceptGetEntries(keys, f.get());
+                    else {
+                        Map<K, CacheEntry<K, V>> res = U.newHashMap(f.get().size());
+
+                        for (Map.Entry<K, T2<V, GridCacheVersion>> e : f.get().entrySet())
+                            res.put(e.getKey(),
+                                new CacheEntryImplEx<>(e.getKey(), e.getValue().get1(), e.getValue().get2()));
+
+                        return res.values();
+                    }
+                }
+            });
+
+        if (statsEnabled)
+            fut.listen(new UpdateGetTimeStatClosure<Map<K, T2<V, GridCacheVersion>>>(metrics0(), start));
+
+        return rf;
+    }
+
     /**
      * Applies cache interceptor on result of 'get' operation.
      *
@@ -1493,6 +1625,53 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Applies cache interceptor on result of 'getEntries' operation.
+     *
+     * @param keys All requested keys.
+     * @param map Result map.
+     * @return Map with values returned by cache interceptor..
+     */
+    @SuppressWarnings("IfMayBeConditional")
+    private Collection<CacheEntry<K, V>> interceptGetEntries(
+        @Nullable Collection<? extends K> keys, Map<K, T2<V, GridCacheVersion>> map) {
+        Map<K, CacheEntry<K, V>> res;
+
+        if (F.isEmpty(keys)) {
+            assert map.isEmpty();
+
+            return Collections.emptySet();
+        }
+
+        res = U.newHashMap(keys.size());
+
+        CacheInterceptor<K, V> interceptor = cacheCfg.getInterceptor();
+
+        assert interceptor != null;
+
+        for (Map.Entry<K, T2<V, GridCacheVersion>> e : map.entrySet()) {
+            V val = interceptor.onGet(e.getKey(), e.getValue().get1());
+
+            if (val != null)
+                res.put(e.getKey(), new CacheEntryImplEx<>(e.getKey(), val, e.getValue().get2()));
+        }
+
+        if (map.size() != keys.size()) { // Not all requested keys were in cache.
+            for (K key : keys) {
+                if (key != null) {
+                    if (!map.containsKey(key)) {
+                        V val = interceptor.onGet(key, null);
+
+                        if (val != null)
+                            res.put(key, new CacheEntryImplEx<>(key, val, null));
+                    }
+                }
+            }
+        }
+
+        return res.values();
+    }
+
+    /**
      * @param key Key.
      * @param forcePrimary Force primary.
      * @param skipTx Skip tx.
@@ -1501,6 +1680,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param deserializeBinary Deserialize binary.
      * @param skipVals Skip values.
      * @param canRemap Can remap flag.
+     * @param needVer Need version.
      * @return Future for the get operation.
      */
     protected IgniteInternalFuture<V> getAsync(
@@ -1511,7 +1691,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         String taskName,
         boolean deserializeBinary,
         final boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        final boolean needVer
     ) {
         return getAllAsync(Collections.singletonList(key),
             forcePrimary,
@@ -1520,7 +1701,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             deserializeBinary,
             skipVals,
-            canRemap).chain(
+            canRemap,
+            needVer).chain(
             new CX1<IgniteInternalFuture<Map<K, V>>, V>() {
                 @Override public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
                     Map<K, V> map = e.get();
@@ -1547,6 +1729,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param deserializeBinary Deserialize binary.
      * @param skipVals Skip values.
      * @param canRemap Can remap flag.
+     * @param needVer Need version.
      * @return Future for the get operation.
      * @see GridCacheAdapter#getAllAsync(Collection)
      */
@@ -1558,7 +1741,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         String taskName,
         boolean deserializeBinary,
         boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        final boolean needVer
     ) {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
@@ -1573,7 +1757,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             forcePrimary,
             skipVals ? null : expiryPolicy(opCtx != null ? opCtx.expiry() : null),
             skipVals,
-            canRemap);
+            canRemap,
+            needVer);
     }
 
     /**
@@ -1587,6 +1772,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param expiry Expiry policy.
      * @param skipVals Skip values.
      * @param canRemap Can remap flag.
+     * @param needVer Need version.
      * @return Future for the get operation.
      * @see GridCacheAdapter#getAllAsync(Collection)
      */
@@ -1599,7 +1785,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         final boolean forcePrimary,
         @Nullable IgniteCacheExpiryPolicy expiry,
         final boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        final boolean needVer
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -1616,7 +1803,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             skipVals,
             false,
             canRemap,
-            false);
+            needVer);
     }
 
     /**
@@ -1711,20 +1898,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                                     ctx.evicts().touch(entry, topVer);
                             }
                             else {
-                                if (needVer) {
-                                    assert keepCacheObjects;
-
-                                    map.put((K1)key, (V1)new T2<>(res.get1(), res.get2()));
-                                }
-                                else {
-                                    ctx.addResult(map,
-                                        key,
-                                        res.get1(),
-                                        skipVals,
-                                        keepCacheObjects,
-                                        deserializeBinary,
-                                        true);
-                                }
+                                ctx.addResult(map,
+                                    key,
+                                    res.get1(),
+                                    skipVals,
+                                    keepCacheObjects,
+                                    deserializeBinary,
+                                    true,
+                                    needVer ? res.get2() : null);
 
                                 if (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED))
                                     ctx.evicts().touch(entry, topVer);
@@ -1786,20 +1967,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
                                                 // Don't put key-value pair into result map if value is null.
                                                 if (val != null) {
-                                                    if (needVer) {
-                                                        assert keepCacheObjects;
-
-                                                        map.put((K1)key, (V1)new T2<>(cacheVal, set ? verSet : ver));
-                                                    }
-                                                    else {
-                                                        ctx.addResult(map,
-                                                            key,
-                                                            cacheVal,
-                                                            skipVals,
-                                                            keepCacheObjects,
-                                                            deserializeBinary,
-                                                            false);
-                                                    }
+                                                    ctx.addResult(map,
+                                                        key,
+                                                        cacheVal,
+                                                        skipVals,
+                                                        keepCacheObjects,
+                                                        deserializeBinary,
+                                                        false,
+                                                        needVer ? set ? verSet : ver : null);
                                                 }
 
                                                 if (tx0 == null || (!tx0.implicit() &&
@@ -1892,11 +2067,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             }
         }
         else {
-            assert !needVer;
-
             return asyncOp(tx, new AsyncOp<Map<K1, V1>>(keys) {
                 @Override public IgniteInternalFuture<Map<K1, V1>> op(IgniteTxLocalAdapter tx) {
-                    return tx.getAllAsync(ctx, keys, deserializeBinary, skipVals, false, !readThrough);
+                    return tx.getAllAsync(ctx, keys, deserializeBinary, skipVals, false, !readThrough, needVer);
                 }
             }, ctx.operationContextPerCall());
         }
@@ -4537,28 +4710,31 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * @param key Key.
      * @param deserializeBinary Deserialize binary flag.
+     * @param needVer Need version.
      * @return Cached value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public V get(K key, boolean deserializeBinary) throws IgniteCheckedException {
+    @Nullable public V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException {
         checkJta();
 
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return get(key, taskName, deserializeBinary);
+        return get(key, taskName, deserializeBinary, needVer);
     }
 
     /**
      * @param key Key.
      * @param taskName Task name.
      * @param deserializeBinary Deserialize binary flag.
+     * @param needVer Need version.
      * @return Cached value.
      * @throws IgniteCheckedException If failed.
      */
     protected V get(
         final K key,
         String taskName,
-        boolean deserializeBinary) throws IgniteCheckedException {
+        boolean deserializeBinary,
+        boolean needVer) throws IgniteCheckedException {
         return getAsync(key,
             !ctx.config().isReadFromBackup(),
             /*skip tx*/false,
@@ -4566,15 +4742,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             deserializeBinary,
             false,
-            /*can remap*/true).get();
+            /*can remap*/true,
+            needVer).get();
     }
 
     /**
      * @param key Key.
      * @param deserializeBinary Deserialize binary flag.
+     * @param needVer Need version.
      * @return Read operation future.
      */
-    public final IgniteInternalFuture<V> getAsync(final K key, boolean deserializeBinary) {
+    public final IgniteInternalFuture<V> getAsync(final K key, boolean deserializeBinary, final boolean needVer) {
         try {
             checkJta();
         }
@@ -4591,28 +4769,32 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             deserializeBinary,
             false,
-            /*can remap*/true);
+            /*can remap*/true,
+            needVer);
     }
 
     /**
      * @param keys Keys.
      * @param deserializeBinary Deserialize binary flag.
+     * @param needVer Need version.
      * @return Map of cached values.
      * @throws IgniteCheckedException If read failed.
      */
-    public Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary) throws IgniteCheckedException {
+    public Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary,
+        boolean needVer) throws IgniteCheckedException {
         checkJta();
 
-        return getAllAsync(keys, deserializeBinary).get();
+        return getAllAsync(keys, deserializeBinary, needVer).get();
     }
 
     /**
      * @param keys Keys.
      * @param deserializeBinary Deserialize binary flag.
+     * @param needVer Need version.
      * @return Read future.
      */
     public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys,
-        boolean deserializeBinary) {
+        boolean deserializeBinary, boolean needVer) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(keys,
@@ -4622,7 +4804,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             taskName,
             deserializeBinary,
             /*skip vals*/false,
-            /*can remap*/true);
+            /*can remap*/true,
+            needVer);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index fc48b9d..e875df0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -49,12 +49,12 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentManager;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManager;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -91,6 +91,7 @@ import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.C1;
 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.CU;
 import org.apache.ignite.internal.util.typedef.internal.GPC;
@@ -1882,6 +1883,7 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param keepCacheObjects Keep cache objects flag.
      * @param deserializeBinary Deserialize binary flag.
      * @param cpy Copy flag.
+     * @param ver GridCacheVersion.
      */
     @SuppressWarnings("unchecked")
     public <K1, V1> void addResult(Map<K1, V1> map,
@@ -1890,7 +1892,8 @@ public class GridCacheContext<K, V> implements Externalizable {
         boolean skipVals,
         boolean keepCacheObjects,
         boolean deserializeBinary,
-        boolean cpy) {
+        boolean cpy,
+        final GridCacheVersion ver) {
         assert key != null;
         assert val != null || skipVals;
 
@@ -1902,10 +1905,32 @@ public class GridCacheContext<K, V> implements Externalizable {
             assert key0 != null : key;
             assert val0 != null : val;
 
-            map.put((K1)key0, (V1)val0);
+            map.put((K1)key0, ver != null ? (V1)new T2<>(val0, ver) : (V1)val0);
         }
         else
-            map.put((K1)key, (V1)(skipVals ? true : val));
+            map.put((K1)key,
+                (V1)(ver != null ?
+                    (V1)new T2<>(skipVals ? true : val, ver) :
+                    skipVals ? true : val));
+    }
+
+    /**
+     * @param map Map.
+     * @param key Key.
+     * @param val Value.
+     * @param skipVals Skip values flag.
+     * @param keepCacheObjects Keep cache objects flag.
+     * @param deserializeBinary Deserialize binary flag.
+     * @param cpy Copy flag.
+     */
+    public <K1, V1> void addResult(Map<K1, V1> map,
+        KeyCacheObject key,
+        CacheObject val,
+        boolean skipVals,
+        boolean keepCacheObjects,
+        boolean deserializeBinary,
+        boolean cpy) {
+        addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, cpy, null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/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 2d25d16..64cfd01 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
@@ -882,7 +882,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 updateTtl(expiryPlc);
 
             if (retVer) {
-                resVer = isNear() ? ((GridNearCacheEntry)this).dhtVersion() : this.ver;
+                resVer = (isNear() && cctx.transactional()) ? ((GridNearCacheEntry)this).dhtVersion() : this.ver;
 
                 if (resVer == null)
                     ret = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index 3a53942..9b4aff3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -31,6 +31,7 @@ import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.affinity.Affinity;
@@ -307,6 +308,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public CacheEntry<K, V> getEntry(K key) throws IgniteCheckedException {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.getEntry(key);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public V getTopologySafe(K key) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -331,6 +344,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<CacheEntry<K, V>> getEntryAsync(K key) {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.getEntryAsync(key);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public V getForcePrimary(K key) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -451,6 +476,19 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<CacheEntry<K, V>> getEntries(
+        @Nullable Collection<? extends K> keys) throws IgniteCheckedException {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.getEntries(keys);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys) {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -463,6 +501,19 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Collection<CacheEntry<K, V>>> getEntriesAsync(
+        @Nullable Collection<? extends K> keys) {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.getEntriesAsync(keys);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public V getAndPut(K key, V val)
         throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 82024ea..d111a62 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -44,6 +44,7 @@ import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheManager;
 import org.apache.ignite.cache.CacheMetrics;
@@ -873,6 +874,31 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public CacheEntry<K, V> getEntry(K key) {
+        try {
+            GridCacheGateway<K, V> gate = this.gate;
+
+            CacheOperationContext prev = onEnter(gate, opCtx);
+
+            try {
+                if (isAsync()) {
+                    setFuture(delegate.getEntryAsync(key));
+
+                    return null;
+                }
+                else
+                    return delegate.getEntry(key);
+            }
+            finally {
+                onLeave(gate, prev);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<K, V> getAll(Set<? extends K> keys) {
         try {
             GridCacheGateway<K, V> gate = this.gate;
@@ -898,6 +924,31 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<CacheEntry<K, V>>  getEntries(Set<? extends K> keys) {
+        try {
+            GridCacheGateway<K, V> gate = this.gate;
+
+            CacheOperationContext prev = onEnter(gate, opCtx);
+
+            try {
+                if (isAsync()) {
+                    setFuture(delegate.getEntriesAsync(keys));
+
+                    return null;
+                }
+                else
+                    return delegate.getEntries(keys);
+            }
+            finally {
+                onLeave(gate, prev);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) {
         try {
             GridCacheGateway<K, V> gate = this.gate;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index 433290c..68d0f06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -31,6 +31,7 @@ import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
@@ -335,6 +336,28 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     @Nullable public V get(K key) throws IgniteCheckedException;
 
     /**
+     * Retrieves value mapped to the specified key from cache. Value will only be returned if
+     * its entry passed the optional filter provided. Filter check is atomic, and therefore the
+     * returned value is guaranteed to be consistent with the filter. The return value of {@code null}
+     * means entry did not pass the provided filter or cache has no mapping for the
+     * key.
+     * <p>
+     * If the value is not present in cache, then it will be looked up from swap storage. If
+     * it's not present in swap, or if swap is disable, and if read-through is allowed, value
+     * will be loaded from {@link CacheStore} persistent storage via
+     * <code>CacheStore#load(Transaction, Object)</code> method.
+     * <h2 class="header">Transactions</h2>
+     * This method is transactional and will enlist the entry into ongoing transaction
+     * if there is one.
+     *
+     * @param key Key to retrieve the value for.
+     * @return Value for the given key.
+     * @throws IgniteCheckedException If get operation failed.
+     * @throws NullPointerException if the key is {@code null}.
+     */
+    @Nullable public CacheEntry<K, V> getEntry(K key) throws IgniteCheckedException;
+
+    /**
      * Asynchronously retrieves value mapped to the specified key from cache. Value will only be returned if
      * its entry passed the optional filter provided. Filter check is atomic, and therefore the
      * returned value is guaranteed to be consistent with the filter. The return value of {@code null}
@@ -356,6 +379,27 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public IgniteInternalFuture<V> getAsync(K key);
 
     /**
+     * Asynchronously retrieves value mapped to the specified key from cache. Value will only be returned if
+     * its entry passed the optional filter provided. Filter check is atomic, and therefore the
+     * returned value is guaranteed to be consistent with the filter. The return value of {@code null}
+     * means entry did not pass the provided filter or cache has no mapping for the
+     * key.
+     * <p>
+     * If the value is not present in cache, then it will be looked up from swap storage. If
+     * it's not present in swap, or if swap is disabled, and if read-through is allowed, value
+     * will be loaded from {@link CacheStore} persistent storage via
+     * <code>CacheStore#load(Transaction, Object)</code> method.
+     * <h2 class="header">Transactions</h2>
+     * This method is transactional and will enlist the entry into ongoing transaction
+     * if there is one.
+     *
+     * @param key Key for the value to get.
+     * @return Future for the get operation.
+     * @throws NullPointerException if the key is {@code null}.
+     */
+    public IgniteInternalFuture<CacheEntry<K, V>> getEntryAsync(K key);
+
+    /**
      * Retrieves values mapped to the specified keys from cache. Value will only be returned if
      * its entry passed the optional filter provided. Filter check is atomic, and therefore the
      * returned value is guaranteed to be consistent with the filter. If requested key-value pair
@@ -377,6 +421,27 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public Map<K, V> getAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException;
 
     /**
+     * Retrieves values mapped to the specified keys from cache. Value will only be returned if
+     * its entry passed the optional filter provided. Filter check is atomic, and therefore the
+     * returned value is guaranteed to be consistent with the filter. If requested key-value pair
+     * is not present in the returned map, then it means that its entry did not pass the provided
+     * filter or cache has no mapping for the key.
+     * <p>
+     * If some value is not present in cache, then it will be looked up from swap storage. If
+     * it's not present in swap, or if swap is disabled, and if read-through is allowed, value
+     * will be loaded from {@link CacheStore} persistent storage via
+     * <code>CacheStore#loadAll(Transaction, Collection, org.apache.ignite.lang.IgniteBiInClosure)</code> method.
+     * <h2 class="header">Transactions</h2>
+     * This method is transactional and will enlist the entry into ongoing transaction
+     * if there is one.
+     *
+     * @param keys Keys to get.
+     * @return Map of key-value pairs.
+     * @throws IgniteCheckedException If get operation failed.
+     */
+    public Collection<CacheEntry<K, V>> getEntries(@Nullable Collection<? extends K> keys) throws IgniteCheckedException;
+
+    /**
      * Asynchronously retrieves values mapped to the specified keys from cache. Value will only be returned if
      * its entry passed the optional filter provided. Filter check is atomic, and therefore the
      * returned value is guaranteed to be consistent with the filter. If requested key-value pair
@@ -397,6 +462,26 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys);
 
     /**
+     * Asynchronously retrieves values mapped to the specified keys from cache. Value will only be returned if
+     * its entry passed the optional filter provided. Filter check is atomic, and therefore the
+     * returned value is guaranteed to be consistent with the filter. If requested key-value pair
+     * is not present in the returned map, then it means that its entry did not pass the provided
+     * filter or cache has no mapping for the key.
+     * <p>
+     * If some value is not present in cache, then it will be looked up from swap storage. If
+     * it's not present in swap, or if swap is disabled, and if read-through is allowed, value
+     * will be loaded from {@link CacheStore} persistent storage via
+     * <code>CacheStore#loadAll(Transaction, Collection, org.apache.ignite.lang.IgniteBiInClosure)</code> method.
+     * <h2 class="header">Transactions</h2>
+     * This method is transactional and will enlist the entry into ongoing transaction
+     * if there is one.
+     *
+     * @param keys Key for the value to get.
+     * @return Future for the get operation.
+     */
+    public IgniteInternalFuture<Collection<CacheEntry<K, V>>> getEntriesAsync(@Nullable Collection<? extends K> keys);
+
+    /**
      * Stores given key-value pair in cache. If filters are provided, then entries will
      * be stored in cache only if they pass the filter. Note that filter check is atomic,
      * so value stored in cache is guaranteed to be consistent with the filters. If cache

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java
index 5dc5e98..4b80387 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/CacheDistributedGetFutureAdapter.java
@@ -147,21 +147,6 @@ public abstract class CacheDistributedGetFutureAdapter<K, V> extends GridCompoun
     }
 
     /**
-     * @param map Result map.
-     * @param key Key.
-     * @param val Value.
-     * @param ver Version.
-     */
-    @SuppressWarnings("unchecked")
-    protected final void versionedResult(Map map, KeyCacheObject key, Object val, GridCacheVersion ver) {
-        assert needVer;
-        assert skipVals || val != null;
-        assert ver != null;
-
-        map.put(key, new T2<>(skipVals ? true : val, ver));
-    }
-
-    /**
      * Affinity node to send get request to.
      *
      * @param affNodes All affinity nodes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 9cf8084..5be4e72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -617,6 +617,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
      * @param keys {@inheritDoc}
      * @param forcePrimary {@inheritDoc}
      * @param skipTx {@inheritDoc}
+     * @param needVer Need version.
      * @return {@inheritDoc}
      */
     @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
@@ -627,7 +628,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
         String taskName,
         boolean deserializeBinary,
         boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        boolean needVer
     ) {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
@@ -640,7 +642,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
             forcePrimary,
             null,
             skipVals,
-            canRemap);
+            canRemap,
+            needVer);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index e410228..e86c885 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -383,7 +383,8 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                     /*deserialize binary*/false,
                     skipVals,
                     /*keep cache objects*/true,
-                    /*skip store*/!readThrough);
+                    /*skip store*/!readThrough,
+                    false);
             }
         }
         else {
@@ -413,7 +414,8 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                                 /*deserialize binary*/false,
                                 skipVals,
                                 /*keep cache objects*/true,
-                                /*skip store*/!readThrough);
+                                /*skip store*/!readThrough,
+                                false);
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 40399b4..b8995bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -929,7 +929,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         if (retVal ||
             !F.isEmpty(e.entryProcessors()) ||
             !F.isEmpty(e.filters()) ||
-            e.serializableReadVersion() != null) {
+            e.entryReadVersion() != null) {
             if (map == null)
                 map = new HashMap<>();
 
@@ -996,7 +996,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         throws IgniteCheckedException {
         try {
             for (IgniteTxEntry entry : entries) {
-                GridCacheVersion serReadVer = entry.serializableReadVersion();
+                GridCacheVersion serReadVer = entry.entryReadVersion();
 
                 if (serReadVer != null) {
                     entry.cached().unswap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 30b19e7..037d408 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -490,17 +490,14 @@ public class GridPartitionedGetFuture<K, V> extends CacheDistributedGetFutureAda
                             cache.removeIfObsolete(key);
                     }
                     else {
-                        if (needVer)
-                            versionedResult(locVals, key, v, ver);
-                        else {
-                            cctx.addResult(locVals,
-                                key,
-                                v,
-                                skipVals,
-                                keepCacheObjects,
-                                deserializeBinary,
-                                true);
-                        }
+                        cctx.addResult(locVals,
+                            key,
+                            v,
+                            skipVals,
+                            keepCacheObjects,
+                            deserializeBinary,
+                            true,
+                            ver);
 
                         return true;
                     }
@@ -552,17 +549,14 @@ public class GridPartitionedGetFuture<K, V> extends CacheDistributedGetFutureAda
             for (GridCacheEntryInfo info : infos) {
                 assert skipVals == (info.value() == null);
 
-                if (needVer)
-                    versionedResult(map, info.key(), info.value(), info.version());
-                else {
-                    cctx.addResult(map,
-                        info.key(),
-                        info.value(),
-                        skipVals,
-                        keepCacheObjects,
-                        deserializeBinary,
-                        false);
-                }
+                cctx.addResult(map,
+                    info.key(),
+                    info.value(),
+                    skipVals,
+                    keepCacheObjects,
+                    deserializeBinary,
+                    false,
+                    needVer ? info.version() : null);
             }
 
             return map;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index 0c811ae..01e61bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -625,20 +625,13 @@ public class GridPartitionedSingleGetFuture extends GridFutureAdapter<Object> im
             assert !skipVals;
 
             if (val != null) {
-                if (needVer) {
-                    assert ver != null;
+                if (!keepCacheObjects) {
+                    Object res = cctx.unwrapBinaryIfNeeded(val, !deserializeBinary);
 
-                    onDone(new T2<>(val, ver));
-                }
-                else {
-                    if (!keepCacheObjects) {
-                        Object res = cctx.unwrapBinaryIfNeeded(val, !deserializeBinary);
-
-                        onDone(res);
-                    }
-                    else
-                        onDone(val);
+                    onDone(needVer ? new T2<>(res, ver) : res);
                 }
+                else
+                    onDone(needVer ? new T2<>(val, ver) : val);
             }
             else
                 onDone(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 6b23550..d413c18 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -93,6 +93,7 @@ import org.apache.ignite.internal.util.typedef.CO;
 import org.apache.ignite.internal.util.typedef.CX1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -317,7 +318,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected V get(K key, String taskName, boolean deserializeBinary) throws IgniteCheckedException {
+    @Override protected V get(K key, String taskName, boolean deserializeBinary, boolean needVer)
+        throws IgniteCheckedException {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (keyCheck)
@@ -339,7 +341,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             expiryPlc,
             false,
             skipStore,
-            true).get();
+            true,
+            needVer).get();
     }
 
     /** {@inheritDoc} */
@@ -350,7 +353,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final String taskName,
         final boolean deserializeBinary,
         final boolean skipVals,
-        final boolean canRemap) {
+        final boolean canRemap,
+        final boolean needVer) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (keyCheck)
@@ -376,7 +380,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     expiryPlc,
                     skipVals,
                     skipStore,
-                    canRemap);
+                    canRemap,
+                    needVer);
             }
         });
     }
@@ -390,7 +395,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final String taskName,
         final boolean deserializeBinary,
         final boolean skipVals,
-        final boolean canRemap
+        final boolean canRemap,
+        final boolean needVer
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -420,7 +426,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     expiryPlc,
                     skipVals,
                     skipStore,
-                    canRemap);
+                    canRemap,
+                    needVer);
             }
         });
     }
@@ -1098,6 +1105,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param skipVals Skip values flag.
      * @param skipStore Skip store flag.
      * @param canRemap Can remap flag.
+     * @param needVer Need version.
      * @return Get future.
      */
     private IgniteInternalFuture<V> getAsync0(KeyCacheObject key,
@@ -1108,7 +1116,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable ExpiryPolicy expiryPlc,
         boolean skipVals,
         boolean skipStore,
-        boolean canRemap
+        boolean canRemap,
+        boolean needVer
     ) {
         AffinityTopologyVersion topVer = canRemap ? ctx.affinity().affinityTopologyVersion() :
             ctx.shared().exchange().readyAffinityVersion();
@@ -1126,7 +1135,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             expiry,
             skipVals,
             canRemap,
-            false,
+            needVer,
             false);
 
         fut.init();
@@ -1145,6 +1154,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param expiryPlc Expiry policy.
      * @param skipVals Skip values flag.
      * @param skipStore Skip store flag.
+     * @param needVer Need version.
      * @return Get future.
      */
     private IgniteInternalFuture<Map<K, V>> getAllAsync0(@Nullable Collection<KeyCacheObject> keys,
@@ -1155,7 +1165,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable ExpiryPolicy expiryPlc,
         boolean skipVals,
         boolean skipStore,
-        boolean canRemap
+        boolean canRemap,
+        boolean needVer
     ) {
         AffinityTopologyVersion topVer = canRemap ? ctx.affinity().affinityTopologyVersion() :
             ctx.shared().exchange().readyAffinityVersion();
@@ -1180,19 +1191,42 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         if (entry != null) {
                             boolean isNew = entry.isNewLocked();
 
-                            CacheObject v = entry.innerGet(null,
-                                /*swap*/true,
-                                /*read-through*/false,
-                                /*fail-fast*/true,
-                                /*unmarshal*/true,
-                                /**update-metrics*/false,
-                                /*event*/!skipVals,
-                                /*temporary*/false,
-                                subjId,
-                                null,
-                                taskName,
-                                expiry,
-                                !deserializeBinary);
+                            CacheObject v = null;
+                            GridCacheVersion ver = null;
+
+                            if (needVer) {
+                                T2<CacheObject, GridCacheVersion> res = entry.innerGetVersioned(
+                                    null,
+                                    /*swap*/true,
+                                    /*unmarshal*/true,
+                                    /**update-metrics*/false,
+                                    /*event*/!skipVals,
+                                    subjId,
+                                    null,
+                                    taskName,
+                                    expiry,
+                                    true);
+
+                                if (res != null) {
+                                    v = res.get1();
+                                    ver = res.get2();
+                                }
+                            }
+                            else {
+                                v = entry.innerGet(null,
+                                    /*swap*/true,
+                                    /*read-through*/false,
+                                    /*fail-fast*/true,
+                                    /*unmarshal*/true,
+                                    /**update-metrics*/false,
+                                    /*event*/!skipVals,
+                                    /*temporary*/false,
+                                    subjId,
+                                    null,
+                                    taskName,
+                                    expiry,
+                                    !deserializeBinary);
+                            }
 
                             // Entry was not in memory or in swap, so we remove it from cache.
                             if (v == null) {
@@ -1204,7 +1238,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 success = false;
                             }
                             else
-                                ctx.addResult(locVals, key, v, skipVals, false, deserializeBinary, true);
+                                ctx.addResult(locVals, key, v, skipVals, false, deserializeBinary, true, ver);
                         }
                         else
                             success = false;
@@ -1256,7 +1290,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             expiry,
             skipVals,
             canRemap,
-            false,
+            needVer,
             false);
 
         fut.init();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index 073043d..dc4b6bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -200,7 +200,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         String taskName,
         final boolean deserializeBinary,
         final boolean skipVals,
-        boolean canRemap) {
+        boolean canRemap,
+        final boolean needVer) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (keyCheck)
@@ -218,7 +219,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                         deserializeBinary,
                         skipVals,
                         false,
-                        opCtx != null && opCtx.skipStore());
+                        opCtx != null && opCtx.skipStore(),
+                        needVer);
 
                     return fut.chain(new CX1<IgniteInternalFuture<Map<Object, Object>>, V>() {
                         @SuppressWarnings("unchecked")
@@ -258,7 +260,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             skipVals ? null : expiryPolicy(opCtx != null ? opCtx.expiry() : null),
             skipVals,
             canRemap,
-            /*needVer*/false,
+            needVer,
             /*keepCacheObjects*/false);
 
         fut.init();
@@ -275,7 +277,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         String taskName,
         final boolean deserializeBinary,
         final boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        final boolean needVer
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -297,7 +300,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                         deserializeBinary,
                         skipVals,
                         false,
-                        opCtx != null && opCtx.skipStore());
+                        opCtx != null && opCtx.skipStore(),
+                        needVer);
                 }
             }, opCtx);
         }
@@ -318,7 +322,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             deserializeBinary,
             skipVals ? null : expiryPolicy(opCtx != null ? opCtx.expiry() : null),
             skipVals,
-            canRemap);
+            canRemap,
+            needVer);
     }
 
     /** {@inheritDoc} */
@@ -345,6 +350,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      * @param expiryPlc Expiry policy.
      * @param skipVals Skip values flag.
      * @param canRemap Can remap flag.
+     * @param needVer Need version.
      * @return Loaded values.
      */
     public IgniteInternalFuture<Map<K, V>> loadAsync(
@@ -357,7 +363,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         boolean deserializeBinary,
         @Nullable IgniteCacheExpiryPolicy expiryPlc,
         boolean skipVals,
-        boolean canRemap) {
+        boolean canRemap,
+        boolean needVer) {
         return loadAsync(keys,
             readThrough,
             forcePrimary,
@@ -367,7 +374,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             expiryPlc,
             skipVals,
             canRemap,
-            false,
+            needVer,
             false);
     }
 
@@ -522,17 +529,14 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                                 if (locVals == null)
                                     locVals = U.newHashMap(keys.size());
 
-                                if (needVer)
-                                    locVals.put((K)key, (V)new T2<>((Object)(skipVals ? true : v), ver));
-                                else {
-                                    ctx.addResult(locVals,
-                                        key,
-                                        v,
-                                        skipVals,
-                                        keepCacheObj,
-                                        deserializeBinary,
-                                        true);
-                                }
+                                ctx.addResult(locVals,
+                                    key,
+                                    v,
+                                    skipVals,
+                                    keepCacheObj,
+                                    deserializeBinary,
+                                    true,
+                                    ver);
                             }
                         }
                         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index a2d5adb..63c073d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -400,7 +400,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
         String taskName,
         boolean deserializeBinary,
         boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        boolean needVer
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -423,7 +424,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
             skipVals ? null : opCtx != null ? opCtx.expiry() : null,
             skipVals,
             opCtx != null && opCtx.skipStore(),
-            canRemap);
+            canRemap,
+            needVer);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 5bf18d9..c750be6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -230,6 +230,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
      * @param skipVal Skip value flag.
      * @param skipStore Skip store flag.
      * @param canRemap Can remap flag.
+     * @param needVer Need version.
      * @return Loaded values.
      */
     public IgniteInternalFuture<Map<K, V>> loadAsync(@Nullable IgniteInternalTx tx,
@@ -241,7 +242,8 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
         @Nullable ExpiryPolicy expiryPlc,
         boolean skipVal,
         boolean skipStore,
-        boolean canRemap
+        boolean canRemap,
+        boolean needVer
     ) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
@@ -261,7 +263,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
             expiry,
             skipVal,
             canRemap,
-            false,
+            needVer,
             false);
 
         // init() will register future for responses if future has remote mappings.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index c0a1617..026fb4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -350,7 +350,8 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
             null,
             false,
             /*skip store*/false,
-            /*can remap*/true
+            /*can remap*/true,
+            false
         ).get().get(keyValue(false));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index dd6e8d5..e65a44a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -650,26 +650,25 @@ public final class GridNearGetFuture<K, V> extends CacheDistributedGetFutureAdap
      */
     @SuppressWarnings("unchecked")
     private void addResult(KeyCacheObject key, CacheObject v, GridCacheVersion ver) {
-        if (needVer) {
-            V val0 = (V)new T2<>(skipVals ? true : v, ver);
+        if (keepCacheObjects) {
+            K key0 = (K)key;
+            V val0 = needVer ?
+                (V)new T2<>(skipVals ? true : v, ver) :
+                (V)(skipVals ? true : v);
 
-            add(new GridFinishedFuture<>(Collections.singletonMap((K)key, val0)));
+            add(new GridFinishedFuture<>(Collections.singletonMap(key0, val0)));
         }
         else {
-            if (keepCacheObjects) {
-                K key0 = (K)key;
-                V val0 = (V)(skipVals ? true : v);
-
-                add(new GridFinishedFuture<>(Collections.singletonMap(key0, val0)));
-            }
-            else {
-                K key0 = (K)cctx.unwrapBinaryIfNeeded(key, !deserializeBinary, false);
-                V val0 = !skipVals ?
+            K key0 = (K)cctx.unwrapBinaryIfNeeded(key, !deserializeBinary, false);
+            V val0 = needVer ?
+                (V)new T2<>(!skipVals ?
+                    (V)cctx.unwrapBinaryIfNeeded(v, !deserializeBinary, false) :
+                    (V)Boolean.TRUE, ver) :
+                !skipVals ?
                     (V)cctx.unwrapBinaryIfNeeded(v, !deserializeBinary, false) :
                     (V)Boolean.TRUE;
 
-                add(new GridFinishedFuture<>(Collections.singletonMap(key0, val0)));
-            }
+            add(new GridFinishedFuture<>(Collections.singletonMap(key0, val0)));
         }
     }
 
@@ -741,16 +740,14 @@ public final class GridNearGetFuture<K, V> extends CacheDistributedGetFutureAdap
 
                     assert skipVals == (info.value() == null);
 
-                    if (needVer)
-                        versionedResult(map, key, val, info.version());
-                    else
-                        cctx.addResult(map,
-                            key,
-                            val,
-                            skipVals,
-                            keepCacheObjects,
-                            deserializeBinary,
-                            false);
+                    cctx.addResult(map,
+                        key,
+                        val,
+                        skipVals,
+                        keepCacheObjects,
+                        deserializeBinary,
+                        false,
+                        needVer ? info.version() : null);
                 }
                 catch (GridCacheEntryRemovedException ignore) {
                     if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
index 37dc564..2050c58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
@@ -104,7 +104,7 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
 
             if (txEntry != null) {
                 if (entry.context().isLocal()) {
-                    GridCacheVersion serReadVer = txEntry.serializableReadVersion();
+                    GridCacheVersion serReadVer = txEntry.entryReadVersion();
 
                     if (serReadVer != null) {
                         GridCacheContext ctx = entry.context();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index a9f158a..79ffffa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -278,6 +278,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
      * @param topLocked {@code True} if thread already acquired lock preventing topology change.
      */
     private void prepareSingle(IgniteTxEntry write, boolean topLocked) {
+        write.clearEntryReadVersion();
+
         AffinityTopologyVersion topVer = tx.topologyVersion();
 
         assert topVer.topologyVersion() > 0;
@@ -338,6 +340,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         Queue<GridDistributedTxMapping> mappings = new ArrayDeque<>();
 
         for (IgniteTxEntry write : writes) {
+            write.clearEntryReadVersion();
+
             GridDistributedTxMapping updated = map(write, topVer, cur, topLocked);
 
             if (cur != updated) {


[2/6] ignite git commit: 2224 (cherry picked from commit 0113506)

Posted by av...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index ffe5373..104af94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -174,6 +174,8 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
         txMapping = new GridDhtTxMapping();
 
         for (IgniteTxEntry txEntry : tx.allEntries()) {
+            txEntry.clearEntryReadVersion();
+
             GridCacheContext cacheCtx = txEntry.context();
 
             List<ClusterNode> nodes = cacheCtx.affinity().nodes(txEntry.key(), topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index a09dec0..a3130cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -122,7 +122,8 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
         String taskName,
         final boolean deserializeBinary,
         final boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        final boolean needVer
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -146,7 +147,8 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
                         deserializeBinary,
                         skipVals,
                         false,
-                        skipStore);
+                        skipStore,
+                        needVer);
                 }
             }, opCtx);
         }
@@ -162,7 +164,8 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
             skipVals ? null : opCtx != null ? opCtx.expiry() : null,
             skipVals,
             skipStore,
-            canRemap);
+            canRemap,
+            needVer);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 6130ead..fed3e33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -477,7 +477,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
 
     @SuppressWarnings("unchecked")
-    @Override @Nullable public V get(K key, boolean deserializeBinary) throws IgniteCheckedException {
+    @Override @Nullable public V get(K key, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         Map<K, V> m = getAllInternal(Collections.singleton(key),
@@ -485,7 +485,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             ctx.readThrough(),
             taskName,
             deserializeBinary,
-            false);
+            false,
+            needVer);
 
         assert m.isEmpty() || m.size() == 1 : m.size();
 
@@ -494,7 +495,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public final Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary)
+    @Override public final Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary, boolean needVer)
         throws IgniteCheckedException {
         A.notNull(keys, "keys");
 
@@ -505,7 +506,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             ctx.readThrough(),
             taskName,
             deserializeBinary,
-            false);
+            false,
+            needVer);
     }
 
 
@@ -519,7 +521,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         final String taskName,
         final boolean deserializeBinary,
         final boolean skipVals,
-        boolean canRemap
+        boolean canRemap,
+        final boolean needVer
     ) {
         A.notNull(keys, "keys");
 
@@ -528,7 +531,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
         return asyncOp(new Callable<Map<K, V>>() {
             @Override public Map<K, V> call() throws Exception {
-                return getAllInternal(keys, swapOrOffheap, storeEnabled, taskName, deserializeBinary, skipVals);
+                return getAllInternal(keys, swapOrOffheap, storeEnabled, taskName, deserializeBinary, skipVals, needVer);
             }
         });
     }
@@ -542,6 +545,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @param taskName Task name.
      * @param deserializeBinary Deserialize binary .
      * @param skipVals Skip value flag.
+     * @param needVer Need version.
      * @return Key-value map.
      * @throws IgniteCheckedException If failed.
      */
@@ -551,7 +555,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         boolean storeEnabled,
         String taskName,
         boolean deserializeBinary,
-        boolean skipVals
+        boolean skipVals,
+        boolean needVer
     ) throws IgniteCheckedException {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -584,24 +589,65 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                     entry = swapOrOffheap ? entryEx(cacheKey) : peekEx(cacheKey);
 
                     if (entry != null) {
-                        CacheObject v = entry.innerGet(null,
-                            /*swap*/swapOrOffheap,
-                            /*read-through*/false,
-                            /*fail-fast*/false,
-                            /*unmarshal*/true,
-                            /**update-metrics*/true,
-                            /**event*/!skipVals,
-                            /**temporary*/false,
-                            subjId,
-                            null,
-                            taskName,
-                            expiry,
-                            !deserializeBinary);
+                        CacheObject v ;
+                        GridCacheVersion ver;
 
-                        if (v != null)
-                            ctx.addResult(vals, cacheKey, v, skipVals, false, deserializeBinary, true);
-                        else
-                            success = false;
+                        if (needVer) {
+                            T2<CacheObject, GridCacheVersion> res = entry.innerGetVersioned(
+                                null,
+                                /*swap*/swapOrOffheap,
+                                /*unmarshal*/true,
+                                /**update-metrics*/false,
+                                /*event*/!skipVals,
+                                subjId,
+                                null,
+                                taskName,
+                                expiry,
+                                !deserializeBinary);
+
+                            if (res != null) {
+                                v = res.get1();
+                                ver = res.get2();
+
+                                ctx.addResult(
+                                    vals,
+                                    cacheKey,
+                                    v,
+                                    skipVals,
+                                    false,
+                                    deserializeBinary,
+                                    true,
+                                    ver);
+                            }else
+                                success = false;
+                        }
+                        else {
+                            v = entry.innerGet(null,
+                                /*swap*/swapOrOffheap,
+                                /*read-through*/false,
+                                /*fail-fast*/false,
+                                /*unmarshal*/true,
+                                /**update-metrics*/true,
+                                /**event*/!skipVals,
+                                /**temporary*/false,
+                                subjId,
+                                null,
+                                taskName,
+                                expiry,
+                                !deserializeBinary);
+
+                            if (v != null) {
+                                ctx.addResult(vals,
+                                    cacheKey,
+                                    v,
+                                    skipVals,
+                                    false,
+                                    deserializeBinary,
+                                    true);
+                            }
+                            else
+                                success = false;
+                        }
                     }
                     else {
                         if (!storeEnabled && configuration().isStatisticsEnabled() && !skipVals)
@@ -638,7 +684,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             /*force primary*/false,
             expiry,
             skipVals,
-            /*can remap*/true).get();
+            /*can remap*/true,
+            needVer).get();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 2c6c3df..3282323 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
@@ -73,6 +74,16 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     /** Dummy version for any existing entry read in SERIALIZABLE transaction. */
     public static final GridCacheVersion SER_READ_NOT_EMPTY_VER = new GridCacheVersion(0, 0, 0, 1);
 
+    /** */
+    public static final GridCacheVersion GET_ENTRY_INVALID_VER_UPDATED = new GridCacheVersion(0, 0, 0, 2);
+
+    /** */
+    public static final GridCacheVersion GET_ENTRY_INVALID_VER_AFTER_GET = new GridCacheVersion(0, 0, 0, 3);
+
+    /** Prepared flag updater. */
+    private static final AtomicIntegerFieldUpdater<IgniteTxEntry> PREPARED_UPD =
+        AtomicIntegerFieldUpdater.newUpdater(IgniteTxEntry.class, "prepared");
+
     /** Owning transaction. */
     @GridToStringExclude
     @GridDirectTransient
@@ -909,13 +920,30 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     }
 
     /**
-     * @param serReadVer Read version for serializable transaction.
+     * Gets stored entry version. Version is stored for all entries in serializable transaction or
+     * when value is read using {@link IgniteCache#getEntry(Object)} method.
+     *
+     * @return Entry version.
      */
-    public void serializableReadVersion(GridCacheVersion serReadVer) {
+    @Nullable public GridCacheVersion entryReadVersion() {
+        return serReadVer;
+    }
+
+    /**
+     * @param ver Entry version.
+     */
+    public void entryReadVersion(GridCacheVersion  ver) {
         assert this.serReadVer == null;
-        assert serReadVer != null;
+        assert ver != null;
 
-        this.serReadVer = serReadVer;
+        this.serReadVer = ver;
+    }
+
+    /**
+     * Clears recorded read version, should be done before starting commit of not serializable/optimistic transaction.
+     */
+    public void clearEntryReadVersion() {
+        serReadVer = null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/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 bf80e0b..10419b5 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
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheUpdateTxResult;
 import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtDetachedCacheEntry;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
 import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -1334,7 +1335,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
     /**
      * Checks if there is a cached or swapped value for
-     * {@link #getAllAsync(GridCacheContext, Collection, boolean, boolean, boolean, boolean)} method.
+     * {@link #getAllAsync(GridCacheContext, Collection, boolean, boolean, boolean, boolean, boolean)} method.
      *
      * @param cacheCtx Cache context.
      * @param keys Key to enlist.
@@ -1360,7 +1361,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         boolean deserializeBinary,
         boolean skipVals,
         boolean keepCacheObjects,
-        boolean skipStore
+        boolean skipStore,
+        final boolean needVer
     ) throws IgniteCheckedException {
         assert !F.isEmpty(keys);
         assert keysCnt == keys.size();
@@ -1373,7 +1375,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
         AffinityTopologyVersion topVer = topologyVersion();
 
-        boolean needReadVer = serializable() && optimistic();
+        boolean needReadVer = (serializable() && optimistic()) || needVer;
 
         // In this loop we cover only read-committed or optimistic transactions.
         // Transactions that are pessimistic and not read-committed are covered
@@ -1395,31 +1397,89 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                     if (!F.isEmpty(txEntry.entryProcessors()))
                         val = txEntry.applyEntryProcessors(val);
 
-                    if (val != null)
-                        cacheCtx.addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, false);
+                    if (val != null) {
+                        GridCacheVersion ver = null;
+
+                        if (needVer) {
+                            if (txEntry.op() != READ)
+                                ver = IgniteTxEntry.GET_ENTRY_INVALID_VER_UPDATED;
+                            else {
+                                ver = txEntry.entryReadVersion();
+
+                                if (ver == null && pessimistic()) {
+                                    while (true) {
+                                        try {
+                                            GridCacheEntryEx cached = txEntry.cached();
+
+                                            ver = cached.isNear() ?
+                                                ((GridNearCacheEntry)cached).dhtVersion() : cached.version();
+
+                                            break;
+                                        }
+                                        catch (GridCacheEntryRemovedException rmvdErr) {
+                                            txEntry.cached(entryEx(cacheCtx, txEntry.txKey(), topVer));
+                                        }
+                                    }
+                                }
+
+                                if (ver == null) {
+                                    assert optimistic() && repeatableRead() : this;
+
+                                    ver = IgniteTxEntry.GET_ENTRY_INVALID_VER_AFTER_GET;
+                                }
+                            }
+
+                            assert ver != null;
+                        }
+
+                        cacheCtx.addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, false, ver);
+                    }
                 }
                 else {
                     assert txEntry.op() == TRANSFORM;
 
                     while (true) {
                         try {
+                            GridCacheVersion readVer = null;
+
                             Object transformClo =
-                                (txEntry.op() == TRANSFORM  && cctx.gridEvents().isRecordable(EVT_CACHE_OBJECT_READ)) ?
+                                (txEntry.op() == TRANSFORM &&
+                                    cctx.gridEvents().isRecordable(EVT_CACHE_OBJECT_READ)) ?
                                     F.first(txEntry.entryProcessors()) : null;
 
-                            val = txEntry.cached().innerGet(this,
-                                /*swap*/true,
-                                /*read-through*/false,
-                                /*fail fast*/true,
-                                /*unmarshal*/true,
-                                /*metrics*/true,
-                                /*event*/!skipVals,
-                                /*temporary*/false,
-                                CU.subjectId(this, cctx),
-                                transformClo,
-                                resolveTaskName(),
-                                null,
-                                txEntry.keepBinary());
+                            if (needVer) {
+                                T2<CacheObject, GridCacheVersion> res = txEntry.cached().innerGetVersioned(
+                                    this,
+                                    /*swap*/true,
+                                    /*unmarshal*/true,
+                                    /*update-metrics*/true,
+                                    /*event*/!skipVals,
+                                    CU.subjectId(this, cctx),
+                                    transformClo,
+                                    resolveTaskName(),
+                                    null,
+                                    txEntry.keepBinary());
+
+                                if (res != null) {
+                                    val = res.get1();
+                                    readVer = res.get2();
+                                }
+                            }
+                            else {
+                                val = txEntry.cached().innerGet(this,
+                                    /*swap*/true,
+                                    /*read-through*/false,
+                                    /*fail fast*/true,
+                                    /*unmarshal*/true,
+                                    /*metrics*/true,
+                                    /*event*/!skipVals,
+                                    /*temporary*/false,
+                                    CU.subjectId(this, cctx),
+                                    transformClo,
+                                    resolveTaskName(),
+                                    null,
+                                    txEntry.keepBinary());
+                            }
 
                             if (val != null) {
                                 if (!readCommitted() && !skipVals)
@@ -1434,7 +1494,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     skipVals,
                                     keepCacheObjects,
                                     deserializeBinary,
-                                    false);
+                                    false,
+                                    readVer);
                             }
                             else
                                 missed.put(key, txEntry.cached().version());
@@ -1509,7 +1570,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     skipVals,
                                     keepCacheObjects,
                                     deserializeBinary,
-                                    false);
+                                    false,
+                                    needVer ? readVer : null);
                             }
                             else
                                 missed.put(key, ver);
@@ -1541,7 +1603,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 if (needReadVer) {
                                     assert readVer != null;
 
-                                    txEntry.serializableReadVersion(readVer);
+                                    txEntry.entryReadVersion(readVer);
                                 }
                             }
                         }
@@ -1592,7 +1654,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
     /**
      * Loads all missed keys for
-     * {@link #getAllAsync(GridCacheContext, Collection, boolean, boolean, boolean, boolean)} method.
+     * {@link #getAllAsync(GridCacheContext, Collection, boolean, boolean, boolean, boolean, boolean)} method.
      *
      * @param cacheCtx Cache context.
      * @param map Return map.
@@ -1610,12 +1672,14 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         final boolean deserializeBinary,
         final boolean skipVals,
         final boolean keepCacheObjects,
-        final boolean skipStore
+        final boolean skipStore,
+        final boolean needVer
+
     ) {
         if (log.isDebugEnabled())
             log.debug("Loading missed values for missed map: " + missedMap);
 
-        final boolean needReadVer = serializable() && optimistic();
+        final boolean needReadVer = (serializable() && optimistic()) || needVer;
 
         return new GridEmbeddedFuture<>(
             new C2<Void, Exception, Map<K, V>>() {
@@ -1677,7 +1741,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     skipVals,
                                     keepCacheObjects,
                                     deserializeBinary,
-                                    false);
+                                    false,
+                                    needVer ? loadVer : null);
                             }
                         }
                         else {
@@ -1688,7 +1753,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             if (needReadVer) {
                                 assert loadVer != null;
 
-                                txEntry.serializableReadVersion(loadVer);
+                                txEntry.entryReadVersion(loadVer);
                             }
 
                             if (visibleVal != null) {
@@ -1698,7 +1763,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     skipVals,
                                     keepCacheObjects,
                                     deserializeBinary,
-                                    false);
+                                    false,
+                                    needVer ? loadVer : null);
                             }
                         }
                     }
@@ -1707,13 +1773,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public <K, V> IgniteInternalFuture<Map<K, V>> getAllAsync(
         final GridCacheContext cacheCtx,
         Collection<KeyCacheObject> keys,
         final boolean deserializeBinary,
         final boolean skipVals,
         final boolean keepCacheObjects,
-        final boolean skipStore) {
+        final boolean skipStore,
+        final boolean needVer) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
@@ -1743,7 +1811,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 deserializeBinary,
                 skipVals,
                 keepCacheObjects,
-                skipStore);
+                skipStore,
+                needVer);
 
             if (single && missed.isEmpty())
                 return new GridFinishedFuture<>(retMap);
@@ -1789,25 +1858,48 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             while (true) {
                                 GridCacheEntryEx cached = txEntry.cached();
 
+                                CacheObject val = null;
+                                GridCacheVersion readVer = null;
+
                                 try {
                                     Object transformClo =
                                         (!F.isEmpty(txEntry.entryProcessors()) &&
                                             cctx.gridEvents().isRecordable(EVT_CACHE_OBJECT_READ)) ?
                                             F.first(txEntry.entryProcessors()) : null;
 
-                                    CacheObject val = cached.innerGet(IgniteTxLocalAdapter.this,
-                                        cacheCtx.isSwapOrOffheapEnabled(),
-                                        /*read-through*/false,
-                                        /*fail-fast*/true,
-                                        /*unmarshal*/true,
-                                        /*metrics*/true,
-                                        /*events*/!skipVals,
-                                        /*temporary*/true,
-                                        CU.subjectId(IgniteTxLocalAdapter.this, cctx),
-                                        transformClo,
-                                        resolveTaskName(),
-                                        null,
-                                        txEntry.keepBinary());
+                                    if (needVer) {
+                                        T2<CacheObject, GridCacheVersion> res = cached.innerGetVersioned(
+                                            IgniteTxLocalAdapter.this,
+                                            /*swap*/cacheCtx.isSwapOrOffheapEnabled(),
+                                            /*unmarshal*/true,
+                                            /**update-metrics*/true,
+                                            /*event*/!skipVals,
+                                            CU.subjectId(IgniteTxLocalAdapter.this, cctx),
+                                            transformClo,
+                                            resolveTaskName(),
+                                            null,
+                                            txEntry.keepBinary());
+
+                                        if (res != null) {
+                                            val = res.get1();
+                                            readVer = res.get2();
+                                        }
+                                    }
+                                    else{
+                                        val = cached.innerGet(IgniteTxLocalAdapter.this,
+                                            cacheCtx.isSwapOrOffheapEnabled(),
+                                            /*read-through*/false,
+                                            /*fail-fast*/true,
+                                            /*unmarshal*/true,
+                                            /*metrics*/true,
+                                            /*events*/!skipVals,
+                                            /*temporary*/true,
+                                            CU.subjectId(IgniteTxLocalAdapter.this, cctx),
+                                            transformClo,
+                                            resolveTaskName(),
+                                            null,
+                                            txEntry.keepBinary());
+                                    }
 
                                     // If value is in cache and passed the filter.
                                     if (val != null) {
@@ -1824,7 +1916,11 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                             skipVals,
                                             keepCacheObjects,
                                             deserializeBinary,
-                                            false);
+                                            false,
+                                            readVer);
+
+                                        if (readVer != null)
+                                            txEntry.entryReadVersion(readVer);
                                     }
 
                                     // Even though we bring the value back from lock acquisition,
@@ -1850,7 +1946,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 deserializeBinary,
                                 skipVals,
                                 keepCacheObjects,
-                                skipStore);
+                                skipStore,
+                                needVer);
                         }
 
                         return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
@@ -1916,7 +2013,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         deserializeBinary,
                         skipVals,
                         keepCacheObjects,
-                        skipStore);
+                        skipStore,
+                        needVer);
                 }
 
                 return new GridFinishedFuture<>(retMap);
@@ -2307,7 +2405,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                     if (needReadVer) {
                         assert loadVer != null;
 
-                        e.serializableReadVersion(singleRmv && val != null ? SER_READ_NOT_EMPTY_VER : loadVer);
+                        e.entryReadVersion(singleRmv && val != null ? SER_READ_NOT_EMPTY_VER : loadVer);
                     }
 
                     if (singleRmv) {
@@ -2500,7 +2598,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             if (needReadVer) {
                                 assert readVer != null;
 
-                                txEntry.serializableReadVersion(singleRmv ? SER_READ_NOT_EMPTY_VER : readVer);
+                                txEntry.entryReadVersion(singleRmv ? SER_READ_NOT_EMPTY_VER : readVer);
                             }
                         }
 
@@ -2553,7 +2651,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             if (needReadVer) {
                                 assert readVer != null;
 
-                                txEntry.serializableReadVersion(singleRmv ? SER_READ_NOT_EMPTY_VER : readVer);
+                                txEntry.entryReadVersion(singleRmv ? SER_READ_NOT_EMPTY_VER : readVer);
                             }
 
                             if (retval && !transform)

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
index a5d3373..78f517c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
@@ -76,7 +76,8 @@ public interface IgniteTxLocalEx extends IgniteInternalTx {
         boolean deserializeBinary,
         boolean skipVals,
         boolean keepCacheObjects,
-        boolean skipStore);
+        boolean skipStore,
+        boolean needVer);
 
     /**
      * @param cacheCtx Cache context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 818ac11..2c1205a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -1417,7 +1417,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
                     assert !entry1.detached() : "Expected non-detached entry for near transaction " +
                         "[locNodeId=" + cctx.localNodeId() + ", entry=" + entry1 + ']';
 
-                    GridCacheVersion serReadVer = txEntry1.serializableReadVersion();
+                    GridCacheVersion serReadVer = txEntry1.entryReadVersion();
 
                     assert serReadVer == null || (tx.optimistic() && tx.serializable()) : txEntry1;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
new file mode 100644
index 0000000..c0ba42c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
@@ -0,0 +1,803 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheEntry;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+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.CacheMode.LOCAL;
+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.transactions.TransactionConcurrency.OPTIMISTIC;
+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;
+
+/**
+ * Test getEntry and getEntries methods.
+ */
+public abstract class CacheGetEntryAbstractTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final String UPDATED_ENTRY_ERR = "Impossible to get version for entry updated in transaction";
+
+    /** */
+    private static final String ENTRY_AFTER_GET_ERR = "Impossible to get entry version after get()";
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /**
+     * @return Transaction concurrency.
+     */
+    abstract protected TransactionConcurrency concurrency();
+
+    /**
+     *
+     * @return Transaction isolation.
+     */
+    abstract protected TransactionIsolation isolation();
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 60_000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setMarshaller(null);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNear() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(PARTITIONED);
+        cfg.setAtomicityMode(ATOMIC);
+        cfg.setName("near");
+        cfg.setNearConfiguration(new NearCacheConfiguration());
+
+        test(cfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearTransactional() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(PARTITIONED);
+        cfg.setAtomicityMode(TRANSACTIONAL);
+        cfg.setName("nearT");
+        cfg.setNearConfiguration(new NearCacheConfiguration());
+
+        test(cfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitioned() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(PARTITIONED);
+        cfg.setAtomicityMode(ATOMIC);
+        cfg.setName("partitioned");
+
+        test(cfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionedTransactional() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(PARTITIONED);
+        cfg.setAtomicityMode(TRANSACTIONAL);
+        cfg.setName("partitionedT");
+
+        test(cfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocal() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(LOCAL);
+        cfg.setAtomicityMode(ATOMIC);
+        cfg.setName("local");
+
+        test(cfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalTransactional() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(LOCAL);
+        cfg.setAtomicityMode(TRANSACTIONAL);
+        cfg.setName("localT");
+
+        test(cfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicated() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(REPLICATED);
+        cfg.setAtomicityMode(ATOMIC);
+        cfg.setName("replicated");
+
+        test(cfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedTransactional() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(REPLICATED);
+        cfg.setAtomicityMode(TRANSACTIONAL);
+        cfg.setName("replicatedT");
+
+        test(cfg);
+    }
+
+    /**
+     * @param cfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void test(CacheConfiguration cfg) throws Exception {
+        test(cfg, true);
+
+        test(cfg, false);
+    }
+
+    /**
+     * @param cfg Cache configuration.
+     * @param oneEntry If {@code true} then single entry is tested.
+     * @throws Exception If failed.
+     */
+    private void test(CacheConfiguration cfg, final boolean oneEntry) throws Exception {
+        final IgniteCache<Integer, TestValue> cache = grid(0).createCache(cfg);
+
+        try {
+            init(cache);
+
+            test(cache, null, null, null, oneEntry);
+
+            if (cfg.getAtomicityMode() == TRANSACTIONAL) {
+                TransactionConcurrency txConcurrency = concurrency();
+                TransactionIsolation txIsolation = isolation();
+
+                try (Transaction tx = grid(0).transactions().txStart(txConcurrency, txIsolation)) {
+                    initTx(cache);
+
+                    test(cache, txConcurrency, txIsolation, tx, oneEntry);
+
+                    tx.commit();
+                }
+
+                testConcurrentTx(cache, OPTIMISTIC, REPEATABLE_READ, oneEntry);
+                testConcurrentTx(cache, OPTIMISTIC, READ_COMMITTED, oneEntry);
+
+                testConcurrentTx(cache, PESSIMISTIC, REPEATABLE_READ, oneEntry);
+                testConcurrentTx(cache, PESSIMISTIC, READ_COMMITTED, oneEntry);
+            }
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param txConcurrency Transaction concurrency.
+     * @param txIsolation Transaction isolation.
+     * @param oneEntry If {@code true} then single entry is tested.
+     * @throws Exception If failed.
+     */
+    private void testConcurrentTx(final IgniteCache<Integer, TestValue> cache,
+        final TransactionConcurrency txConcurrency,
+        final TransactionIsolation txIsolation,
+        final boolean oneEntry) throws Exception {
+        GridTestUtils.runMultiThreaded(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                IgniteTransactions txs = grid(0).transactions();
+
+                long stopTime = System.currentTimeMillis() + 3000;
+
+                while (System.currentTimeMillis() < stopTime) {
+                    Set<Integer> keys = new LinkedHashSet<>();
+
+                    for (int i = 0; i < 100; i++)
+                        keys.add(i);
+
+                    try (Transaction tx = txs.txStart(txConcurrency, txIsolation)) {
+                        if (oneEntry) {
+                            for (int i = 0; i < 100; i++)
+                                cache.getEntry(i);
+                        }
+                        else
+                            cache.getEntries(keys);
+
+                        for (int i = 0; i < 100; i++)
+                            cache.put(i, new TestValue(i));
+
+                        tx.commit();
+                    }
+                }
+
+                return null;
+            }
+        }, 10, "tx-thread");
+    }
+
+    /**
+     * @param base Start value.
+     * @return Keys.
+     */
+    private Set<Integer> getKeys(int base) {
+        int start = 0;
+        int finish = 100;
+
+        Set<Integer> keys = new HashSet<>(finish - start);
+
+        for (int i = base + start; i < base + finish; ++i)
+            keys.add(i);
+
+        return keys;
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdBeforeTxKeys() {
+        return getKeys(0);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdBeforeTxWithBinaryKeys() {
+        return getKeys(1_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdBeforeTxKeys2() {
+        return getKeys(2_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdBeforeTxWithBinaryKeys2() {
+        return getKeys(3_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdBeforeTxKeys3() {
+        return getKeys(4_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdBeforeTxWithBinaryKeys3() {
+        return getKeys(5_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> removedBeforeTxKeys() {
+        return getKeys(6_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> removedBeforeTxWithBinaryKeys() {
+        return getKeys(7_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdAtTxKeys() {
+        return getKeys(8_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> createdAtTxWithBinaryKeys() {
+        return getKeys(9_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> removedAtTxKeys() {
+        return getKeys(10_000);
+    }
+
+    /**
+     * @return Keys.
+     */
+    private Set<Integer> removedAtTxWithBinaryKeys() {
+        return getKeys(11_000);
+    }
+
+    /**
+     * @param cache Cacge.
+     */
+    private void init(IgniteCache<Integer, TestValue> cache) {
+        Set<Integer> keys = new HashSet<>();
+
+        keys.addAll(createdBeforeTxKeys());
+        keys.addAll(createdBeforeTxWithBinaryKeys());
+        keys.addAll(createdBeforeTxKeys2());
+        keys.addAll(createdBeforeTxWithBinaryKeys2());
+        keys.addAll(createdBeforeTxKeys3());
+        keys.addAll(createdBeforeTxWithBinaryKeys3());
+        keys.addAll(removedBeforeTxKeys());
+        keys.addAll(removedBeforeTxWithBinaryKeys());
+        keys.addAll(removedAtTxKeys());
+        keys.addAll(removedAtTxWithBinaryKeys());
+
+        for (int i : keys)
+            cache.put(i, new TestValue(i));
+
+        for (int i : removedBeforeTxKeys())
+            cache.remove(i);
+
+        for (int i : removedBeforeTxWithBinaryKeys())
+            cache.remove(i);
+    }
+
+    /**
+     * @param cache Cache.
+     */
+    private void initTx(IgniteCache<Integer, TestValue> cache) {
+        for (int i : createdAtTxKeys())
+            cache.put(i, new TestValue(i));
+
+        for (int i : createdAtTxWithBinaryKeys())
+            cache.put(i, new TestValue(i));
+
+        for (int i : removedAtTxKeys())
+            cache.remove(i);
+
+        for (int i : removedAtTxWithBinaryKeys())
+            cache.remove(i);
+    }
+
+    /**
+     * @param e Entry.
+     * @param cache Cache.
+     * @throws Exception If failed.
+     */
+    private void compareVersionWithPrimaryNode(CacheEntry<Integer, ?> e, IgniteCache<Integer, TestValue> cache)
+        throws Exception {
+        CacheConfiguration cfg = cache.getConfiguration(CacheConfiguration.class);
+
+        if (cfg.getCacheMode() != LOCAL) {
+            Ignite prim = primaryNode(e.getKey(), cache.getName());
+
+            GridCacheAdapter<Object, Object> cacheAdapter = ((IgniteKernal)prim).internalCache(cache.getName());
+
+            if (cfg.getNearConfiguration() != null)
+                cacheAdapter = ((GridNearCacheAdapter)cacheAdapter).dht();
+
+            IgniteCacheObjectProcessor cacheObjects = cacheAdapter.context().cacheObjects();
+
+            CacheObjectContext cacheObjCtx = cacheAdapter.context().cacheObjectContext();
+
+            GridCacheMapEntry mapEntry = cacheAdapter.map().getEntry(cacheObjects.toCacheKeyObject(
+                cacheObjCtx, e.getKey(), true));
+
+            assertNotNull("No entry for key: " + e.getKey(), mapEntry);
+            assertEquals(mapEntry.version(), e.version());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param i Key.
+     * @param oneEntry If {@code true} then single entry is tested.
+     * @param getVerErr Not null error if entry version access should fail.
+     * @param expKeys Expected keys with values.
+     * @throws Exception If failed.
+     */
+    private void checkData(IgniteCache<Integer, TestValue> cache,
+        int i,
+        boolean oneEntry,
+        @Nullable String getVerErr,
+        Set<Integer> expKeys) throws Exception {
+        if (oneEntry) {
+            final CacheEntry<Integer, TestValue> e = cache.getEntry(i);
+
+            if (getVerErr == null)
+                compareVersionWithPrimaryNode(e, cache);
+            else {
+                Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        e.version();
+
+                        return null;
+                    }
+                }, IgniteException.class, null);
+
+                assertTrue("Unexpected error message: " + err.getMessage(), err.getMessage().startsWith(getVerErr));
+            }
+
+            assertEquals(e.getValue().val, i);
+        }
+        else {
+            Set<Integer> set = new HashSet<>();
+
+            int expCnt = 0;
+
+            for (int j = 0; j < 10; j++) {
+                Integer key = i + j;
+
+                set.add(key);
+
+                if (expKeys.contains(key))
+                    expCnt++;
+            }
+
+            Collection<CacheEntry<Integer, TestValue>> entries = cache.getEntries(set);
+
+            assertEquals(expCnt, entries.size());
+
+            for (final CacheEntry<Integer, TestValue> e : entries) {
+                if (getVerErr == null)
+                    compareVersionWithPrimaryNode(e, cache);
+                else {
+                    Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            e.version();
+
+                            return null;
+                        }
+                    }, IgniteException.class, null);
+
+                    assertTrue("Unexpected error message: " + err.getMessage(), err.getMessage().startsWith(getVerErr));
+                }
+
+                assertEquals((Integer)e.getValue().val, e.getKey());
+
+                assertTrue(set.contains(e.getValue().val));
+            }
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param i Key.
+     * @param oneEntry If {@code true} then single entry is tested.
+     * @param getVerErr Not null error if entry version access should fail.
+     * @param expKeys Expected keys with values.
+     * @throws Exception If failed.
+     */
+    private void checkBinaryData(IgniteCache<Integer, TestValue> cache,
+        int i,
+        boolean oneEntry,
+        @Nullable String getVerErr,
+        Set<Integer> expKeys) throws Exception {
+        IgniteCache<Integer, BinaryObject> cacheB = cache.withKeepBinary();
+
+        if (oneEntry) {
+            final CacheEntry<Integer, BinaryObject> e = cacheB.getEntry(i);
+
+            if (getVerErr == null)
+                compareVersionWithPrimaryNode(e, cache);
+            else {
+                Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        e.version();
+
+                        return null;
+                    }
+                }, IgniteException.class, null);
+
+                assertTrue("Unexpected error message: " + err.getMessage(), err.getMessage().startsWith(getVerErr));
+            }
+
+            assertEquals(((TestValue)e.getValue().deserialize()).val, i);
+        }
+        else {
+            Set<Integer> set = new HashSet<>();
+
+            int expCnt = 0;
+
+            for (int j = 0; j < 10; j++) {
+                Integer key = i + j;
+
+                set.add(key);
+
+                if (expKeys.contains(key))
+                    expCnt++;
+            }
+
+            Collection<CacheEntry<Integer, BinaryObject>> entries = cacheB.getEntries(set);
+
+            assertEquals(expCnt, entries.size());
+
+            for (final CacheEntry<Integer, BinaryObject> e : entries) {
+                if (getVerErr == null)
+                    compareVersionWithPrimaryNode(e, cache);
+                else {
+                    Throwable err = GridTestUtils.assertThrows(log, new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            e.version();
+
+                            return null;
+                        }
+                    }, IgniteException.class, null);
+
+                    assertTrue("Unexpected error message: " + err.getMessage(), err.getMessage().startsWith(getVerErr));
+                }
+
+                TestValue tv = e.getValue().deserialize();
+
+                assertEquals((Integer)tv.val, e.getKey());
+
+                assertTrue(set.contains((tv).val));
+            }
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param i Key.
+     * @param oneEntry If {@code true} then single entry is tested.
+     */
+    private void checkRemoved(IgniteCache<Integer, TestValue> cache, int i, boolean oneEntry) {
+        if (oneEntry) {
+            CacheEntry<Integer, TestValue> e = cache.getEntry(i);
+
+            assertNull(e);
+        }
+        else {
+            Set<Integer> set = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                set.add(i + j);
+
+            Collection<CacheEntry<Integer, TestValue>> es = cache.getEntries(set);
+
+            assertTrue(es.isEmpty());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param i Key.
+     * @param oneEntry If {@code true} then single entry is tested.
+     */
+    private void checkBinaryRemoved(IgniteCache<Integer, TestValue> cache, int i, boolean oneEntry) {
+        IgniteCache<Integer, BinaryObject> cacheB = cache.withKeepBinary();
+
+        if (oneEntry) {
+            CacheEntry<Integer, BinaryObject> e = cacheB.getEntry(i);
+
+            assertNull(e);
+        }
+        else {
+            Set<Integer> set = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                set.add(i + j);
+
+            Collection<CacheEntry<Integer, BinaryObject>> es = cacheB.getEntries(set);
+
+            assertTrue(es.isEmpty());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param txConcurrency Transaction concurrency.
+     * @param txIsolation Transaction isolation.
+     * @param tx Transaction.
+     * @param oneEntry If {@code true} then single entry is tested.
+     * @throws Exception If failed.
+     */
+    private void test(IgniteCache<Integer, TestValue> cache,
+        TransactionConcurrency txConcurrency,
+        TransactionIsolation txIsolation,
+        Transaction tx,
+        boolean oneEntry) throws Exception {
+        if (tx == null) {
+            Set<Integer> keys = createdBeforeTxKeys();
+
+            for (int i : keys)
+                checkData(cache, i, oneEntry, null, keys);
+
+            keys = createdBeforeTxWithBinaryKeys();
+
+            for (int i : keys)
+                checkBinaryData(cache, i, oneEntry, null, keys);
+
+            for (int i : removedBeforeTxKeys())
+                checkRemoved(cache, i, oneEntry);
+
+            for (int i : removedBeforeTxWithBinaryKeys())
+                checkBinaryRemoved(cache, i, oneEntry);
+        }
+        else {
+            Set<Integer> keys = createdBeforeTxKeys2();
+
+            for (int i : keys) {
+                checkData(cache, i, oneEntry, null, keys);
+                checkData(cache, i, oneEntry, null, keys);
+            }
+
+            keys = createdBeforeTxWithBinaryKeys2();
+
+            for (int i : keys) {
+                checkBinaryData(cache, i, oneEntry, null, keys);
+                checkBinaryData(cache, i, oneEntry, null, keys);
+            }
+
+            String verGetErr = null;
+
+            if (txConcurrency == OPTIMISTIC && txIsolation == REPEATABLE_READ)
+                verGetErr = ENTRY_AFTER_GET_ERR;
+
+            keys = createdBeforeTxKeys3();
+
+            for (int i : keys) {
+                if (oneEntry)
+                    cache.get(i);
+                else {
+                    Set<Integer> set = new HashSet<>();
+
+                    for (int j = 0; j < 10; j++)
+                        set.add(i + j);
+
+                    cache.getAll(set);
+                }
+
+                checkData(cache, i, oneEntry, verGetErr, keys);
+            }
+
+            keys = createdBeforeTxWithBinaryKeys3();
+
+            for (int i : keys) {
+                if (oneEntry)
+                    cache.get(i);
+                else {
+                    Set<Integer> set = new HashSet<>();
+
+                    for (int j = 0; j < 10; j++)
+                        set.add(i + j);
+
+                    cache.getAll(set);
+                }
+
+                checkBinaryData(cache, i, oneEntry, verGetErr, keys);
+            }
+
+            keys = createdAtTxKeys();
+
+            for (int i : keys)
+                checkData(cache, i, oneEntry, UPDATED_ENTRY_ERR, keys);
+
+            keys = createdAtTxWithBinaryKeys();
+
+            for (int i : keys)
+                checkBinaryData(cache, i, oneEntry, UPDATED_ENTRY_ERR, keys);
+
+            for (int i : removedBeforeTxKeys())
+                checkRemoved(cache, i, oneEntry);
+
+            for (int i : removedBeforeTxWithBinaryKeys())
+                checkBinaryRemoved(cache, i, oneEntry);
+
+            for (int i : removedAtTxKeys())
+                checkRemoved(cache, i, oneEntry);
+
+            for (int i : removedAtTxWithBinaryKeys())
+                checkBinaryRemoved(cache, i, oneEntry);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestValue implements Serializable {
+        /** */
+        private int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(int val) {
+            this.val = val;
+        }
+
+        /**
+         * @return Value.
+         */
+        public int value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticReadCommittedSeltTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticReadCommittedSeltTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticReadCommittedSeltTest.java
new file mode 100644
index 0000000..acc21df
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticReadCommittedSeltTest.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;
+
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ * Test getEntry and getEntries methods.
+ */
+public class CacheGetEntryOptimisticReadCommittedSeltTest extends CacheGetEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected TransactionConcurrency concurrency() {
+        return TransactionConcurrency.OPTIMISTIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TransactionIsolation isolation() {
+        return TransactionIsolation.READ_COMMITTED;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticRepeatableReadSeltTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticRepeatableReadSeltTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticRepeatableReadSeltTest.java
new file mode 100644
index 0000000..6153869
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticRepeatableReadSeltTest.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;
+
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ * Test getEntry and getEntries methods.
+ */
+public class CacheGetEntryOptimisticRepeatableReadSeltTest extends CacheGetEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected TransactionConcurrency concurrency() {
+        return TransactionConcurrency.OPTIMISTIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TransactionIsolation isolation() {
+        return TransactionIsolation.REPEATABLE_READ;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticSerializableSeltTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticSerializableSeltTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticSerializableSeltTest.java
new file mode 100644
index 0000000..6ded4a9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryOptimisticSerializableSeltTest.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;
+
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ * Test getEntry and getEntries methods.
+ */
+public class CacheGetEntryOptimisticSerializableSeltTest extends CacheGetEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected TransactionConcurrency concurrency() {
+        return TransactionConcurrency.OPTIMISTIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TransactionIsolation isolation() {
+        return TransactionIsolation.SERIALIZABLE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticReadCommittedSeltTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticReadCommittedSeltTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticReadCommittedSeltTest.java
new file mode 100644
index 0000000..975d271
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticReadCommittedSeltTest.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;
+
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ * Test getEntry and getEntries methods.
+ */
+public class CacheGetEntryPessimisticReadCommittedSeltTest extends CacheGetEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected TransactionConcurrency concurrency() {
+        return TransactionConcurrency.PESSIMISTIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TransactionIsolation isolation() {
+        return TransactionIsolation.READ_COMMITTED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticRepeatableReadSeltTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticRepeatableReadSeltTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticRepeatableReadSeltTest.java
new file mode 100644
index 0000000..dac64d9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticRepeatableReadSeltTest.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;
+
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ * Test getEntry and getEntries methods.
+ */
+public class CacheGetEntryPessimisticRepeatableReadSeltTest extends CacheGetEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected TransactionConcurrency concurrency() {
+        return TransactionConcurrency.PESSIMISTIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TransactionIsolation isolation() {
+        return TransactionIsolation.REPEATABLE_READ;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticSerializableSeltTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticSerializableSeltTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticSerializableSeltTest.java
new file mode 100644
index 0000000..70f71ce
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryPessimisticSerializableSeltTest.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;
+
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ * Test getEntry and getEntries methods.
+ */
+public class CacheGetEntryPessimisticSerializableSeltTest extends CacheGetEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected TransactionConcurrency concurrency() {
+        return TransactionConcurrency.PESSIMISTIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TransactionIsolation isolation() {
+        return TransactionIsolation.SERIALIZABLE;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java
index c606a2a..b60ada7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java
@@ -85,6 +85,20 @@ public class CacheReadThroughRestartSelfTest extends GridCacheAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testReadThroughInTx() throws Exception {
+        testReadThroughInTx(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadEntryThroughInTx() throws Exception {
+        testReadThroughInTx(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void testReadThroughInTx(boolean needVer) throws Exception {
         IgniteCache<String, Integer> cache = grid(1).cache(null);
 
         for (int k = 0; k < 1000; k++)
@@ -104,7 +118,14 @@ public class CacheReadThroughRestartSelfTest extends GridCacheAbstractSelfTest {
                     for (int k = 0; k < 1000; k++) {
                         String key = "key" + k;
 
-                        assertNotNull("Null value for key: " + key, cache.get(key));
+                        if (needVer) {
+                            assertNotNull("Null value for key: " + key, cache.getEntry(key));
+                            assertNotNull("Null value for key: " + key, cache.getEntry(key));
+                        }
+                        else {
+                            assertNotNull("Null value for key: " + key, cache.get(key));
+                            assertNotNull("Null value for key: " + key, cache.get(key));
+                        }
                     }
 
                     tx.commit();
@@ -117,6 +138,20 @@ public class CacheReadThroughRestartSelfTest extends GridCacheAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testReadThrough() throws Exception {
+        testReadThrough(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadEntryThrough() throws Exception {
+        testReadThrough(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void testReadThrough(boolean needVer) throws Exception {
         IgniteCache<String, Integer> cache = grid(1).cache(null);
 
         for (int k = 0; k < 1000; k++)
@@ -132,8 +167,10 @@ public class CacheReadThroughRestartSelfTest extends GridCacheAbstractSelfTest {
 
         for (int k = 0; k < 1000; k++) {
             String key = "key" + k;
-
-            assertNotNull("Null value for key: " + key, cache.get(key));
+            if (needVer)
+                assertNotNull("Null value for key: " + key, cache.getEntry(key));
+            else
+                assertNotNull("Null value for key: " + key, cache.get(key));
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f527656d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
index 9906ad3..4baef66 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -46,6 +47,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.store.CacheStore;
@@ -575,6 +577,20 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testTxCommitReadOnlyGetAll() throws Exception {
+        testTxCommitReadOnlyGetAll(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxCommitReadOnlyGetEntries() throws Exception {
+        testTxCommitReadOnlyGetAll(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxCommitReadOnlyGetAll(boolean needVer) throws Exception {
         Ignite ignite0 = ignite(0);
 
         final IgniteTransactions txs = ignite0.transactions();
@@ -591,9 +607,16 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                     keys.add(i);
 
                 try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
-                    Map<Integer, Integer> map = cache.getAll(keys);
+                    if (needVer) {
+                        Collection<CacheEntry<Integer, Integer>> c = cache.getEntries(keys);
 
-                    assertTrue(map.isEmpty());
+                        assertTrue(c.isEmpty());
+                    }
+                    else {
+                        Map<Integer, Integer> map = cache.getAll(keys);
+
+                        assertTrue(map.isEmpty());
+                    }
 
                     tx.commit();
                 }
@@ -602,9 +625,16 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                     checkValue(key, null, cache.getName());
 
                 try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
-                    Map<Integer, Integer> map = cache.getAll(keys);
+                    if (needVer) {
+                        Collection<CacheEntry<Integer, Integer>> c = cache.getEntries(keys);
 
-                    assertTrue(map.isEmpty());
+                        assertTrue(c.isEmpty());
+                    }
+                    else {
+                        Map<Integer, Integer> map = cache.getAll(keys);
+
+                        assertTrue(map.isEmpty());
+                    }
 
                     tx.rollback();
                 }
@@ -653,21 +683,35 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testTxConflictRead1() throws Exception {
-        txConflictRead(true);
+        txConflictRead(true, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTxConflictRead2() throws Exception {
-        txConflictRead(false);
+        txConflictRead(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxConflictReadEntry1() throws Exception {
+        txConflictRead(true, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxConflictReadEntry2() throws Exception {
+        txConflictRead(false, true);
     }
 
     /**
      * @param noVal If {@code true} there is no cache value when read in tx.
      * @throws Exception If failed.
      */
-    private void txConflictRead(boolean noVal) throws Exception {
+    private void txConflictRead(boolean noVal, boolean needVer) throws Exception {
         Ignite ignite0 = ignite(0);
 
         final IgniteTransactions txs = ignite0.transactions();
@@ -693,9 +737,16 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
                     try {
                         try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
-                            Integer val = cache.get(key);
+                            if (needVer) {
+                                CacheEntry<Integer, Integer> val = cache.getEntry(key);
 
-                            assertEquals(expVal, val);
+                                assertEquals(expVal, val == null ? null : val.getValue());
+                            }
+                            else {
+                                Integer val = cache.get(key);
+
+                                assertEquals(expVal, val);
+                            }
 
                             updateKey(cache, key, 1);
 
@@ -711,9 +762,16 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                     checkValue(key, 1, cache.getName());
 
                     try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
-                        Object val = cache.get(key);
+                        if (needVer) {
+                            CacheEntry<Integer, Integer> val = cache.getEntry(key);
 
-                        assertEquals(1, val);
+                            assertEquals((Integer)1, val.getValue());
+                        }
+                        else {
+                            Object val = cache.get(key);
+
+                            assertEquals(1, val);
+                        }
 
                         tx.commit();
                     }
@@ -731,28 +789,56 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testTxConflictReadWrite1() throws Exception {
-        txConflictReadWrite(true, false);
+        txConflictReadWrite(true, false, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTxConflictReadWrite2() throws Exception {
-        txConflictReadWrite(false, false);
+        txConflictReadWrite(false, false, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTxConflictReadRemove1() throws Exception {
-        txConflictReadWrite(true, true);
+        txConflictReadWrite(true, true, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTxConflictReadRemove2() throws Exception {
-        txConflictReadWrite(false, true);
+        txConflictReadWrite(false, true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxConflictReadEntryWrite1() throws Exception {
+        txConflictReadWrite(true, false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxConflictReadEntryWrite2() throws Exception {
+        txConflictReadWrite(false, false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxConflictReadEntryRemove1() throws Exception {
+        txConflictReadWrite(true, true, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxConflictReadEntryRemove2() throws Exception {
+        txConflictReadWrite(false, true, true);
     }
 
     /**
@@ -760,7 +846,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
      * @param rmv If {@code true} tests remove, otherwise put.
      * @throws Exception If failed.
      */
-    private void txConflictReadWrite(boolean noVal, boolean rmv) throws Exception {
+    private void txConflictReadWrite(boolean noVal, boolean rmv, boolean needVer) throws Exception {
         Ignite ignite0 = ignite(0);
 
         final IgniteTransactions txs = ignite0.transactions();
@@ -786,9 +872,16 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
                     try {
                         try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
-                            Integer val = cache.get(key);
+                            if (needVer) {
+                                CacheEntry<Integer, Integer> val = cache.getEntry(key);
 
-                            assertEquals(expVal, val);
+                                assertEquals(expVal, val == null ? null : val.getValue());
+                            }
+                            else {
+                                Integer val = cache.get(key);
+
+                                assertEquals(expVal, val);
+                            }
 
                             updateKey(cache, key, 1);
 
@@ -809,9 +902,16 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                     checkValue(key, 1, cache.getName());
 
                     try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
-                        Integer val = cache.get(key);
+                        if (needVer) {
+                            CacheEntry<Integer, Integer> val = cache.getEntry(key);
+
+                            assertEquals(1, (Object)val.getValue());
+                        }
+                        else {
+                            Integer val = cache.get(key);
 
-                        assertEquals(1, (Object) val);
+                            assertEquals(1, (Object)val);
+                        }
 
                         if (rmv)
                             cache.remove(key);
@@ -4239,7 +4339,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                 ignite.destroyCache(cacheName);
             }
             catch (IgniteException ignore) {
-                // No-op.                
+                // No-op.
             }
 
             GridTestSwapSpaceSpi spi = (GridTestSwapSpaceSpi)ignite.configuration().getSwapSpaceSpi();


[6/6] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5.7' into ignite-1.5.7

Posted by av...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5.7' into ignite-1.5.7


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

Branch: refs/heads/ignite-1.5.7
Commit: 1842861f54c2e33dd7060846cf2372397cc8dc6c
Parents: 5787d6e e5fe263
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Feb 2 18:23:09 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Feb 2 18:23:09 2016 +0300

----------------------------------------------------------------------
 examples/pom.xml                                               | 2 +-
 examples/schema-import/pom.xml                                 | 2 +-
 modules/aop/pom.xml                                            | 2 +-
 modules/apache-license-gen/pom.xml                             | 2 +-
 modules/aws/pom.xml                                            | 2 +-
 modules/camel/pom.xml                                          | 2 +-
 modules/clients/pom.xml                                        | 2 +-
 modules/cloud/pom.xml                                          | 2 +-
 modules/codegen/pom.xml                                        | 2 +-
 modules/core/pom.xml                                           | 2 +-
 modules/core/src/main/resources/ignite.properties              | 2 +-
 modules/extdata/p2p/pom.xml                                    | 2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml             | 2 +-
 modules/extdata/uri/pom.xml                                    | 2 +-
 modules/flume/pom.xml                                          | 2 +-
 modules/gce/pom.xml                                            | 2 +-
 modules/geospatial/pom.xml                                     | 2 +-
 modules/hadoop/pom.xml                                         | 2 +-
 modules/hibernate/pom.xml                                      | 2 +-
 modules/indexing/pom.xml                                       | 2 +-
 modules/jcl/pom.xml                                            | 2 +-
 modules/jms11/pom.xml                                          | 2 +-
 modules/jta/pom.xml                                            | 2 +-
 modules/kafka/pom.xml                                          | 2 +-
 modules/log4j/pom.xml                                          | 2 +-
 modules/log4j2/pom.xml                                         | 2 +-
 modules/mesos/pom.xml                                          | 2 +-
 modules/mqtt/pom.xml                                           | 2 +-
 modules/osgi-karaf/pom.xml                                     | 2 +-
 modules/osgi-paxlogging/pom.xml                                | 2 +-
 modules/osgi/pom.xml                                           | 2 +-
 modules/platforms/cpp/common/configure.ac                      | 2 +-
 modules/platforms/cpp/core-test/configure.ac                   | 2 +-
 modules/platforms/cpp/core/configure.ac                        | 2 +-
 modules/platforms/cpp/examples/configure.ac                    | 2 +-
 modules/platforms/cpp/ignite/configure.ac                      | 2 +-
 .../dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs | 6 +++---
 .../Properties/AssemblyInfo.cs                                 | 6 +++---
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs | 6 +++---
 .../dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs       | 6 +++---
 .../platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs  | 6 +++---
 .../examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs | 6 +++---
 .../Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs       | 6 +++---
 modules/rest-http/pom.xml                                      | 2 +-
 modules/scalar-2.10/pom.xml                                    | 2 +-
 modules/scalar/pom.xml                                         | 2 +-
 modules/schedule/pom.xml                                       | 2 +-
 modules/schema-import/pom.xml                                  | 2 +-
 modules/slf4j/pom.xml                                          | 2 +-
 modules/spark-2.10/pom.xml                                     | 2 +-
 modules/spark/pom.xml                                          | 2 +-
 modules/spring/pom.xml                                         | 2 +-
 modules/ssh/pom.xml                                            | 2 +-
 modules/tools/pom.xml                                          | 2 +-
 modules/twitter/pom.xml                                        | 2 +-
 modules/urideploy/pom.xml                                      | 2 +-
 modules/visor-console-2.10/pom.xml                             | 2 +-
 modules/visor-console/pom.xml                                  | 2 +-
 modules/visor-plugins/pom.xml                                  | 2 +-
 modules/web/pom.xml                                            | 2 +-
 modules/yardstick/pom.xml                                      | 2 +-
 modules/yarn/pom.xml                                           | 2 +-
 modules/zookeeper/pom.xml                                      | 2 +-
 pom.xml                                                        | 2 +-
 64 files changed, 78 insertions(+), 78 deletions(-)
----------------------------------------------------------------------



[5/6] ignite git commit: 2224 (cherry picked from commit dcb572e) Merge fix

Posted by av...@apache.org.
2224
(cherry picked from commit dcb572e)
Merge fix


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

Branch: refs/heads/ignite-1.5.7
Commit: 5787d6e4c76c4f274f0b96559ce2c8430e08a027
Parents: b630fbf
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Feb 2 18:21:42 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Feb 2 18:21:42 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/transactions/IgniteTxEntry.java       | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5787d6e4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 3282323..6cdac07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.LinkedList;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCache;


[4/6] ignite git commit: 1523 (cherry picked from commit 28a5247)

Posted by av...@apache.org.
1523
(cherry picked from commit 28a5247)


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

Branch: refs/heads/ignite-1.5.7
Commit: b630fbf9e6198078d69f589598fa1cdbfdb34eb8
Parents: f527656
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Feb 2 15:37:12 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Feb 2 18:13:50 2016 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryEnumObjectImpl.java   |   2 +-
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |   7 ++
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   6 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  11 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   1 -
 .../TcpDiscoveryCustomEventMessage.java         |  10 +-
 .../GridCacheReplicatedPreloadSelfTest.java     | 108 ++++++++++++++++++
 .../tests/p2p/CacheDeploymentTestEnumValue.java |  47 ++++++++
 .../p2p/CacheDeploymentTestStoreFactory.java    | 113 +++++++++++++++++++
 10 files changed, 297 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index 536c582..99c5d48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -112,7 +112,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T> T deserialize() throws BinaryObjectException {
-        Class cls = BinaryUtils.resolveClass(ctx, typeId, clsName, null, true);
+        Class cls = BinaryUtils.resolveClass(ctx, typeId, clsName, ctx.configuration().getClassLoader(), true);
 
         return BinaryEnumCache.get(cls, ord);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/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 6b734b5..3e9e01b 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
@@ -3403,7 +3403,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         try {
-            return marshaller.unmarshal(marshaller.marshal(val), val.getClass().getClassLoader());
+            return marshaller.unmarshal(marshaller.marshal(val), U.resolveClassLoader(ctx.config().getClassLoader()));
         }
         catch (IgniteCheckedException e) {
             throw new IgniteCheckedException("Failed to validate cache configuration " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 5784a4d..af2114e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -2193,6 +2193,13 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @return Class loader passed as an argument or classloader used to load Ignite itself in case argument is null.
+     */
+    public static ClassLoader resolveClassLoader(ClassLoader ldr) {
+        return ldr != null ? ldr : gridClassLoader;
+    }
+
+    /**
      * @param parent Parent to find.
      * @param ldr Loader to check.
      * @return {@code True} if parent found.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 850cc24..b12f7a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -1665,7 +1665,8 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     if (dataMap != null) {
                         for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                            spi.onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(), null);
+                            spi.onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(),
+                                U.resolveClassLoader(spi.ignite().configuration().getClassLoader()));
                     }
 
                     locNode.setAttributes(msg.clientNodeAttributes());
@@ -1961,7 +1962,8 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     if (node != null && node.visible()) {
                         try {
-                            DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh);
+                            DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh,
+                                spi.ignite().configuration().getClassLoader());
 
                             notifyDiscovery(EVT_DISCOVERY_CUSTOM_EVT, topVer, node, allVisibleNodes(), msgObj);
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 471c36b..3a009b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -3601,7 +3601,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                     Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
 
                     if (data != null)
-                        spi.onExchange(node.id(), node.id(), data, U.gridClassLoader());
+                        spi.onExchange(node.id(), node.id(), data,
+                            U.resolveClassLoader(spi.ignite().configuration().getClassLoader()));
 
                     msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id()));
 
@@ -3680,7 +3681,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                 // Notify outside of synchronized block.
                 if (dataMap != null) {
                     for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                        spi.onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader());
+                        spi.onExchange(node.id(), entry.getKey(), entry.getValue(),
+                            U.resolveClassLoader(spi.ignite().configuration().getClassLoader()));
                 }
 
                 processMessageFailedNodes(msg);
@@ -4605,7 +4607,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     DiscoverySpiCustomMessage msgObj = null;
 
                     try {
-                        msgObj = msg.message(spi.marsh);
+                        msgObj = msg.message(spi.marsh, spi.ignite().configuration().getClassLoader());
                     }
                     catch (Throwable e) {
                         U.error(log, "Failed to unmarshal discovery custom message.", e);
@@ -4728,7 +4730,8 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 if (node != null) {
                     try {
-                        DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh);
+                        DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh,
+                            spi.ignite().configuration().getClassLoader());
 
                         lsnr.onDiscovery(DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT,
                             msg.topologyVersion(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index f9273c3..0d41cd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1700,7 +1700,6 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * @param joiningNodeID Joining node ID.
      * @param nodeId Remote node ID for which data is provided.
      * @param data Collection of marshalled discovery data objects from different components.
-     * @param clsLdr Class loader for discovery data unmarshalling.
      */
     protected void onExchange(UUID joiningNodeID,
         UUID nodeId,

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
index 8f14459..e10de46 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
@@ -74,8 +74,16 @@ public class TcpDiscoveryCustomEventMessage extends TcpDiscoveryAbstractMessage
      * @throws java.lang.Throwable if unmarshal failed.
      */
     @Nullable public DiscoverySpiCustomMessage message(@NotNull Marshaller marsh) throws Throwable {
+        return message(marsh, null);
+    }
+
+    /**
+     * @return Deserialized message,
+     * @throws java.lang.Throwable if unmarshal failed.
+     */
+    @Nullable public DiscoverySpiCustomMessage message(@NotNull Marshaller marsh, ClassLoader ldr) throws Throwable {
         if (msg == null) {
-            msg = marsh.unmarshal(msgBytes, U.gridClassLoader());
+            msg = marsh.unmarshal(msgBytes, U.resolveClassLoader(ldr));
 
             assert msg != null;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
index 5649b34..523f641 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.UUID;
+import javax.cache.configuration.Factory;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CachePeekMode;
@@ -70,6 +71,12 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     private int poolSize = 2;
 
     /** */
+    private volatile boolean needStore = false;
+
+    /** */
+    private volatile boolean isClient = false;
+
+    /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** {@inheritDoc} */
@@ -104,6 +111,9 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
         if (getTestGridName(1).equals(gridName) || cfg.getMarshaller() instanceof BinaryMarshaller)
             cfg.setClassLoader(getExternalClassLoader());
 
+        if (isClient)
+            cfg.setClientMode(true);
+
         return cfg;
     }
 
@@ -122,6 +132,20 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
         cacheCfg.setRebalanceBatchSize(batchSize);
         cacheCfg.setRebalanceThreadPoolSize(poolSize);
 
+        if (needStore) {
+            Object sf = null;
+
+            try {
+                sf = getExternalClassLoader().
+                    loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
+            }
+            catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+
+            cacheCfg.setCacheStoreFactory((Factory)sf);
+        }
+
         return cacheCfg;
     }
 
@@ -233,6 +257,18 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             assert v2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader") ||
                 grid(2).configuration().getMarshaller() instanceof BinaryMarshaller;
 
+            Object e1 = ldr.loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestEnumValue").getEnumConstants()[0];
+
+            cache1.put(2, e1);
+
+            Object e2 = cache2.get(2);
+
+            assert e2 != null;
+            assert e2.toString().equals(e1.toString());
+            assert !e2.getClass().getClassLoader().equals(getClass().getClassLoader());
+            assert e2.getClass().getClassLoader().getClass().getName().contains("GridDeploymentClassLoader") ||
+                grid(2).configuration().getMarshaller() instanceof BinaryMarshaller;
+
             stopGrid(1);
 
             Ignite g3 = startGrid(3);
@@ -259,6 +295,78 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If test failed.
      */
+    public void testStore() throws Exception {
+        try {
+            Ignite g1 = startGrid(1);
+
+            if (g1.configuration().getMarshaller() instanceof BinaryMarshaller) {
+                stopAllGrids();
+
+                needStore = true;
+
+                g1 = startGrid(1);
+
+                ClassLoader ldr = grid(1).configuration().getClassLoader();
+
+                CacheConfiguration cfg = defaultCacheConfiguration();
+
+                Ignite g2 = startGrid(2);  // Checks deserialization at node join.
+
+                isClient = true;
+
+                Ignite g3 = startGrid(3);
+
+                isClient = false;
+
+                IgniteCache<Integer, Object> cache1 = g1.cache(null);
+                IgniteCache<Integer, Object> cache2 = g2.cache(null);
+                IgniteCache<Integer, Object> cache3 = g3.cache(null);
+
+                cache1.put(1, 1);
+
+                assertEquals(1, cache2.get(1));
+                assertEquals(1, cache3.get(1));
+
+                needStore = false;
+
+                stopAllGrids();
+
+                g1 = startGrid(1);
+                g2 = startGrid(2);
+
+                isClient = true;
+
+                g3 = startGrid(3);
+
+                isClient = false;
+
+                Object sf = ldr.loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
+
+                cfg.setCacheStoreFactory((Factory)sf);
+                cfg.setName("customStore");
+
+                cache1 = g1.createCache(cfg);
+
+                cache2 = g2.getOrCreateCache(cfg); // Checks deserialization at cache creation.
+                cache3 = g3.getOrCreateCache(cfg); // Checks deserialization at cache creation.
+
+                cache1.put(1, 1);
+
+                assertEquals(1, cache2.get(1));
+                assertEquals(1, cache3.get(1));
+            }
+        }
+        finally {
+            needStore = false;
+            isClient = false;
+
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
     public void testSync() throws Exception {
         preloadMode = SYNC;
         batchSize = 512;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestEnumValue.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestEnumValue.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestEnumValue.java
new file mode 100644
index 0000000..cc70e2d
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestEnumValue.java
@@ -0,0 +1,47 @@
+/*
+ * 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.tests.p2p;
+
+/**
+ * Test enum for cache deployment tests.
+ */
+public enum CacheDeploymentTestEnumValue {
+    ONE("one"),
+    TWO("two"),
+    THREE("three");
+
+    /** */
+    private String value;
+
+    /** */
+    CacheDeploymentTestEnumValue(String value) {
+        this.value = value;
+    }
+
+    /** */
+    public String getValue() {
+        return value;
+    }
+
+    /** */
+    @Override public String toString() {
+        return "CacheDeploymentTestEnumValue{" +
+            "value='" + value + '\'' +
+            '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b630fbf9/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestStoreFactory.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestStoreFactory.java
new file mode 100644
index 0000000..1a0fc17
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentTestStoreFactory.java
@@ -0,0 +1,113 @@
+/*
+ * 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.tests.p2p;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.store.CacheLocalStore;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test store factory for cache deployment tests.
+ */
+public class CacheDeploymentTestStoreFactory implements Factory<CacheStore<Integer, String>>, Serializable {
+    /** {@inheritDoc} */
+    @Override public CacheStore<Integer, String> create() {
+        return new TestLocalStore();
+    }
+
+    /**
+     *
+     */
+    @CacheLocalStore
+    public static class TestLocalStore<K, V> implements CacheStore<K, IgniteBiTuple<V, ?>>, Serializable {
+        /** */
+        private Map<K, IgniteBiTuple<V, ?>> map = new ConcurrentHashMap<>();
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<K, IgniteBiTuple<V, ?>> clo, @Nullable Object... args)
+            throws CacheLoaderException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) throws CacheWriterException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteBiTuple<V, ?> load(K key) throws CacheLoaderException {
+            return map.get(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<K, IgniteBiTuple<V, ?>> loadAll(Iterable<? extends K> keys) throws CacheLoaderException {
+            Map<K, IgniteBiTuple<V, ?>> res = new HashMap<>();
+
+            for (K key : keys) {
+                IgniteBiTuple<V, ?> val = map.get(key);
+
+                if (val != null)
+                    res.put(key, val);
+            }
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends K, ? extends IgniteBiTuple<V, ?>> entry)
+            throws CacheWriterException {
+            map.put(entry.getKey(), entry.getValue());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeAll(Collection<Cache.Entry<? extends K, ? extends IgniteBiTuple<V, ?>>> entries)
+            throws CacheWriterException {
+            for (Cache.Entry<? extends K, ? extends IgniteBiTuple<V, ?>> e : entries)
+                map.put(e.getKey(), e.getValue());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            map.remove(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void deleteAll(Collection<?> keys) throws CacheWriterException {
+            for (Object key : keys)
+                map.remove(key);
+        }
+
+        /**
+         * Clear store.
+         */
+        public void clear() {
+            map.clear();
+        }
+    }
+}
\ No newline at end of file