You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/03/02 09:11:29 UTC

[10/19] ignite git commit: ignite-2719 Value is not copied in entry processor

ignite-2719 Value is not copied in entry processor


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

Branch: refs/heads/ignite-1232
Commit: 9777198aa25b9127764c5c539d77c39f5a770661
Parents: 8917269
Author: agura <ag...@gridgain.com>
Authored: Sun Feb 28 19:43:58 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Mar 1 13:54:51 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheLazyEntry.java        |   3 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  | 213 +++++++++
 .../GridCacheOnCopyFlagAbstractSelfTest.java    | 473 +++++++++++--------
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 4 files changed, 491 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
index 30933e5..6ec17c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
@@ -114,9 +114,10 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
      * @param keepBinary Flag to keep binary if needed.
      * @return the value corresponding to this entry
      */
+    @SuppressWarnings("unchecked")
     public V getValue(boolean keepBinary) {
         if (val == null)
-            val = (V)cctx.unwrapBinaryIfNeeded(valObj, keepBinary, false);
+            val = (V)cctx.unwrapBinaryIfNeeded(valObj, keepBinary, true);
 
         return val;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
new file mode 100644
index 0000000..21395e6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for value copy in entry processor.
+ */
+public class CacheEntryProcessorCopySelfTest extends GridCommonAbstractTest {
+    /** Old value. */
+    private static final int OLD_VAL = 100;
+
+    /** New value. */
+    private static final int NEW_VAL = 200;
+
+    /** Empty array. */
+    private static final int[] EMPTY_ARR = new int[0];
+
+    /** Deserializations counter. */
+    private static final AtomicInteger cnt = new AtomicInteger();
+
+    /** p2p enabled. */
+    private boolean p2pEnabled;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setIncludeEventTypes(EMPTY_ARR);
+
+        cfg.setPeerClassLoadingEnabled(p2pEnabled);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMutableEntryWithP2PEnabled() throws Exception {
+        doTestMutableEntry(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMutableEntryWithP2PDisabled() throws Exception {
+        doTestMutableEntry(false);
+    }
+
+    /**
+     *
+     */
+    private void doTestMutableEntry(boolean p2pEnabled) throws Exception {
+        this.p2pEnabled = p2pEnabled;
+
+        Ignite grid = startGrid();
+
+        assertEquals(p2pEnabled, grid.configuration().isPeerClassLoadingEnabled());
+
+        try {
+            // One deserialization due to copyOnRead == true.
+            doTest(true, false, OLD_VAL, 1);
+
+            // One deserialization due to copyOnRead == true.
+            // Additional deserialization in case p2p enabled due to storeValue == true on update entry.
+            doTest(true, true, NEW_VAL, p2pEnabled ? 2 : 1);
+
+            // No deserialization.
+            doTest(false, false, NEW_VAL, 0);
+
+            // One deserialization due to storeValue == true.
+            doTest(false, true, NEW_VAL, 1);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @param cpOnRead Copy on read.
+     * @param mutate Mutate.
+     * @param expVal Expected value.
+     * @param expCnt Expected deserializations count.
+     */
+    @SuppressWarnings("unchecked")
+    private void doTest(boolean cpOnRead, final boolean mutate, int expVal, int expCnt) throws Exception {
+
+        Ignite ignite = grid();
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setCopyOnRead(cpOnRead);
+        ccfg.setNearConfiguration(null);
+
+        IgniteCache<Integer, Value> cache = null;
+
+        try {
+            cache = ignite.createCache(ccfg);
+
+            cache.put(0, new Value(OLD_VAL));
+
+            cache.get(0);
+
+            cnt.set(0);
+
+            cache.invoke(0, new CacheEntryProcessor<Integer, Value, Object>() {
+                @Override public Object process(MutableEntry<Integer, Value> entry, Object... args) {
+                    Value val = entry.getValue();
+
+                    val.i = NEW_VAL;
+
+                    if (mutate)
+                        entry.setValue(val);
+
+                    return null;
+                }
+            });
+
+            CacheObject obj = ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(0).peekVisibleValue();
+
+            int actCnt = cnt.get();
+
+            if (obj instanceof BinaryObject)
+                if (cpOnRead)
+                    assertEquals(expVal, (int)((BinaryObject)obj).field("i"));
+                else
+                    assertEquals(expVal, ((Value)U.field(obj, "obj")).i);
+            else {
+                if (storeValue(cache))
+                    assertEquals(expVal, U.<Value>field(obj, "val").i);
+                else
+                    assertEquals(expVal, CU.<Value>value(obj, ((IgniteCacheProxy)cache).context(), false).i);
+            }
+
+            assertEquals(expCnt, actCnt);
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     */
+    private static boolean storeValue(IgniteCache cache) {
+        return ((IgniteCacheProxy)cache).context().cacheObjectContext().storeValue();
+    }
+
+    /**
+     *
+     */
+    private static class Value implements Externalizable {
+        /**  */
+        private int i;
+
+        /**
+         * Default constructor (required by Externalizable).
+         */
+        public Value() {
+        }
+
+        /**
+         * @param i I.
+         */
+        public Value(int i) {
+            this.i = i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(i);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            i = in.readInt();
+
+            cnt.incrementAndGet();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
index 6f73e0f..8d8aa02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
@@ -32,12 +28,17 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheInterceptorAdapter;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
 import static org.junit.Assert.assertNotEquals;
@@ -45,7 +46,7 @@ import static org.junit.Assert.assertNotEquals;
 /**
  * Tests that cache value is copied for get, interceptor and invoke closure.
  */
-public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstractSelfTest {
+public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCommonAbstractTest {
     /** */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
@@ -61,36 +62,21 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
     /** */
     private static boolean noInterceptor;
 
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        interceptor = new Interceptor();
-
-        super.beforeTestsStarted();
-
-        awaitPartitionMapExchange();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        noInterceptor = false;
-    }
+    /** p2p enabled. */
+    private boolean p2pEnabled;
 
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
+    /**
+     * Returns cache mode for tests.
+     * @return cache mode.
+     */
+    protected abstract CacheMode cacheMode();
 
-        interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>());
+    /**
+     * Returns cache atomicity mode for cache.
+     * @return cache atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
 
-        for (int i = 0; i < gridCount(); i++)
-            jcache(i, null).localClearAll(keySet(jcache(i, null)));
-    }
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -102,7 +88,7 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
 
         c.setDiscoverySpi(spi);
 
-        c.setPeerClassLoadingEnabled(false);
+        c.setPeerClassLoadingEnabled(p2pEnabled);
 
         c.getTransactionConfiguration().setTxSerializableEnabled(true);
 
@@ -111,12 +97,12 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+    protected CacheConfiguration cacheConfiguration() throws Exception {
+        CacheConfiguration ccfg = defaultCacheConfiguration();
 
         assertTrue(ccfg.isCopyOnRead());
 
-        assertNotNull(interceptor);
+        interceptor = new Interceptor();
 
         ccfg.setInterceptor(interceptor);
 
@@ -127,288 +113,370 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
         return ccfg;
     }
 
-    /** {@inheritDoc} */
-    @Override protected abstract CacheAtomicityMode atomicityMode();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCopyOnReadFlagP2PEnabled() throws Exception {
+        doTest(true);
+    }
 
     /**
      * @throws Exception If failed.
      */
-    public void testInterceptor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).cache(null);
+    public void testCopyOnReadFlagP2PDisbaled() throws Exception {
+        doTest(false);
+    }
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            final TestValue val = new TestValue(i);
-            final TestKey key = new TestKey(i, i);
+    /**
+     * @param p2pEnabled P 2 p enabled.
+     */
+    private void doTest(boolean p2pEnabled) throws Exception {
+        this.p2pEnabled = p2pEnabled;
 
-            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
-                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
+        IgniteEx grid = startGrid(0);
 
-                    assertSame(entry.getValue(), entry.getValue());
-                    assertSame(entry.getKey(), entry.getKey());
+        assertEquals(p2pEnabled, grid.configuration().isPeerClassLoadingEnabled());
 
-                    // Try change value.
-                    entry.getValue().val(WRONG_VALUE);
-                }
-            });
+        try {
+            interceptor();
+            invokeAndInterceptor();
+            putGet();
+            putGetByteArray();
+            putGetKnownImmutable();
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
 
-            cache.put(key, val);
+    /**
+     * @throws Exception If failed.
+     */
+    private void interceptor() throws Exception {
+        noInterceptor = false;
 
-            Cache.Entry<Object, Object> entry = grid(0).cache(null).localEntries().iterator().next();
+        IgniteCache<TestKey, TestValue> cache = grid(0).createCache(cacheConfiguration());
 
-            // Check thar internal entry wasn't changed.
-            assertEquals(i, ((TestKey)entry.getKey()).field());
-            assertEquals(i, ((TestValue)entry.getValue()).val());
+        try {
+            for (int i = 0; i < ITER_CNT; i++) {
+                final TestValue val = new TestValue(i);
+                final TestKey key = new TestKey(i, i);
 
-            final TestValue newTestVal = new TestValue(-i);
+                interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
+                    @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
 
-            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
-                @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(val, entry.getValue());
+                        assertSame(entry.getValue(), entry.getValue());
+                        assertSame(entry.getKey(), entry.getKey());
 
-                    assertEquals(newTestVal, newVal);
+                        // Try change value.
+                        entry.getValue().val(WRONG_VALUE);
+                    }
+                });
 
-                    // Try change value.
-                    entry.getValue().val(WRONG_VALUE);
+                cache.put(key, val);
 
-                    return newVal;
-                }
+                CacheObject obj =
+                    ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
 
-                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
+                // Check thar internal entry wasn't changed.
+                if (storeValue(cache))
+                    assertEquals(i, ((TestValue)U.field(obj, "val")).val());
+                else
+                    assertEquals(i, CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
 
-                    assertSame(entry.getValue(), entry.getValue());
-                    assertSame(entry.getKey(), entry.getKey());
+                final TestValue newTestVal = new TestValue(-i);
 
-                    // Try change value.
-                    entry.getValue().val(WRONG_VALUE);
-                }
-            });
+                interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
+                    @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
+                        assertNotSame(key, entry.getKey());
+                        assertNotSame(val, entry.getValue());
 
-            cache.put(key, newTestVal);
+                        assertEquals(newTestVal, newVal);
 
-            entry = grid(0).cache(null).localEntries().iterator().next();
+                        // Try change value.
+                        entry.getValue().val(WRONG_VALUE);
 
-            // Check thar internal entry wasn't changed.
-            assertEquals(i, ((TestKey)entry.getKey()).field());
-            assertEquals(-i, ((TestValue)entry.getValue()).val());
+                        return newVal;
+                    }
 
-            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
-                @Override public IgniteBiTuple onBeforeRemove(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(newTestVal, entry.getValue());
+                    @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
 
-                    return super.onBeforeRemove(entry);
-                }
+                        assertSame(entry.getValue(), entry.getValue());
+                        assertSame(entry.getKey(), entry.getKey());
 
-                @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(newTestVal, entry.getValue());
-                }
-            });
+                        // Try change value.
+                        entry.getValue().val(WRONG_VALUE);
+                    }
+                });
+
+                cache.put(key, newTestVal);
+
+                obj = ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
+
+                // Check thar internal entry wasn't changed.
+                if (storeValue(cache))
+                    assertEquals(-i, ((TestValue)U.field(obj, "val")).val());
+                else
+                    assertEquals(-i, CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
+
+                interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
+                    @Override public IgniteBiTuple onBeforeRemove(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
+                        assertNotSame(newTestVal, entry.getValue());
+
+                        return super.onBeforeRemove(entry);
+                    }
 
-            cache.remove(key);
+                    @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
+                        assertNotSame(newTestVal, entry.getValue());
+                    }
+                });
+
+                cache.remove(key);
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testInvokeAndInterceptor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).cache(null);
-
-        for (int i = 0; i < ITER_CNT; i++)
-            cache.put(new TestKey(i, i), new TestValue(i));
-
-        interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>(){
-            @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
-                // Check that we have correct value and key.
-                assertEquals(entry.getKey().key(), entry.getKey().field());
+    private void invokeAndInterceptor() throws Exception {
+        noInterceptor = false;
 
-                // Try changed entry.
-                entry.getValue().val(WRONG_VALUE);
+        IgniteCache<TestKey, TestValue> cache = grid(0).createCache(cacheConfiguration());
 
-                return super.onBeforePut(entry, newVal);
-            }
+        try {
+            for (int i = 0; i < ITER_CNT; i++)
+                cache.put(new TestKey(i, i), new TestValue(i));
 
-            @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                assertEquals(entry.getKey().key(), entry.getKey().field());
+            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>(){
+                @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
+                    // Check that we have correct value and key.
+                    assertEquals(entry.getKey().key(), entry.getKey().field());
 
-                entry.getValue().val(WRONG_VALUE);
+                    // Try changed entry.
+                    entry.getValue().val(WRONG_VALUE);
 
-                super.onAfterPut(entry);
-            }
-        });
+                    return super.onBeforePut(entry, newVal);
+                }
 
-        for (int i = 0; i < ITER_CNT; i++)
-            cache.invoke(new TestKey(i, i), new EntryProcessor<TestKey, TestValue, Object>() {
-                @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments)
-                    throws EntryProcessorException {
-                    // Check that we have correct value.
-                    assertEquals(entry.getKey().key(), entry.getValue().val());
+                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                    assertEquals(entry.getKey().key(), entry.getKey().field());
 
-                    // Try changed entry.
                     entry.getValue().val(WRONG_VALUE);
 
-                    return -1;
+                    super.onAfterPut(entry);
                 }
             });
 
-        // Check that entries weren't changed.
-        for (Cache.Entry<Object, Object> e : grid(0).cache(null).localEntries()) {
-            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
-            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
+            for (int i = 0; i < ITER_CNT; i++) {
+                TestKey key = new TestKey(i, i);
+
+                cache.invoke(key, new EntryProcessor<TestKey, TestValue, Object>() {
+                    @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments)
+                        throws EntryProcessorException {
+                        TestValue val = entry.getValue();
+
+                        // Check that we have correct value.
+                        assertEquals(entry.getKey().key(), val.val());
+
+                        // Try changed entry.
+                        val.val(WRONG_VALUE);
+
+                        return -1;
+                    }
+                });
+
+                CacheObject obj =
+                    ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
+
+                if (storeValue(cache))
+                    assertNotEquals(WRONG_VALUE, ((TestValue)U.field(obj, "val")).val());
+                else
+                    assertNotEquals(WRONG_VALUE,
+                        CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutGet() throws Exception {
+    private void putGet() throws Exception {
         noInterceptor = true;
 
-        IgniteCache<TestKey, TestValue> cache = grid(0).cache(null);
+        IgniteCache<TestKey, TestValue> cache = grid(0).createCache(cacheConfiguration());
 
-        Map<TestKey, TestValue> map = new HashMap<>();
+        try {
+            Map<TestKey, TestValue> map = new HashMap<>();
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            TestKey key = new TestKey(i, i);
-            TestValue val = new TestValue(i);
+            for (int i = 0; i < ITER_CNT; i++) {
+                TestKey key = new TestKey(i, i);
+                TestValue val = new TestValue(i);
 
-            cache.put(key, val);
+                cache.put(key, val);
 
-            map.put(key, val);
-        }
+                map.put(key, val);
+            }
 
-        GridCacheAdapter cache0 = internalCache(cache);
+            GridCacheAdapter cache0 = internalCache(cache);
 
-        GridCacheContext cctx = cache0.context();
+            GridCacheContext cctx = cache0.context();
 
-        boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
+            boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
 
-        for (Map.Entry<TestKey, TestValue> e : map.entrySet()) {
-            GridCacheEntryEx entry = cache0.peekEx(e.getKey());
+            for (Map.Entry<TestKey, TestValue> e : map.entrySet()) {
+                GridCacheEntryEx entry = cache0.peekEx(e.getKey());
 
-            assertNotNull("No entry for key: " + e.getKey(), entry);
+                assertNotNull("No entry for key: " + e.getKey(), entry);
 
-            TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
+                TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(key0, e.getKey());
+                assertNotSame(key0, e.getKey());
 
-            TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
+                TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
 
-            if (!binary)
-                assertSame(key0, key1);
-            else
-                assertNotSame(key0, key1);
+                if (!binary)
+                    assertSame(key0, key1);
+                else
+                    assertNotSame(key0, key1);
 
-            TestValue val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
+                TestValue val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(val0, e.getValue());
+                assertNotSame(val0, e.getValue());
 
-            TestValue val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
+                TestValue val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
 
-            assertNotSame(val0, val1);
+                assertNotSame(val0, val1);
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutGetByteArray() throws Exception {
+    private void putGetByteArray() throws Exception {
         noInterceptor = true;
 
-        IgniteCache<TestKey, byte[]> cache = grid(0).cache(null);
+        IgniteCache<TestKey, byte[]> cache = grid(0).createCache(cacheConfiguration());
 
-        Map<TestKey, byte[]> map = new HashMap<>();
+        try {
+            Map<TestKey, byte[]> map = new HashMap<>();
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            TestKey key = new TestKey(i, i);
-            byte[] val = new byte[10];
+            for (int i = 0; i < ITER_CNT; i++) {
+                TestKey key = new TestKey(i, i);
+                byte[] val = new byte[10];
 
-            cache.put(key, val);
+                cache.put(key, val);
 
-            map.put(key, val);
-        }
+                map.put(key, val);
+            }
 
-        GridCacheAdapter cache0 = internalCache(cache);
+            GridCacheAdapter cache0 = internalCache(cache);
 
-        GridCacheContext cctx = cache0.context();
+            GridCacheContext cctx = cache0.context();
 
-        boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
+            boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
 
-        for (Map.Entry<TestKey, byte[]> e : map.entrySet()) {
-            GridCacheEntryEx entry = cache0.peekEx(e.getKey());
+            for (Map.Entry<TestKey, byte[]> e : map.entrySet()) {
+                GridCacheEntryEx entry = cache0.peekEx(e.getKey());
 
-            assertNotNull("No entry for key: " + e.getKey(), entry);
+                assertNotNull("No entry for key: " + e.getKey(), entry);
 
-            TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
+                TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(key0, e.getKey());
+                assertNotSame(key0, e.getKey());
 
-            TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
+                TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
 
-            if (!binary)
-                assertSame(key0, key1);
-            else
-                assertNotSame(key0, key1);
+                if (!binary)
+                    assertSame(key0, key1);
+                else
+                    assertNotSame(key0, key1);
 
-            byte[] val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
+                byte[] val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(val0, e.getValue());
+                assertNotSame(val0, e.getValue());
 
-            byte[] val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
+                byte[] val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
 
-            assertNotSame(val0, val1);
+                assertNotSame(val0, val1);
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutGetKnownImmutable() throws Exception {
+    private void putGetKnownImmutable() throws Exception {
         noInterceptor = true;
 
-        IgniteCache<String, Long> cache = grid(0).cache(null);
+        IgniteCache<String, Long> cache = grid(0).createCache(cacheConfiguration());
 
-        Map<String, Long> map = new HashMap<>();
+        try {
+            Map<String, Long> map = new HashMap<>();
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            String key = String.valueOf(i);
-            Long val = Long.MAX_VALUE - i;
+            for (int i = 0; i < ITER_CNT; i++) {
+                String key = String.valueOf(i);
+                Long val = Long.MAX_VALUE - i;
 
-            cache.put(key, val);
+                cache.put(key, val);
 
-            map.put(key, val);
-        }
+                map.put(key, val);
+            }
 
-        GridCacheAdapter cache0 = internalCache(cache);
+            GridCacheAdapter cache0 = internalCache(cache);
 
-        GridCacheContext cctx = cache0.context();
+            GridCacheContext cctx = cache0.context();
 
-        for (Map.Entry<String, Long> e : map.entrySet()) {
-            GridCacheEntryEx entry = cache0.peekEx(e.getKey());
+            for (Map.Entry<String, Long> e : map.entrySet()) {
+                GridCacheEntryEx entry = cache0.peekEx(e.getKey());
 
-            assertNotNull("No entry for key: " + e.getKey(), entry);
+                assertNotNull("No entry for key: " + e.getKey(), entry);
 
-            String key0 = entry.key().value(cctx.cacheObjectContext(), false);
+                String key0 = entry.key().value(cctx.cacheObjectContext(), false);
 
-            assertSame(key0, e.getKey());
+                assertSame(key0, e.getKey());
 
-            String key1 = entry.key().value(cctx.cacheObjectContext(), true);
+                String key1 = entry.key().value(cctx.cacheObjectContext(), true);
 
-            assertSame(key0, key1);
+                assertSame(key0, key1);
 
-            Long val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
+                if (!storeValue(cache)) {
+                    Long val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(val0, e.getValue());
+                    assertNotSame(val0, e.getValue());
 
-            Long val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
+                    Long val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
 
-            assertNotSame(val0, val1);
+                    assertNotSame(val0, val1);
 
-            assertNotSame(e.getValue(), cache.get(e.getKey()));
+                    assertNotSame(e.getValue(), cache.get(e.getKey()));
+                }
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
@@ -481,6 +549,13 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
     }
 
     /**
+     * @param cache Cache.
+     */
+    private static boolean storeValue(IgniteCache cache) {
+        return ((IgniteCacheProxy)cache).context().cacheObjectContext().storeValue();
+    }
+
+    /**
      *
      */
     public static class TestValue implements Serializable {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 5af37a6..045ff6f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -37,6 +37,7 @@ import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSel
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
+import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheFutureExceptionSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheNamesSelfTest;
 import org.apache.ignite.internal.processors.cache.CachePutEventListenerErrorSelfTest;
@@ -178,6 +179,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheAtomicLocalWithStoreInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxInvokeTest.class);
         suite.addTestSuite(IgniteCacheEntryProcessorCallTest.class);
+        suite.addTestSuite(CacheEntryProcessorCopySelfTest.class);
         suite.addTestSuite(IgniteCacheTxNearEnabledInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxLocalInvokeTest.class);
         suite.addTestSuite(IgniteCrossCacheTxStoreSelfTest.class);