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 2015/03/05 09:55:47 UTC

[49/50] [abbrv] incubator-ignite git commit: IGNITE-51 Added tests for interceptors.

IGNITE-51 Added tests for interceptors.


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

Branch: refs/heads/ignite-user-req
Commit: 5ed9eb861471ee52ccf61b66d34db98b44234604
Parents: 7bf8e61
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Mar 4 17:24:31 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Mar 5 09:11:13 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |   2 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    | 550 ++++++++++++++++++
 .../GridCacheOnCopyFlagAtomicSelfTest.java      |  39 ++
 .../cache/GridCacheOnCopyFlagLocalSelfTest.java |  39 ++
 .../GridCacheOnCopyFlagReplicatedSelfTest.java  |  39 ++
 ...ridCacheOnCopyFlagTxPartitionedSelfTest.java |  39 ++
 .../cache/GridCacheOnFlagAbstractSelfTest.java  | 553 -------------------
 .../cache/GridCacheOnFlagAtomicSelfTest.java    |  39 --
 .../cache/GridCacheOnFlagLocalSelfTest.java     |  39 --
 .../GridCacheOnFlagReplicatedSelfTest.java      |  39 --
 .../GridCacheOnFlagTxPartitionedSelfTest.java   |  39 --
 .../IgniteCacheInterceptorSelfTestSuite.java    |   5 +
 12 files changed, 712 insertions(+), 710 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/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 4c1b654..75d1fd4 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
@@ -1860,8 +1860,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                     Object computed = entryProcessor.process(entry, invokeArgs);
 
                     if (entry.modified()) {
-                        updated = cctx.toCacheObject(updated0);
                         updated0 = cctx.unwrapTemporary(entry.getValue());
+                        updated = cctx.toCacheObject(updated0);
                     }
                     else
                         updated = oldVal;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/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
new file mode 100644
index 0000000..750a3aa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
@@ -0,0 +1,550 @@
+/*
+ * 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.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import javax.cache.processor.*;
+import java.io.*;
+import java.util.*;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
+ */
+public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    public static final int ITER_CNT = 1000;
+
+    /** */
+    public static final int WRONG_VALUE = -999999;
+
+    /** */
+    private static Interceptor interceptor;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        interceptor = new Interceptor();
+
+        super.beforeTestsStarted();
+
+        awaitPartitionMapExchange();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        interceptor = new Interceptor();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(IP_FINDER);
+
+        c.setDiscoverySpi(spi);
+
+        c.getTransactionConfiguration().setTxSerializableEnabled(true);
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+
+        assertNotNull(interceptor);
+
+        ccfg.setInterceptor(interceptor);
+        
+        ccfg.setAtomicityMode(atomicityMode());
+        ccfg.setDistributionMode(distributionMode());
+        ccfg.setCacheMode(cacheMode());
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected abstract CacheAtomicityMode atomicityMode();
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInterceptor() throws Exception {
+        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(null);
+
+        for (int i = 0; i < ITER_CNT; i++) {
+            final TestValue val = new TestValue(i);
+            final TestKey key = new TestKey(i, i);
+
+            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
+                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                    assertNotSame(key, entry.getKey());
+                    assertNotSame(val, entry.getValue());
+
+                    assertSame(entry.getValue(), entry.getValue());
+                    assertSame(entry.getKey(), entry.getKey());
+
+                    // Try change key and value.
+                    entry.getKey().field(WRONG_VALUE);
+                    entry.getValue().val(WRONG_VALUE);
+                }
+            });
+
+            cache.put(key, val);
+
+            Cache.Entry<Object, Object> entry = internalCache(0).entrySet().iterator().next();
+
+            // Check thar internal entry wasn't changed.
+            assertEquals(i, ((TestKey)entry.getKey()).field());
+            assertEquals(i, ((TestValue)entry.getValue()).val());
+
+            final TestValue newTestVal = new TestValue(-i);
+
+            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());
+
+                    assertEquals(newTestVal, newVal);
+
+                    // Try change key and value.
+                    entry.getKey().field(WRONG_VALUE);
+                    entry.getValue().val(WRONG_VALUE);
+
+                    return newVal;
+                }
+
+                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                    assertNotSame(key, entry.getKey());
+                    assertNotSame(newTestVal, entry.getValue());
+
+                    assertSame(entry.getValue(), entry.getValue());
+                    assertSame(entry.getKey(), entry.getKey());
+
+                    // Try change key and value.
+                    entry.getKey().field(WRONG_VALUE);
+                    entry.getValue().val(WRONG_VALUE);
+                }
+            });
+
+            cache.put(key, newTestVal);
+
+            entry = internalCache(0).entrySet().iterator().next();
+
+            // Check thar internal entry wasn't changed.
+            assertEquals(i, ((TestKey)entry.getKey()).field());
+            assertEquals(-i, ((TestValue)entry.getValue()).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);
+                }
+
+                @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
+                    assertNotSame(key, entry.getKey());
+                    assertNotSame(newTestVal, entry.getValue());
+                }
+            });
+
+            cache.remove(key);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEntryProcessor() throws Exception {
+        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(null);
+
+        Set<TestKey> keys = new LinkedHashSet<>();
+
+        for (int i = 0; i < ITER_CNT; i++) {
+            TestKey key = new TestKey(i, i);
+            keys.add(key);
+
+            cache.put(key, new TestValue(i));
+        }
+
+        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 {
+                    // Try change entry.
+                    entry.getKey().field(WRONG_VALUE);
+                    entry.getValue().val(WRONG_VALUE);
+   
+                    return -1;
+                }
+            });
+
+            // Check that internal entry isn't changed.
+            Cache.Entry<Object, Object> e = internalCache(0).entry(new TestKey(i, i));
+
+            assertEquals(i, ((TestKey)e.getKey()).field());
+            assertEquals(i, ((TestValue)e.getValue()).val());
+        }
+        
+        cache.invokeAll(keys, new EntryProcessor<TestKey, TestValue, Object>() {
+            @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments) 
+                throws EntryProcessorException {
+                // Try change entry.
+                entry.getKey().field(WRONG_VALUE);
+                entry.getValue().val(WRONG_VALUE);
+                
+                return -1;
+            }
+        });
+
+        // Check that entries weren't changed.
+        for (Cache.Entry<Object, Object> e : internalCache(0).entrySet()) {
+            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
+            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAndInterceptor() throws Exception {
+        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(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());
+                assertEquals(entry.getKey().key(), entry.getValue().val());
+
+                // Try changed entry.
+                entry.getKey().field(WRONG_VALUE);
+                entry.getValue().val(WRONG_VALUE);
+
+                return super.onBeforePut(entry, newVal);
+            }
+
+            @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                assertEquals(entry.getKey().key(), entry.getKey().field());
+                assertEquals(entry.getKey().key(), entry.getValue().val());
+
+                entry.getValue().val(WRONG_VALUE);
+                entry.getKey().field(WRONG_VALUE);
+
+                super.onAfterPut(entry);
+            }
+        });
+
+        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 and key.
+                    assertEquals(entry.getKey().key(), entry.getKey().field());
+                    assertEquals(entry.getKey().key(), entry.getValue().val());
+
+                    // Try changed entry.
+                    entry.getKey().field(WRONG_VALUE);
+                    entry.getValue().val(WRONG_VALUE);
+                    
+                    return -1;
+                }
+            });
+
+        // Check that entries weren't changed.
+        for (Cache.Entry<Object, Object> e : internalCache(0).entrySet()) {
+            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
+            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
+        }
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllAndInterceptor() throws Exception {
+        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(null);
+
+        Set<TestKey> keys = new LinkedHashSet<>();
+        
+        for (int i = 0; i < ITER_CNT; i++) {
+            TestKey key = new TestKey(i, i);
+
+            keys.add(key);
+
+            cache.put(key, 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());
+                assertEquals(entry.getKey().key(), entry.getValue().val());
+
+                // Try changed entry.
+                entry.getKey().field(WRONG_VALUE);
+                entry.getValue().val(WRONG_VALUE);
+
+                return super.onBeforePut(entry, newVal);
+            }
+
+            @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                // Check that we have correct value and key.
+                assertEquals(entry.getKey().key(), entry.getKey().field());
+                assertEquals(entry.getKey().key(), entry.getValue().val());
+
+                // Try changed entry.
+                entry.getValue().val(WRONG_VALUE);
+                entry.getKey().field(WRONG_VALUE);
+
+                super.onAfterPut(entry);
+            }
+        });
+
+        cache.invokeAll(keys, new EntryProcessor<TestKey, TestValue, Object>() {
+            @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments)
+                throws EntryProcessorException {
+                // Check that we have correct value and key.
+                assertEquals(entry.getKey().key(), entry.getKey().field());
+                assertEquals(entry.getKey().key(), entry.getValue().val());
+
+                // Try changed entry.
+                entry.getKey().field(WRONG_VALUE);
+                entry.getValue().val(WRONG_VALUE);
+
+                return -1;
+            }
+        });
+
+        // Check that entries weren't changed.
+        for (Cache.Entry<Object, Object> e : internalCache(0).entrySet()) {
+            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
+            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
+        }
+    }
+
+    /**
+     *
+     */
+    public static class TestKey implements Externalizable {
+        /** */
+        private int key;
+
+        /** */
+        private int field;
+
+        /**
+         * Constructor.
+         *
+         * @param key Key.
+         */
+        public TestKey(int key, int field) {
+            this.key = key;
+            this.field = field;
+        }
+
+        /**
+         * Default constructor.
+         */
+        public TestKey() {
+            // No-op.
+        }
+
+        /**
+         * @return key Key.
+         */
+        public int key(){
+            return key;
+        }
+
+        /**
+         * @return Test field.
+         */
+        public int field() {
+            return field;
+        }
+
+        /**
+         * *
+         * @param field Test field.
+         */
+        public void field(int field) {
+            this.field = field;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TestKey testKey = (TestKey) o;
+
+            return key == testKey.key;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(key);
+            out.writeInt(field);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            key = in.readInt();
+            field = in.readInt();
+        }
+    }
+
+
+    /**
+     *
+     */
+    public static class TestValue implements Externalizable {
+        /** */
+        private int val;
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public TestValue(int val) {
+            this.val = val;
+        }
+
+        /**
+         * Default constructor.
+         */
+        public TestValue() {
+            // No-op.
+        }
+
+        /**
+         * @return Value.
+         */
+        public int val() {
+            return val;
+        }
+
+        /**
+         * @param val Value.
+         */
+        public void val(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TestValue testKey = (TestValue)o;
+
+            return val == testKey.val;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            val = in.readInt();
+        }
+    }
+
+    /**
+     *
+     */
+    private class Interceptor implements CacheInterceptor<TestKey, TestValue> {
+        /** */
+        CacheInterceptor<TestKey, TestValue> delegate = new CacheInterceptorAdapter<>();
+
+        /** {@inheritDoc} */
+        @Override public TestValue onGet(TestKey key, @Nullable TestValue val) {
+            return delegate.onGet(key, val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
+            return delegate.onBeforePut(entry, newVal);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+            delegate.onAfterPut(entry);
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteBiTuple<Boolean, TestValue> onBeforeRemove(Cache.Entry<TestKey, TestValue> entry) {
+            return delegate.onBeforeRemove(entry);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
+            delegate.onAfterRemove(entry);
+        }
+
+        /**
+         * @param delegate Cache interceptor delegate.
+         */
+        public void delegate(CacheInterceptor<TestKey, TestValue> delegate) {
+            this.delegate = delegate;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAtomicSelfTest.java
new file mode 100644
index 0000000..bc9eb1b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAtomicSelfTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
+ */
+public class GridCacheOnCopyFlagAtomicSelfTest extends
+    GridCacheOnCopyFlagAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagLocalSelfTest.java
new file mode 100644
index 0000000..1da5d0e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagLocalSelfTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
+ */
+public class GridCacheOnCopyFlagLocalSelfTest extends
+    GridCacheOnCopyFlagAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return LOCAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagReplicatedSelfTest.java
new file mode 100644
index 0000000..72a4dc5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagReplicatedSelfTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
+ */
+public class GridCacheOnCopyFlagReplicatedSelfTest extends
+    GridCacheOnCopyFlagAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagTxPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagTxPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagTxPartitionedSelfTest.java
new file mode 100644
index 0000000..7a24b3b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagTxPartitionedSelfTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
+ */
+public class GridCacheOnCopyFlagTxPartitionedSelfTest extends
+    GridCacheOnCopyFlagAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAbstractSelfTest.java
deleted file mode 100644
index c486efe..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAbstractSelfTest.java
+++ /dev/null
@@ -1,553 +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.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.jetbrains.annotations.*;
-
-import javax.cache.*;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-import java.io.*;
-import java.util.LinkedHashSet;
-import java.util.Set;
-
-import static org.junit.Assert.assertNotEquals;
-
-/**
- * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
- */
-public abstract class GridCacheOnFlagAbstractSelfTest extends GridCacheAbstractSelfTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    public static final int ITER_CNT = 1000;
-
-    /** */
-    public static final int WRONG_VALUE = -999999;
-
-    /** */
-    private static Interceptor interceptor;
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        interceptor = new Interceptor();
-
-        super.beforeTestsStarted();
-
-        awaitPartitionMapExchange();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        interceptor = new Interceptor();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(IP_FINDER);
-
-        c.setDiscoverySpi(spi);
-
-        c.getTransactionConfiguration().setTxSerializableEnabled(true);
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
-
-        assertNotNull(interceptor);
-
-        ccfg.setInterceptor(interceptor);
-        
-        ccfg.setAtomicityMode(atomicityMode());
-        ccfg.setDistributionMode(distributionMode());
-        ccfg.setCacheMode(cacheMode());
-
-        return ccfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected abstract CacheAtomicityMode atomicityMode();
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testInterceptor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(null);
-
-        for (int i = 0; i < ITER_CNT; i++) {
-            final TestValue val = new TestValue(i);
-            final TestKey key = new TestKey(i, i);
-
-            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
-                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(val, entry.getValue());
-
-                    assertSame(entry.getValue(), entry.getValue());
-                    assertSame(entry.getKey(), entry.getKey());
-
-                    // Try change key and value.
-                    entry.getKey().field(WRONG_VALUE);
-                    entry.getValue().val(WRONG_VALUE);
-                }
-            });
-
-            cache.put(key, val);
-
-            Cache.Entry<Object, Object> entry = internalCache(0).entrySet().iterator().next();
-
-            // Check thar internal entry wasn't changed.
-            assertEquals(i, ((TestKey)entry.getKey()).field());
-            assertEquals(i, ((TestValue)entry.getValue()).val());
-
-            final TestValue newTestVal = new TestValue(-i);
-
-            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());
-
-                    assertEquals(newTestVal, newVal);
-
-                    // Try change key and value.
-                    entry.getKey().field(WRONG_VALUE);
-                    entry.getValue().val(WRONG_VALUE);
-
-                    return newVal;
-                }
-
-                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(newTestVal, entry.getValue());
-
-                    assertSame(entry.getValue(), entry.getValue());
-                    assertSame(entry.getKey(), entry.getKey());
-
-                    // Try change key and value.
-                    entry.getKey().field(WRONG_VALUE);
-                    entry.getValue().val(WRONG_VALUE);
-                }
-            });
-
-            cache.put(key, newTestVal);
-
-            entry = internalCache(0).entrySet().iterator().next();
-
-            // Check thar internal entry wasn't changed.
-            assertEquals(i, ((TestKey)entry.getKey()).field());
-            assertEquals(-i, ((TestValue)entry.getValue()).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);
-                }
-
-                @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(newTestVal, entry.getValue());
-                }
-            });
-
-            cache.remove(key);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testEntryProcessor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(null);
-
-        Set<TestKey> keys = new LinkedHashSet<>();
-
-        for (int i = 0; i < ITER_CNT; i++) {
-            TestKey key = new TestKey(i, i);
-            keys.add(key);
-
-            cache.put(key, new TestValue(i));
-        }
-
-        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 {
-                    // Try change entry.
-                    entry.getKey().field(WRONG_VALUE);
-                    entry.getValue().val(WRONG_VALUE);
-   
-                    return -1;
-                }
-            });
-
-            // Check that internal entry isn't changed.
-            Cache.Entry<Object, Object> e = internalCache(0).entry(new TestKey(i, i));
-
-            assertEquals(i, ((TestKey)e.getKey()).field());
-            assertEquals(i, ((TestValue)e.getValue()).val());
-        }
-        
-        cache.invokeAll(keys, new EntryProcessor<TestKey, TestValue, Object>() {
-            @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments) 
-                throws EntryProcessorException {
-                // Try change entry.
-                entry.getKey().field(WRONG_VALUE);
-                entry.getValue().val(WRONG_VALUE);
-                
-                return -1;
-            }
-        });
-
-        // Check that entries weren't changed.
-        for (Cache.Entry<Object, Object> e : internalCache(0).entrySet()) {
-            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
-            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testInvokeAndInterceptor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(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());
-                assertEquals(entry.getKey().key(), entry.getValue().val());
-
-                // Try changed entry.
-                entry.getKey().field(WRONG_VALUE);
-                entry.getValue().val(WRONG_VALUE);
-
-                return super.onBeforePut(entry, newVal);
-            }
-
-            @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                assertEquals(entry.getKey().key(), entry.getKey().field());
-                assertEquals(entry.getKey().key(), entry.getValue().val());
-
-                entry.getValue().val(WRONG_VALUE);
-                entry.getKey().field(WRONG_VALUE);
-
-                super.onAfterPut(entry);
-            }
-        });
-
-        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 and key.
-                    assertEquals(entry.getKey().key(), entry.getKey().field());
-                    assertEquals(entry.getKey().key(), entry.getValue().val());
-
-                    // Try changed entry.
-                    entry.getKey().field(WRONG_VALUE);
-                    entry.getValue().val(WRONG_VALUE);
-                    
-                    return -1;
-                }
-            });
-
-        // Check that entries weren't changed.
-        for (Cache.Entry<Object, Object> e : internalCache(0).entrySet()) {
-            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
-            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
-        }
-    }
-
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testInvokeAllAndInterceptor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).jcache(null);
-
-        Set<TestKey> keys = new LinkedHashSet<>();
-        
-        for (int i = 0; i < ITER_CNT; i++) {
-            TestKey key = new TestKey(i, i);
-
-            keys.add(key);
-
-            cache.put(key, 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());
-                assertEquals(entry.getKey().key(), entry.getValue().val());
-
-                // Try changed entry.
-                entry.getKey().field(WRONG_VALUE);
-                entry.getValue().val(WRONG_VALUE);
-
-                return super.onBeforePut(entry, newVal);
-            }
-
-            @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                // Check that we have correct value and key.
-                assertEquals(entry.getKey().key(), entry.getKey().field());
-                assertEquals(entry.getKey().key(), entry.getValue().val());
-
-                // Try changed entry.
-                entry.getValue().val(WRONG_VALUE);
-                entry.getKey().field(WRONG_VALUE);
-
-                super.onAfterPut(entry);
-            }
-        });
-
-        cache.invokeAll(keys, new EntryProcessor<TestKey, TestValue, Object>() {
-            @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments)
-                throws EntryProcessorException {
-                // Check that we have correct value and key.
-                assertEquals(entry.getKey().key(), entry.getKey().field());
-                assertEquals(entry.getKey().key(), entry.getValue().val());
-
-                // Try changed entry.
-                entry.getKey().field(WRONG_VALUE);
-                entry.getValue().val(WRONG_VALUE);
-
-                return -1;
-            }
-        });
-
-        // Check that entries weren't changed.
-        for (Cache.Entry<Object, Object> e : internalCache(0).entrySet()) {
-            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
-            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
-        }
-    }
-
-    /**
-     *
-     */
-    public static class TestKey implements Externalizable {
-        /** */
-        private int key;
-
-        /** */
-        private int field;
-
-        /**
-         * Constructor.
-         *
-         * @param key Key.
-         */
-        public TestKey(int key, int field) {
-            this.key = key;
-            this.field = field;
-        }
-
-        /**
-         * Default constructor.
-         */
-        public TestKey() {
-            // No-op.
-        }
-
-        /**
-         * @return key Key.
-         */
-        public int key(){
-            return key;
-        }
-
-        /**
-         * @return Test field.
-         */
-        public int field() {
-            return field;
-        }
-
-        /**
-         * *
-         * @param field Test field.
-         */
-        public void field(int field) {
-            this.field = field;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o) return true;
-
-            if (o == null || getClass() != o.getClass()) return false;
-
-            TestKey testKey = (TestKey) o;
-
-            return key == testKey.key;
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return key;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeInt(key);
-            out.writeInt(field);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            key = in.readInt();
-            field = in.readInt();
-        }
-    }
-
-
-    /**
-     *
-     */
-    public static class TestValue implements Externalizable {
-        /** */
-        private int val;
-
-        /**
-         * Constructor.
-         *
-         * @param val Value.
-         */
-        public TestValue(int val) {
-            this.val = val;
-        }
-
-        /**
-         * Default constructor.
-         */
-        public TestValue() {
-            // No-op.
-        }
-
-        /**
-         * @return Value.
-         */
-        public int val() {
-            return val;
-        }
-
-        /**
-         * @param val Value.
-         */
-        public void val(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o) return true;
-
-            if (o == null || getClass() != o.getClass()) return false;
-
-            TestValue testKey = (TestValue)o;
-
-            return val == testKey.val;
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeInt(val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            val = in.readInt();
-        }
-    }
-
-    /**
-     *
-     */
-    private class Interceptor implements CacheInterceptor<TestKey, TestValue> {
-        /** */
-        CacheInterceptor<TestKey, TestValue> delegate = new CacheInterceptorAdapter<>();
-
-        /** {@inheritDoc} */
-        @Override public TestValue onGet(TestKey key, @Nullable TestValue val) {
-            return delegate.onGet(key, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
-            return delegate.onBeforePut(entry, newVal);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-            delegate.onAfterPut(entry);
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteBiTuple<Boolean, TestValue> onBeforeRemove(Cache.Entry<TestKey, TestValue> entry) {
-            return delegate.onBeforeRemove(entry);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
-            delegate.onAfterRemove(entry);
-        }
-
-        /**
-         * @param delegate Cache interceptor delegate.
-         */
-        public void delegate(CacheInterceptor<TestKey, TestValue> delegate) {
-            this.delegate = delegate;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAtomicSelfTest.java
deleted file mode 100644
index 07beef5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagAtomicSelfTest.java
+++ /dev/null
@@ -1,39 +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.apache.ignite.cache.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
- */
-public class GridCacheOnFlagAtomicSelfTest extends
-    GridCacheOnFlagAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagLocalSelfTest.java
deleted file mode 100644
index 9bc8032..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagLocalSelfTest.java
+++ /dev/null
@@ -1,39 +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.apache.ignite.cache.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
- */
-public class GridCacheOnFlagLocalSelfTest extends
-    GridCacheOnFlagAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return LOCAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagReplicatedSelfTest.java
deleted file mode 100644
index 66c33e8..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagReplicatedSelfTest.java
+++ /dev/null
@@ -1,39 +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.apache.ignite.cache.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
- */
-public class GridCacheOnFlagReplicatedSelfTest extends
-    GridCacheOnFlagAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagTxPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagTxPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagTxPartitionedSelfTest.java
deleted file mode 100644
index 688c4f8..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnFlagTxPartitionedSelfTest.java
+++ /dev/null
@@ -1,39 +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.apache.ignite.cache.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests {@link org.apache.ignite.cache.CacheInterceptor}.
- */
-public class GridCacheOnFlagTxPartitionedSelfTest extends
-    GridCacheOnFlagAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ed9eb86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java
index 87b9e4b..ee4d64b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java
@@ -51,6 +51,11 @@ public class IgniteCacheInterceptorSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheInterceptorReplicatedSelfTest.class);
         suite.addTestSuite(GridCacheInterceptorReplicatedWithStoreSelfTest.class);
 
+        suite.addTestSuite(GridCacheOnCopyFlagTxPartitionedSelfTest.class);
+        suite.addTestSuite(GridCacheOnCopyFlagReplicatedSelfTest.class);
+        suite.addTestSuite(GridCacheOnCopyFlagLocalSelfTest.class);
+        suite.addTestSuite(GridCacheOnCopyFlagAtomicSelfTest.class);
+
         return suite;
     }
 }