You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2015/02/01 14:18:52 UTC

[3/3] incubator-ignite git commit: # IGNITE-56 Use IgniteCache in ignite-core module (1).

# IGNITE-56 Use IgniteCache in ignite-core module (1).


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

Branch: refs/heads/ignite-56
Commit: 7d4002ae39c356b2dd098fad9d0ab5c8c467cf97
Parents: ece021f
Author: sevdokimov <se...@jetbrains.com>
Authored: Sun Feb 1 16:18:33 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Sun Feb 1 16:18:33 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheUtils.java        |  19 +
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |  12 +-
 .../internal/GridCacheProjectionRemoveTest.java |   6 +-
 .../GridCacheAbstractFailoverSelfTest.java      |  87 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java | 222 ++---
 .../cache/GridCacheAbstractFullApiSelfTest.java |  44 +-
 .../GridCacheAbstractIteratorsSelfTest.java     |  84 +-
 .../GridCacheAbstractProjectionSelfTest.java    | 884 -------------------
 .../cache/GridCacheAbstractSelfTest.java        |  33 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |  42 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |   7 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |  52 +-
 ...acheBasicStoreMultithreadedAbstractTest.java |   7 +-
 ...acheEntrySetIterationPreloadingSelfTest.java |   6 +-
 .../distributed/GridCacheEventAbstractTest.java |   2 +-
 .../dht/GridCacheAtomicFullApiSelfTest.java     |  12 +-
 ...dCachePartitionedOnlyProjectionSelfTest.java |  32 -
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |  79 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |  52 +-
 ...AtomicPartitionedTckMetricsSelfTestImpl.java | 132 +--
 .../GridCachePartitionedProjectionSelfTest.java | 155 ----
 .../GridCacheReplicatedProjectionSelfTest.java  |  66 --
 ...dCacheAtomicLocalTckMetricsSelfTestImpl.java | 132 +--
 .../GridCacheLocalAtomicProjectionSelfTest.java |  32 -
 .../local/GridCacheLocalProjectionSelfTest.java |  38 -
 .../junits/common/GridCommonAbstractTest.java   |  14 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   5 -
 27 files changed, 426 insertions(+), 1830 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 7368a02..9ff44ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1635,6 +1635,25 @@ public class GridCacheUtils {
     }
 
     /**
+     * Execute closure inside cache transaction.
+     *
+     * @param cache Cache.
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @param clo Closure.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static <K, V> void inTx(Ignite ignite, IgniteCache<K, V> cache, IgniteTxConcurrency concurrency,
+        IgniteTxIsolation isolation, IgniteInClosureX<IgniteCache<K ,V>> clo) throws IgniteCheckedException {
+
+        try (IgniteTx tx = ignite.transactions().txStart(concurrency, isolation)) {
+            clo.applyx(cache);
+
+            tx.commit();
+        }
+    }
+
+    /**
      * Gets subject ID by transaction.
      *
      * @param tx Transaction.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
index 7b080af..84fcb56 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
@@ -117,7 +117,7 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr
 
             @Override public Object call() throws Exception {
                 for (int i = 0; i < TX_CNT; i++) {
-                    GridCache<Integer, String> cache = cache(rnd.nextInt(GRID_CNT));
+                    IgniteCache<Object, Object> cache = jcache(rnd.nextInt(GRID_CNT));
 
                     cache.put(rnd.nextInt(1000), "value");
                 }
@@ -131,7 +131,7 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr
 
             @Override public Object call() throws Exception {
                 for (int i = 0; i < TX_CNT; i++) {
-                    GridCache<Integer, String> cache = cache(rnd.nextInt(GRID_CNT));
+                    IgniteCache<Object, Object> cache = jcache(rnd.nextInt(GRID_CNT));
 
                     cache.putIfAbsent(rnd.nextInt(1000), "value");
                 }
@@ -160,7 +160,7 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr
                     for (int j = 0; j < 10; j++)
                         map.put(rnd.nextInt(1000), "value");
 
-                    GridCache<Integer, String> cache = cache(rnd.nextInt(GRID_CNT));
+                    IgniteCache<Object, Object> cache = jcache(rnd.nextInt(GRID_CNT));
 
                     cache.putAll(map);
                 }
@@ -181,9 +181,11 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr
 
             @Override public Object call() throws Exception {
                 for (int i = 0; i < TX_CNT; i++) {
-                    GridCache<Integer, String> cache = cache(rnd.nextInt(GRID_CNT));
+                    IgniteEx ignite = grid(rnd.nextInt(GRID_CNT));
 
-                    try (IgniteTx tx = cache.txStart()) {
+                    IgniteCache<Object, Object> cache = ignite.jcache(null);
+
+                    try (IgniteTx tx = ignite.transactions().txStart()) {
                         cache.put(1, "value");
                         cache.put(2, "value");
                         cache.put(3, "value");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/GridCacheProjectionRemoveTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheProjectionRemoveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheProjectionRemoveTest.java
index 061313a..85ce758 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheProjectionRemoveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheProjectionRemoveTest.java
@@ -33,9 +33,9 @@ public class GridCacheProjectionRemoveTest extends GridCacheAbstractSelfTest {
      * @throws IgniteCheckedException If failed.
      */
     public void testRemove() throws IgniteCheckedException {
-        cache().put("key", 1);
+        jcache().put("key", 1);
 
-        assert cache().remove("key", 1);
-        assert !cache().remove("key", 1);
+        assert jcache().remove("key", 1);
+        assert !jcache().remove("key", 1);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index b50f061..288877f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -22,17 +22,15 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
+import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.*;
 import java.util.*;
 
-import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CachePreloadMode.*;
 import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
 import static org.apache.ignite.transactions.IgniteTxIsolation.*;
@@ -214,9 +212,9 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         boolean tx = concurrency != null && isolation != null;
 
         if (tx)
-            put(cache(), ENTRY_CNT, concurrency, isolation);
+            put(ignite(0), jcache(), ENTRY_CNT, concurrency, isolation);
         else
-            put(cache(), ENTRY_CNT);
+            put(jcache(), ENTRY_CNT);
 
         Ignite g = startGrid(NEW_GRID_NAME);
 
@@ -225,8 +223,8 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         int half = ENTRY_CNT / 2;
 
         if (tx) {
-            remove(cache(g), half, concurrency, isolation);
-            put(cache(g), half, concurrency, isolation);
+            remove(g, cache(g), half, concurrency, isolation);
+            put(g, cache(g), half, concurrency, isolation);
         }
         else {
             remove(cache(g), half);
@@ -235,7 +233,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         stopGrid(NEW_GRID_NAME);
 
-        check(cache(), ENTRY_CNT);
+        check(jcache(), ENTRY_CNT);
     }
 
     /**
@@ -248,11 +246,11 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         final boolean tx = concurrency != null && isolation != null;
 
         if (tx)
-            put(cache(), ENTRY_CNT, concurrency, isolation);
+            put(ignite(0), jcache(), ENTRY_CNT, concurrency, isolation);
         else
-            put(cache(), ENTRY_CNT);
+            put(jcache(), ENTRY_CNT);
 
-        check(cache(), ENTRY_CNT);
+        check(jcache(), ENTRY_CNT);
 
         final int half = ENTRY_CNT / 2;
 
@@ -285,12 +283,12 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         while (!fut.isDone()) {
             if (tx) {
-                remove(cache(), half, concurrency, isolation);
-                put(cache(), half, concurrency, isolation);
+                remove(grid(0), jcache(), half, concurrency, isolation);
+                put(grid(0), jcache(), half, concurrency, isolation);
             }
             else {
-                remove(cache(), half);
-                put(cache(), half);
+                remove(jcache(), half);
+                put(jcache(), half);
             }
         }
 
@@ -302,12 +300,12 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
      * @param cnt Entry count.
      * @throws IgniteCheckedException If failed.
      */
-    private void put(CacheProjection<String, Integer> cache, int cnt) throws Exception {
+    private void put(IgniteCache<String, Integer> cache, int cnt) throws Exception {
         try {
             for (int i = 0; i < cnt; i++)
-                assertTrue("Failed to put key: 'key" + i + "'",  cache.putx("key" + i, i));
+                cache.put("key" + i, i);
         }
-        catch (IgniteCheckedException e) {
+        catch (CacheException e) {
             // It is ok to fail with topology exception.
             if (!X.hasCause(e, ClusterTopologyException.class))
                 throw e;
@@ -321,16 +319,15 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
      * @param isolation Isolation level.
      * @throws IgniteCheckedException If failed.
      */
-    private void put(CacheProjection<String, Integer> cache, final int cnt,
+    private void put(Ignite ignite, IgniteCache<String, Integer> cache, final int cnt,
         IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) throws Exception {
         try {
             info("Putting values to cache [0," + cnt + ')');
 
-            CU.inTx(cache, concurrency, isolation, new CIX1<CacheProjection<String, Integer>>() {
-                @Override public void applyx(CacheProjection<String, Integer> cache)
-                    throws IgniteCheckedException {
+            CU.inTx(ignite, cache, concurrency, isolation, new CIX1<IgniteCache<String, Integer>>() {
+                @Override public void applyx(IgniteCache<String, Integer> cache) {
                     for (int i = 0; i < cnt; i++)
-                        assertTrue("Failed to put key: 'key" + i + "'", cache.putx("key" + i, i));
+                        cache.put("key" + i, i);
                 }
             });
         }
@@ -348,12 +345,12 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
      * @param cnt Entry count.
      * @throws IgniteCheckedException If failed.
      */
-    private void remove(CacheProjection<String, Integer> cache, int cnt) throws Exception {
+    private void remove(IgniteCache<String, Integer> cache, int cnt) throws Exception {
         try {
             for (int i = 0; i < cnt; i++)
-                cache.removex("key" + i);
+                cache.remove("key" + i);
         }
-        catch (IgniteCheckedException e) {
+        catch (CacheException e) {
             // It is ok to fail with topology exception.
             if (!X.hasCause(e, ClusterTopologyException.class))
                 throw e;
@@ -367,16 +364,15 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
      * @param isolation Isolation level.
      * @throws IgniteCheckedException If failed.
      */
-    private void remove(CacheProjection<String, Integer> cache, final int cnt,
+    private void remove(Ignite g, IgniteCache<String, Integer> cache, final int cnt,
         IgniteTxConcurrency concurrency, IgniteTxIsolation isolation) throws Exception {
         try {
             info("Removing values form cache [0," + cnt + ')');
 
-            CU.inTx(cache, concurrency, isolation, new CIX1<CacheProjection<String, Integer>>() {
-                @Override public void applyx(CacheProjection<String, Integer> cache)
-                    throws IgniteCheckedException {
+            CU.inTx(g, cache, concurrency, isolation, new CIX1<IgniteCache<String, Integer>>() {
+                @Override public void applyx(IgniteCache<String, Integer> cache) {
                     for (int i = 0; i < cnt; i++)
-                        cache.removex("key" + i);
+                        cache.remove("key" + i);
                 }
             });
         }
@@ -392,28 +388,9 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
     /**
      * @param cache Cache.
      * @param expSize Minimum expected cache size.
-     * @throws IgniteCheckedException If failed.
      */
-    private void check(CacheProjection<String,Integer> cache, int expSize) throws IgniteCheckedException {
-        int size;
-
-        if (cacheMode() == PARTITIONED) {
-            Collection<Integer> res = compute(cache.gridProjection()).broadcast(new IgniteCallable<Integer>() {
-                @IgniteInstanceResource
-                private Ignite g;
-
-                @Override public Integer call() {
-                    return cache(g).projection(F.<String, Integer>cachePrimary()).size();
-                }
-            });
-
-            size = 0 ;
-
-            for (Integer size0 : res)
-                size += size0;
-        }
-        else
-            size = cache.size();
+    private void check(IgniteCache<String,Integer> cache, int expSize) {
+        int size = cache.size();
 
         assertTrue("Key set size is lesser then the expected size [size=" + size + ", expSize=" + expSize + ']',
             size >= expSize);
@@ -426,7 +403,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
      * @param g Grid.
      * @return Cache.
      */
-    private CacheProjection<String,Integer> cache(Ignite g) {
-        return g.cache(null);
+    private IgniteCache<String,Integer> cache(Ignite g) {
+        return g.jcache(null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
index 15a4a92..8285c67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
@@ -17,14 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import com.google.common.collect.*;
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.*;
 
+import javax.cache.*;
 import java.util.*;
 import java.util.concurrent.atomic.*;
 
@@ -72,16 +73,16 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @param c Test closure.
      * @throws Exception In case of error.
      */
-    private void runTest(final IgniteInClosure<GridCache<String, Integer>> c) throws Exception {
+    private void runTest(final IgniteInClosure<IgniteCache<String, Integer>> c) throws Exception {
         final IgniteInternalFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
+            @Override public void applyx() {
                 while (true) {
                     int i = cnt.getAndIncrement();
 
                     if (i >= PUT_CNT)
                         break;
 
-                    cache().put("key" + i, i);
+                    jcache().put("key" + i, i);
 
                     set.add(i);
 
@@ -93,7 +94,7 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
 
         IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new CA() {
             @Override public void apply() {
-                GridCache<String, Integer> cache = cache();
+                IgniteCache<String, Integer> cache = jcache();
 
                 while (!fut1.isDone())
                     if (guard.get())
@@ -111,14 +112,14 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws IgniteCheckedException If failed.
      */
     private void checkConsistency() throws IgniteCheckedException {
-        for (CacheEntry<String, Integer> e : cache())
+        for (Cache.Entry<String, Integer> e : jcache())
             for (int i = 1; i < gridCount(); i++) {
                 Integer val = cache(i).get(e.getKey());
 
                 if (val == null)
-                    assert e.get() == null;
+                    assert e.getValue() == null;
                 else
-                    assert val.equals(e.get());
+                    assert val.equals(e.getValue());
             }
     }
 
@@ -140,20 +141,20 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @param toExcl Exclusive stop of the range.
      * @return Range of keys.
      */
-    private Collection<String> rangeKeys(int fromIncl, int toExcl) {
-        return F.transform(F.range(fromIncl, toExcl), new C1<Integer, String>() {
+    private Set<String> rangeKeys(int fromIncl, int toExcl) {
+        return new HashSet<>(F.transform(F.range(fromIncl, toExcl), new C1<Integer, String>() {
             @Override public String apply(Integer i) {
                 return "key" + i;
             }
-        });
+        }));
     }
 
     /**
      * @throws Exception In case of error.
      */
     public void testContainsKey() throws Exception {
-        runTest(new CI1<GridCache<String,Integer>>() {
-            @Override public void apply(GridCache<String, Integer> cache) {
+        runTest(new CI1<IgniteCache<String,Integer>>() {
+            @Override public void apply(IgniteCache<String, Integer> cache) {
                 assert cache.containsKey("key" + random());
                 assert !cache.containsKey("wrongKey");
             }
@@ -163,62 +164,9 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
     /**
      * @throws Exception In case of error.
      */
-    public void testContainsKeyFiltered() throws Exception {
-        runTest(new CI1<GridCache<String,Integer>>() {
-            @Override public void apply(GridCache<String, Integer> cache) {
-                assert cache.projection(F.<String, Integer>cacheHasPeekValue()).containsKey("key");
-                assert !cache.projection(F.<String, Integer>cacheNoPeekValue()).containsKey("key" + random());
-            }
-        });
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testContainsValue() throws Exception {
-        runTest(new CI1<GridCache<String,Integer>>() {
-            @Override public void apply(GridCache<String, Integer> cache) {
-                assert cache.containsValue(random());
-                assert !cache.containsValue(-1);
-            }
-        });
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testContainsValueFiltered() throws Exception {
-        runTest(new CI1<GridCache<String,Integer>>() {
-            @Override public void apply(GridCache<String, Integer> cache) {
-                assert cache.projection(F.<String, Integer>cacheHasPeekValue()).containsValue(random());
-                assert !cache.projection(F.<String, Integer>cacheNoPeekValue()).containsValue(random());
-            }
-        });
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testForAll() throws Exception {
-        runTest(new CI1<GridCache<String,Integer>>() {
-            @Override public void apply(GridCache<String, Integer> cache) {
-                assert cache.forAll(new P1<CacheEntry<String, Integer>>() {
-                    @Override public boolean apply(CacheEntry<String, Integer> e) {
-                        Integer val = e.peek();
-
-                        return val == null || val <= PUT_CNT;
-                    }
-                });
-            }
-        });
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
     public void testGet() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) throws IgniteCheckedException {
                 int rnd = random();
 
                 assert cache.get("key" + rnd) == rnd;
@@ -231,12 +179,19 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws Exception In case of error.
      */
     public void testGetAsync() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) throws IgniteCheckedException {
                 int rnd = random();
 
-                assert cache.getAsync("key" + rnd).get() == rnd;
-                assert cache.getAsync("wrongKey").get() == null;
+                IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+                cacheAsync.get("key" + rnd);
+
+                assert cacheAsync.future().get() == rnd;
+
+                cache.get("wrongKey");
+
+                assert cacheAsync.future().get() == null;
             }
         });
     }
@@ -245,12 +200,12 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws Exception In case of error.
      */
     public void testGetAll() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) {
                 int rnd1 = random();
                 int rnd2 = random();
 
-                Map<String, Integer> map = cache.getAll(F.asList("key" + rnd1, "key" + rnd2));
+                Map<String, Integer> map = cache.getAll(ImmutableSet.of("key" + rnd1, "key" + rnd2));
 
                 assert map.size() == (rnd1 != rnd2 ? 2 : 1);
                 assert map.get("key" + rnd1) == rnd1;
@@ -263,12 +218,13 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws Exception In case of error.
      */
     public void testGetAllAsync() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) throws IgniteCheckedException {
                 int rnd1 = random();
                 int rnd2 = random();
 
-                Map<String, Integer> map = cache.getAllAsync(F.asList("key" + rnd1, "key" + rnd2)).get();
+                cache.withAsync().getAll(ImmutableSet.of("key" + rnd1, "key" + rnd2));
+                Map<String, Integer> map = cache.withAsync().<Map<String, Integer>>future().get();
 
                 assert map.size() == (rnd1 != rnd2 ? 2 : 1);
                 assert map.get("key" + rnd1) == rnd1;
@@ -281,22 +237,22 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws Exception In case of error.
      */
     public void testRemove() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) throws IgniteCheckedException {
                 int rnd1 = random();
                 int rnd2 = random();
 
-                assert cache.remove("wrongKey") == null;
+                assert cache.getAndRemove("wrongKey") == null;
                 assert !cache.remove("key" + rnd1, -1);
 
-                assert cache.peek("key" + rnd1) == null || cache.peek("key" + rnd1) == rnd1;
-                assert cache.peek("key" + rnd2) == null || cache.peek("key" + rnd2) == rnd2;
+                assert cache.localPeek("key" + rnd1) == null || cache.localPeek("key" + rnd1) == rnd1;
+                assert cache.localPeek("key" + rnd2) == null || cache.localPeek("key" + rnd2) == rnd2;
 
-                assert cache.peek("key" + rnd1) == null || cache.remove("key" + rnd1) == rnd1;
-                assert cache.peek("key" + rnd2) == null || cache.remove("key" + rnd2, rnd2);
+                assert cache.localPeek("key" + rnd1) == null || cache.getAndRemove("key" + rnd1) == rnd1;
+                assert cache.localPeek("key" + rnd2) == null || cache.remove("key" + rnd2, rnd2);
 
-                assert cache.peek("key" + rnd1) == null;
-                assert cache.peek("key" + rnd2) == null;
+                assert cache.localPeek("key" + rnd1) == null;
+                assert cache.localPeek("key" + rnd2) == null;
             }
         });
     }
@@ -305,22 +261,29 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws Exception In case of error.
      */
     public void testRemoveAsync() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) throws IgniteCheckedException {
                 int rnd1 = random();
                 int rnd2 = random();
 
-                assert cache.removeAsync("wrongKey").get() == null;
-                assert !cache.removeAsync("key" + rnd1, -1).get();
+                IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+                cacheAsync.getAndRemove("wrongKey");
+
+                assert cacheAsync.future().get() == null;
+
+                cacheAsync.remove("key" + rnd1, -1);
 
-                assert cache.peek("key" + rnd1) == null || cache.peek("key" + rnd1) == rnd1;
-                assert cache.peek("key" + rnd2) == null || cache.peek("key" + rnd2) == rnd2;
+                assert !cacheAsync.<Boolean>future().get();
 
-                assert cache.peek("key" + rnd1) == null || cache.removeAsync("key" + rnd1).get() == rnd1;
-                assert cache.peek("key" + rnd2) == null || cache.removeAsync("key" + rnd2, rnd2).get();
+                assert cache.localPeek("key" + rnd1) == null || cache.localPeek("key" + rnd1) == rnd1;
+                assert cache.localPeek("key" + rnd2) == null || cache.localPeek("key" + rnd2) == rnd2;
 
-                assert cache.peek("key" + rnd1) == null;
-                assert cache.peek("key" + rnd2) == null;
+                assert cache.localPeek("key" + rnd1) == null || removeAsync(cache, "key" + rnd1) == rnd1;
+                assert cache.localPeek("key" + rnd2) == null || removeAsync(cache, "key" + rnd2, rnd2);
+
+                assert cache.localPeek("key" + rnd1) == null;
+                assert cache.localPeek("key" + rnd2) == null;
             }
         });
     }
@@ -329,14 +292,14 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws Exception In case of error.
      */
     public void testRemoveAll() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) {
                 int rnd = random();
 
                 cache.removeAll(rangeKeys(0, rnd));
 
                 for (int i = 0; i < rnd; i++)
-                    assert cache.peek("key" + i) == null;
+                    assert cache.localPeek("key" + i) == null;
             }
         });
     }
@@ -344,60 +307,39 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
     /**
      * @throws Exception In case of error.
      */
-    public void testRemoveAllFiltered() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
-                final int rnd = random();
+    public void testRemoveAllAsync() throws Exception {
+        runTest(new CIX1<IgniteCache<String,Integer>>() {
+            @Override public void applyx(IgniteCache<String, Integer> cache) throws IgniteCheckedException {
+                int rnd = random();
 
-                cache.removeAll(new P1<CacheEntry<String, Integer>>() {
-                    @Override public boolean apply(CacheEntry<String, Integer> e) {
-                        Integer val = e.peek();
+                cache.withAsync().removeAll(rangeKeys(0, rnd));
 
-                        return val != null && val < rnd;
-                    }
-                });
+                cache.withAsync().future().get();
 
                 for (int i = 0; i < rnd; i++)
-                    assert cache.peek("key" + i) == null;
+                    assert cache.localPeek("key" + i) == null;
             }
         });
     }
 
     /**
-     * @throws Exception In case of error.
+     * @param cache Cache.
+     * @param key Key.
      */
-    public void testRemoveAllAsync() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
-                int rnd = random();
-
-                cache.removeAllAsync(rangeKeys(0, rnd)).get();
+    private <K, V> V removeAsync(IgniteCache<K, V> cache, K key) throws IgniteCheckedException {
+        cache.withAsync().getAndRemove(key);
 
-                for (int i = 0; i < rnd; i++)
-                    assert cache.peek("key" + i) == null;
-            }
-        });
+        return cache.withAsync().<V>future().get();
     }
 
     /**
-     * @throws Exception In case of error.
+     * @param cache Cache.
+     * @param key Key.
+     * @param val Value.
      */
-    public void testRemoveAllAsyncFiltered() throws Exception {
-        runTest(new CIX1<GridCache<String,Integer>>() {
-            @Override public void applyx(GridCache<String, Integer> cache) throws IgniteCheckedException {
-                final int rnd = random();
-
-                cache.removeAllAsync(new P1<CacheEntry<String, Integer>>() {
-                    @Override public boolean apply(CacheEntry<String, Integer> e) {
-                        Integer val = e.peek();
-
-                        return val != null && val < rnd;
-                    }
-                }).get();
+    private <K, V> boolean removeAsync(IgniteCache<K, V> cache, K key, V val) throws IgniteCheckedException {
+        cache.withAsync().remove(key, val);
 
-                for (int i = 0; i < rnd; i++)
-                    assert cache.peek("key" + i) == null;
-            }
-        });
+        return cache.withAsync().<Boolean>future().get();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 4f55e2f..3c434c4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -3155,7 +3155,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         for (String key : keys)
             assertNull(cache.get(key));
 
-        Map<String, Integer> vals = new HashMap<>(keys.size());
+        Map<String, Integer> vals = new HashMap<>();
 
         int i = 0;
 
@@ -3343,7 +3343,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     public void testClear() throws Exception {
-        GridCache<String, Integer> cache = cache();
+        IgniteCache<String, Integer> cache = jcache();
 
         Collection<String> keys = primaryKeysForCache(cache, 3);
 
@@ -3365,7 +3365,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         for (String key : keys)
             assertEquals(vals.get(key), peek(cache, key));
 
-        cache.clearAll();
+        cache.clear();
 
         for (String key : keys)
             assertNull(peek(cache, key));
@@ -3385,29 +3385,28 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         String first = F.first(keys);
 
         if (lockingEnabled()) {
-            assertTrue(cache.lock(first, 0));
+            Lock lock = cache.lock(first);
 
-            cache.clearAll();
+            lock.lock();
 
-            assertEquals(vals.get(first), peek(cache, first));
+            try {
+                cache.clear();
 
-            cache.unlock(first);
+                assertEquals(vals.get(first), peek(cache, first));
+            }
+            finally {
+                lock.unlock();
+            }
         }
         else {
-            cache.clearAll();
+            cache.clear();
 
             cache.put(first, vals.get(first));
         }
 
-        cache.projection(gte100).clear(first);
+        cache.clear();
 
-        assertNotNull(peek(cache, first));
-
-        cache.put(first, 101);
-
-        cache.projection(gte100).clear(first);
-
-        assert cache.isEmpty() : "Values after clear: " + cache.values();
+        assert cache.localSize() == 0 : "Values after clear.";
 
         i = 0;
 
@@ -3419,15 +3418,10 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             i++;
         }
 
-        for (String key : keys) {
-            if (!first.equals(key))
-                assertEquals(vals.get(key), peek(cache, key));
-        }
-
-        cache().put("key1", 1);
-        cache().put("key2", 2);
+        cache.put("key1", 1);
+        cache.put("key2", 2);
 
-        cache().evictAll();
+        cache.localEvict(ImmutableSet.of("key1", "key2"));
 
         assert cache().isEmpty();
 
@@ -3454,7 +3448,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         subKeys.remove(lastKey);
 
-        Map<String, Integer> vals = new HashMap<>(keys.size());
+        Map<String, Integer> vals = new HashMap<>();
 
         int i = 0;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
index 021a736..b8a839b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
@@ -18,11 +18,11 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 
+import javax.cache.*;
 import java.util.*;
 
 /**
@@ -37,7 +37,7 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
         super.beforeTest();
 
         for (int i = 0; i < entryCount(); i++)
-            cache().put(KEY_PREFIX + i, i);
+            jcache().put(KEY_PREFIX + i, i);
     }
 
     /**
@@ -51,40 +51,19 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
     public void testCacheIterator() throws Exception {
         int cnt = 0;
 
-        for (CacheEntry<String, Integer> entry : cache()) {
+        for (Cache.Entry<String, Integer> entry : jcache()) {
             assert entry != null;
             assert entry.getKey() != null;
             assert entry.getKey().contains(KEY_PREFIX);
             assert entry.getValue() != null;
             assert entry.getValue() >= 0 && entry.getValue() < entryCount();
-            assert entry.get() != null;
-            assert entry.get() >= 0 && entry.get() < entryCount();
-
-            cnt++;
-        }
-
-        assertEquals(cnt, entryCount());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCacheProjectionIterator() throws Exception {
-        int cnt = 0;
-
-        for (CacheEntry<String, Integer> entry : cache().projection(lt50)) {
-            assert entry != null;
-            assert entry.getKey() != null;
-            assert entry.getKey().contains(KEY_PREFIX);
             assert entry.getValue() != null;
-            assert entry.getValue() >= 0 && entry.getValue() < 50;
-            assert entry.get() != null;
-            assert entry.get() >= 0 && entry.get() < 50;
+            assert entry.getValue() >= 0 && entry.getValue() < entryCount();
 
             cnt++;
         }
 
-        assert cnt == 50;
+        assertEquals(cnt, entryCount());
     }
 
     /**
@@ -92,19 +71,19 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
      */
     public void testCacheIteratorMultithreaded() throws Exception {
         for (int i = 0; i < gridCount(); i++)
-            cache(i).removeAll();
+            jcache(i).removeAll();
 
         final IgniteInternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
+            @Override public void applyx() {
                 for (int i = 0; i < entryCount(); i++)
-                    cache().put(KEY_PREFIX + i, i);
+                    jcache().put(KEY_PREFIX + i, i);
             }
         }, 1, "put-thread");
 
         GridTestUtils.runMultiThreaded(new CA() {
             @Override public void apply() {
                 while (!putFut.isDone()) {
-                    for (CacheEntry<String, Integer> entry : cache()) {
+                    for (Cache.Entry<String, Integer> entry : jcache()) {
                         assert entry != null;
                         assert entry.getKey() != null;
                         assert entry.getKey().contains(KEY_PREFIX);
@@ -118,52 +97,23 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
      * @throws Exception If failed.
      */
     public void testEntrySetIterator() throws Exception {
-        Set<CacheEntry<String, Integer>> entries = cache().entrySet();
-
-        assert entries != null;
-        assert entries.size() == entryCount();
+        assert jcache().size() == entryCount();
 
         int cnt = 0;
 
-        for (CacheEntry<String, Integer> entry : entries) {
+        for (Cache.Entry<String, Integer> entry : jcache()) {
             assert entry != null;
             assert entry.getKey() != null;
             assert entry.getKey().contains(KEY_PREFIX);
             assert entry.getValue() != null;
             assert entry.getValue() >= 0 && entry.getValue() < entryCount();
-            assert entry.get() != null;
-            assert entry.get() >= 0 && entry.get() < entryCount();
-
-            cnt++;
-        }
-
-        assert cnt == entryCount();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testEntrySetIteratorFiltered() throws Exception {
-        Set<CacheEntry<String, Integer>> entries = cache().projection(lt50).entrySet();
-
-        assert entries != null;
-        assert entries.size() == 50;
-
-        int cnt = 0;
-
-        for (CacheEntry<String, Integer> entry : entries) {
-            assert entry != null;
-            assert entry.getKey() != null;
-            assert entry.getKey().contains(KEY_PREFIX);
             assert entry.getValue() != null;
-            assert entry.getValue() >= 0 && entry.getValue() < 50;
-            assert entry.get() != null;
-            assert entry.get() >= 0 && entry.get() < 50;
+            assert entry.getValue() >= 0 && entry.getValue() < entryCount();
 
             cnt++;
         }
 
-        assert cnt == 50;
+        assert cnt == entryCount();
     }
 
     /**
@@ -171,19 +121,19 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
      */
     public void testEntrySetIteratorMultithreaded() throws Exception {
         for (int i = 0; i < gridCount(); i++)
-            cache(i).removeAll();
+            jcache(i).removeAll();
 
         final IgniteInternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
+            @Override public void applyx() {
                 for (int i = 0; i < entryCount(); i++)
-                    cache().put(KEY_PREFIX + i, i);
+                    jcache().put(KEY_PREFIX + i, i);
             }
         }, 1, "put-thread");
 
         GridTestUtils.runMultiThreaded(new CA() {
             @Override public void apply() {
                 while (!putFut.isDone()) {
-                    for (CacheEntry<String, Integer> entry : cache().entrySet()) {
+                    for (Cache.Entry<String, Integer> entry : jcache()) {
                         assert entry != null;
                         assert entry.getKey() != null;
                         assert entry.getKey().contains(KEY_PREFIX);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractProjectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractProjectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractProjectionSelfTest.java
deleted file mode 100644
index acdeede..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractProjectionSelfTest.java
+++ /dev/null
@@ -1,884 +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.transactions.*;
-import org.apache.ignite.internal.util.typedef.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-import static java.util.concurrent.TimeUnit.*;
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.internal.processors.cache.CacheFlag.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * Tests for custom cache projection (with filters and flags).
- */
-public abstract class GridCacheAbstractProjectionSelfTest extends GridCacheAbstractSelfTest {
-    /** Test timeout */
-    private static final long TEST_TIMEOUT = 120 * 1000;
-
-    /** Number of grids to start. */
-    private static final int GRID_CNT = 1;
-
-    /** {@inheritDoc} */
-    @Override protected long getTestTimeout() {
-        return TEST_TIMEOUT;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return GRID_CNT;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setCacheMode(cacheMode());
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
-        cfg.setPreloadMode(CachePreloadMode.SYNC);
-
-        return cfg;
-    }
-
-    /**
-     * @return Cache mode.
-     */
-    @Override protected abstract CacheMode cacheMode();
-
-    /**
-     * @return Cache instance.
-     */
-    @SuppressWarnings({"TypeMayBeWeakened"})
-    private GridCache<String, TestCloneable> cacheCloneable() {
-        return grid(0).cache(null);
-    }
-
-    /**
-     * Test cloneable.
-     */
-    private static class TestCloneable implements Cloneable {
-        /** */
-        private String str;
-
-        /**
-         * Default constructor.
-         */
-        private TestCloneable() {
-            // No-op.
-        }
-
-        /**
-         * @param str String value.
-         */
-        private TestCloneable(String str) {
-            this.str = str;
-        }
-
-        /**
-         * @return str value.
-         */
-        private String str() {
-            return str;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object clone() throws CloneNotSupportedException {
-            return super.clone();
-        }
-    }
-
-    /** */
-    private IgniteBiPredicate<String, Integer> kvFilter = new P2<String, Integer>() {
-        @Override public boolean apply(String key, Integer val) {
-            return key.contains("key") && val >= 0;
-        }
-    };
-
-    /** */
-    private IgnitePredicate<CacheEntry<String, Integer>> entryFilter = new P1<CacheEntry<String, Integer>>() {
-        @Override public boolean apply(CacheEntry<String, Integer> e) {
-            Integer val = e.peek();
-
-            // Let's assume that null values will be passed through, otherwise we won't be able
-            // to put any new values to cache using projection with this entry filter.
-            return e.getKey().contains("key") && (val == null || val >= 0);
-        }
-    };
-
-    /**
-     * Asserts that given runnable throws specified exception.
-     *
-     * @param exCls Expected exception.
-     * @param r Runnable to check.
-     * @throws Exception If check failed.
-     */
-    private void assertException(Class<? extends Exception> exCls, Runnable r) throws Exception {
-        assert exCls != null;
-        assert r != null;
-
-        try {
-            r.run();
-
-            fail(exCls.getSimpleName() + " must have been thrown.");
-        }
-        catch (Exception e) {
-            if (e.getClass() != exCls)
-                throw e;
-
-            info("Caught expected exception: " + e);
-        }
-    }
-
-    /**
-     * @param r Runnable.
-     * @throws Exception If check failed.
-     */
-    private void assertFlagException(Runnable r) throws Exception {
-        assertException(CacheFlagException.class, r);
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testTypeProjection() throws Exception {
-        GridCache<String, Integer> cache = cache();
-
-        cache.putAll(F.asMap("k1", 1 , "k2", 2, "k3", 3));
-
-        GridCache<Double, Boolean> anotherCache = grid(0).cache(null);
-
-        assert anotherCache != null;
-
-        anotherCache.put(3.14, true);
-
-        CacheProjection<String, Integer> prj = cache.projection(String.class, Integer.class);
-
-        List<String> keys = F.asList("k1", "k2", "k3");
-
-        for (String key : keys)
-            assert prj.containsKey(key);
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testSize() throws Exception {
-        CacheProjection<String, Integer> prj = cache().projection(kvFilter);
-
-        assert prj.cache() != null;
-
-        int size = 10;
-
-        if (atomicityMode() == TRANSACTIONAL) {
-            IgniteTx tx = prj.txStart();
-
-            for (int i = 0; i < size; i++)
-                prj.put("key" + i, i);
-
-            prj.put("k", 11);
-            prj.put("key", -1);
-
-            tx.commit();
-        }
-        else {
-            for (int i = 0; i < size; i++)
-                prj.put("key" + i, i);
-
-            prj.put("k", 11);
-            prj.put("key", -1);
-        }
-
-        assertEquals(size, cache().size());
-        assertEquals(size, prj.size());
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testContainsKey() throws Exception {
-        cache().put("key", 1);
-        cache().put("k", 2);
-
-        assert cache().containsKey("key");
-        assert cache().containsKey("k");
-        assert !cache().containsKey("wrongKey");
-
-        CacheProjection<String, Integer> prj = cache().projection(kvFilter);
-
-        assert prj.containsKey("key");
-        assert !prj.containsKey("k");
-        assert !prj.containsKey("wrongKey");
-
-        assert prj.projection(F.<CacheEntry<String, Integer>>alwaysTrue()).containsKey("key");
-        assert !prj.projection(F.<CacheEntry<String, Integer>>alwaysFalse()).containsKey("key");
-        assert !prj.projection(F.<CacheEntry<String, Integer>>alwaysFalse()).containsKey("k");
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testPut() throws Exception {
-        final CacheProjection<String, Integer> prj = cache().projection(kvFilter);
-
-        prj.put("key", 1);
-        prj.put("k", 2);
-
-        assert prj.containsKey("key");
-        assert !prj.containsKey("k");
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                prj.flagsOn(LOCAL).put("key", 1);
-            }
-        });
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                prj.flagsOn(READ).put("key", 1);
-            }
-        });
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testLocalFlag() throws Exception {
-        CacheProjection<String, Integer> prj = cache().projection(entryFilter);
-
-        final CacheProjection<String, Integer> locPrj = prj.flagsOn(LOCAL);
-
-        prj.put("key", 1);
-
-        Integer one = 1;
-
-        assertEquals(one, prj.get("key"));
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locPrj.put("key", 1);
-            }
-        });
-
-        prj.get("key");
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locPrj.get("key");
-            }
-        });
-
-        prj.getAll(F.asList("key", "key1"));
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locPrj.getAll(F.asList("key", "key1"));
-            }
-        });
-
-        prj.remove("key");
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locPrj.remove("key");
-            }
-        });
-
-        prj.put("key", 1);
-
-        assertEquals(one, prj.replace("key", 2));
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locPrj.replace("key", 3);
-            }
-        });
-
-        prj.removeAll(F.asList("key"));
-
-        assert !prj.containsKey("key");
-
-        prj.put("key", 1);
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locPrj.removeAll(F.asList("key"));
-            }
-        });
-
-        assert prj.containsKey("key");
-
-        assert locPrj.containsKey("key");
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locPrj.reload("key");
-            }
-        });
-
-        assertEquals(one, locPrj.peek("key"));
-
-        locPrj.evict("key");
-
-        assert !locPrj.containsKey("key");
-
-        locPrj.promote("key");
-
-        assert locPrj.containsKey("key");
-
-        locPrj.clear("key");
-
-        assert !locPrj.containsKey("key");
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testEntryLocalFlag() throws Exception {
-        CacheProjection<String, Integer> prj = cache().projection(entryFilter);
-
-        CacheProjection<String, Integer> loc = prj.flagsOn(LOCAL);
-
-        prj.put("key", 1);
-
-        CacheEntry<String, Integer> prjEntry = prj.entry("key");
-        final CacheEntry<String, Integer> locEntry = loc.entry("key");
-
-        assert prjEntry != null;
-        assert locEntry != null;
-
-        Integer one = 1;
-
-        assertEquals(one, prjEntry.getValue());
-
-        assertFlagException(new CA() {
-            @Override public void apply() {
-                locEntry.setValue(1);
-            }
-        });
-
-        assertEquals(one, prjEntry.getValue());
-
-        assertFlagException(new CA() {
-            @Override public void apply() {
-                locEntry.getValue();
-            }
-        });
-
-        prjEntry.remove();
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locEntry.remove();
-            }
-        });
-
-        prjEntry.set(1);
-
-        assertEquals(one, prjEntry.replace(2));
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locEntry.replace(3);
-            }
-        });
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                locEntry.reload();
-            }
-        });
-
-        prj.put("key", 1);
-
-        assertEquals(one, locEntry.peek());
-
-        locEntry.evict();
-
-        assert locEntry.peek() == null;
-
-        loc.promote("key");
-
-        assert loc.containsKey("key");
-
-        locEntry.clear();
-
-        assert locEntry.peek() == null;
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testReadFlag() throws Exception {
-        CacheProjection<String, Integer> prj = cache().projection(entryFilter);
-
-        final CacheProjection<String, Integer> readPrj = prj.flagsOn(READ);
-
-        prj.put("key", 1);
-
-        Integer one = 1;
-
-        assertEquals(one, prj.get("key"));
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                readPrj.put("key", 1);
-            }
-        });
-
-        prj.remove("key");
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                readPrj.remove("key");
-            }
-        });
-
-        prj.put("key", 1);
-
-        assertEquals(one, prj.replace("key", 2));
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                readPrj.replace("key", 3);
-            }
-        });
-
-        prj.removeAll(F.asList("key"));
-
-        assert !prj.containsKey("key");
-
-        prj.put("key", 1);
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                readPrj.removeAll(F.asList("key"));
-            }
-        });
-
-        assertFlagException(new CA() {
-            @Override public void apply() {
-                readPrj.evict("key");
-            }
-        });
-
-        assert prj.containsKey("key");
-
-        assertFlagException(new CA() {
-            @Override public void apply() {
-                readPrj.clear("key");
-            }
-        });
-
-        assert prj.containsKey("key");
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                readPrj.reload("key");
-            }
-        });
-
-        assert prj.containsKey("key");
-
-        assertFlagException(new CAX() {
-            @Override public void applyx() throws IgniteCheckedException {
-                readPrj.promote("key");
-            }
-        });
-
-        assert prj.containsKey("key");
-
-        readPrj.get("key");
-
-        readPrj.getAll(F.asList("key", "key1"));
-
-        assertEquals(one, readPrj.peek("key"));
-    }
-
-    /**
-     * @param clone Cloned value.
-     * @param original Original value.
-     */
-    private void checkClone(TestCloneable clone, TestCloneable original) {
-        assert original != null;
-        assert clone != null;
-        assert clone != original;
-        assertEquals(clone.str(), original.str());
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    @SuppressWarnings({"UnnecessaryFinalOnLocalVariable"})
-    public void testCloneFlag() throws Exception {
-        CacheProjection<String, TestCloneable> prj = cacheCloneable().flagsOn(CLONE);
-
-        final TestCloneable val = new TestCloneable("val");
-
-        prj.put("key", val);
-
-        checkClone(prj.get("key"), val);
-
-        checkClone(prj.getAsync("key").get(), val);
-
-        Map<String, TestCloneable> map = prj.getAll(F.asList("key"));
-
-        assertEquals(1, map.size());
-
-        checkClone(map.get("key"), val);
-
-        map = prj.getAllAsync(F.asList("key")).get();
-
-        assertEquals(1, map.size());
-
-        checkClone(map.get("key"), val);
-
-        checkClone(prj.peek("key"), val);
-
-        Collection<TestCloneable> vals = prj.values();
-
-        assert vals != null;
-        assertEquals(1, vals.size());
-
-        checkClone(vals.iterator().next(), val);
-
-        Set<CacheEntry<String, TestCloneable>> entries = prj.entrySet();
-
-        assertEquals(1, entries.size());
-
-        checkClone(entries.iterator().next().getValue(), val);
-
-        CacheEntry<String, TestCloneable> entry = prj.entry("key");
-
-        assert entry != null;
-
-        checkClone(entry.peek(), val);
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testEntryParent() throws Exception {
-        cache().put("key", 1);
-
-        GridCacheProxyImpl<String, Integer> prj = (GridCacheProxyImpl<String, Integer>)cache().
-            flagsOn(CLONE, INVALIDATE);
-
-        CacheEntry<String, Integer> entry = prj.entry("key");
-
-        assert entry != null;
-
-        GridCacheProxyImpl<String, Integer> entryPrj = (GridCacheProxyImpl<String, Integer>)entry.projection();
-
-        assert entryPrj.delegate() == prj.delegate();
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testSkipStoreFlag() throws Exception {
-        assertNull(cache().put("kk1", 100500));
-        assertEquals(100500, map.get("kk1"));
-
-        IgniteCache<String, Integer> c = jcache().withSkipStore();
-
-        assertNull(c.getAndPut("noStore", 123));
-        assertEquals(123, (Object) c.get("noStore"));
-        assertNull(map.get("noStore"));
-
-        assertTrue(c.remove("kk1", 100500));
-        assertEquals(100500, map.get("kk1"));
-        assertNull(c.get("kk1"));
-        assertEquals(100500, (Object) cache().get("kk1"));
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testSkipStoreIterator() throws Exception {
-        assertNull(cache().put("1", 100500));
-
-        IgniteCache<String, Integer> c = jcache().withSkipStore();
-
-        Iterator i = c.iterator();
-
-        assertTrue(i.hasNext());
-
-        i.next();
-
-        i.remove();
-
-        i = c.iterator();
-
-        assertFalse(i.hasNext());
-
-        assertNull(c.get("1"));
-
-        assertEquals(100500, map.get("1"));
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testNotSkipStoreIterator() throws Exception {
-        assertNull(cache().put("1", 100500));
-
-        IgniteCache<String, Integer> c = jcache();
-
-        Iterator i = c.iterator();
-
-        assertTrue(i.hasNext());
-
-        i.next();
-
-        i.remove();
-
-        i = c.iterator();
-
-        assertFalse(i.hasNext());
-
-        assertNull(c.get("1"));
-
-        assertNull(map.get("1"));
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    // TODO: enable when GG-7579 is fixed.
-    public void _testSkipStoreFlagMultinode() throws Exception {
-        final int nGrids = 3;
-
-        // Start additional grids.
-        for (int i = 1; i < nGrids; i++)
-            startGrid(i);
-
-        try {
-            testSkipStoreFlag();
-        }
-        finally {
-            for (int i = 1; i < nGrids; i++)
-                stopGrid(i);
-        }
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testSkipSwapFlag() throws Exception {
-        cache().put("key", 1);
-
-        cache().evict("key");
-
-        assert cache().peek("key") == null;
-
-        Integer one = 1;
-
-        assertEquals(one, cache().get("key"));
-
-        cache().evict("key");
-
-        assertEquals(one, cache().reload("key"));
-
-        cache().remove("key");
-
-        assertFalse(cache().containsKey("key"));
-        assertNull(cache().get("key"));
-
-        CacheProjection<String, Integer> prj = cache().flagsOn(SKIP_SWAP, SKIP_STORE);
-
-        prj.put("key", 1);
-
-        assertEquals(one, prj.get("key"));
-        assertEquals(one, prj.peek("key"));
-
-        assert prj.evict("key");
-
-        assert prj.peek("key") == null;
-        assert prj.get("key") == null;
-    }
-
-    /**
-     * Checks that previous entry in update operations is taken
-     * from swap after eviction, even if SKIP_SWAP is enabled.
-     *
-     * @throws Exception If error happens.
-     */
-    public void testSkipSwapFlag2() throws Exception {
-        cache().put("key", 1);
-
-        cache().evict("key");
-
-        CacheProjection<String, Integer> prj = cache().flagsOn(SKIP_SWAP, SKIP_STORE);
-
-        assertNull(prj.get("key"));
-
-        Integer old = prj.put("key", 2);
-
-        assertEquals(Integer.valueOf(1), old); // Update operations on cache should not take into account SKIP_SWAP flag.
-
-        prj.remove("key");
-    }
-
-    /**
-     * Tests {@link CacheFlag#SKIP_SWAP} flag on multiple nodes.
-     *
-     * @throws Exception If error occurs.
-     */
-    public void testSkipSwapFlagMultinode() throws Exception {
-        final int nGrids = 3;
-
-        // Start additional grids.
-        for (int i = 1; i < nGrids; i++)
-            startGrid(i);
-
-        try {
-            final int nEntries = 100;
-
-            // Put the values in cache.
-            for (int i = 1; i <= nEntries; i++)
-                grid(0).cache(null).put(i, i);
-
-            // Evict values from cache. Values should go to swap.
-            for (int i = 0; i < nGrids; i++) {
-                grid(i).cache(null).evictAll();
-
-                assertTrue("Grid #" + i + " has empty swap.", grid(i).cache(null).swapIterator().hasNext());
-            }
-
-            // Set SKIP_SWAP flag.
-            CacheProjection<Object, Object> cachePrj = grid(0).cache(null).flagsOn(SKIP_SWAP, SKIP_STORE);
-
-            // Put new values.
-            for (int i = 1; i <= nEntries; i++)
-                assertEquals(i, cachePrj.put(i, i + 1)); // We should get previous values from swap, disregarding SKIP_SWAP.
-
-            // Swap should be empty now.
-            for (int i = 0; i < nGrids; i++)
-                assertFalse("Grid #" + i + " has non-empty swap.", grid(i).cache(null).swapIterator().hasNext());
-        }
-        finally {
-            // Stop started grids.
-            for (int i = 1; i < nGrids; i++)
-                stopGrid(i);
-        }
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testTx() throws Exception {
-        if (atomicityMode() == ATOMIC)
-            return;
-
-        IgniteTx tx = cache().txStart();
-
-        CacheProjection<String, Integer> typePrj = cache().projection(String.class, Integer.class);
-
-        typePrj.put("key", 1);
-        typePrj.put("k", 2);
-
-        CacheProjection<String, Integer> kvFilterPrj = cache().projection(kvFilter);
-
-        Integer one = 1;
-
-        assertEquals(one, kvFilterPrj.get("key"));
-        assert kvFilterPrj.get("k") == null;
-
-        CacheProjection<String, Integer> entryFilterPrj = cache().projection(entryFilter);
-
-        assertEquals(one, entryFilterPrj.get("key"));
-        assert entryFilterPrj.get("k") == null;
-
-        // Now will check projection on projection.
-        kvFilterPrj = typePrj.projection(kvFilter);
-
-        assertEquals(one, kvFilterPrj.get("key"));
-        assert kvFilterPrj.get("k") == null;
-
-        entryFilterPrj = typePrj.projection(entryFilter);
-
-        assertEquals(one, entryFilterPrj.get("key"));
-        assert entryFilterPrj.get("k") == null;
-
-        typePrj = cache().projection(entryFilter).projection(String.class, Integer.class);
-
-        assertEquals(one, typePrj.get("key"));
-        assertNull(typePrj.get("k"));
-
-        tx.commit();
-
-        TransactionsConfiguration tCfg = grid(0).configuration().getTransactionsConfiguration();
-
-        tx = cache().txStart(
-            tCfg.getDefaultTxConcurrency(),
-            tCfg.getDefaultTxIsolation(),
-            tCfg.getDefaultTxTimeout(),
-            0
-        );
-
-        // Try to change tx property.
-        assertFlagException(new CA() {
-            @Override public void apply() {
-                cache().flagsOn(INVALIDATE);
-            }
-        });
-
-        assertFlagException(new CA() {
-            @Override public void apply() {
-                cache().projection(entryFilter).flagsOn(INVALIDATE);
-            }
-        });
-
-        tx.commit();
-    }
-
-    /**
-     * @throws IgniteCheckedException In case of error.
-     */
-    public void testTypedProjection() throws Exception {
-        GridCache<Object, Object> cache = grid(0).cache(null);
-
-        cache.putx("1", "test string");
-        cache.putx("2", 0);
-
-        final CacheProjection<String, String> prj = cache.projection(String.class, String.class);
-
-        final CountDownLatch latch = new CountDownLatch(1);
-
-        prj.removeAll(new P1<CacheEntry<String, String>>() {
-            @Override
-            public boolean apply(CacheEntry<String, String> e) {
-                info(" --> " + e.peek().getClass());
-
-                latch.countDown();
-
-                return true;
-            }
-        });
-
-        assertTrue(latch.await(1, SECONDS));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index 0f9a11f..8dd2a5e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -89,14 +89,13 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        assert cache().tx() == null;
-        assert cache().isEmpty() : "Cache is not empty: " + cache().entrySet();
-        assert cache().keySet().isEmpty() : "Key set is not empty: " + cache().keySet();
+        assert grid(0).transactions().tx() == null;
+        assert jcache().localSize() == 0;
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        IgniteTx tx = cache().tx();
+        IgniteTx tx = grid(0).transactions().tx();
 
         if (tx != null) {
             tx.close();
@@ -176,10 +175,8 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
             }
         }
 
-        assert cache().tx() == null;
-        assert cache().isEmpty() : "Cache is not empty: " + cache().entrySet();
-        assertEquals("Cache is not empty: " + cache().entrySet(), 0, cache().size());
-        assert cache().keySet().isEmpty() : "Key set is not empty: " + cache().keySet();
+        assert grid(0).transactions().tx() == null;
+        assertEquals("Cache is not empty", 0, jcache().size());
 
         resetStore();
     }
@@ -409,6 +406,14 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param cache Cache.
+     * @return {@code True} if cache has OFFHEAP_TIERED memory mode.
+     */
+    protected <K, V> boolean offheapTiered(IgniteCache<K, V> cache) {
+        return cache.getConfiguration(CacheConfiguration.class).getMemoryMode() == OFFHEAP_TIERED;
+    }
+
+    /**
      * Executes regular peek or peek from swap.
      *
      * @param prj Cache projection.
@@ -421,6 +426,18 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Executes regular peek or peek from swap.
+     *
+     * @param cache Cache projection.
+     * @param key Key.
+     * @return Value.
+     * @throws Exception If failed.
+     */
+    @Nullable protected <K, V> V peek(IgniteCache<K, V> cache, K key) throws Exception {
+        return offheapTiered(cache) ? cache.localPeek(key, CachePeekMode.SWAP) : cache.localPeek(key);
+    }
+
+    /**
      * @param cache Cache.
      * @param key Key.
      * @return {@code True} if cache contains given key.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
index 1bf6d8e..21b9183 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
@@ -76,7 +76,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testPartitions() throws Exception {
-        assertEquals(affinity().partitions(), cache().affinity().partitions());
+        assertEquals(affinity().partitions(), grid(0).affinity(null).partitions());
     }
 
     /**
@@ -87,7 +87,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
     public void testPartition() throws Exception {
         String key = "key";
 
-        assertEquals(affinity().partition(key), cache().affinity().partition(key));
+        assertEquals(affinity().partition(key), grid(0).affinity(null).partition(key));
     }
 
     /**
@@ -102,7 +102,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
         List<List<ClusterNode>> assignment = affinity().assignPartitions(ctx);
 
         for (ClusterNode node : grid(0).nodes()) {
-            int[] parts = cache().affinity().primaryPartitions(node);
+            int[] parts = grid(0).affinity(null).primaryPartitions(node);
 
             assert !F.isEmpty(parts);
 
@@ -127,7 +127,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
         // Pick 2 nodes and create a projection over them.
         ClusterNode n0 = grid(0).localNode();
 
-        int[] parts = cache().affinity().primaryPartitions(n0);
+        int[] parts = grid(0).affinity(null).primaryPartitions(n0);
 
         info("Primary partitions count: " + parts.length);
 
@@ -164,7 +164,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
         ClusterNode n0 = grid(0).localNode();
 
         // Get backup partitions without explicitly specified levels.
-        int[] parts = cache().affinity().backupPartitions(n0);
+        int[] parts = grid(0).affinity(null).backupPartitions(n0);
 
         assert !F.isEmpty(parts);
 
@@ -197,7 +197,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
         // Pick 2 nodes and create a projection over them.
         ClusterNode n0 = grid(0).localNode();
 
-        int[] parts = cache().affinity().allPartitions(n0);
+        int[] parts = grid(0).affinity(null).allPartitions(n0);
 
         assert !F.isEmpty(parts);
 
@@ -230,7 +230,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
 
         List<List<ClusterNode>> assignment = aff.assignPartitions(ctx);
 
-        assertEquals(F.first(nodes(assignment, aff, part)), cache().affinity().mapPartitionToNode(part));
+        assertEquals(F.first(nodes(assignment, aff, part)), grid(0).affinity(null).mapPartitionToNode(part));
     }
 
     /**
@@ -239,7 +239,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testMapPartitionsToNode() throws Exception {
-        Map<Integer, ClusterNode> map = cache().affinity().mapPartitionsToNodes(F.asList(0, 1, 5, 19, 12));
+        Map<Integer, ClusterNode> map = grid(0).affinity(null).mapPartitionsToNodes(F.asList(0, 1, 5, 19, 12));
 
         CacheAffinityFunctionContext ctx =
             new GridCacheAffinityFunctionContextImpl(new ArrayList<>(grid(0).nodes()), null, null, 1, 1);
@@ -258,7 +258,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testMapPartitionsToNodeArray() throws Exception {
-        Map<Integer, ClusterNode> map = cache().affinity().mapPartitionsToNodes(F.asList(0, 1, 5, 19, 12));
+        Map<Integer, ClusterNode> map = grid(0).affinity(null).mapPartitionsToNodes(F.asList(0, 1, 5, 19, 12));
 
         CacheAffinityFunctionContext ctx =
             new GridCacheAffinityFunctionContextImpl(new ArrayList<>(grid(0).nodes()), null, null, 1, 1);
@@ -282,7 +282,7 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
         for (int p = 0; p < affinity().partitions(); p++)
             parts.add(p);
 
-        Map<Integer, ClusterNode> map = cache().affinity().mapPartitionsToNodes(parts);
+        Map<Integer, ClusterNode> map = grid(0).affinity(null).mapPartitionsToNodes(parts);
 
         CacheAffinityFunctionContext ctx =
             new GridCacheAffinityFunctionContextImpl(new ArrayList<>(grid(0).nodes()), null, null, 1, 1);
@@ -317,28 +317,6 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
     }
 
     /**
-     * JUnit.
-     *
-     * @throws Exception If failed.
-     */
-    public void testEntryPartition() throws Exception {
-        int keyCnt = 100;
-
-        for (int kv = 0; kv < keyCnt; kv++)
-            cache().put(String.valueOf(kv), kv);
-
-        for (int kv = 0; kv < keyCnt; kv++) {
-            String key = String.valueOf(kv);
-
-            CacheEntry<String, Integer> entry = cache().entry(key);
-
-            assert entry != null;
-
-            assertEquals(affinity().partition(key), entry.partition());
-        }
-    }
-
-    /**
      * @throws Exception If failed.
      */
     public void testPartitionWithAffinityMapper() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
index b890acc..d40f59e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
@@ -57,7 +58,11 @@ public class GridCacheAsyncOperationsLimitSelfTest extends GridCacheAbstractSelf
 
             cnt.incrementAndGet();
 
-            IgniteInternalFuture<Boolean> fut = cache().putxAsync("key" + i, i);
+            IgniteCache<String, Integer> cacheAsync = jcache().withAsync();
+
+            cacheAsync.put("key" + i, i);
+
+            IgniteInternalFuture<Boolean> fut = cacheAsync.future();
 
             fut.listenAsync(new CI1<IgniteInternalFuture<Boolean>>() {
                 @Override public void apply(IgniteInternalFuture<Boolean> t) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7d4002ae/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
index 912f424..2047c16 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
@@ -118,16 +118,16 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
      * @throws IgniteCheckedException If failed.
      */
     public void testNotExistingKeys() throws IgniteCheckedException {
-        GridCache<Integer, String> cache = cache();
+        IgniteCache<Integer, String> cache = jcache();
         Map<Integer, String> map = store.getMap();
 
         cache.put(100, "hacuna matata");
         assertEquals(1, map.size());
 
-        cache.evict(100);
+        cache.localEvict(Collections.<Integer>singleton(100));
         assertEquals(1, map.size());
 
-        assertEquals("hacuna matata", cache.remove(100));
+        assertEquals("hacuna matata", cache.getAndRemove(100));
         assertTrue(map.isEmpty());
 
         store.resetLastMethod();
@@ -142,16 +142,16 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
     /** @throws Exception If test fails. */
     public void testWriteThrough() throws Exception {
-        GridCache<Integer, String> cache = cache();
+        IgniteCache<Integer, String> cache = jcache();
 
         Map<Integer, String> map = store.getMap();
 
         assert map.isEmpty();
 
         if (atomicityMode() == TRANSACTIONAL) {
-            try (IgniteTx tx = cache.txStart(OPTIMISTIC, REPEATABLE_READ)) {
+            try (IgniteTx tx = grid().transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
                 for (int i = 1; i <= 10; i++) {
-                    cache.putx(i, Integer.toString(i));
+                    cache.put(i, Integer.toString(i));
 
                     checkLastMethod(null);
                 }
@@ -182,9 +182,9 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
         store.resetLastMethod();
 
         if (atomicityMode() == TRANSACTIONAL) {
-            try (IgniteTx tx = cache.txStart()) {
+            try (IgniteTx tx = grid().transactions().txStart()) {
                 for (int i = 1; i <= 10; i++) {
-                    String val = cache.remove(i);
+                    String val = cache.getAndRemove(i);
 
                     checkLastMethod(null);
 
@@ -198,7 +198,7 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
             }
         }
         else {
-            Collection<Integer> keys = new ArrayList<>(10);
+            Set<Integer> keys = new HashSet<>();
 
             for (int i = 1; i <= 10; i++)
                 keys.add(i);
@@ -213,16 +213,16 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
     /** @throws Exception If test failed. */
     public void testReadThrough() throws Exception {
-        GridCache<Integer, String> cache = cache();
+        IgniteCache<Integer, String> cache = jcache();
 
         Map<Integer, String> map = store.getMap();
 
         assert map.isEmpty();
 
         if (atomicityMode() == TRANSACTIONAL) {
-            try (IgniteTx tx = cache.txStart(OPTIMISTIC, REPEATABLE_READ)) {
+            try (IgniteTx tx = grid().transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
                 for (int i = 1; i <= 10; i++)
-                    cache.putx(i, Integer.toString(i));
+                    cache.put(i, Integer.toString(i));
 
                 checkLastMethod(null);
 
@@ -247,10 +247,10 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
             assert val.equals(Integer.toString(i));
         }
 
-        cache.clearAll();
+        cache.clear();
 
-        assert cache.isEmpty();
-        assert cache.isEmpty();
+        assert cache.localSize() == 0;
+        assert cache.localSize() == 0;
 
         assert map.size() == 10;
 
@@ -266,14 +266,14 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
         assert cache.size() == 10;
 
-        cache.clearAll();
+        cache.clear();
 
-        assert cache.isEmpty();
-        assert cache.isEmpty();
+        assert cache.localSize() == 0;
+        assert cache.localSize() == 0;
 
         assert map.size() == 10;
 
-        Collection<Integer> keys = new ArrayList<>();
+        Set<Integer> keys = new HashSet<>();
 
         for (int i = 1; i <= 10; i++)
             keys.add(i);
@@ -298,15 +298,15 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
         checkLastMethod("removeAll");
 
-        assert cache.isEmpty();
-        assert cache.isEmpty();
+        assert cache.localSize() == 0;
+        assert cache.localSize() == 0;
 
         assert map.isEmpty();
     }
 
     /** @throws Exception If test failed. */
     public void testLoadCache() throws Exception {
-        GridCache<Integer, String> cache = cache();
+        IgniteCache<Integer, String> cache = jcache();
 
         int cnt = 1;
 
@@ -314,9 +314,9 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
         checkLastMethod("loadAllFull");
 
-        assert !cache.isEmpty();
+        assert !(cache.localSize() == 0);
 
-        Map<Integer, String> map = cache.getAll(cache.keySet());
+        Map<Integer, String> map = cache.getAll(keySet(cache));
 
         assert map.size() == cnt : "Invalid map size: " + map.size();
 
@@ -336,7 +336,7 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
     /** @throws Exception If test failed. */
     public void testLoadCacheWithPredicate() throws Exception {
-        GridCache<Integer, String> cache = cache();
+        IgniteCache<Integer, String> cache = jcache();
 
         int cnt = 10;
 
@@ -349,7 +349,7 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
         checkLastMethod("loadAllFull");
 
-        Map<Integer, String> map = cache.getAll(cache.keySet());
+        Map<Integer, String> map = cache.getAll(keySet(cache));
 
         assert map.size() == cnt / 2 : "Invalid map size: " + map.size();