You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2015/07/31 11:34:30 UTC

[1/9] incubator-ignite git commit: #ignite-gg-10610: Security hole if DataStreamer is used for populating the cache

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-946 a12775621 -> 13dd417e1


#ignite-gg-10610: Security hole if DataStreamer is used for populating the cache


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

Branch: refs/heads/ignite-946
Commit: 5288b2d8b882bbb86d69e1019821d51803685861
Parents: a127756
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jul 29 15:27:31 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jul 29 15:27:31 2015 +0300

----------------------------------------------------------------------
 .../datastreamer/DataStreamerImpl.java          | 22 ++++++++++++++++++++
 .../datastreamer/DataStreamerUpdateJob.java     | 20 +++++++++++++++++-
 2 files changed, 41 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5288b2d8/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 605f478..5fae676 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.stream.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
@@ -413,6 +414,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     @Override public IgniteFuture<?> addData(Collection<? extends Map.Entry<K, V>> entries) {
         A.notEmpty(entries, "entries");
 
+        checkSecurityPermission(SecurityPermission.CACHE_PUT);
+
         enterBusy();
 
         try {
@@ -520,6 +523,11 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     @Override public IgniteFuture<?> addData(K key, V val) {
         A.notNull(key, "key");
 
+        if (val == null)
+            checkSecurityPermission(SecurityPermission.CACHE_REMOVE);
+        else
+            checkSecurityPermission(SecurityPermission.CACHE_PUT);
+
         KeyCacheObject key0 = cacheObjProc.toCacheKeyObject(cacheObjCtx, key, true);
         CacheObject val0 = cacheObjProc.toCacheObject(cacheObjCtx, val, true);
 
@@ -980,6 +988,20 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     }
 
     /**
+     * Check permissions for streaming.
+     *
+     * @param perm Security permission.
+     * @throws org.apache.ignite.plugin.security.SecurityException If permissions are not enough for streaming.
+     */
+    private void checkSecurityPermission(SecurityPermission perm)
+        throws org.apache.ignite.plugin.security.SecurityException{
+        if (!ctx.security().enabled())
+            return;
+
+        ctx.security().authorize(cacheName, perm, null);
+    }
+
+    /**
      *
      */
     private class Buffer {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5288b2d8/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateJob.java
index 21ba3ac..9e0703a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerUpdateJob.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.stream.*;
 import org.jetbrains.annotations.*;
 
@@ -106,8 +107,13 @@ class DataStreamerUpdateJob implements GridPlainCallable<Object> {
 
                 CacheObject val = e.getValue();
 
-                if (val != null)
+                if (val != null) {
+                    checkSecurityPermission(SecurityPermission.CACHE_PUT);
+
                     val.finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader());
+                }
+                else
+                    checkSecurityPermission(SecurityPermission.CACHE_REMOVE);
             }
 
             if (unwrapEntries()) {
@@ -139,4 +145,16 @@ class DataStreamerUpdateJob implements GridPlainCallable<Object> {
     private boolean unwrapEntries() {
         return !(rcvr instanceof DataStreamerCacheUpdaters.InternalUpdater);
     }
+
+    /**
+     * @param perm Security permission.
+     * @throws org.apache.ignite.plugin.security.SecurityException If permission is not enough.
+     */
+    private void checkSecurityPermission(SecurityPermission perm)
+        throws org.apache.ignite.plugin.security.SecurityException {
+        if (!ctx.security().enabled())
+            return;
+
+        ctx.security().authorize(cacheName, perm, null);
+    }
 }


[6/9] incubator-ignite git commit: ignite-946: improving tests

Posted by nt...@apache.org.
ignite-946: improving tests


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

Branch: refs/heads/ignite-946
Commit: 76badc94242bc0ad650764966bf02cd42ec37f0b
Parents: c252cc1
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Jul 31 10:20:22 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jul 31 12:33:08 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/query/GridCacheQueryManager.java      | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/76badc94/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 400d997..5d3f6a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -845,7 +845,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                         try {
                             val = prj.localPeek(key, CachePeekModes.ONHEAP_ONLY, expiryPlc);
-
                         }
                         catch (IgniteCheckedException e) {
                             if (log.isDebugEnabled())


[5/9] incubator-ignite git commit: ignite-946: improving tests

Posted by nt...@apache.org.
ignite-946: improving tests


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

Branch: refs/heads/ignite-946
Commit: c252cc1b48ec34fa7a577d334cfc09d15b170ce6
Parents: 3d5cf6a
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Jul 31 10:19:46 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jul 31 12:33:07 2015 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryManager.java      |  1 +
 .../CacheVersionedEntryAbstractTest.java        | 24 ++++++-----
 .../CacheVersionedEntryLocalAtomicSelfTest.java | 40 -----------------
 ...nedEntryLocalAtomicSwapDisabledSelfTest.java | 45 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |  2 +-
 5 files changed, 60 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c252cc1b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 5d3f6a3..400d997 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -845,6 +845,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                         try {
                             val = prj.localPeek(key, CachePeekModes.ONHEAP_ONLY, expiryPlc);
+
                         }
                         catch (IgniteCheckedException e) {
                             if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c252cc1b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
index 951d05a..4cfacb7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
@@ -55,23 +55,25 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS
     public void testInvoke() throws Exception {
         Cache<Integer, String> cache = grid(0).cache(null);
 
-        final AtomicBoolean invoked = new AtomicBoolean(false);
+        final AtomicInteger invoked = new AtomicInteger();
 
-        cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
-            @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
-                throws EntryProcessorException {
+        for (int i = 0; i < ENTRIES_NUM; i++) {
+            cache.invoke(i, new EntryProcessor<Integer, String, Object>() {
+                @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
+                    throws EntryProcessorException {
 
-                invoked.set(true);
+                    invoked.incrementAndGet();
 
-                VersionedEntry<Integer, String> verEntry = entry.unwrap(VersionedEntry.class);
+                    VersionedEntry<Integer, String> verEntry = entry.unwrap(VersionedEntry.class);
 
-                checkVersionedEntry(verEntry);
+                    checkVersionedEntry(verEntry);
 
-                return entry;
-            }
-        });
+                    return entry;
+                }
+            });
+        }
 
-        assertTrue(invoked.get());
+        assert invoked.get() > 0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c252cc1b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java
deleted file mode 100644
index a340413..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.version;
-
-import org.apache.ignite.cache.*;
-
-/**
- *
- */
-public class CacheVersionedEntryLocalAtomicSelfTest extends CacheVersionedEntryAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return CacheMode.LOCAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return CacheAtomicityMode.ATOMIC;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c252cc1b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSwapDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSwapDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSwapDisabledSelfTest.java
new file mode 100644
index 0000000..b0035d1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSwapDisabledSelfTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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.version;
+
+import org.apache.ignite.cache.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryLocalAtomicSwapDisabledSelfTest extends CacheVersionedEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.LOCAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean swapEnabled() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c252cc1b/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 dd9c799..3ac7879 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
@@ -155,7 +155,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheReadThroughAtomicRestartSelfTest.class);
 
         // Versioned entry tests
-        suite.addTestSuite(CacheVersionedEntryLocalAtomicSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryLocalAtomicSwapDisabledSelfTest.class);
         suite.addTestSuite(CacheVersionedEntryLocalTransactionalSelfTest.class);
         suite.addTestSuite(CacheVersionedEntryPartitionedAtomicSelfTest.class);
         suite.addTestSuite(CacheVersionedEntryPartitionedTransactionalSelfTest.class);


[7/9] incubator-ignite git commit: ignite-946: supported VersionedEntry for IgniteCache.localEntries() when OFF_HEAP mode is used

Posted by nt...@apache.org.
ignite-946: supported VersionedEntry for IgniteCache.localEntries() when OFF_HEAP mode is used


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

Branch: refs/heads/ignite-946
Commit: 0ac76bc77e907715bfebc1e7e60a29d4ef7b5c67
Parents: 76badc9
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Jul 31 11:39:47 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jul 31 12:33:08 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheEntryImpl0.java       |  5 +++
 .../processors/cache/GridCacheSwapManager.java  |  8 ++++-
 .../cache/version/GridVersionedMapEntry.java    | 33 +++++++++++++++++++
 .../CacheVersionedEntryAbstractTest.java        | 34 ++++++--------------
 4 files changed, 55 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ac76bc7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl0.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl0.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl0.java
index d2b1923..a5e27d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl0.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl0.java
@@ -17,6 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import org.apache.ignite.cache.version.*;
+import org.apache.ignite.internal.processors.cache.version.*;
+
 import javax.cache.*;
 import java.util.*;
 
@@ -49,6 +52,8 @@ public class CacheEntryImpl0<K, V> implements Cache.Entry<K, V> {
     @Override public <T> T unwrap(Class<T> cls) {
         if(cls.isAssignableFrom(getClass()))
             return cls.cast(this);
+        else if (cls.isAssignableFrom(VersionedEntry.class) && e instanceof GridVersionedMapEntry)
+            return (T)new CacheVersionedEntryImpl<>(e.getKey(), e.getValue(), ((GridVersionedMapEntry)e).version());
 
         throw new IllegalArgumentException("Unwrapping to class is not supported: " + cls);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ac76bc7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index 9e9c958..0530c19 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -1513,7 +1513,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             @Override protected Map.Entry<K, V> onNext() {
                 final Map.Entry<byte[], byte[]> cur0 = it.next();
 
-                cur = new Map.Entry<K, V>() {
+                cur = new GridVersionedMapEntry<K, V>() {
                     @Override public K getKey() {
                         try {
                             KeyCacheObject key = cctx.toCacheKeyObject(cur0.getKey());
@@ -1538,6 +1538,12 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         }
                     }
 
+                    @Override public GridCacheVersion version() {
+                        GridCacheSwapEntry e = unmarshalSwapEntry(cur0.getValue());
+
+                        return e.version();
+                    }
+
                     @Override public V setValue(V val) {
                         throw new UnsupportedOperationException();
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ac76bc7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridVersionedMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridVersionedMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridVersionedMapEntry.java
new file mode 100644
index 0000000..f653fac
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridVersionedMapEntry.java
@@ -0,0 +1,33 @@
+/*
+ * 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.version;
+
+import java.util.*;
+
+/**
+ * This interface extends {@link java.util.Map.Entry} by adding the method that returns entry's
+ * {@link GridCacheVersion}.
+ */
+public interface GridVersionedMapEntry<K, V> extends Map.Entry<K, V> {
+    /**
+     * Gets entry version.
+     *
+     * @return Entry version.
+     */
+    public GridCacheVersion version();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ac76bc7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
index 4cfacb7..25a2a42 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
@@ -32,7 +32,7 @@ import java.util.concurrent.atomic.*;
  */
 public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractSelfTest {
     /** Entries number to store in a cache. */
-    private static final int ENTRIES_NUM = 1000;
+    private static final int ENTRIES_NUM = 500;
 
     /** {@inheritDoc} */
     @Override protected int gridCount() {
@@ -57,21 +57,19 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS
 
         final AtomicInteger invoked = new AtomicInteger();
 
-        for (int i = 0; i < ENTRIES_NUM; i++) {
-            cache.invoke(i, new EntryProcessor<Integer, String, Object>() {
-                @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
-                    throws EntryProcessorException {
+        cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
+            @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
+                throws EntryProcessorException {
 
-                    invoked.incrementAndGet();
+                invoked.incrementAndGet();
 
-                    VersionedEntry<Integer, String> verEntry = entry.unwrap(VersionedEntry.class);
+                VersionedEntry<Integer, String> verEntry = entry.unwrap(VersionedEntry.class);
 
-                    checkVersionedEntry(verEntry);
+                checkVersionedEntry(verEntry);
 
-                    return entry;
-                }
-            });
-        }
+                return entry;
+            }
+        });
 
         assert invoked.get() > 0;
     }
@@ -119,18 +117,6 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS
     /**
      * @throws Exception If failed.
      */
-    public void testIterator() throws Exception {
-        IgniteCache<Integer, String> cache = grid(0).cache(null);
-
-        Iterator<Cache.Entry<Integer, String>> entries = cache.iterator();
-
-        while (entries.hasNext())
-            checkVersionedEntry(entries.next().unwrap(VersionedEntry.class));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testLocalPeek() throws Exception {
         IgniteCache<Integer, String> cache = grid(0).cache(null);
 


[2/9] incubator-ignite git commit: #ignite-1175: Add test for dht local partition map.

Posted by nt...@apache.org.
#ignite-1175: Add test for dht local partition map.


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

Branch: refs/heads/ignite-946
Commit: 7ed4d15f16c71e1683fd659865653a383d99259e
Parents: 5288b2d
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jul 30 14:12:27 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jul 30 14:12:27 2015 +0300

----------------------------------------------------------------------
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java | 107 +++++++++++++++++++
 1 file changed, 107 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ed4d15f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
new file mode 100644
index 0000000..b04e41a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+/**
+ * Test for remove operation.
+ */
+public class CacheDhtLocalPartitionAfterRemoveSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setEvictSynchronized(false);
+        ccfg.setNearConfiguration(null);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMemoryUsage() throws Exception {
+        IgniteCache<TestKey, Integer> cache = grid(0).cache(null);
+
+        for (int i = 0; i < 1000; ++i)
+            cache.put(new TestKey("" + i), i);
+
+        for (int i = 0; i < 1000; ++i)
+            assert cache.getAndRemove(new TestKey("" + i)).equals(i);
+
+        assertEquals(0, cache.size());
+
+        int size = 0;
+
+        for (GridDhtLocalPartition p : dht(cache).topology().localPartitions()) {
+            int pSize = p.size();
+
+            size += pSize;
+        }
+
+        System.out.println("All size: " + size);
+    }
+
+    /**
+     * Test key.
+     */
+    private static class TestKey {
+        /** Key. */
+        private String key;
+
+        /**
+         * @param key Key.
+         */
+        public TestKey(String key) {
+            this.key = key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            if (obj == null || !(obj instanceof TestKey))
+                return false;
+
+            return key.equals(((TestKey)obj).key);
+        }
+    }
+}


[9/9] incubator-ignite git commit: ignite-946: formatting fixes

Posted by nt...@apache.org.
ignite-946: formatting fixes


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

Branch: refs/heads/ignite-946
Commit: 13dd417e1140a901247c75075fef2cd0aaef304a
Parents: d7f672d
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Jul 31 12:18:58 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jul 31 12:33:09 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/13dd417e/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 45ff619..ebcb908 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
@@ -609,16 +609,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         @Nullable IgniteCacheExpiryPolicy expirePlc)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         return innerGet0(tx,
-                         readSwap,
-                         readThrough,
-                         evt,
-                         unmarshal,
-                         updateMetrics,
-                         tmp,
-                         subjId,
-                         transformClo,
-                         taskName,
-                         expirePlc);
+            readSwap,
+            readThrough,
+            evt,
+            unmarshal,
+            updateMetrics,
+            tmp,
+            subjId,
+            transformClo,
+            taskName,
+            expirePlc);
     }
 
     /** {@inheritDoc} */


[8/9] incubator-ignite git commit: ignite-946: added documentation

Posted by nt...@apache.org.
ignite-946: added documentation


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

Branch: refs/heads/ignite-946
Commit: d7f672d026bdea471822507ca932024261fcde0d
Parents: 0ac76bc
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Jul 31 12:14:54 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jul 31 12:33:09 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/version/VersionedEntry.java    | 31 +++++++++++++++++++-
 1 file changed, 30 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d7f672d0/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java
index 6f9d8f6..e669f15 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java
@@ -17,11 +17,40 @@
 
 package org.apache.ignite.cache.version;
 
+import org.apache.ignite.*;
+
 import javax.cache.*;
+import javax.cache.processor.*;
 import java.util.*;
 
 /**
- * Cache entry along with version information.
+ * Cache entry that stores entry's version related information.
+ *
+ * To get a {@code VersionedEntry} of an {@link Cache.Entry} use {@link Cache.Entry#unwrap(Class)} by passing
+ * {@code VersionedEntry} class to it as the argument.
+ * <p>
+ * {@code VersionedEntry} is supported only for {@link Cache.Entry} returned by one of the following methods:
+ * <ul>
+ * <li>{@link Cache#invoke(Object, EntryProcessor, Object...)}</li>
+ * <li>{@link Cache#invokeAll(Set, EntryProcessor, Object...)}</li>
+ * <li>invoke and invokeAll methods of {@link IgniteCache}</li>
+ * <li>{@link IgniteCache#randomEntry()}</li>
+ * </ul>
+ * <p>
+ * {@code VersionedEntry} is not supported for {@link Cache#iterator()} because of performance reasons.
+ * {@link Cache#iterator()} loads entries from all the cluster nodes and to speed up the load version information
+ * is excluded from responses.
+ * <h2 class="header">Java Example</h2>
+ * <pre name="code" class="java">
+ * Cache<Integer, String> cache = grid(0).cache(null);
+ *
+ *  cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
+ *      public Object process(MutableEntry<Integer, String> entry, Object... arguments) throws EntryProcessorException {
+ *          VersionedEntry<Integer, String> verEntry = entry.unwrap(VersionedEntry.class);
+ *          return entry;
+ *       }
+ *   });
+ * </pre>
  */
 public interface VersionedEntry<K, V> extends Cache.Entry<K, V> {
     /**


[4/9] incubator-ignite git commit: ignite-946: added tests

Posted by nt...@apache.org.
ignite-946: added tests


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

Branch: refs/heads/ignite-946
Commit: 3d5cf6a1698c9a8624c8e878cd756662b8dfa4fd
Parents: 9fd86b1
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 30 16:03:18 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jul 31 12:33:06 2015 +0300

----------------------------------------------------------------------
 .../cache/version/CacheVersionedEntryImpl.java  |   2 -
 .../CacheVersionedEntryAbstractTest.java        | 184 +++++++++++++++++++
 .../CacheVersionedEntryLocalAtomicSelfTest.java |  40 ++++
 ...ersionedEntryLocalTransactionalSelfTest.java |  40 ++++
 ...edEntryPartitionedAtomicOffHeapSelfTest.java |  35 ++++
 ...VersionedEntryPartitionedAtomicSelfTest.java |  35 ++++
 ...PartitionedTransactionalOffHeapSelfTest.java |  36 ++++
 ...edEntryPartitionedTransactionalSelfTest.java |  35 ++++
 ...nedEntryReplicatedAtomicOffHeapSelfTest.java |  35 ++++
 ...eVersionedEntryReplicatedAtomicSelfTest.java |  35 ++++
 ...yReplicatedTransactionalOffHeapSelfTest.java |  36 ++++
 ...nedEntryReplicatedTransactionalSelfTest.java |  35 ++++
 .../testsuites/IgniteCacheTestSuite4.java       |  13 ++
 13 files changed, 559 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java
index 6d1e0c9..924eff9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java
@@ -48,8 +48,6 @@ public class CacheVersionedEntryImpl<K, V> extends CacheEntryImpl<K, V> implemen
     public CacheVersionedEntryImpl(K key, V val, GridCacheVersion ver) {
         super(key, val);
 
-        assert val == null;
-
         this.ver = ver;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
new file mode 100644
index 0000000..951d05a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.version.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+import javax.cache.*;
+import javax.cache.processor.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Versioned entry abstract test.
+ */
+public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractSelfTest {
+    /** Entries number to store in a cache. */
+    private static final int ENTRIES_NUM = 1000;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        Cache<Integer, String> cache = grid(0).cache(null);
+
+        for (int i = 0 ; i < ENTRIES_NUM; i++)
+            cache.put(i, "value_" + i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvoke() throws Exception {
+        Cache<Integer, String> cache = grid(0).cache(null);
+
+        final AtomicBoolean invoked = new AtomicBoolean(false);
+
+        cache.invoke(100, new EntryProcessor<Integer, String, Object>() {
+            @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
+                throws EntryProcessorException {
+
+                invoked.set(true);
+
+                VersionedEntry<Integer, String> verEntry = entry.unwrap(VersionedEntry.class);
+
+                checkVersionedEntry(verEntry);
+
+                return entry;
+            }
+        });
+
+        assertTrue(invoked.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAll() throws Exception {
+        Cache<Integer, String> cache = grid(0).cache(null);
+
+        Set<Integer> keys = new HashSet<>();
+
+        for (int i = 0; i < ENTRIES_NUM; i++)
+            keys.add(i);
+
+        final AtomicInteger invoked = new AtomicInteger();
+
+        cache.invokeAll(keys, new EntryProcessor<Integer, String, Object>() {
+            @Override public Object process(MutableEntry<Integer, String> entry, Object... arguments)
+                throws EntryProcessorException {
+
+                invoked.incrementAndGet();
+
+                VersionedEntry<Integer, String> verEntry = entry.unwrap(VersionedEntry.class);
+
+                checkVersionedEntry(verEntry);
+
+                return null;
+            }
+        });
+
+        assert invoked.get() > 0;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomEntry() throws Exception {
+        IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+        for (int i = 0; i < 5; i++)
+            checkVersionedEntry(cache.randomEntry().unwrap(VersionedEntry.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIterator() throws Exception {
+        IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+        Iterator<Cache.Entry<Integer, String>> entries = cache.iterator();
+
+        while (entries.hasNext())
+            checkVersionedEntry(entries.next().unwrap(VersionedEntry.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalPeek() throws Exception {
+        IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+        Iterable<Cache.Entry<Integer, String>> entries = offheapTiered(cache) ?
+            cache.localEntries(CachePeekMode.SWAP, CachePeekMode.OFFHEAP) :
+            cache.localEntries(CachePeekMode.ONHEAP);
+
+        for (Cache.Entry<Integer, String> entry : entries)
+            checkVersionedEntry(entry.unwrap(VersionedEntry.class));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testVersionComparision() throws Exception {
+        IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+        VersionedEntry<String, Integer> ver1 = cache.invoke(100,
+            new EntryProcessor<Integer, String, VersionedEntry<String, Integer>>() {
+                @Override public VersionedEntry<String, Integer> process(MutableEntry<Integer, String> entry,
+                    Object... arguments) throws EntryProcessorException {
+                        return entry.unwrap(VersionedEntry.class);
+                    }
+            });
+
+        cache.put(100, "new value 100");
+
+        VersionedEntry<String, Integer> ver2 = cache.invoke(100,
+            new EntryProcessor<Integer, String, VersionedEntry<String, Integer>>() {
+                @Override public VersionedEntry<String, Integer> process(MutableEntry<Integer, String> entry,
+                    Object... arguments) throws EntryProcessorException {
+                        return entry.unwrap(VersionedEntry.class);
+                    }
+            });
+
+        assert VersionedEntry.VERSIONS_COMPARATOR.compare(ver1, ver2) < 0;
+    }
+
+    /**
+     * @param entry Versioned entry.
+     */
+    private void checkVersionedEntry(VersionedEntry<Integer, String> entry) {
+        assertNotNull(entry);
+
+        assert entry.topologyVersion() > 0;
+        assert entry.order() > 0;
+        assert entry.nodeOrder() > 0;
+        assert entry.globalTime() > 0;
+
+        assertNotNull(entry.getKey());
+        assertNotNull(entry.getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java
new file mode 100644
index 0000000..a340413
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalAtomicSelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryLocalAtomicSelfTest extends CacheVersionedEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.LOCAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalTransactionalSelfTest.java
new file mode 100644
index 0000000..4833c2c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryLocalTransactionalSelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryLocalTransactionalSelfTest extends CacheVersionedEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.LOCAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicOffHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicOffHeapSelfTest.java
new file mode 100644
index 0000000..2f33d84
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicOffHeapSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryPartitionedAtomicOffHeapSelfTest extends CacheVersionedEntryPartitionedAtomicSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicSelfTest.java
new file mode 100644
index 0000000..f2197ad
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedAtomicSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryPartitionedAtomicSelfTest extends CacheVersionedEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalOffHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalOffHeapSelfTest.java
new file mode 100644
index 0000000..7494690
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalOffHeapSelfTest.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.version;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryPartitionedTransactionalOffHeapSelfTest extends
+    CacheVersionedEntryPartitionedTransactionalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalSelfTest.java
new file mode 100644
index 0000000..95a63c8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryPartitionedTransactionalSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryPartitionedTransactionalSelfTest extends CacheVersionedEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicOffHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicOffHeapSelfTest.java
new file mode 100644
index 0000000..dd3fd0c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicOffHeapSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryReplicatedAtomicOffHeapSelfTest extends CacheVersionedEntryReplicatedAtomicSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicSelfTest.java
new file mode 100644
index 0000000..fd9617d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedAtomicSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryReplicatedAtomicSelfTest extends CacheVersionedEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalOffHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalOffHeapSelfTest.java
new file mode 100644
index 0000000..d1bc5c3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalOffHeapSelfTest.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.version;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryReplicatedTransactionalOffHeapSelfTest extends
+    CacheVersionedEntryReplicatedTransactionalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalSelfTest.java
new file mode 100644
index 0000000..8d37e7b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryReplicatedTransactionalSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryReplicatedTransactionalSelfTest extends CacheVersionedEntryAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d5cf6a1/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 18b2409..dd9c799 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
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.integration.*;
+import org.apache.ignite.internal.processors.cache.version.*;
 
 /**
  * Test suite.
@@ -153,6 +154,18 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheReadThroughLocalAtomicRestartSelfTest.class);
         suite.addTestSuite(CacheReadThroughAtomicRestartSelfTest.class);
 
+        // Versioned entry tests
+        suite.addTestSuite(CacheVersionedEntryLocalAtomicSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryLocalTransactionalSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryPartitionedAtomicSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryPartitionedTransactionalSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryPartitionedAtomicOffHeapSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryPartitionedTransactionalOffHeapSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryReplicatedAtomicSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryReplicatedTransactionalSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryReplicatedAtomicOffHeapSelfTest.class);
+        suite.addTestSuite(CacheVersionedEntryPartitionedTransactionalOffHeapSelfTest.class);
+
         return suite;
     }
 }


[3/9] incubator-ignite git commit: ignite-946: introduced VersionedEntry, supported versioned entry for Cache.invoke/randomEntry/localEntries methods

Posted by nt...@apache.org.
ignite-946: introduced VersionedEntry, supported versioned entry for Cache.invoke/randomEntry/localEntries methods


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

Branch: refs/heads/ignite-946
Commit: 9fd86b15ef4940cf7b2214dff6bd59d57a8586a6
Parents: 7ed4d15
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 30 13:50:40 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jul 31 12:33:05 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/version/VersionedEntry.java    |  73 +++++++++++++
 .../ignite/cache/version/package-info.java      |  21 ++++
 .../processors/cache/CacheEntryImpl.java        |  20 ++++
 .../processors/cache/CacheInvokeEntry.java      |  24 ++++-
 .../cache/CacheVersionedEntryImpl.java          |  80 ---------------
 .../processors/cache/GridCacheAdapter.java      |  13 ++-
 .../processors/cache/GridCacheMapEntry.java     |  42 +++++---
 .../distributed/dht/GridDhtTxPrepareFuture.java |   2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   3 +-
 .../local/atomic/GridLocalAtomicCache.java      |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |   2 +-
 .../cache/transactions/IgniteTxEntry.java       |   3 +-
 .../transactions/IgniteTxLocalAdapter.java      |   3 +-
 .../cache/version/CacheVersionedEntryImpl.java  | 102 +++++++++++++++++++
 .../resources/META-INF/classnames.properties    |   2 +-
 15 files changed, 287 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.java
new file mode 100644
index 0000000..6f9d8f6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/version/VersionedEntry.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.cache.version;
+
+import javax.cache.*;
+import java.util.*;
+
+/**
+ * Cache entry along with version information.
+ */
+public interface VersionedEntry<K, V> extends Cache.Entry<K, V> {
+    /**
+     * Versions comparator.
+     */
+    public static final Comparator<VersionedEntry> VERSIONS_COMPARATOR = new Comparator<VersionedEntry>() {
+        @Override public int compare(VersionedEntry o1, VersionedEntry o2) {
+            int res = Integer.compare(o1.topologyVersion(), o2.topologyVersion());
+
+            if (res != 0)
+                return res;
+
+            res = Long.compare(o1.order(), o2.order());
+
+            if (res != 0)
+                return res;
+
+            return Integer.compare(o1.nodeOrder(), o2.nodeOrder());
+        }
+    };
+
+    /**
+     * Gets entry's topology version.
+     *
+     * @return Topology version plus number of seconds from the start time of the first grid node.
+     */
+    public int topologyVersion();
+
+    /**
+     * Gets entry's order.
+     *
+     * @return Version order.
+     */
+    public long order();
+
+    /**
+     * Gets entry's node order.
+     *
+     * @return Node order on which this version was assigned.
+     */
+    public int nodeOrder();
+
+    /**
+     * Gets entry's global time.
+     *
+     * @return Adjusted time.
+     */
+    public long globalTime();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/java/org/apache/ignite/cache/version/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/version/package-info.java b/modules/core/src/main/java/org/apache/ignite/cache/version/package-info.java
new file mode 100644
index 0000000..9aeaba2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/version/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains cache version based implementations.
+ */
+package org.apache.ignite.cache.version;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
index 3bd7ef4..98f3616 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
@@ -17,6 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import org.apache.ignite.cache.version.*;
+import org.apache.ignite.internal.processors.cache.version.*;
+
 import javax.cache.*;
 import java.io.*;
 
@@ -33,6 +36,9 @@ public class CacheEntryImpl<K, V> implements Cache.Entry<K, V>, Externalizable {
     /** */
     private V val;
 
+    /** Entry version. */
+    private GridCacheVersion ver;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -49,6 +55,17 @@ public class CacheEntryImpl<K, V> implements Cache.Entry<K, V>, Externalizable {
         this.val = val;
     }
 
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param ver Entry version.
+     */
+    public CacheEntryImpl(K key, V val, GridCacheVersion ver) {
+        this.key = key;
+        this.val = val;
+        this.ver = ver;
+    }
+
     /** {@inheritDoc} */
     @Override public K getKey() {
         return key;
@@ -65,6 +82,9 @@ public class CacheEntryImpl<K, V> implements Cache.Entry<K, V>, Externalizable {
         if(cls.isAssignableFrom(getClass()))
             return cls.cast(this);
 
+        if (ver != null && cls.isAssignableFrom(VersionedEntry.class))
+            return (T)new CacheVersionedEntryImpl<>(key, val, ver);
+
         throw new IllegalArgumentException("Unwrapping to class is not supported: " + cls);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java
index 2817748..e6f8d4e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import org.apache.ignite.cache.version.*;
+import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -35,17 +37,23 @@ public class CacheInvokeEntry<K, V> extends CacheLazyEntry<K, V> implements Muta
     /** */
     private V oldVal;
 
+    /** Entry version. */
+    private GridCacheVersion ver;
+
     /**
      * @param cctx Cache context.
      * @param keyObj Key cache object.
      * @param valObj Cache object value.
+     * @param ver Entry version.
      */
     public CacheInvokeEntry(GridCacheContext cctx,
         KeyCacheObject keyObj,
-        @Nullable CacheObject valObj) {
+        @Nullable CacheObject valObj,
+        GridCacheVersion ver) {
         super(cctx, keyObj, valObj);
 
         this.hadVal = valObj != null;
+        this.ver = ver;
     }
 
     /** 
@@ -54,15 +62,18 @@ public class CacheInvokeEntry<K, V> extends CacheLazyEntry<K, V> implements Muta
      * @param key Key value.
      * @param valObj Value cache object.
      * @param val Value.
+     * @param ver Entry version.
      */
     public CacheInvokeEntry(GridCacheContext<K, V> ctx,
         KeyCacheObject keyObj,
         @Nullable K key,
         @Nullable CacheObject valObj,
-        @Nullable V val) {
+        @Nullable V val,
+        GridCacheVersion ver) {
         super(ctx, keyObj, key, valObj, val);
 
         this.hadVal = valObj != null || val != null;
+        this.ver = ver;
     }
 
     /** {@inheritDoc} */
@@ -108,6 +119,15 @@ public class CacheInvokeEntry<K, V> extends CacheLazyEntry<K, V> implements Muta
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unwrap(Class<T> cls) {
+        if (cls.isAssignableFrom(VersionedEntry.class))
+            return (T)new CacheVersionedEntryImpl<>(getKey(), getValue(), ver);
+
+        return super.unwrap(cls);
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheInvokeEntry.class, this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheVersionedEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheVersionedEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheVersionedEntryImpl.java
deleted file mode 100644
index 59394f5..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheVersionedEntryImpl.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- *
- */
-public class CacheVersionedEntryImpl<K, V> extends CacheEntryImpl<K, V> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Version. */
-    private Object ver;
-
-    /**
-     * Required by {@link Externalizable}.
-     */
-    public CacheVersionedEntryImpl() {
-        // No-op.
-    }
-
-    /**
-     * @param key Key.
-     * @param val Value (always null).
-     * @param ver Version.
-     */
-    public CacheVersionedEntryImpl(K key, V val, Object ver) {
-        super(key, val);
-
-        assert val == null;
-
-        this.ver = ver;
-    }
-
-    /**
-     * @return Version.
-     */
-    @Nullable public Object version() {
-        return ver;
-    }
-
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        super.writeExternal(out);
-
-        out.writeObject(ver);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        super.readExternal(in);
-
-        ver = in.readObject();
-    }
-
-    /** {@inheritDoc} */
-    public String toString() {
-        return "VersionedEntry [key=" + getKey() + ", val=" + getValue() + ", ver=" + ver + ']';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/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 94bcc93..d125382 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
@@ -3691,7 +3691,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     try {
                         V val = localPeek(lazyEntry.getKey(), CachePeekModes.ONHEAP_ONLY, expiryPlc);
 
-                        return new CacheEntryImpl<>(lazyEntry.getKey(), val);
+                        GridCacheVersion ver = null;
+
+                        try {
+                            ver = lazyEntry.unwrap(GridCacheVersion.class);
+                        }
+                        catch (IllegalArgumentException e) {
+                            log.error("Failed to unwrap entry version information", e);
+                        }
+
+                        return new CacheEntryImpl<>(lazyEntry.getKey(), val, ver);
                     }
                     catch (IgniteCheckedException e) {
                         throw CU.convertToCacheException(e);
@@ -4614,7 +4623,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 val0 = ctx.unwrapPortableIfNeeded(val0, true);
             }
 
-            return new CacheEntryImpl<>((K)key0, (V)val0);
+            return new CacheEntryImpl<>((K)key0, (V)val0, entry.version());
         }
         catch (GridCacheFilterFailedException ignore) {
             assert false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/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 f85a18b..45ff619 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
@@ -609,16 +609,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         @Nullable IgniteCacheExpiryPolicy expirePlc)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         return innerGet0(tx,
-            readSwap,
-            readThrough,
-            evt,
-            unmarshal,
-            updateMetrics,
-            tmp,
-            subjId,
-            transformClo,
-            taskName,
-            expirePlc);
+                         readSwap,
+                         readThrough,
+                         evt,
+                         unmarshal,
+                         updateMetrics,
+                         tmp,
+                         subjId,
+                         transformClo,
+                         taskName,
+                         expirePlc);
     }
 
     /** {@inheritDoc} */
@@ -1385,7 +1385,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 assert entryProcessor != null;
 
-                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry<>(cctx, key, old);
+                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry<>(cctx, key, old, this.ver);
 
                 try {
                     Object computed = entryProcessor.process(entry, invokeArgs);
@@ -1653,7 +1653,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         oldVal = rawGetOrUnmarshalUnlocked(true);
 
-                        CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal);
+                        CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal, this.ver);
 
                         try {
                             Object computed = entryProcessor.process(entry, invokeArgs);
@@ -1878,7 +1878,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 EntryProcessor<Object, Object, ?> entryProcessor = (EntryProcessor<Object, Object, ?>)writeObj;
 
-                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal);
+                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry(cctx, key, oldVal, this.ver);
 
                 try {
                     Object computed = entryProcessor.process(entry, invokeArgs);
@@ -3531,7 +3531,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 val = rawGetOrUnmarshal(false);
 
             return new CacheEntryImpl<>(key.<K>value(cctx.cacheObjectContext(), false),
-                CU.<V>value(val, cctx, false));
+                CU.<V>value(val, cctx, false), ver);
         }
         catch (GridCacheFilterFailedException ignored) {
             throw new IgniteException("Should never happen.");
@@ -3593,6 +3593,15 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         return new CacheVersionedEntryImpl<>(key.<K>value(cctx.cacheObjectContext(), false), null, ver);
     }
 
+    /**
+     * @return Entry which holds key, value and version.
+     */
+    private synchronized <K, V> CacheVersionedEntryImpl<K, V> wrapVersionedWithValue() {
+        V val = this.val == null ? null : this.val.<V>value(cctx.cacheObjectContext(), false);
+
+        return new CacheVersionedEntryImpl<>(key.<K>value(cctx.cacheObjectContext(), false), val, ver);
+    }
+
     /** {@inheritDoc} */
     @Override public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer,
         @Nullable CacheEntryPredicate[] filter) throws IgniteCheckedException {
@@ -4020,7 +4029,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 return (T)wrapEviction();
 
             if (cls.isAssignableFrom(CacheVersionedEntryImpl.class))
-                return (T)wrapVersioned();
+                return cls == CacheVersionedEntryImpl.class ? (T)wrapVersioned() : (T)wrapVersionedWithValue();
+
+            if (cls.isAssignableFrom(GridCacheVersion.class))
+                return (T)ver;
 
             if (cls.isAssignableFrom(GridCacheMapEntry.this.getClass()))
                 return (T)GridCacheMapEntry.this;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/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 fbc8c84..9bd5de2 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
@@ -324,7 +324,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                              for (T2<EntryProcessor<Object, Object, Object>, Object[]> t : txEntry.entryProcessors()) {
                                 try {
                                     CacheInvokeEntry<Object, Object> invokeEntry =
-                                        new CacheInvokeEntry<>(txEntry.context(), key, val);
+                                        new CacheInvokeEntry<>(txEntry.context(), key, val, txEntry.cached().version());
 
                                     EntryProcessor<Object, Object, Object> processor = t.get1();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/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 0a21979..5dff4ea 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
@@ -1313,7 +1313,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     Object oldVal = null;
                     Object updatedVal = null;
 
-                    CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry(ctx, entry.key(), old);
+                    CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry(ctx, entry.key(), old,
+                        entry.version());
 
                     CacheObject updated;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/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 bcbdec4..8dd3276 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
@@ -1057,7 +1057,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
                         Object oldVal = null;
 
-                        CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry<>(ctx, entry.key(), old);
+                        CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry<>(ctx, entry.key(), old,
+                            entry.version());
 
                         CacheObject updated;
                         Object updatedVal = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 7190249..0d14012 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -1230,7 +1230,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
                 for (T2<EntryProcessor<Object, Object, Object>, Object[]> t : txEntry.entryProcessors()) {
                     CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry(txEntry.context(),
-                        txEntry.key(), key, cacheVal, val);
+                        txEntry.key(), key, cacheVal, val, txEntry.cached().version());
 
                     try {
                         EntryProcessor<Object, Object, Object> processor = t.get1();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/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 247d350..7f06380 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
@@ -573,7 +573,8 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
 
         for (T2<EntryProcessor<Object, Object, Object>, Object[]> t : entryProcessors()) {
             try {
-                CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry(ctx, key, keyVal, cacheVal, val);
+                CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry(ctx, key, keyVal, cacheVal, val,
+                    entry.version());
 
                 EntryProcessor processor = t.get1();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/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 0a61b1a..d8797fe 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
@@ -2522,7 +2522,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
             for (T2<EntryProcessor<Object, Object, Object>, Object[]> t : txEntry.entryProcessors()) {
                 CacheInvokeEntry<Object, Object> invokeEntry =
-                    new CacheInvokeEntry(txEntry.context(), txEntry.key(), key0, cacheVal, val0);
+                    new CacheInvokeEntry(txEntry.context(), txEntry.key(), key0, cacheVal, val0,
+                        txEntry.cached().version());
 
                 EntryProcessor<Object, Object, ?> entryProcessor = t.get1();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java
new file mode 100644
index 0000000..6d1e0c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryImpl.java
@@ -0,0 +1,102 @@
+/*
+ * 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.version;
+
+import org.apache.ignite.cache.version.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ *
+ */
+public class CacheVersionedEntryImpl<K, V> extends CacheEntryImpl<K, V> implements VersionedEntry<K, V> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Version. */
+    private GridCacheVersion ver;
+
+    /**
+     * Required by {@link Externalizable}.
+     */
+    public CacheVersionedEntryImpl() {
+        // No-op.
+    }
+
+    /**
+     * @param key Key.
+     * @param val Value (always null).
+     * @param ver Version.
+     */
+    public CacheVersionedEntryImpl(K key, V val, GridCacheVersion ver) {
+        super(key, val);
+
+        assert val == null;
+
+        this.ver = ver;
+    }
+
+    /**
+     * @return Version.
+     */
+    @Nullable public GridCacheVersion version() {
+        return ver;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int topologyVersion() {
+        return ver.topologyVersion();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int nodeOrder() {
+        return ver.nodeOrder();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long order() {
+        return ver.order();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long globalTime() {
+        return ver.globalTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        out.writeObject(ver);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        ver = (GridCacheVersion)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    public String toString() {
+        return "VersionedEntry [key=" + getKey() + ", val=" + getValue() + ", topVer=" + ver.topologyVersion() +
+            ", nodeOrder=" + ver.nodeOrder() + ", order=" + ver.order() + ", globalTime=" + ver.globalTime() + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fd86b15/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index b3eed46..ff75b02 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -289,7 +289,7 @@ org.apache.ignite.internal.processors.cache.CacheOperationContext
 org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException
 org.apache.ignite.internal.processors.cache.CacheStorePartialUpdateException
 org.apache.ignite.internal.processors.cache.CacheType
-org.apache.ignite.internal.processors.cache.CacheVersionedEntryImpl
+org.apache.ignite.internal.processors.cache.version.CacheVersionedEntryImpl
 org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQueryFutureIterator
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest