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/02 16:52:31 UTC

incubator-ignite git commit: IGNITE-56 Change all tests to use new cache API, add JCache TCK (4)

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-56 c10fad8bd -> 625b33f9d


IGNITE-56 Change all tests to use new cache API, add JCache TCK (4)


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

Branch: refs/heads/ignite-56
Commit: 625b33f9d35947d4f53cc6ca215a9cd33c99294e
Parents: c10fad8
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Feb 2 18:51:48 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Feb 2 18:51:48 2015 +0300

----------------------------------------------------------------------
 .../near/GridCacheNearMultiNodeSelfTest.java    | 157 +++++++-------
 .../near/GridCacheNearOneNodeSelfTest.java      |  36 ++--
 .../near/GridCacheNearReadersSelfTest.java      |  14 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |  47 +++--
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |   4 +-
 .../GridCachePartitionedEntryLockSelfTest.java  |  74 -------
 ...titionedExplicitLockNodeFailureSelfTest.java |  24 +--
 ...GridCachePartitionedFilteredPutSelfTest.java |  60 +-----
 .../GridCachePartitionedFullApiSelfTest.java    |   4 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |   9 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java | 211 +++----------------
 .../GridCachePartitionedTxSalvageSelfTest.java  |   4 +-
 .../near/GridPartitionedBackupLoadSelfTest.java |  18 +-
 13 files changed, 193 insertions(+), 469 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
index 7aa9482..d2c5a25 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
@@ -19,21 +19,20 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.GridCache;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 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.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.configuration.*;
@@ -43,8 +42,8 @@ import java.util.concurrent.atomic.*;
 import java.util.concurrent.locks.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CacheDistributionMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.GridCachePeekMode.*;
 import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
 import static org.apache.ignite.transactions.IgniteTxIsolation.*;
@@ -341,7 +340,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
 
         info("Local node: " + U.toShortString(loc));
 
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
         int cnt = 10;
 
@@ -382,13 +381,13 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"ConstantConditions"})
     public void testOptimisticWriteThrough() throws Exception {
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
         if (transactional()) {
-            try (IgniteTx tx = near.txStart(OPTIMISTIC, REPEATABLE_READ, 0, 0)) {
-                near.putx(2, "2");
+            try (IgniteTx tx = grid(0).transactions().txStart(OPTIMISTIC, REPEATABLE_READ, 0, 0)) {
+                near.put(2, "2");
 
-                String s = near.put(3, "3");
+                String s = near.getAndPut(3, "3");
 
                 assertNotNull(s);
                 assertEquals("3", s);
@@ -407,54 +406,54 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
             }
         }
         else {
-            near.putx(2, "2");
+            near.put(2, "2");
 
-            String s = near.put(3, "3");
+            String s = near.getAndPut(3, "3");
 
             assertNotNull(s);
             assertEquals("3", s);
         }
 
-        assertEquals("2", near.peek(2));
-        assertEquals("3", near.peek(3));
+        assertEquals("2", near.localPeek(2));
+        assertEquals("3", near.localPeek(3));
 
         assertEquals("2", dht(primaryGrid(2)).peek(2));
         assertEquals("3", dht(primaryGrid(3)).peek(3));
 
-        assertEquals(2, near.size());
-        assertEquals(2, near.size());
+        assertEquals(2, near.localSize());
+        assertEquals(2, near.localSize());
     }
 
     /** @throws Exception If failed. */
     public void testNoTransactionSinglePutx() throws Exception {
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
-        near.putx(2, "2");
+        near.put(2, "2");
 
-        assertEquals("2", near.peek(2));
+        assertEquals("2", near.localPeek(2));
         assertEquals("2", near.get(2));
 
         assertEquals("2", dht(primaryGrid(2)).peek(2));
 
-        assertEquals(1, near.size());
-        assertEquals(1, near.size());
+        assertEquals(1, near.localSize());
+        assertEquals(1, near.localSize());
 
         assertEquals(1, dht(primaryGrid(2)).size());
     }
 
     /** @throws Exception If failed. */
     public void testNoTransactionSinglePut() throws Exception {
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
         // There should be a not-null previously mapped value because
         // we use a store implementation that just returns values which
         // are string representations of requesting integer keys.
-        String s = near.put(3, "3");
+        String s = near.getAndPut(3, "3");
 
         assertNotNull(s);
         assertEquals("3", s);
 
-        assertEquals("3", near.peek(3));
+        assertEquals("3", near.localPeek(3));
         assertEquals("3", near.get(3));
 
         Ignite primaryIgnite = primaryGrid(3);
@@ -465,8 +464,8 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
 
         assertEquals("3", dht(primaryIgnite).peek(3));
 
-        assertEquals(1, near.size());
-        assertEquals(1, near.size());
+        assertEquals(1, near.localSize());
+        assertEquals(1, near.localSize());
 
         assertEquals(1, dht(primaryIgnite).size());
 
@@ -486,17 +485,17 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testNoTransactionWriteThrough() throws Exception {
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
-        near.putx(2, "2");
+        near.put(2, "2");
 
-        String s = near.put(3, "3");
+        String s = near.getAndPut(3, "3");
 
         assertNotNull(s);
         assertEquals("3", s);
 
-        assertEquals("2", near.peek(2));
-        assertEquals("3", near.peek(3));
+        assertEquals("2", near.localPeek(2));
+        assertEquals("3", near.localPeek(3));
 
         assertEquals("2", near.get(2));
         assertEquals("3", near.get(3));
@@ -504,8 +503,8 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
         assertEquals("2", dht(primaryGrid(2)).peek(2));
         assertEquals("3", dht(primaryGrid(3)).peek(3));
 
-        assertEquals(2, near.size());
-        assertEquals(2, near.size());
+        assertEquals(2, near.localSize());
+        assertEquals(2, near.localSize());
     }
 
     /**
@@ -515,19 +514,19 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"ConstantConditions"})
     public void testPessimisticWriteThrough() throws Exception {
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
         if (transactional()) {
-            try (IgniteTx tx = near.txStart(PESSIMISTIC, REPEATABLE_READ, 0, 0)) {
-                assertTrue(near.putx(2, "2"));
+            try (IgniteTx tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 0, 0)) {
+                near.put(2, "2");
 
-                String s = near.put(3, "3");
+                String s = near.getAndPut(3, "3");
 
                 assertNotNull(s);
                 assertEquals("3", s);
 
-                assertEquals("2", near.peek(2));
-                assertEquals("3", near.peek(3));
+                assertEquals("2", near.localPeek(2));
+                assertEquals("3", near.localPeek(3));
 
                 assertNotNull(dht(primaryGrid(3)).peek(3, F.asList(GLOBAL)));
 
@@ -539,22 +538,22 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
             }
         }
         else {
-            assertTrue(near.putx(2, "2"));
+            near.put(2, "2");
 
-            String s = near.put(3, "3");
+            String s = near.getAndPut(3, "3");
 
             assertNotNull(s);
             assertEquals("3", s);
         }
 
-        assertEquals("2", near.peek(2));
-        assertEquals("3", near.peek(3));
+        assertEquals("2", near.localPeek(2));
+        assertEquals("3", near.localPeek(3));
 
         assertEquals("2", dht(primaryGrid(2)).peek(2));
         assertEquals("3", dht(primaryGrid(3)).peek(3));
 
-        assertEquals(2, near.size());
-        assertEquals(2, near.size());
+        assertEquals(2, near.localSize());
+        assertEquals(2, near.localSize());
     }
 
     /** @throws Exception If failed. */
@@ -562,30 +561,30 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
         // Don't create missing values.
         store.create(false);
 
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
         int key = 1;
 
-        assertTrue(near.putxIfAbsent(key, "1"));
-        assertFalse(near.putxIfAbsent(key, "1"));
-        assertEquals("1", near.putIfAbsent(key, "2"));
+        assertTrue(near.putIfAbsent(key, "1"));
+        assertFalse(near.putIfAbsent(key, "1"));
+        assertEquals("1", near.getAndPutIfAbsent(key, "2"));
 
-        assertEquals("1", near.peek(key));
-        assertEquals(1, near.size());
-        assertEquals(1, near.size());
+        assertEquals("1", near.localPeek(key));
+        assertEquals(1, near.localSize());
+        assertEquals(1, near.localSize());
 
-        assertEquals("1", near.replace(key, "2"));
-        assertEquals("2", near.peek(key));
+        assertEquals("1", near.getAndReplace(key, "2"));
+        assertEquals("2", near.localPeek(key));
 
-        assertTrue(near.replacex(key, "2"));
+        assertTrue(near.replace(key, "2"));
 
-        assertEquals("2", near.peek(key));
-        assertEquals(1, near.size());
-        assertEquals(1, near.size());
+        assertEquals("2", near.localPeek(key));
+        assertEquals(1, near.localSize());
+        assertEquals(1, near.localSize());
 
         assertTrue(near.remove(key, "2"));
 
-        assertEquals(0, near.size());
+        assertEquals(0, near.localSize());
     }
 
     /** @throws Exception If failed. */
@@ -603,7 +602,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     private void checkBackupConsistency(int key) throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         String val = Integer.toString(key);
 
@@ -657,11 +656,11 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
         if (!transactional())
             return;
 
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         String val = Integer.toString(key);
 
-        Collection<ClusterNode> affNodes = cache.affinity().mapKeyToPrimaryAndBackups(key);
+        Collection<ClusterNode> affNodes = grid(0).affinity(null).mapKeyToPrimaryAndBackups(key);
 
         info("Affinity for key [nodeId=" + U.nodeIds(affNodes) + ", key=" + key + ']');
 
@@ -673,7 +672,9 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
 
         info("Primary local: " + primary.isLocal());
 
-        cache.lock(key, 0);
+        Lock lock = cache.lock(key);
+
+        lock.lock();
 
         try {
             long topVer = grid(0).topologyVersion();
@@ -685,8 +686,8 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
             assertNotNull(nearEntry1);
             assertTrue("Invalid near entry: " + nearEntry1, nearEntry1.valid(topVer));
 
-            assertTrue(cache.isLocked(key));
-            assertTrue(cache.isLockedByThread(key));
+            assertTrue(cache.isLocalLocked(key, false));
+            assertTrue(cache.isLocalLocked(key, true));
 
             cache.put(key, val);
 
@@ -699,7 +700,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
             assertNotNull(nearEntry2);
             assertTrue("Invalid near entry [hash=" + nearEntry2, nearEntry2.valid(topVer));
 
-            assertEquals(val, cache.peek(key));
+            assertEquals(val, cache.localPeek(key));
             assertEquals(val, dht(0).peek(key));
             assertEquals(val, dht(1).peek(key));
 
@@ -716,23 +717,23 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
             assertNull(near(1).peekNearOnly(key));
 
             assertEquals(val, cache.get(key));
-            assertEquals(val, cache.remove(key));
+            assertEquals(val, cache.getAndRemove(key));
 
-            assertNull(cache.peek(key));
+            assertNull(cache.localPeek(key));
             assertNull(dht(primaryGrid(key)).peek(key));
 
-            assertTrue(cache.isLocked(key));
-            assertTrue(cache.isLockedByThread(key));
+            assertTrue(cache.isLocalLocked(key, false));
+            assertTrue(cache.isLocalLocked(key, true));
         }
         finally {
-            cache.unlock(key);
+            lock.unlock();
         }
 
         assertNull(near(0).peekNearOnly(key));
         assertNull(near(1).peekNearOnly(key));
 
         assertFalse(near(0).isLockedNearOnly(key));
-        assertFalse(cache.isLockedByThread(key));
+        assertFalse(cache.isLocalLocked(key, true));
     }
 
     /** @throws Throwable If failed. */
@@ -817,7 +818,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     private void checkTransactionSingleGet(int key) throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         String val = Integer.toString(key);
 
@@ -831,7 +832,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
 
         if (transactional()) {
 
-            try (IgniteTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            try (IgniteTx tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
                 // Simple transaction get.
                 assertEquals(val, cache.get(key));
 
@@ -863,7 +864,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void checkTransactionSingleGetRemove(int key) throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Object, Object> cache = jcache(0);
 
         String val = Integer.toString(key);
 
@@ -876,12 +877,12 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
         assertNull(near(1).peekNearOnly(key));
 
         if (transactional()) {
-            try (IgniteTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            try (IgniteTx tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
                 // Read.
                 assertEquals(val, cache.get(key));
 
                 // Remove.
-                assertTrue(cache.removex(key));
+                assertTrue(cache.remove(key));
 
                 tx.commit();
             }
@@ -891,7 +892,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
             assertEquals(val, cache.get(key));
 
             // Remove.
-            assertTrue(cache.removex(key));
+            assertTrue(cache.remove(key));
         }
 
         assertNull(dht(0).peek(key));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
index d39715d..6e68702 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
@@ -56,11 +56,11 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
     @Override protected void afterTest() throws Exception {
         store.reset();
 
-        cache().removeAll();
+        jcache().removeAll();
 
         assertEquals("DHT entries: " + dht().entries(), 0, dht().size());
         assertEquals("Near entries: " + near().entries(), 0, near().size());
-        assertEquals("Cache entries: " + cache().entrySet(), 0, cache().size());
+        assertEquals(0, jcache().size());
     }
 
     /** {@inheritDoc} */
@@ -95,35 +95,35 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testRemove() throws Exception {
-        GridCache<Integer, String> near = cache();
+        IgniteCache<Object, Object> near = jcache();
 
         assertEquals("DHT entries: " + dht().entries(), 0, dht().size());
         assertEquals("Near entries: " + near().entries(), 0, near().size());
-        assertEquals("Cache entries: " + cache().entrySet(), 0, cache().size());
+        assertEquals(0, near.size());
 
         for (int i = 0; i < 10; i++)
             near.put(i, Integer.toString(i));
 
         assertEquals("DHT entries: " + dht().entries(), 10, dht().size());
         assertEquals("Near entries: " + near().entries(), 10, near().size());
-        assertEquals("Cache entries: " + cache().entrySet(), 10, cache().size());
+        assertEquals(10, near.size());
 
-        cache().remove(0);
+        near.remove(0);
 
         assertEquals("DHT entries: " + dht().entries(), 9, dht().size());
         assertEquals("Near entries: " + near().entries(), 9, near().size());
-        assertEquals("Cache entries: " + cache().entrySet(), 9, cache().size());
+        assertEquals(9, near.size());
 
-        cache().removeAll();
+        near.removeAll();
 
         assertEquals("DHT entries: " + dht().entries(), 0, dht().size());
         assertEquals("Near entries: " + near().entries(), 0, near().size());
-        assertEquals("Cache entries: " + cache().entrySet(), 0, cache().size());
+        assertEquals(0, near.size());
     }
 
     /** @throws Exception If failed. */
     public void testReadThrough() throws Exception {
-        GridCache<Integer, String> near = cache();
+        IgniteCache<Integer, String> near = jcache();
 
         GridCache<Integer, String> dht = dht();
 
@@ -153,11 +153,11 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"ConstantConditions"})
     public void testOptimisticTxWriteThrough() throws Exception {
-        GridCache<Integer, String> near = cache();
+        IgniteCache<Object, Object> near = jcache();
         GridCacheAdapter<Integer, String> dht = dht();
 
-        try (IgniteTx tx = cache().txStart(OPTIMISTIC, REPEATABLE_READ) ) {
-            near.putx(2, "2");
+        try (IgniteTx tx = grid().transactions().txStart(OPTIMISTIC, REPEATABLE_READ) ) {
+            near.put(2, "2");
             near.put(3, "3");
 
             assert "2".equals(near.get(2));
@@ -283,14 +283,14 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testTransactionSingleGet() throws Exception {
-        GridCache<Integer, String> cache = cache();
+        IgniteCache<Object, Object> cache = jcache();
 
         cache.put(1, "val1");
 
         assertEquals("val1", dht().peek(1));
         assertNull(near().peekNearOnly(1));
 
-        IgniteTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ);
+        IgniteTx tx = grid().transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
 
         assertEquals("val1", cache.get(1));
 
@@ -302,18 +302,18 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testTransactionSingleGetRemove() throws Exception {
-        GridCache<Integer, String> cache = cache();
+        IgniteCache<Object, Object> cache = jcache();
 
         cache.put(1, "val1");
 
         assertEquals("val1", dht().peek(1));
         assertNull(near().peekNearOnly(1));
 
-        IgniteTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ);
+        IgniteTx tx = grid().transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
 
         assertEquals("val1", cache.get(1));
 
-        assertTrue(cache.removex(1));
+        assertTrue(cache.remove(1));
 
         tx.commit();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
index f658a15..df7a925 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
@@ -432,9 +432,9 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
 
         assertEquals(grid(1).localNode(), F.first(aff.nodes(aff.partition(key2), grid(1).nodes())));
 
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
-        assertNull(cache.put(key1, val1));
+        assertNull(cache.getAndPut(key1, val1));
 
         assertEquals(val1, dht(0).peek(key1));
         assertEquals(val1, dht(1).peek(key1));
@@ -444,7 +444,7 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
         assertNull(near(1).peekNearOnly(key1));
         assertNull(near(2).peekNearOnly(key1));
 
-        assertTrue(cache.putx(key2, val2));
+        cache.put(key2, val2);
 
         assertNull(dht(0).peek(key2));
         assertEquals(val2, dht(1).peek(key2));
@@ -487,12 +487,10 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
         assertNull(near(2).peekNearOnly(key1));
 
         for (int i = 0; i < grids; i++) {
-            assert !cache(i).isLocked(key1);
-            assert !cache(i).isLocked(key2);
+            assert !jcache(i).isLocalLocked(key1, false);
+            assert !jcache(i).isLocalLocked(key2, false);
 
-            assert cache(i).size() == 0;
-            assert cache(i).isEmpty();
-            assert cache(i).size() == 0;
+            assert jcache(i).localSize() == 0;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
index a0d68f6..d7d9b2a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
@@ -60,7 +61,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         for (int i = 0; i < GRID_CNT; i++)
-            cache(i).removeAll();
+            jcache(i).removeAll();
 
         for (GridCacheTestStore store : stores)
             store.reset();
@@ -124,7 +125,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testPutFromPrimary() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         int key = 0;
 
@@ -132,7 +133,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
             boolean found = false;
 
             for (ClusterNode n : grid(0).nodes()) {
-                if (cache.affinity().isPrimary(n, key)) {
+                if (grid(0).affinity(null).isPrimary(n, key)) {
                     found = true;
 
                     break;
@@ -143,7 +144,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
                 break;
         }
 
-        assertNull(cache.put(key, "val"));
+        assertNull(cache.getAndPut(key, "val"));
 
         checkStoreUsage(1, 1, 0, 1);
     }
@@ -152,7 +153,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testPutFromBackup() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         int key = 0;
 
@@ -160,7 +161,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
             boolean found = false;
 
             for (ClusterNode n : grid(0).nodes()) {
-                if (cache.affinity().isBackup(n, key)) {
+                if (grid(0).affinity(null).isBackup(n, key)) {
                     found = true;
 
                     break;
@@ -171,7 +172,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
                 break;
         }
 
-        assertNull(cache.put(key, "val"));
+        assertNull(cache.getAndPut(key, "val"));
 
         checkStoreUsage(1, 1, 0, 1);
     }
@@ -180,7 +181,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testPutFromNear() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         int key = 0;
 
@@ -188,7 +189,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
             boolean found = false;
 
             for (ClusterNode n : grid(0).nodes()) {
-                if (!cache.affinity().isPrimaryOrBackup(n, key)) {
+                if (!grid(0).affinity(null).isPrimaryOrBackup(n, key)) {
                     found = true;
 
                     break;
@@ -199,7 +200,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
                 break;
         }
 
-        assertNull(cache.put(key, "val"));
+        assertNull(cache.getAndPut(key, "val"));
 
         checkStoreUsage(1, 1, 0, 1);
     }
@@ -208,7 +209,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testPutIfAbsentFromPrimary() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         int key = 0;
 
@@ -216,7 +217,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
             boolean found = false;
 
             for (ClusterNode n : grid(0).nodes()) {
-                if (cache.affinity().isPrimary(n, key)) {
+                if (grid(0).affinity(null).isPrimary(n, key)) {
                     found = true;
 
                     break;
@@ -227,7 +228,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
                 break;
         }
 
-        assertNull(cache.putIfAbsent(key, "val"));
+        assertTrue(cache.putIfAbsent(key, "val"));
 
         checkStoreUsage(1, 1, 0, 1);
     }
@@ -236,7 +237,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testPutIfAbsentFromBackup() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         int key = 0;
 
@@ -244,7 +245,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
             boolean found = false;
 
             for (ClusterNode n : grid(0).nodes()) {
-                if (cache.affinity().isBackup(n, key)) {
+                if (grid(0).affinity(null).isBackup(n, key)) {
                     found = true;
 
                     break;
@@ -255,7 +256,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
                 break;
         }
 
-        assertNull(cache.putIfAbsent(key, "val"));
+        assertTrue(cache.putIfAbsent(key, "val"));
 
         checkStoreUsage(1, 1, 0, 1);
     }
@@ -264,7 +265,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testPutIfAbsentFromNear() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
         int key = 0;
 
@@ -272,7 +273,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
             boolean found = false;
 
             for (ClusterNode n : grid(0).nodes()) {
-                if (!cache.affinity().isPrimaryOrBackup(n, key)) {
+                if (!grid(0).affinity(null).isPrimaryOrBackup(n, key)) {
                     found = true;
 
                     break;
@@ -283,7 +284,7 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
                 break;
         }
 
-        assertNull(cache.putIfAbsent(key, "val"));
+        assertTrue(cache.putIfAbsent(key, "val"));
 
         checkStoreUsage(1, 1, 0, 1);
     }
@@ -292,9 +293,9 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testPutAll() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
-        Map<Integer, String> map = new HashMap<>(10);
+        Map<Integer, String> map = new HashMap<>();
 
         for (int i = 0; i < 10; i++)
             map.put(i, "val");
@@ -308,9 +309,9 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     public void testMultipleOperations() throws Exception {
-        GridCache<Integer, String> cache = cache(0);
+        IgniteCache<Integer, String> cache = jcache(0);
 
-        try (IgniteTx tx = cache.txStart(OPTIMISTIC, REPEATABLE_READ)) {
+        try (IgniteTx tx = grid(0).transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
             cache.put(1, "val");
             cache.put(2, "val");
             cache.put(3, "val");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
index 8656154..c506127 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
@@ -38,14 +38,14 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
      *
      */
     public void testMapKeysToNodes() {
-        cache().affinity().mapKeysToNodes(Arrays.asList("1", "2"));
+        grid(0).affinity(null).mapKeysToNodes(Arrays.asList("1", "2"));
     }
 
     /**
      *
      */
     public void testMapKeyToNode() {
-        assert cache().affinity().mapKeyToNode("1") == null;
+        assert grid(0).affinity(null).mapKeyToNode("1") == null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java
index 2d0c63e..1981ee7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java
@@ -17,13 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
-import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
-
-import java.util.concurrent.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
 
@@ -40,73 +35,4 @@ public class GridCachePartitionedEntryLockSelfTest extends GridCacheAbstractSelf
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;
     }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("BusyWait")
-    public void testLockAsyncWithTimeout() throws Exception {
-        cache().put("key", 1);
-
-        for (int i = 0; i < gridCount(); i++) {
-            final CacheEntry<String, Integer> e = cache(i).entry("key");
-
-            if (e.backup()) {
-                assert !e.isLocked();
-
-                e.lockAsync(2000).get();
-
-                assert e.isLocked();
-
-                IgniteCompute comp = compute(grid(i).forLocal()).withAsync();
-
-                comp.call(new Callable<Boolean>() {
-                    @Override public Boolean call() throws Exception {
-                        IgniteInternalFuture<Boolean> f = e.lockAsync(1000);
-
-                        try {
-                            f.get(100);
-
-                            fail();
-                        }
-                        catch (IgniteFutureTimeoutException ex) {
-                            info("Caught expected exception: " + ex);
-                        }
-
-                        try {
-                            assert f.get();
-                        }
-                        finally {
-                            e.unlock();
-                        }
-
-                        return true;
-                    }
-                });
-
-                IgniteInternalFuture<Boolean> f = comp.future();
-
-                // Let another thread start.
-                Thread.sleep(300);
-
-                assert e.isLocked();
-                assert e.isLockedByThread();
-
-                cache().unlock("key");
-
-                assert f.get();
-
-                for (int j = 0; j < 100; j++)
-                    if (cache().isLocked("key") || cache().isLockedByThread("key"))
-                        Thread.sleep(10);
-                    else
-                        break;
-
-                assert !cache().isLocked("key");
-                assert !cache().isLockedByThread("key");
-
-                break;
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index da3bab4..2cdb4ef 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
@@ -30,6 +31,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
+import java.util.concurrent.locks.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.cache.CacheMode.*;
@@ -96,22 +98,20 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
 
         info("Primary node for key [id=" + node.id() + ", order=" + node.order() + ", key=" + key + ']');
 
-        GridCache<Integer, String> cache = cache(idx);
+        IgniteCache<Integer, String> cache = jcache(idx);
 
         cache.put(key, "val");
 
-        assert cache.lock(key, -1);
+        Lock lock = cache.lock(key);
+
+        assert lock.tryLock();
 
         for (int checkIdx = 1; checkIdx < GRID_CNT; checkIdx++) {
             info("Check grid index: " + checkIdx);
 
-            GridCache<Integer, String> checkCache = cache(checkIdx);
-
-            assert !checkCache.lock(key, -1);
-
-            CacheEntry e = checkCache.entry(key);
+            IgniteCache<Integer, String> checkCache = jcache(checkIdx);
 
-            assert e.isLocked() : "Entry is not locked for grid [idx=" + checkIdx + ", entry=" + e + ']';
+            assert !checkCache.lock(key).tryLock();
         }
 
         Collection<IgniteInternalFuture<?>> futs = new LinkedList<>();
@@ -137,13 +137,9 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
                 for (int checkIdx = 1; checkIdx < GRID_CNT; checkIdx++) {
                     info("Check grid index: " + checkIdx);
 
-                    GridCache<Integer, String> checkCache = cache(checkIdx);
-
-                    CacheEntry e = checkCache.entry(key);
-
-                    info("Checking entry: " + e);
+                    IgniteCache<Integer, String> checkCache = jcache(checkIdx);
 
-                    assert !e.isLocked() : "Entry is locked for grid [idx=" + checkIdx + ", entry=" + e + ']';
+                    assert !checkCache.isLocalLocked(key, false);
                 }
             }
             catch (AssertionError e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFilteredPutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFilteredPutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFilteredPutSelfTest.java
index 3dc1d7e..f00b4b7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFilteredPutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFilteredPutSelfTest.java
@@ -17,19 +17,19 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.*;
 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.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CacheDistributionMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
 
 /**
  * Test filtered put.
@@ -86,35 +86,12 @@ public class GridCachePartitionedFilteredPutSelfTest extends GridCommonAbstractT
     /**
      * @throws Exception If failed.
      */
-    public void testFilteredPutCheckNear() throws Exception {
-        doFilteredPut();
-
-        GridCache<Integer, Integer> c = cache();
-
-        assert c.entrySet().isEmpty() : "Actual size: " + c.entrySet().size();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFilteredPutCheckDht() throws Exception {
-        doFilteredPut();
-
-        GridCache<Integer, Integer> c =
-            ((GridNearCacheAdapter<Integer, Integer>)cache().<Integer, Integer>cache()).dht();
-
-        assert c.entrySet().isEmpty() : "Actual size: " + c.entrySet().size();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testPutAndRollbackCheckNear() throws Exception {
         doPutAndRollback();
 
-        GridCache<Integer, Integer> c = cache();
+        IgniteCache<Object, Object> c = jcache();
 
-        assert c.entrySet().isEmpty() : "Actual size: " + c.entrySet().size();
+        assert c.size() == 0 : "Actual size: " + c.size();
     }
 
     /**
@@ -132,34 +109,17 @@ public class GridCachePartitionedFilteredPutSelfTest extends GridCommonAbstractT
     /**
      * @throws Exception If failed.
      */
-    private void doFilteredPut() throws Exception {
-        GridCache<Integer, Integer> c = cache();
-
-        try (IgniteTx tx = c.txStart()) {
-            assert !c.putx(1, 1, F.<Integer, Integer>cacheHasPeekValue());
-
-            tx.commit();
-        }
-
-        assert c.isEmpty();
-        assert c.peek(1) == null;
-        assert c.get(1) == null;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     private void doPutAndRollback() throws Exception {
-        GridCache<Integer, Integer> c = cache();
+        IgniteCache<Object, Object> c = jcache();
 
-        try (IgniteTx tx = c.txStart()) {
-            assert c.putx(1, 1);
+        try (IgniteTx tx = grid().transactions().txStart()) {
+            c.put(1, 1);
 
             tx.rollback();
         }
 
-        assert c.isEmpty();
-        assert c.peek(1) == null;
+        assert c.localSize() == 0;
+        assert c.localPeek(1) == null;
         assert c.get(1) == null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
index f3ce6fa..f01cfde 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
@@ -190,14 +190,14 @@ public class GridCachePartitionedFullApiSelfTest extends GridCacheAbstractFullAp
      * @throws Exception If failed.
      */
     public void testPartitionEntrySetRemove() throws Exception {
-        GridCache<String, Integer> cache = cache(0);
+        IgniteCache<String, Integer> cache = jcache(0);
 
         Map<Integer, Collection<String>> partMap = new HashMap<>();
 
         for (int i = 0; i < 1000; i++) {
             String key = String.valueOf(i);
 
-            int part = cache.affinity().partition(key);
+            int part = grid(0).affinity(null).partition(key);
 
             cache.put(key, i);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
index 1563356..d03d995 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
@@ -19,15 +19,14 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.GridCache;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 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.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
@@ -109,9 +108,7 @@ public class GridCachePartitionedLoadCacheSelfTest extends GridCommonAbstractTes
             else
                 cache.localLoadCache(null, PUT_CNT);
 
-            GridCache<Integer, String> cache0 = cache(0);
-
-            CacheAffinity aff = cache0.affinity();
+            CacheAffinity<Integer> aff = grid(0).affinity(null);
 
             int[] parts = aff.allPartitions(grid(0).localNode());
 
@@ -126,7 +123,7 @@ public class GridCachePartitionedLoadCacheSelfTest extends GridCommonAbstractTes
 
             int cnt2 = 0;
 
-            for (CacheEntry<Integer, String> e : cache0.entrySet()) {
+            for (CacheEntry<Object, Object> e : cache(0).entrySet()) {
                 assert e.primary() || e.backup();
 
                 cnt2++;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index e18421f..91de083 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -17,24 +17,20 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
-import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 
 import java.util.*;
-import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.events.IgniteEventType.*;
-import static org.apache.ignite.cache.GridCachePeekMode.*;
 import static org.apache.ignite.cache.CachePreloadMode.*;
+import static org.apache.ignite.cache.GridCachePeekMode.*;
+import static org.apache.ignite.events.IgniteEventType.*;
 
 /**
  * Multi-node tests for partitioned cache.
@@ -198,7 +194,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
             }, EVT_CACHE_OBJECT_SWAPPED, EVT_CACHE_OBJECT_UNSWAPPED);
         }
 
-        cache().put("key", 1);
+        jcache().put("key", 1);
 
         for (int i = 0; i < gridCount(); i++) {
             CacheEntry<String, Integer> e = cache(i).entry("key");
@@ -224,7 +220,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
      * @throws Exception If failed.
      */
     public void testPeekPartitionedModes() throws Exception {
-        cache().put("key", 1);
+        jcache().put("key", 1);
 
         for (int i = 0; i < gridCount(); i++) {
             boolean nearEnabled = nearEnabled(cache(i));
@@ -261,7 +257,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
      * @throws Exception If failed.
      */
     public void testPeekAsyncPartitionedModes() throws Exception {
-        cache().put("key", 1);
+        jcache().put("key", 1);
 
         for (int i = 0; i < gridCount(); i++) {
             boolean nearEnabled = nearEnabled(cache(i));
@@ -304,15 +300,17 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
 
         info("Generating keys for test...");
 
+        GridCache<String, Integer> cache0 = cache(0);
+
         for (int i = 0; i < 5; i++) {
             while (true) {
                 String key = UUID.randomUUID().toString();
 
-                if (cache().affinity().isPrimary(grid(0).localNode(), key) &&
-                    cache().affinity().isBackup(grid(1).localNode(), key)) {
+                if (ignite(0).affinity(null).isPrimary(grid(0).localNode(), key) &&
+                    ignite(0).affinity(null).isBackup(grid(1).localNode(), key)) {
                     keys.add(key);
 
-                    assertTrue(cache(0).putx(key, i));
+                    assertTrue(cache0.putx(key, i));
 
                     break;
                 }
@@ -321,17 +319,21 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
 
         info("Finished generating keys for test.");
 
-        assertEquals(Integer.valueOf(0), cache(2).get(keys.get(0)));
-        assertEquals(Integer.valueOf(1), cache(2).get(keys.get(1)));
+        GridCache<String, Integer> cache2 = cache(2);
+
+        assertEquals(Integer.valueOf(0), cache2.get(keys.get(0)));
+        assertEquals(Integer.valueOf(1), cache2.get(keys.get(1)));
 
-        assertEquals(0, cache(0).nearSize());
-        assertEquals(5, cache(0).size() - cache(0).nearSize());
+        assertEquals(0, cache0.nearSize());
+        assertEquals(5, cache0.size() - cache0.nearSize());
 
-        assertEquals(0, cache(1).nearSize());
-        assertEquals(5, cache(1).size() - cache(1).nearSize());
+        GridCache<String, Integer> cache1 = cache(1);
 
-        assertEquals(nearEnabled() ? 2 : 0, cache(2).nearSize());
-        assertEquals(0, cache(2).size() - cache(2).nearSize());
+        assertEquals(0, cache1.nearSize());
+        assertEquals(5, cache1.size() - cache1.nearSize());
+
+        assertEquals(nearEnabled() ? 2 : 0, cache2.nearSize());
+        assertEquals(0, cache2.size() - cache2.nearSize());
 
         IgniteBiPredicate<String, Integer> prjFilter = new P2<String, Integer>() {
             @Override public boolean apply(String key, Integer val) {
@@ -339,14 +341,14 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
             }
         };
 
-        assertEquals(0, cache(0).projection(prjFilter).nearSize());
-        assertEquals(3, cache(0).projection(prjFilter).size() - cache(0).projection(prjFilter).nearSize());
+        assertEquals(0, cache0.projection(prjFilter).nearSize());
+        assertEquals(3, cache0.projection(prjFilter).size() - cache0.projection(prjFilter).nearSize());
 
-        assertEquals(0, cache(1).projection(prjFilter).nearSize());
-        assertEquals(3, cache(1).projection(prjFilter).size() - cache(1).projection(prjFilter).nearSize());
+        assertEquals(0, cache1.projection(prjFilter).nearSize());
+        assertEquals(3, cache1.projection(prjFilter).size() - cache1.projection(prjFilter).nearSize());
 
-        assertEquals(nearEnabled() ? 1 : 0, cache(2).projection(prjFilter).nearSize());
-        assertEquals(0, cache(2).projection(prjFilter).size() - cache(2).projection(prjFilter).nearSize());
+        assertEquals(nearEnabled() ? 1 : 0, cache2.projection(prjFilter).nearSize());
+        assertEquals(0, cache2.projection(prjFilter).size() - cache2.projection(prjFilter).nearSize());
     }
 
     /** {@inheritDoc} */
@@ -355,161 +357,6 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
     }
 
     /**
-     * @throws Exception In case of error.
-     */
-    public void testLockAsyncWithTimeoutEntryPrimary() throws Exception {
-        if (!lockingEnabled())
-            return;
-
-        ClusterNode node = CU.primary(cache().affinity().mapKeyToPrimaryAndBackups("key"));
-
-        assert node != null;
-
-        info("Node: " + node);
-
-        GridCache<String, Integer> cache = G.ignite(node.id()).cache(null);
-
-        checkLockAsyncWithTimeoutEntry("key", cache);
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testLockAsyncWithTimeoutEntryBackup() throws Exception {
-        if (!lockingEnabled())
-            return;
-
-        ClusterNode node = F.first(CU.backups(cache().affinity().mapKeyToPrimaryAndBackups("key")));
-
-        assert node != null;
-
-        info("Node: " + node);
-
-        GridCache<String, Integer> cache = G.ignite(node.id()).cache(null);
-
-        checkLockAsyncWithTimeoutEntry("key", cache);
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    public void testLockAsyncWithTimeoutEntryNear() throws Exception {
-        if (!lockingEnabled())
-            return;
-
-        Collection<ClusterNode> affNodes = cache().affinity().mapKeyToPrimaryAndBackups("key");
-
-        ClusterNode node = null;
-
-        for (ClusterNode n : grid(0).nodes()) {
-            if (!affNodes.contains(n)) {
-                node = n;
-
-                break;
-            }
-        }
-
-        assert node != null;
-
-        info("Node: " + node);
-
-        GridCache<String, Integer> cache = G.ignite(node.id()).cache(null);
-
-        checkLockAsyncWithTimeoutEntry("key", cache);
-    }
-
-    /**
-     * @param key Key.
-     * @param cache Cache.
-     * @throws Exception If failed.
-     */
-    private void checkLockAsyncWithTimeoutEntry(String key, CacheProjection<String,Integer> cache)
-        throws Exception {
-        assert lockingEnabled();
-
-        final CacheEntry<String, Integer> e = cache.entry(key);
-
-        info("Entry [e=" + e + ", primary=" + e.primary() + ", backup=" + e.backup() + ']');
-
-        assert e != null;
-
-        assert !e.isLocked();
-
-        final AtomicBoolean locked = new AtomicBoolean(e.lock(0));
-
-        info("Entry after lock [e=" + e + ", primary=" + e.primary() + ", backup=" + e.backup() + ']');
-
-        try {
-            assert e.isLocked();
-            assert e.isLockedByThread();
-
-            assert cache.isLockedByThread(key);
-
-            final CountDownLatch syncLatch = new CountDownLatch(1);
-
-            IgniteCompute comp = compute(dfltIgnite.cluster().forLocal()).withAsync();
-
-            comp.call(new Callable<Boolean>() {
-                    @Override public Boolean call() throws Exception {
-                        syncLatch.countDown();
-
-                        IgniteInternalFuture<Boolean> f = e.lockAsync(15000);
-
-                        try {
-                            f.get(100);
-
-                            info(">>>> Entry locked async [e=" + e + ", primary=" + e.primary() +
-                                ", backup=" + e.backup() + ']');
-
-                            fail();
-                        } catch (IgniteFutureTimeoutException ex) {
-                            info("Caught expected exception: " + ex);
-                        }
-
-                        try {
-                            assert f.get();
-                        } finally {
-                            e.unlock();
-
-                            locked.set(false);
-                        }
-
-                        return true;
-                    }
-                });
-
-            IgniteInternalFuture<Boolean> f = comp.future();
-
-            syncLatch.await();
-
-            // Make 1st future in closure fail.
-            Thread.sleep(300);
-
-            assert e.isLocked();
-            assert e.isLockedByThread();
-
-            cache.unlock(key);
-
-            locked.set(false);
-
-            assert f.get();
-
-            for (int i = 0; i < 100; i++)
-                if (cache.isLocked(key) || cache.isLockedByThread(key))
-                    U.sleep(10);
-                else
-                    break;
-
-            assert !cache.isLocked(key);
-            assert !cache.isLockedByThread(key);
-        }
-        finally {
-            if (locked.get())
-                e.unlock();
-        }
-    }
-
-    /**
      * @throws Exception If failed.
      */
     public void testAffinity() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index 590568e..07d2800 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@ -197,10 +197,10 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
 
         IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
-                GridCache<Object, Object> c = cache(0);
+                IgniteCache<Object, Object> c = jcache(0);
 
                 try {
-                    IgniteTx tx = c.txStart(mode, REPEATABLE_READ);
+                    IgniteTx tx = grid(0).transactions().txStart(mode, REPEATABLE_READ);
 
                     for (Integer key : keys)
                         c.put(key, "val" + key);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/625b33f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridPartitionedBackupLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridPartitionedBackupLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridPartitionedBackupLoadSelfTest.java
index b7391cb..a91f380 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridPartitionedBackupLoadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridPartitionedBackupLoadSelfTest.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.GridCache;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.spi.discovery.*;
@@ -104,26 +104,24 @@ public class GridPartitionedBackupLoadSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testBackupLoad() throws Exception {
-        assert grid(0).cache(null).putx(1, 1);
+        grid(0).jcache(null).put(1, 1);
 
         assert store.get(1) == 1;
 
         for (int i = 0; i < GRID_CNT; i++) {
-            GridCache<Integer, Integer> cache = cache(i);
+            IgniteCache<Integer, Integer> cache = jcache(i);
 
-            CacheEntry<Integer, Integer> entry = cache.entry(1);
+            if (grid(i).affinity(null).isBackup(grid(i).localNode(), 1)) {
+                assert cache.localPeek(1) == 1;
 
-            if (entry.backup()) {
-                assert entry.peek() == 1;
+                assert cache.clear(Collections.singleton(1));
 
-                assert entry.clear();
-
-                assert entry.peek() == null;
+                assert cache.localPeek(1) == null;
 
                 // Store is called in putx method, so we reset counter here.
                 cnt.set(0);
 
-                assert entry.get() == 1;
+                assert cache.get(1) == 1;
 
                 assert cnt.get() == 0;
             }