You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/01/19 07:42:28 UTC

[11/15] incubator-ignite git commit: IGNITE-59 Support lock, lockAll: Fix tests.

IGNITE-59 Support lock, lockAll: Fix tests.


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

Branch: refs/heads/ignite-43
Commit: 7737e759a4aa651fa95abb263e87045c4c551208
Parents: 26d7964
Author: sevdokimov <se...@gridgain.com>
Authored: Thu Jan 15 18:43:28 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri Jan 16 15:45:41 2015 +0300

----------------------------------------------------------------------
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |  18 +--
 .../cache/GridCacheAbstractFullApiSelfTest.java | 156 ++++---------------
 .../cache/GridCacheBasicApiAbstractTest.java    |  79 +++++-----
 .../GridCacheFinishPartitionsSelfTest.java      |  22 +--
 .../GridCacheGroupLockAbstractSelfTest.java     |  24 +--
 .../cache/GridCacheNestedTxAbstractTest.java    |  28 ++--
 .../GridCacheOffHeapTieredAbstractSelfTest.java |  38 +----
 .../distributed/GridCacheLockAbstractTest.java  |  25 ++-
 .../GridCacheMultiNodeLockAbstractTest.java     | 144 ++++++++++++-----
 .../dht/GridCacheColocatedDebugTest.java        |  26 ++--
 ...dCachePartitionedTopologyChangeSelfTest.java |   8 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |  14 +-
 .../near/GridCacheNearOneNodeSelfTest.java      |  35 +++--
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |  42 -----
 .../near/GridCacheNearReadersSelfTest.java      |  14 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |   1 +
 .../near/GridCachePartitionedLockSelfTest.java  |   2 +-
 .../GridCacheEvictionLockUnlockSelfTest.java    |   7 +-
 .../cache/local/GridCacheLocalLockSelfTest.java |  34 ++--
 .../GridCacheLocalMultithreadedSelfTest.java    |  14 +-
 20 files changed, 322 insertions(+), 409 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
index 5d9c289..54157c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
@@ -113,8 +113,8 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
         }
 
         // Now, grid1 is always primary node for key 1.
-        final GridCache<Integer, String> cache1 = ignite1.cache(null);
-        final GridCache<Integer, String> cache2 = ignite2.cache(null);
+        final IgniteCache<Integer, String> cache1 = ignite1.jcache(null);
+        final IgniteCache<Integer, String> cache2 = ignite2.jcache(null);
 
         info(">>> Primary: " + ignite1.cluster().localNode().id());
         info(">>>  Backup: " + ignite2.cluster().localNode().id());
@@ -125,7 +125,7 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
             @Nullable @Override public Object call() throws Exception {
                 info("Before lock for key: " + kv);
 
-                assert cache1.lock(kv, 0L);
+                cache1.lock(kv).lock();
 
                 info("After lock for key: " + kv);
 
@@ -144,7 +144,7 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
                 finally {
                     Thread.sleep(1000);
 
-                    cache1.unlockAll(Collections.singleton(kv));
+                    cache1.lockAll(Collections.singleton(kv)).unlock();
 
                     info("Unlocked key in thread 1: " + kv);
                 }
@@ -161,7 +161,7 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
 
                 l1.await();
 
-                assert cache2.lock(kv, 0L);
+                cache2.lock(kv).lock();
 
                 try {
                     String v = cache2.get(kv);
@@ -170,7 +170,7 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
                     assertEquals(Integer.toString(kv), v);
                 }
                 finally {
-                    cache2.unlockAll(Collections.singleton(kv));
+                    cache2.lockAll(Collections.singleton(kv)).unlock();
 
                     info("Unlocked key in thread 2: " + kv);
                 }
@@ -193,12 +193,12 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
 
         info("Remove all completed");
 
-        if (!cache2.isEmpty()) {
-            String failMsg = cache2.entrySet().toString();
+        if (cache2.size() > 0) {
+            String failMsg = cache2.toString();
 
             long start = System.currentTimeMillis();
 
-            while (!cache2.isEmpty())
+            while (cache2.size() > 0)
                 U.sleep(100);
 
             long clearDuration = System.currentTimeMillis() - start;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 870da16..4447383 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -37,6 +37,7 @@ import javax.cache.processor.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
 
 import static java.util.concurrent.TimeUnit.*;
 import static org.gridgain.grid.cache.GridCacheMode.*;
@@ -3308,7 +3309,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 }
             }, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
 
-            GridCache<String, Integer> cache = cache();
+            IgniteCache<String, Integer> cache = jcache();
 
             String key = primaryKeysForCache(cache, 1).get(0);
 
@@ -3316,13 +3317,13 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
             assert !cache.isLocked(key);
 
-            cache.lock(key, 0);
+            cache.lock(key).lock();
 
             lockCnt.await();
 
             assert cache.isLocked(key);
 
-            cache.unlock(key);
+            cache.lock(key).unlock();
 
             unlockCnt.await();
 
@@ -3397,22 +3398,24 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      */
     public void testLockWithTimeout() throws Exception {
         if (lockingEnabled()) {
-            cache().put("key", 1);
+            jcache().put("key", 1);
 
-            assert !cache().isLocked("key");
+            assert !jcache().isLocked("key");
 
-            cache().lock("key", 2000);
+            final Lock lock = jcache().lock("key");
 
-            assert cache().isLocked("key");
-            assert cache().isLockedByThread("key");
+            lock.tryLock(2000, MILLISECONDS);
+
+            assert jcache().isLocked("key");
+            assert jcache().isLockedByThread("key");
 
             assert !forLocal(dfltIgnite).call(new Callable<Boolean>() {
-                @Override public Boolean call() throws IgniteCheckedException {
-                    return cache().lock("key", 100);
+                @Override public Boolean call() throws InterruptedException {
+                    return lock.tryLock(100, MILLISECONDS);
                 }
             });
 
-            cache().unlock("key");
+            lock.unlock();
         }
     }
 
@@ -3594,61 +3597,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     @SuppressWarnings("BusyWait")
-    public void testLockFiltered() throws Exception {
-        if (lockingEnabled()) {
-            cache().put("key1", 1);
-            cache().put("key2", 100);
-
-            for (int i = 0; i < gridCount(); i++) {
-                assert !cache(i).isLocked("key1");
-                assert !cache(i).isLocked("key2");
-            }
-
-            cache().projection(F.<GridCacheEntry<String, Integer>>alwaysFalse()).lock("key1", 0);
-            cache().projection(F.<GridCacheEntry<String, Integer>>alwaysTrue()).lock("key2", 0);
-
-            boolean passed = false;
-
-            for (int i = 0; i < gridCount(); i++) {
-                assertFalse("key1 is locked for cache: " + i, cache(i).isLocked("key1"));
-
-                if (cache(i).isLocked("key2"))
-                    passed = true;
-            }
-
-            assert passed;
-
-            // Must pass keys as lock() was called with keys.
-            cache().unlockAll(F.asList("key2"), F.<GridCacheEntry<String, Integer>>alwaysTrue());
-
-            for (int i = 0; i < 100; i++) {
-                boolean sleep = false;
-
-                for (int j = 0; j < gridCount(); j++) {
-                    if (cache(j).isLocked("key1") || cache(j).isLocked("key2")) {
-                        sleep = true;
-
-                        break;
-                    }
-                }
-
-                if (sleep)
-                    Thread.sleep(10);
-                else
-                    break;
-            }
-
-            for (int i = 0; i < gridCount(); i++) {
-                assert !cache(i).isLocked("key1");
-                assert !cache(i).isLocked("key2");
-            }
-        }
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    @SuppressWarnings("BusyWait")
     public void testLockFilteredEntry() throws Exception {
         if (lockingEnabled()) {
             cache().put("key1", 1);
@@ -3703,69 +3651,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     @SuppressWarnings("BusyWait")
-    public void testUnlockFiltered() throws Exception {
-        if (lockingEnabled()) {
-            List<String> keys = primaryKeysForCache(cache(), 2);
-
-            info("Keys: " + keys);
-
-            final String key1 = keys.get(0);
-            final String key2 = keys.get(1);
-
-            cache().put(key1, 1);
-            cache().put(key2, 100);
-
-            assert !cache().isLocked(key1);
-            assert !cache().isLocked(key2);
-
-            cache().lock(key1, 0);
-            cache().lock(key2, 0);
-
-            assert cache().isLocked(key1);
-            assert cache().isLocked(key2);
-
-            cache().unlock(key1, gte100);
-            cache().unlock(key2, gte100);
-
-            GridTestUtils.waitForCondition(new PA() {
-                @Override public boolean apply() {
-                    for (int g = 0; g < gridCount(); g++) {
-                        if (cache(g).isLocked(key2)) {
-                            info(key2 + " is locked on grid: " + g);
-
-                            return false;
-                        }
-                    }
-
-                    return true;
-                }
-            }, 2000);
-
-            assert cache().isLocked(key1);
-            assert !cache().isLocked(key2);
-
-            cache().unlockAll(F.asList(key1, key2));
-
-            GridTestUtils.waitForCondition(new PA() {
-                @Override public boolean apply() {
-                    for (int g = 0; g < gridCount(); g++) {
-                        if (cache(g).isLocked(key1))
-                            info(key1 + " is locked on grid: " + g);
-
-                        if (cache(g).isLocked(key2))
-                            info(key2 + " is locked on grid: " + g);
-                    }
-
-                    return true;
-                }
-            }, 2000);
-        }
-    }
-
-    /**
-     * @throws Exception In case of error.
-     */
-    @SuppressWarnings("BusyWait")
     public void testUnlockFilteredEntry() throws Exception {
         if (lockingEnabled()) {
             cache().put("key1", 1);
@@ -5248,4 +5133,17 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         throw new IgniteCheckedException("Unable to find " + cnt + " keys as primary for cache.");
     }
+
+    /**
+     * @param cache Cache.
+     * @param cnt Keys count.
+     * @return Collection of keys for which given cache is primary.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected List<String> primaryKeysForCache(IgniteCache<String, Integer> cache, int cnt)
+        throws IgniteCheckedException {
+        GridCacheProjection<String, Integer> prj = GridTestUtils.getFieldValue(cache, "delegate");
+
+        return primaryKeysForCache(prj, cnt);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheBasicApiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheBasicApiAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheBasicApiAbstractTest.java
index 1eae0a3..54c141e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheBasicApiAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheBasicApiAbstractTest.java
@@ -21,10 +21,9 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.cache.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.gridgain.grid.cache.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.*;
@@ -34,6 +33,7 @@ import org.jetbrains.annotations.*;
 import javax.cache.expiry.*;
 import java.util.*;
 import java.util.concurrent.*;
+import java.util.concurrent.locks.*;
 
 import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.events.IgniteEventType.*;
@@ -80,13 +80,15 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testBasicLock() throws Exception {
-        GridCache<Integer, String> cache = ignite.cache(null);
+        IgniteCache<Integer, String> cache = ignite.jcache(null);
 
-        assert cache.lock(1, 0);
+        Lock lock = cache.lock(1);
+
+        assert lock.tryLock();
 
         assert cache.isLocked(1);
 
-        cache.unlock(1);
+        lock.unlock();
 
         assert !cache.isLocked(1);
     }
@@ -95,21 +97,23 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws IgniteCheckedException If test failed.
      */
     public void testSingleLockReentry() throws IgniteCheckedException {
-        GridCache<Integer, String> cache = ignite.cache(null);
+        IgniteCache<Integer, String> cache = ignite.jcache(null);
+
+        Lock lock = cache.lock(1);
 
-        assert cache.lock(1, 0);
+        lock.lock();
 
         try {
             assert cache.isLockedByThread(1);
 
-            assert cache.lock(1, 0);
+            lock.lock();
 
-            cache.unlock(1);
+            lock.unlock();
 
             assert cache.isLockedByThread(1);
         }
         finally {
-            cache.unlock(1);
+            lock.unlock();
         }
 
         assert !cache.isLockedByThread(1);
@@ -121,34 +125,34 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testReentry() throws Exception {
-        GridCache<Integer, String> cache = ignite.cache(null);
+        IgniteCache<Integer, String> cache = ignite.jcache(null);
 
-        assert cache.lock(1, 0);
+        Lock lock = cache.lock(1);
 
         assert cache.isLocked(1);
         assert cache.isLockedByThread(1);
 
-        assert cache.lock(1, 0);
+        lock.lock();
 
         assert cache.isLocked(1);
         assert cache.isLockedByThread(1);
 
-        assert cache.lock(1, 0);
+        lock.lock();
 
         assert cache.isLocked(1);
         assert cache.isLockedByThread(1);
 
-        cache.unlock(1);
+        lock.unlock();
 
         assert cache.isLocked(1);
         assert cache.isLockedByThread(1);
 
-        cache.unlock(1);
+        lock.unlock();
 
         assert cache.isLocked(1);
         assert cache.isLockedByThread(1);
 
-        cache.unlock(1);
+        lock.unlock();
 
         assert !cache.isLocked(1);
         assert !cache.isLockedByThread(1);
@@ -158,21 +162,21 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws IgniteCheckedException If test failed.
      */
     public void testManyLockReentries() throws IgniteCheckedException {
-        GridCache<Integer, String> cache = ignite.cache(null);
+        IgniteCache<Integer, String> cache = ignite.jcache(null);
 
         Integer key = 1;
 
-        assert cache.lock(key, 0);
+        cache.lock(key).lock();
 
         try {
             assert cache.get(key) == null;
-            assert cache.put(key, "1") == null;
+            assert cache.getAndPut(key, "1") == null;
             assert "1".equals(cache.get(key));
 
             assert cache.isLocked(key);
             assert cache.isLockedByThread(key);
 
-            assert cache.lock(key, 0);
+            cache.lock(key).lock();
 
             assert cache.isLocked(key);
             assert cache.isLockedByThread(key);
@@ -181,14 +185,14 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
                 assert "1".equals(cache.remove(key));
             }
             finally {
-                cache.unlock(key);
+                cache.lock(key).unlock();
             }
 
             assert cache.isLocked(key);
             assert cache.isLockedByThread(key);
         }
         finally {
-            cache.unlock(key);
+            cache.lock(key).unlock();
 
             assert !cache.isLocked(key);
             assert !cache.isLockedByThread(key);
@@ -199,7 +203,7 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If test failed.
      */
     public void testLockMultithreaded() throws Exception {
-        final GridCache<Integer, String> cache = ignite.cache(null);
+        final IgniteCache<Integer, String> cache = ignite.jcache(null);
 
         final CountDownLatch l1 = new CountDownLatch(1);
         final CountDownLatch l2 = new CountDownLatch(1);
@@ -210,7 +214,7 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
             @Nullable @Override public Object call() throws Exception {
                 info("Before lock for.key 1");
 
-                assert cache.lock(1, 0);
+                cache.lock(1).lock();
 
                 info("After lock for key 1");
 
@@ -223,22 +227,22 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
                     info("Let thread2 proceed.");
 
                     // Reentry.
-                    assert cache.lock(1, -1L);
+                    assert cache.lock(1).tryLock();
 
                     // Nested lock.
-                    assert cache.lock(2, -1L);
+                    assert cache.lock(2).tryLock();
 
                     l2.await();
 
-                    cache.unlock(1);
+                    cache.lock(1).unlock();
 
                     // Unlock in reverse order.
-                    cache.unlock(2);
+                    cache.lock(2).unlock();
 
                     info("Waited for latch 2");
                 }
                 finally {
-                    cache.unlock(1);
+                    cache.lock(1).unlock();
 
                     info("Unlocked entry for key 1.");
                 }
@@ -258,7 +262,7 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
 
                 info("Latch1 released.");
 
-                assert !cache.lock(1, -1L);
+                assert !cache.lock(1).tryLock();
 
                 if (!cache.isLocked(1))
                     throw new IllegalArgumentException();
@@ -273,7 +277,7 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
 
                 l3.await();
 
-                assert cache.lock(1, -1L);
+                assert cache.lock(1).tryLock();
 
                 try {
                     info("Locked cache for key 1");
@@ -284,7 +288,7 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
                     info("Checked that cache is locked for key 1");
                 }
                 finally {
-                    cache.unlock(1);
+                    cache.lock(1).unlock();
 
                     info("Unlocked cache for key 1");
                 }
@@ -374,13 +378,13 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
      * @throws Exception If error occur.
      */
     public void testBasicOpsWithReentry() throws Exception {
-        GridCache<Integer, String> cache = ignite.cache(null);
+        IgniteCache<Integer, String> cache = ignite.jcache(null);
 
         int key = (int)System.currentTimeMillis();
 
         assert !cache.containsKey(key);
 
-        assert cache.lock(key, 0);
+        cache.lock(key).lock();
 
         CountDownLatch latch = new CountDownLatch(1);
 
@@ -428,17 +432,16 @@ public abstract class GridCacheBasicApiAbstractTest extends GridCommonAbstractTe
 
             info("Stop latch wait 3");
 
-            assert cache.keySet().contains(key);
+            assert cache.containsKey(key);
             assert cache.isLocked(key);
         }
         finally {
-            cache.unlock(key);
+            cache.lock(key).unlock();
 
             ignite.events().stopLocalListen(lsnr, EVTS_CACHE);
         }
 
         // Entry should be evicted since allowEmptyEntries is false.
-        assert !cache.keySet().contains(key) : "Key set: " + cache.keySet();
         assert !cache.isLocked(key);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java
index 8ddeb5e..b3796be 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
@@ -28,6 +29,7 @@ import org.gridgain.testframework.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
 
 import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
 import static org.gridgain.grid.cache.GridCacheMode.*;
@@ -221,9 +223,9 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
 
         final CountDownLatch latch = new CountDownLatch(1);
 
-        GridCache<Integer, String> cache = grid.cache(null);
+        IgniteCache<Integer, String> cache = grid.jcache(null);
 
-        assert cache.lock(key, 0);
+        cache.lock(key).lock();
 
         long start = System.currentTimeMillis();
 
@@ -243,17 +245,17 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
             }
         });
 
-        assert cache.lock(key + 1, 0);
+        cache.lock(key + 1).lock();
 
-        cache.unlock(key);
+        cache.lock(key).unlock();
 
-        assert cache.lock(key + 2, 0);
+        cache.lock(key + 2).lock();
 
-        cache.unlock(key + 1);
+        cache.lock(key + 1).unlock();
 
         assert !fut.isDone() : "Failed waiting for locks";
 
-        cache.unlock(key + 2);
+        cache.lock(key + 2).unlock();
 
         latch.await();
     }
@@ -273,9 +275,9 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
 
         final CountDownLatch latch = new CountDownLatch(1);
 
-        GridCache<String, String> cache = grid.cache(null);
+        IgniteCache<String, String> cache = grid.jcache(null);
 
-        assert cache.lock(key, 0);
+        cache.lock(key).lock();
 
         long start;
         try {
@@ -301,7 +303,7 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
                 + fut.isDone() + ']';
         }
         finally {
-            cache.unlock(key);
+            cache.lock(key).unlock();
         }
 
         latch.await();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
index c4a7471..f0abccb 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
@@ -37,6 +37,7 @@ import org.gridgain.testframework.junits.common.*;
 import org.jdk8.backport.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -355,7 +356,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
 
         final GridCacheAffinityKey<String> key1 = new GridCacheAffinityKey<>("key1", affinityKey);
 
-        final GridCache<GridCacheAffinityKey<String>, String> cache = grid(0).cache(null);
+        final IgniteCache<GridCacheAffinityKey<String>, String> cache = grid(0).jcache(null);
 
         // Populate cache.
         cache.put(key1, "val1");
@@ -375,17 +376,17 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
         IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
-                    assertTrue(cache.lock(key1, 0));
+                    cache.lock(key1).lock();
 
                     try {
                         lockLatch.countDown();
                         unlockLatch.await();
                     }
                     finally {
-                        cache.unlock(key1);
+                        cache.lock(key1).unlock();
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (CacheException e) {
                     fail(e.getMessage());
                 }
                 catch (InterruptedException ignored) {
@@ -399,7 +400,8 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
 
             GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    try (IgniteTx tx = cache.txStartAffinity(affinityKey, concurrency, READ_COMMITTED, 0, 1)) {
+                    try (IgniteTx tx = grid(0).transactions().txStartAffinity(null, affinityKey, concurrency,
+                        READ_COMMITTED, 0, 1)) {
                         cache.put(key1, "val01");
 
                         tx.commit();
@@ -436,11 +438,13 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
     private void checkSanityCheckDisabled(final IgniteTxConcurrency concurrency) throws Exception {
         assert !sanityCheckEnabled();
 
-        final UUID affinityKey = primaryKeyForCache(grid(0));
+        GridEx grid = grid(0);
+
+        final UUID affinityKey = primaryKeyForCache(grid);
 
         final GridCacheAffinityKey<String> key1 = new GridCacheAffinityKey<>("key1", affinityKey);
 
-        final GridCache<GridCacheAffinityKey<String>, String> cache = grid(0).cache(null);
+        final IgniteCache<GridCacheAffinityKey<String>, String> cache = grid.jcache(null);
 
         // Populate cache.
         cache.put(key1, "val1");
@@ -454,10 +458,10 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
                 assertEquals("For index: " + i, "val1", gCache.peek(key1));
         }
 
-        assertTrue(cache.lock(key1, 0));
+        cache.lock(key1).lock();
 
         try {
-            try (IgniteTx tx = cache.txStartAffinity(affinityKey, concurrency, READ_COMMITTED, 0, 1)) {
+            try (IgniteTx tx = grid.transactions().txStartAffinity(null, affinityKey, concurrency, READ_COMMITTED, 0, 1)) {
                 cache.put(key1, "val01");
 
                 tx.commit();
@@ -473,7 +477,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
             }
         }
         finally {
-            cache.unlock(key1);
+            cache.lock(key1).unlock();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheNestedTxAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheNestedTxAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheNestedTxAbstractTest.java
index 7918c45..ad3048c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheNestedTxAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheNestedTxAbstractTest.java
@@ -136,7 +136,7 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testLockAndTx() throws Exception {
-        final GridCache<String, Integer> c = grid(0).cache(null);
+        final IgniteCache<String, Integer> c = grid(0).jcache(null);
 
         Collection<Thread> threads = new LinkedList<>();
 
@@ -147,7 +147,7 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
 
             threads.add(new Thread(new Runnable() {
                 @Override public void run() {
-                    IgniteTx tx = c.txStart(PESSIMISTIC, REPEATABLE_READ);
+                    IgniteTx tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
 
                     try {
                         int cntr = c.get(CNTR_KEY);
@@ -172,7 +172,7 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
                 @Override public void run() {
 
                     try {
-                        c.lock(CNTR_KEY, 0);
+                        c.lock(CNTR_KEY).lock();
 
                         int cntr = c.get(CNTR_KEY);
 
@@ -184,12 +184,7 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
                         error("Failed lock thread", e);
                     }
                     finally {
-                        try {
-                            c.unlock(CNTR_KEY);
-                        }
-                        catch (IgniteCheckedException e) {
-                            error("Failed unlock", e);
-                        }
+                        c.lock(CNTR_KEY).unlock();
                     }
                 }
             }));
@@ -215,9 +210,9 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testLockAndTx1() throws Exception {
-        final GridCache<String, Integer> c = grid(0).cache(null);
+        final IgniteCache<String, Integer> c = grid(0).jcache(null);
 
-        final GridCache<Integer, Integer> c1 = grid(0).cache(null);
+        final IgniteCache<Integer, Integer> c1 = grid(0).jcache(null);
 
         Collection<Thread> threads = new LinkedList<>();
 
@@ -230,13 +225,13 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
                 @Override public void run() {
 
                     try {
-                        c.lock(CNTR_KEY, 0);
+                        c.lock(CNTR_KEY).lock();
 
                         int cntr = c.get(CNTR_KEY);
 
                         info("*** Cntr in lock thread: " + cntr);
 
-                        IgniteTx tx = c.txStart(OPTIMISTIC, READ_COMMITTED);
+                        IgniteTx tx = grid(0).transactions().txStart(OPTIMISTIC, READ_COMMITTED);
 
                         try {
 
@@ -262,12 +257,7 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
                         error("Failed lock thread", e);
                     }
                     finally {
-                        try {
-                            c.unlock(CNTR_KEY);
-                        }
-                        catch (IgniteCheckedException e) {
-                            error("Failed unlock", e);
-                        }
+                        c.lock(CNTR_KEY).unlock();
                     }
                 }
             }));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTieredAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTieredAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTieredAbstractSelfTest.java
index 30564ca..1a6b6ce 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTieredAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTieredAbstractSelfTest.java
@@ -524,51 +524,23 @@ public abstract class GridCacheOffHeapTieredAbstractSelfTest extends GridCacheAb
      */
     @SuppressWarnings("UnnecessaryLocalVariable")
     private void checkLockUnlock(Integer key) throws Exception {
-        GridCache<Integer, Integer> c = grid(0).cache(null);
+        IgniteCache<Integer, Integer> c = grid(0).jcache(null);
 
         Integer val = key;
 
         c.put(key, val);
 
-        assertNull(c.peek(key));
-
-        assertTrue(c.lock(key, 0));
-
-        assertTrue(c.isLocked(key));
-
-        c.unlock(key);
-
-        assertFalse(c.isLocked(key));
-
-        assertNull(c.peek(key));
-
-        checkValue(key, val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("UnnecessaryLocalVariable")
-    public void _testLockUnlockFiltered() throws Exception { // TODO: 9288, enable when fixed.
-        if (atomicityMode() == ATOMIC)
-            return;
-
-        GridCache<Integer, Integer> c = grid(0).cache(null);
+        assertNull(c.localPeek(key));
 
-        Integer key = primaryKey(c);
-        Integer val = key;
-
-        c.put(key, val);
-
-        assertTrue(c.lock(key, 0, new TestEntryPredicate(val)));
+        c.lock(key).lock();
 
         assertTrue(c.isLocked(key));
 
-        c.unlock(key);
+        c.lock(key).unlock();
 
         assertFalse(c.isLocked(key));
 
-        assertNull(c.peek(key));
+        assertNull(c.localPeek(key));
 
         checkValue(key, val);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheLockAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheLockAbstractTest.java
index f8a1636..fd77a1d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheLockAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheLockAbstractTest.java
@@ -172,12 +172,15 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"TooBroadScope"})
     public void testLockSingleThread() throws Exception {
+        final IgniteCache<Integer, String> cache1 = ignite1.jcache(null);
+        final IgniteCache<Integer, String> cache2 = ignite1.jcache(null);
+
         int k = 1;
         String v = String.valueOf(k);
 
         info("Before lock for key: " + k);
 
-        assert cache1.lock(k, 0L);
+        cache1.lock(k).lock();
 
         info("After lock for key: " + k);
 
@@ -191,7 +194,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
             info("Put " + k + '=' + k + " key pair into cache.");
         }
         finally {
-            cache1.unlock(k);
+            cache1.lock(k).unlock();
 
             info("Unlocked key: " + k);
         }
@@ -205,6 +208,9 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"TooBroadScope"})
     public void testLock() throws Exception {
+        final IgniteCache<Integer, String> cache1 = ignite1.jcache(null);
+        final IgniteCache<Integer, String> cache2 = ignite1.jcache(null);
+
         final int kv = 1;
 
         final CountDownLatch l1 = new CountDownLatch(1);
@@ -214,7 +220,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
             @Nullable @Override public Object call() throws Exception {
                 info("Before lock for key: " + kv);
 
-                assert cache1.lock(kv, 0L);
+                cache1.lock(kv).lock();
 
                 info("After lock for key: " + kv);
 
@@ -233,7 +239,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
                 finally {
                     Thread.sleep(1000);
 
-                    cache1.unlockAll(F.asList(kv));
+                    cache1.lockAll(Collections.singleton(kv)).unlock();
 
                     info("Unlocked key in thread 1: " + kv);
                 }
@@ -253,7 +259,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
 
                 l1.await();
 
-                assert cache2.lock(kv, 0L);
+                cache2.lock(kv).lock();
 
                 try {
                     String v = cache2.get(kv);
@@ -263,7 +269,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
                     assertEquals(Integer.toString(kv), v);
                 }
                 finally {
-                    cache2.unlockAll(F.asList(kv));
+                    cache2.lockAll(Collections.singleton(kv)).unlock();
 
                     info("Unlocked key in thread 2: " + kv);
                 }
@@ -293,12 +299,15 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
      * @throws Exception If test failed.
      */
     public void testLockAndPut() throws Exception {
+        final IgniteCache<Integer, String> cache1 = ignite1.jcache(null);
+        final IgniteCache<Integer, String> cache2 = ignite1.jcache(null);
+
         final CountDownLatch l1 = new CountDownLatch(1);
         final CountDownLatch l2 = new CountDownLatch(1);
 
         GridTestThread t1 = new GridTestThread(new Callable<Object>() {
             @Nullable @Override public Object call() throws Exception {
-                assert cache1.lock(1, 0L);
+                cache1.lock(1).lock();
 
                 info("Locked cache key: 1");
 
@@ -322,7 +331,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
                     info("Woke up from sleep.");
                 }
                 finally {
-                    cache1.unlockAll(F.asList(1));
+                    cache1.lockAll(Collections.singleton(1)).unlock();
 
                     info("Unlocked cache key: 1");
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
index 2336fb5..1374be2 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
@@ -148,6 +148,15 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
      * @param cache Cache.
      * @param key Key.
      */
+    private void checkLocked(IgniteCache<Integer,String> cache, Integer key) {
+        assert cache.isLocked(key);
+        assert cache.isLockedByThread(key);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     */
     private void checkRemoteLocked(GridCacheProjection<Integer,String> cache, Integer key) {
         assert cache.isLocked(key);
         assert !cache.isLockedByThread(key);
@@ -157,6 +166,15 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
      * @param cache Cache.
      * @param key Key.
      */
+    private void checkRemoteLocked(IgniteCache<Integer,String> cache, Integer key) {
+        assert cache.isLocked(key);
+        assert !cache.isLockedByThread(key);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     */
     @SuppressWarnings({"BusyWait"})
     private void checkUnlocked(GridCacheProjection<Integer,String> cache, Integer key) {
         assert !cache.isLockedByThread(key);
@@ -177,6 +195,30 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
 
         assertFalse("Key locked [key=" + key + ", entries=" + entries(key) + "]", cache.isLocked(key));
     }
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     */
+    @SuppressWarnings({"BusyWait"})
+    private void checkUnlocked(IgniteCache<Integer,String> cache, Integer key) {
+        assert !cache.isLockedByThread(key);
+
+        if (partitioned()) {
+            for(int i = 0; i < 200; i++)
+                if (cache.isLocked(key)) {
+                    try {
+                        Thread.sleep(10);
+                    }
+                    catch (InterruptedException e) {
+                        e.printStackTrace();
+                    }
+                }
+                else
+                    return;
+        }
+
+        assertFalse("Key locked [key=" + key + ", entries=" + entries(key) + "]", cache.isLocked(key));
+    }
 
     /**
      * @param cache Cache.
@@ -192,6 +234,15 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
      * @param cache Cache.
      * @param keys Keys.
      */
+    private void checkLocked(IgniteCache<Integer,String> cache, Iterable<Integer> keys) {
+        for (Integer key : keys)
+            checkLocked(cache, key);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param keys Keys.
+     */
     private void checkRemoteLocked(GridCacheProjection<Integer,String> cache, Iterable<Integer> keys) {
         for (Integer key : keys) {
             checkRemoteLocked(cache, key);
@@ -199,6 +250,15 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
     }
 
     /**
+     * @param cache Cache.
+     * @param keys Keys.
+     */
+    private void checkRemoteLocked(IgniteCache<Integer,String> cache, Iterable<Integer> keys) {
+        for (Integer key : keys)
+            checkRemoteLocked(cache, key);
+    }
+
+    /**
      *
      * @param cache Cache.
      * @param keys Keys.
@@ -210,17 +270,27 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
 
     /**
      *
+     * @param cache Cache.
+     * @param keys Keys.
+     */
+    private void checkUnlocked(IgniteCache<Integer,String> cache, Iterable<Integer> keys) {
+        for (Integer key : keys)
+            checkUnlocked(cache, key);
+    }
+
+    /**
+     *
      * @throws Exception If test failed.
      */
     public void testBasicLock() throws Exception {
-        GridCache<Integer, String> cache = ignite1.cache(null);
+        IgniteCache<Integer, String> cache = ignite1.jcache(null);
 
-        assert cache.lock(1, 0L);
+        cache.lock(1).lock();
 
         assert cache.isLocked(1);
         assert cache.isLockedByThread(1);
 
-        cache.unlockAll(F.asList(1));
+        cache.lockAll(Collections.singleton(1)).unlock();
 
         checkUnlocked(cache, 1);
     }
@@ -250,10 +320,10 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
      * @throws Exception If test fails.
      */
     public void testMultiNodeLock() throws Exception {
-        GridCache<Integer, String> cache1 = ignite1.cache(null);
-        GridCache<Integer, String> cache2 = ignite2.cache(null);
+        IgniteCache<Integer, String> cache1 = ignite1.jcache(null);
+        IgniteCache<Integer, String> cache2 = ignite2.jcache(null);
 
-        assert cache1.lock(1, 0L);
+        cache1.lock(1).lock();
 
         assert cache1.isLocked(1) : entries(1);
         assert cache1.isLockedByThread(1);
@@ -262,18 +332,18 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
         assert !cache2.isLockedByThread(1);
 
         try {
-            assert !cache2.lock(1, -1L);
+            assert !cache2.lock(1).tryLock();
 
             assert cache2.isLocked(1) : entries(1);
             assert !cache2.isLockedByThread(1);
         }
         finally {
-            cache1.unlock(1);
+            cache1.lock(1).unlock();
 
             checkUnlocked(cache1, 1);
         }
 
-        assert cache2.lock(1, 0L);
+        cache2.lock(1).lock();
 
         assert cache2.isLocked(1) : entries(1);
         assert cache2.isLockedByThread(1);
@@ -286,13 +356,13 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
         addListener(ignite1, new UnlockListener(latch, 1));
 
         try {
-            assert !cache1.lock(1, -1L);
+            assert !cache1.lock(1).tryLock();
 
             assert cache1.isLocked(1) : entries(1);
             assert !cache1.isLockedByThread(1);
         }
         finally {
-            cache2.unlockAll(F.asList(1));
+            cache2.lockAll(Collections.singleton(1)).unlock();
         }
 
         latch.await();
@@ -357,21 +427,21 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
      * @throws Exception If test fails.
      */
     public void testMultiNodeLockWithKeyLists() throws Exception {
-        GridCache<Integer, String> cache1 = ignite1.cache(null);
-        GridCache<Integer, String> cache2 = ignite2.cache(null);
+        IgniteCache<Integer, String> cache1 = ignite1.jcache(null);
+        IgniteCache<Integer, String> cache2 = ignite2.jcache(null);
 
-        Collection<Integer> keys1 = new ArrayList<>();
-        Collection<Integer> keys2 = new ArrayList<>();
+        Set<Integer> keys1 = new HashSet<>();
+        Set<Integer> keys2 = new HashSet<>();
 
         Collections.addAll(keys1, 1, 2, 3);
         Collections.addAll(keys2, 2, 3, 4);
 
-        assert cache1.lockAll(keys1, 0);
+        cache1.lockAll(keys1).lock();
 
         checkLocked(cache1, keys1);
 
         try {
-            assert !cache2.lockAll(keys2, -1);
+            assert !cache2.lockAll(keys2).tryLock();
 
             assert cache2.isLocked(2);
             assert cache2.isLocked(3);
@@ -384,7 +454,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
             assert !cache2.isLockedByThread(4);
         }
         finally {
-            cache1.unlockAll(keys1);
+            cache1.lockAll(keys1).unlock();
         }
 
         checkUnlocked(cache1, keys1);
@@ -392,7 +462,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
         checkUnlocked(cache1, keys2);
         checkUnlocked(cache2, 4);
 
-        assert cache2.lockAll(keys2, 0);
+        cache2.lockAll(keys2).lock();
 
         CountDownLatch latch1 = new CountDownLatch(keys2.size());
         CountDownLatch latch2 = new CountDownLatch(1);
@@ -405,7 +475,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
 
             checkUnlocked(cache2, 1);
 
-            assert cache1.lock(1, -1L);
+            assert cache1.lock(1).tryLock();
 
             checkLocked(cache1, 1);
 
@@ -414,9 +484,9 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
             checkRemoteLocked(cache2, 1);
         }
         finally {
-            cache2.unlockAll(keys2);
+            cache2.lockAll(keys2).unlock();
 
-            cache1.unlockAll(F.asList(1));
+            cache1.lockAll(Collections.singleton(1)).unlock();
         }
 
         latch1.await();
@@ -509,23 +579,23 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
      * @throws IgniteCheckedException If test failed.
      */
     public void testLockReentry() throws IgniteCheckedException {
-        GridCache<Integer, String> cache = ignite1.cache(null);
+        IgniteCache<Integer, String> cache = ignite1.jcache(null);
 
-        assert cache.lock(1, 0L);
+        cache.lock(1).lock();
 
         try {
             checkLocked(cache, 1);
 
-            assert cache.lock(1, 0L);
+            cache.lock(1).lock();
 
             checkLocked(cache, 1);
 
-            cache.unlockAll(F.asList(1));
+            cache.lockAll(Collections.singleton(1)).unlock();
 
             checkLocked(cache, 1);
         }
         finally {
-            cache.unlockAll(F.asList(1));
+            cache.lockAll(Collections.singleton(1)).unlock();
         }
 
         checkUnlocked(cache, 1);
@@ -535,7 +605,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
      * @throws Exception If test failed.
      */
     public void testLockMultithreaded() throws Exception {
-        final GridCache<Integer, String> cache = ignite1.cache(null);
+        final IgniteCache<Integer, String> cache = ignite1.jcache(null);
 
         final CountDownLatch l1 = new CountDownLatch(1);
         final CountDownLatch l2 = new CountDownLatch(1);
@@ -545,7 +615,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
             @Nullable @Override public Object call() throws Exception {
                 info("Before lock for.key 1");
 
-                assert cache.lock(1, 0L);
+                cache.lock(1).lock();
 
                 info("After lock for key 1");
 
@@ -557,23 +627,23 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
                     info("Let thread2 proceed.");
 
                     // Reentry.
-                    assert cache.lock(1, 0L);
+                    cache.lock(1).lock();
 
                     checkLocked(cache, 1);
 
                     // Nested lock.
-                    assert cache.lock(2, -1L);
+                    assert cache.lock(2).tryLock();
 
                     checkLocked(cache, 2);
 
                     // Unlock reentry.
-                    cache.unlockAll(F.asList(1));
+                    cache.lockAll(Collections.singleton(1)).unlock();
 
                     // Outer should still be owned.
                     checkLocked(cache, 1);
 
                     // Unlock in reverse order.
-                    cache.unlockAll(F.asList(2));
+                    cache.lockAll(Collections.singleton(2)).unlock();
 
                     checkUnlocked(cache, 2);
 
@@ -582,7 +652,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
                     info("Waited for latch 2");
                 }
                 finally {
-                    cache.unlockAll(F.asList(1));
+                    cache.lockAll(Collections.singleton(1)).unlock();
 
                     info("Unlocked entry for key 1.");
                 }
@@ -603,7 +673,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
 
                 info("Latch1 released.");
 
-                assert !cache.lock(1, -1L);
+                assert !cache.lock(1).tryLock();
 
                 info("Tried to lock cache for key1");
 
@@ -611,7 +681,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
 
                 info("Released latch2");
 
-                assert cache.lock(1, 0L);
+                cache.lock(1).lock();
 
                 try {
                     info("Locked cache for key 1");
@@ -621,7 +691,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
                     info("Checked that cache is locked for key 1");
                 }
                 finally {
-                    cache.unlockAll(F.asList(1));
+                    cache.lockAll(Collections.singleton(1)).unlock();
 
                     info("Unlocked cache for key 1");
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
index 6f81490..acc565c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
@@ -877,13 +877,15 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
         startGrid();
 
         try {
-            assert cache().lock(1, 0);
+            IgniteCache<Object, Object> cache = jcache();
 
-            assertNull(cache().put(1, "key1"));
-            assertEquals("key1", cache().put(1, "key2"));
-            assertEquals("key2", cache().get(1));
+            cache.lock(1).lock();
 
-            cache().unlock(1);
+            assertNull(cache.getAndPut(1, "key1"));
+            assertEquals("key1", cache.getAndPut(1, "key2"));
+            assertEquals("key2", cache.get(1));
+
+            cache.lock(1).unlock();
         }
         finally {
             stopAllGrids();
@@ -907,11 +909,11 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
             Integer k1 = forPrimary(g1);
             Integer k2 = forPrimary(g2);
 
-            GridCache<Object, Object> cache = cache(0);
+            IgniteCache<Object, Object> cache = jcache(0);
 
-            assert cache.lock(k0, 0);
-            assert cache.lock(k1, 0);
-            assert cache.lock(k2, 0);
+            cache.lock(k0).lock();
+            cache.lock(k1).lock();
+            cache.lock(k2).lock();
 
             cache.put(k0, "val0");
 
@@ -921,9 +923,9 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
             assertEquals("val1", cache.get(k1));
             assertEquals("val2", cache.get(k2));
 
-            cache.unlock(k0);
-            cache.unlock(k1);
-            cache.unlock(k2);
+            cache.lock(k0).unlock();
+            cache.lock(k1).unlock();
+            cache.lock(k2).unlock();
         }
         finally {
             stopAllGrids();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
index 92acdd8..24521a0 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
@@ -150,9 +150,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
                         @Override public void run() {
                             try {
                                 try {
-                                    boolean locked = node.cache(null).lock(key, 0);
-
-                                    assert locked;
+                                    node.jcache(null).lock(key).lock();
 
                                     info(">>> Acquired explicit lock for key: " + key);
 
@@ -160,9 +158,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
                                     info(">>> Acquiring explicit lock for key: " + key * 10);
 
-                                    locked = node.cache(null).lock(key * 10, 0);
-
-                                    assert locked;
+                                    node.jcache(null).lock(key * 10).lock();
 
                                     info(">>> Releasing locks [key1=" + key + ", key2=" + key * 10 + ']');
                                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
index 2145a30..f63d2d7 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
@@ -742,35 +742,35 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
         if (!transactional())
             return;
 
-        GridCache<Integer, String> near = cache(0);
+        IgniteCache<Integer, String> near = jcache(0);
 
         String val = Integer.toString(key);
 
-        near.lock(key, 0);
+        near.lock(key).lock();
 
         try {
             near.put(key, val);
 
-            assertEquals(val, near.peek(key));
+            assertEquals(val, near.localPeek(key));
             assertEquals(val, dht(primaryGrid(key)).peek(key));
 
             assertTrue(near.isLocked(key));
             assertTrue(near.isLockedByThread(key));
 
-            near.lock(key, 0); // Reentry.
+            near.lock(key).lock(); // Reentry.
 
             try {
                 assertEquals(val, near.get(key));
                 assertEquals(val, near.remove(key));
 
-                assertNull(near.peek(key));
+                assertNull(near.localPeek(key));
                 assertNull(dht(primaryGrid(key)).peek(key));
 
                 assertTrue(near.isLocked(key));
                 assertTrue(near.isLockedByThread(key));
             }
             finally {
-                near.unlock(key);
+                near.lock(key).unlock();
             }
 
             assertTrue(near.isLocked(key));
@@ -782,7 +782,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
             throw t;
         }
         finally {
-            near.unlock(key);
+            near.lock(key).unlock();
         }
 
         assertFalse(near(0).isLockedNearOnly(key));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
index be64e75..25adeea 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOneNodeSelfTest.java
@@ -29,6 +29,7 @@ import org.gridgain.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
 import java.util.concurrent.*;
+import java.util.concurrent.locks.*;
 
 import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
 import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
@@ -179,47 +180,49 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testSingleLockPut() throws Exception {
-        GridCache<Integer, String> near = cache();
+        IgniteCache<Integer, String> near = jcache();
 
-        near.lock(1, 0);
+        near.lock(1).lock();
 
         try {
             near.put(1, "1");
             near.put(2, "2");
 
-            String one = near.put(1, "3");
+            String one = near.getAndPut(1, "3");
 
             assertNotNull(one);
             assertEquals("1", one);
         }
         finally {
-            near.unlock(1);
+            near.lock(1).unlock();
         }
     }
 
     /** @throws Exception If failed. */
     public void testSingleLock() throws Exception {
-        GridCache<Integer, String> near = cache();
+        IgniteCache<Integer, String> near = jcache();
 
-        near.lock(1, 0);
+        Lock lock = near.lock(1);
+
+        lock.lock();
 
         try {
             near.put(1, "1");
 
-            assertEquals("1", near.peek(1));
+            assertEquals("1", near.localPeek(1));
             assertEquals("1", dht().peek(1));
 
             assertEquals("1", near.get(1));
             assertEquals("1", near.remove(1));
 
-            assertNull(near.peek(1));
+            assertNull(near.localPeek(1));
             assertNull(dht().peek(1));
 
             assertTrue(near.isLocked(1));
             assertTrue(near.isLockedByThread(1));
         }
         finally {
-            near.unlock(1);
+            near.lock(1).unlock();
         }
 
         assertFalse(near.isLocked(1));
@@ -228,40 +231,40 @@ public class GridCacheNearOneNodeSelfTest extends GridCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testSingleLockReentry() throws Exception {
-        GridCache<Integer, String> near = cache();
+        IgniteCache<Integer, String> near = jcache();
 
-        near.lock(1, 0);
+        near.lock(1).lock();
 
         try {
             near.put(1, "1");
 
-            assertEquals("1", near.peek(1));
+            assertEquals("1", near.localPeek(1));
             assertEquals("1", dht().peek(1));
 
             assertTrue(near.isLocked(1));
             assertTrue(near.isLockedByThread(1));
 
-            near.lock(1, 0); // Reentry.
+            near.lock(1).lock(); // Reentry.
 
             try {
                 assertEquals("1", near.get(1));
                 assertEquals("1", near.remove(1));
 
-                assertNull(near.peek(1));
+                assertNull(near.localPeek(1));
                 assertNull(dht().peek(1));
 
                 assertTrue(near.isLocked(1));
                 assertTrue(near.isLockedByThread(1));
             }
             finally {
-                near.unlock(1);
+                near.lock(1).unlock();
             }
 
             assertTrue(near.isLocked(1));
             assertTrue(near.isLockedByThread(1));
         }
         finally {
-            near.unlock(1);
+            near.lock(1).unlock();
         }
 
         assertFalse(near.isLocked(1));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
index 61ba20e..7cb1a5b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
@@ -417,48 +417,6 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("BusyWait")
-    @Override public void testUnlockFiltered() throws Exception {
-        if (lockingEnabled()) {
-            List<String> keys = primaryKeysForCache(fullCache(), 2);
-
-            String key1 = keys.get(0);
-            String key2 = keys.get(1);
-
-            cache().put(key1, 1);
-            cache().put(key2, 100);
-
-            assert !cache().isLocked(key1);
-            assert !cache().isLocked(key2);
-            assert !fullCache().isLocked(key1);
-            assert !fullCache().isLocked(key2);
-
-            cache().lock(key1, 0L);
-            cache().lock(key2, 0L);
-
-            assert cache().isLocked(key1);
-            assert cache().isLocked(key2);
-
-            cache().unlock(key1, gte100);
-            cache().unlock(key2, gte100);
-
-            for (int i = 0; i < 100; i++) {
-                if (fullCache().isLocked(key2))
-                    Thread.sleep(10);
-                else
-                    break;
-            }
-
-            assert cache().isLocked(key1);
-            assert fullCache().isLocked(key1);
-            assert !cache().isLocked(key2);
-            assert !fullCache().isLocked(key2);
-
-            cache().unlockAll(F.asList(key1, key2));
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public void testPrimaryData() throws Exception {
         // Not needed for near-only cache.
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
index 88b27f3..ffc83e5 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
@@ -515,16 +515,16 @@ 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);
 
-        assertTrue(cache.lock(key1, 0L));
+        cache.lock(key1).lock();
 
         try {
             // Nested lock.
-            assertTrue(cache.lock(key2, 0L));
+            cache.lock(key2).lock();
 
             try {
-                assertNull(cache.put(key1, val1));
+                assertNull(cache.getAndPut(key1, val1));
 
                 assertEquals(val1, dht(0).peek(key1));
                 assertEquals(val1, dht(1).peek(key1));
@@ -537,7 +537,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));
@@ -574,11 +574,11 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
                 assertNull(near(2).peekEx(key2));
             }
             finally {
-                cache.unlock(key2);
+                cache.lock(key2).lock();
             }
         }
         finally {
-            cache.unlock(key1);
+            cache.lock(key1).unlock();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index f3e3e8c0..94a13b0 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.gridgain.grid.kernal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLockSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLockSelfTest.java
index 3b94d87..26edf39 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLockSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLockSelfTest.java
@@ -72,7 +72,7 @@ public class GridCachePartitionedLockSelfTest extends GridCacheLockAbstractTest
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                return g0.cache(null).lock(1, Long.MAX_VALUE);
+                return g0.jcache(null).lock(1).tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
             }
         }, IgniteCheckedException.class, "Locks are not supported");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionLockUnlockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionLockUnlockSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionLockUnlockSelfTest.java
index 2e772da..88d4f87 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionLockUnlockSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionLockUnlockSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.gridgain.grid.kernal.processors.cache.eviction;
 
+import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
@@ -121,10 +122,10 @@ public class GridCacheEvictionLockUnlockSelfTest extends GridCommonAbstractTest
             for (int i = 0; i < gridCnt; i++) {
                 reset();
 
-                GridCache<Object, Object> cache = cache(i);
+                IgniteCache<Object, Object> cache = jcache(i);
 
-                cache.lock("key", 0L);
-                cache.unlock("key");
+                cache.lock("key").lock();
+                cache.lock("key").unlock();
 
                 assertTrue(evictLatch.await(3, SECONDS));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalLockSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalLockSelfTest.java
index 5ad1475..49a83ac 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalLockSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalLockSelfTest.java
@@ -79,39 +79,39 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If test failed.
      */
     public void testLockReentry() throws IgniteCheckedException {
-        GridCache<Integer, String> cache = ignite.cache(null);
+        IgniteCache<Integer, String> cache = ignite.jcache(null);
 
         assert !cache.isLocked(1);
         assert !cache.isLockedByThread(1);
 
-        assert cache.lock(1, 0L);
+        cache.lock(1).lock();
 
         assert cache.isLocked(1);
         assert cache.isLockedByThread(1);
 
         try {
             assert cache.get(1) == null;
-            assert cache.put(1, "1") == null;
+            assert cache.getAndPut(1, "1") == null;
             assert "1".equals(cache.get(1));
 
             // Reentry.
-            assert cache.lock(1, 0L);
+            cache.lock(1).lock();
 
             assert cache.isLocked(1);
             assert cache.isLockedByThread(1);
 
             try {
-                assert "1".equals(cache.remove(1));
+                assert "1".equals(cache.getAndRemove(1));
             }
             finally {
-                cache.unlock(1);
+                cache.lock(1).unlock();
             }
 
             assert cache.isLocked(1);
             assert cache.isLockedByThread(1);
         }
         finally {
-            cache.unlock(1);
+            cache.lock(1).unlock();
         }
 
         assert !cache.isLocked(1);
@@ -122,7 +122,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
      * @throws Exception If test failed.
      */
     public void testLock() throws Throwable {
-        final GridCache<Integer, String> cache = ignite.cache(null);
+        final IgniteCache<Integer, String> cache = ignite.jcache(null);
 
         final CountDownLatch latch1 = new CountDownLatch(1);
         final CountDownLatch latch2 = new CountDownLatch(1);
@@ -133,7 +133,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
             @Nullable @Override public Object call() throws Exception {
                 info("Before lock for.key 1");
 
-                assert cache.lock(1, 0L);
+                cache.lock(1).lock();
 
                 info("After lock for key 1");
 
@@ -154,7 +154,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
                     info("Waited for latch 2");
                 }
                 finally {
-                    cache.unlock(1);
+                    cache.lock(1).unlock();
 
                     info("Unlocked entry for key 1.");
 
@@ -174,7 +174,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
 
                 info("Latch1 released.");
 
-                assert !cache.lock(1, -1L);
+                assert !cache.lock(1).tryLock();
 
                 assert cache.isLocked(1);
                 assert !cache.isLockedByThread(1);
@@ -187,7 +187,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
 
                 latch3.await();
 
-                assert cache.lock(1, -1L);
+                assert cache.lock(1).tryLock();
 
                 assert cache.isLocked(1);
                 assert cache.isLockedByThread(1);
@@ -199,7 +199,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
 
                     info("Read value for key 1");
 
-                    assert "1".equals(cache.remove(1));
+                    assert "1".equals(cache.getAndRemove(1));
 
                     info("Removed value for key 1");
 
@@ -209,7 +209,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
                     info("Checked that cache is locked for key 1");
                 }
                 finally {
-                    cache.unlock(1);
+                    cache.lock(1).unlock();
 
                     info("Unlocked cache for key 1");
                 }
@@ -238,13 +238,13 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
      * @throws Exception If test failed.
      */
     public void testLockAndPut() throws Throwable {
-        final GridCache<Integer, String> cache = ignite.cache(null);
+        final IgniteCache<Integer, String> cache = ignite.jcache(null);
 
         final CountDownLatch latch1 = new CountDownLatch(1);
 
         GridTestThread t1 = new GridTestThread(new Callable<Object>() {
             @Nullable @Override public Object call() throws Exception {
-                assert cache.lock(1, 0L);
+                cache.lock(1).lock();
 
                 info("Locked cache key: 1");
 
@@ -268,7 +268,7 @@ public class GridCacheLocalLockSelfTest extends GridCommonAbstractTest {
                     info("Woke up from sleep.");
                 }
                 finally {
-                    cache.unlock(1);
+                    cache.lock(1).unlock();
 
                     info("Unlocked cache key: 1");
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7737e759/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
index ce1ebfe..1a42c15 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalMultithreadedSelfTest.java
@@ -83,10 +83,12 @@ public class GridCacheLocalMultithreadedSelfTest extends GridCommonAbstractTest
      * @throws Exception If test fails.
      */
     public void testBasicLocks() throws Throwable {
+        final IgniteCache<Object, Object> cache = grid().jcache(null);
+
         GridTestUtils.runMultiThreaded(new Callable<Object>() {
             /** {@inheritDoc} */
             @Override public Object call() throws Exception {
-                assert cache.lock(1, 1000L);
+                assert cache.lock(1).tryLock(1000L, TimeUnit.MILLISECONDS);
 
                 info("Locked key from thread: " + thread());
 
@@ -94,7 +96,7 @@ public class GridCacheLocalMultithreadedSelfTest extends GridCommonAbstractTest
 
                 info("Unlocking key from thread: " + thread());
 
-                cache.unlock(1);
+                cache.lock(1).unlock();
 
                 info("Unlocked key from thread: " + thread());
 
@@ -167,6 +169,8 @@ public class GridCacheLocalMultithreadedSelfTest extends GridCommonAbstractTest
      * @throws Exception If test fails.
      */
     public void testSingleLockTimeout() throws Exception {
+        final IgniteCache<Object, Object> cache = grid().jcache(null);
+
         final CountDownLatch l1 = new CountDownLatch(1);
         final CountDownLatch l2 = new CountDownLatch(1);
 
@@ -175,7 +179,7 @@ public class GridCacheLocalMultithreadedSelfTest extends GridCommonAbstractTest
             @Override public Object call() throws Exception {
                 assert !cache.isLocked(1);
 
-                assert cache.lock(1, 0);
+                cache.lock(1).lock();
 
                 assert cache.isLockedByThread(1);
                 assert cache.isLocked(1);
@@ -184,7 +188,7 @@ public class GridCacheLocalMultithreadedSelfTest extends GridCommonAbstractTest
 
                 l2.await();
 
-                cache.unlock(1);
+                cache.lock(1).unlock();
 
                 assert !cache.isLockedByThread(1);
                 assert !cache.isLocked(1);
@@ -201,7 +205,7 @@ public class GridCacheLocalMultithreadedSelfTest extends GridCommonAbstractTest
                 assert cache.isLocked(1);
                 assert !cache.isLockedByThread(1);
 
-                assert !cache.lock(1, 100L);
+                assert !cache.lock(1).tryLock(100L, TimeUnit.MILLISECONDS);
 
                 assert cache.isLocked(1);
                 assert !cache.isLockedByThread(1);