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/07/14 15:07:24 UTC

[04/16] incubator-ignite git commit: # ignite-929 close does not destroy cache

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
index 1ba24e3..5093af5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
@@ -107,7 +107,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
         assertMetrics(cache1);
         assertMetrics(cache2);
 
-        closeCaches();
+        destroyCaches();
     }
 
     /**
@@ -135,7 +135,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
         assertMetrics(cache1);
         assertMetrics(cache2);
 
-        closeCaches();
+        destroyCaches();
     }
 
     /**
@@ -157,9 +157,9 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
     /**
      * Closes caches.
      */
-    private void closeCaches() {
-        cache1.close();
-        cache2.close();
+    private void destroyCaches() {
+        cache1.destroy();
+        cache2.destroy();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
index 8c7d33d..f4d7607 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheOffheapMapEntrySelfTest.java
@@ -149,7 +149,9 @@ public class CacheOffheapMapEntrySelfTest extends GridCacheAbstractSelfTest {
             cacheMode,
             "Cache");
 
-        try (IgniteCache jcache = grid(0).getOrCreateCache(cfg)) {
+        IgniteCache jcache = grid(0).getOrCreateCache(cfg);
+
+        try {
             GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(0)).internalCache(jcache.getName());
 
             Integer key = primaryKey(grid(0).cache(null));
@@ -164,5 +166,8 @@ public class CacheOffheapMapEntrySelfTest extends GridCacheAbstractSelfTest {
 
             assertEquals(entry.getClass(), entryCls);
         }
+        finally {
+            jcache.destroy();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java
new file mode 100644
index 0000000..20284a8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStopAndDestroySelfTest.java
@@ -0,0 +1,859 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.communication.tcp.*;
+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.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import javax.cache.CacheManager;
+import javax.cache.configuration.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Checks stop and destroy methods behavior.
+ */
+public class CacheStopAndDestroySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** key-value used at test. */
+    protected static String KEY_VAL = "1";
+
+    /** cache name 1. */
+    protected static String CACHE_NAME_DHT = "cache";
+
+    /** cache name 2. */
+    protected static String CACHE_NAME_CLIENT = "cache_client";
+
+    /** near cache name. */
+    protected static String CACHE_NAME_NEAR = "cache_near";
+
+    /** local cache name. */
+    protected static String CACHE_NAME_LOC = "cache_local";
+
+    /** */
+    private static volatile boolean stop;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGridsMultiThreaded(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @return Grids count to start.
+     */
+    protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration iCfg = super.getConfiguration(gridName);
+
+        if (getTestGridName(2).equals(gridName))
+            iCfg.setClientMode(true);
+
+        ((TcpDiscoverySpi)iCfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)iCfg.getDiscoverySpi()).setForceServerMode(true);
+
+        iCfg.setCacheConfiguration();
+
+        TcpCommunicationSpi commSpi = new CountingTxRequestsToClientNodeTcpCommunicationSpi();
+
+        commSpi.setLocalPort(GridTestUtils.getNextCommPort(getClass()));
+        commSpi.setTcpNoDelay(true);
+
+        iCfg.setCommunicationSpi(commSpi);
+
+        return iCfg;
+    }
+
+    /**
+     * Helps to count messages.
+     */
+    public static class CountingTxRequestsToClientNodeTcpCommunicationSpi extends TcpCommunicationSpi {
+        /** Counter. */
+        public static AtomicInteger cnt = new AtomicInteger();
+
+        /** Node filter. */
+        public static UUID nodeFilter;
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+            super.sendMessage(node, msg);
+
+            if (nodeFilter != null &&
+                node.id().equals(nodeFilter) &&
+                msg instanceof GridIoMessage &&
+                ((GridIoMessage)msg).message() instanceof GridDhtTxPrepareRequest)
+                cnt.incrementAndGet();
+        }
+    }
+
+    /**
+     * @return dht config
+     */
+    private CacheConfiguration getDhtConfig() {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(CACHE_NAME_DHT);
+        cfg.setCacheMode(PARTITIONED);
+        cfg.setNearConfiguration(null);
+
+        return cfg;
+    }
+
+    /**
+     * @return client config
+     */
+    private CacheConfiguration getClientConfig() {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(CACHE_NAME_CLIENT);
+        cfg.setCacheMode(PARTITIONED);
+        cfg.setNearConfiguration(null);
+
+        return cfg;
+    }
+
+    /**
+     * @return near config
+     */
+    private CacheConfiguration getNearConfig() {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(CACHE_NAME_NEAR);
+        cfg.setCacheMode(PARTITIONED);
+        cfg.setNearConfiguration(new NearCacheConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @return local config
+     */
+    private CacheConfiguration getLocalConfig() {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(CACHE_NAME_LOC);
+        cfg.setCacheMode(LOCAL);
+        cfg.setNearConfiguration(null);
+
+        return cfg;
+    }
+
+    /**
+     * Test Double Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDhtDoubleDestroy() throws Exception {
+        dhtDestroy();
+
+        dhtDestroy();
+    }
+
+    /**
+     * Test DHT Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    private void dhtDestroy() throws Exception {
+        grid(0).getOrCreateCache(getDhtConfig());
+
+        assertNull(grid(0).cache(CACHE_NAME_DHT).get(KEY_VAL));
+
+        grid(0).cache(CACHE_NAME_DHT).put(KEY_VAL, KEY_VAL);
+
+        assertEquals(KEY_VAL, grid(0).cache(CACHE_NAME_DHT).get(KEY_VAL));
+        assertEquals(KEY_VAL, grid(1).cache(CACHE_NAME_DHT).get(KEY_VAL));
+        assertEquals(KEY_VAL, grid(2).cache(CACHE_NAME_DHT).get(KEY_VAL));
+
+        assertFalse(grid(0).configuration().isClientMode());
+
+        // DHT Destroy. Cache should be removed from each node.
+
+        IgniteCache<Object, Object> cache = grid(0).cache(CACHE_NAME_DHT);
+
+        cache.destroy();
+
+        checkDestroyed(cache);
+    }
+
+    /**
+     * Test Double Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientDoubleDestroy() throws Exception {
+        clientDestroy();
+
+        clientDestroy();
+    }
+
+    /**
+     * Test Client Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    private void clientDestroy() throws Exception {
+        grid(0).getOrCreateCache(getClientConfig());
+
+        assertNull(grid(0).cache(CACHE_NAME_CLIENT).get(KEY_VAL));
+
+        grid(0).cache(CACHE_NAME_CLIENT).put(KEY_VAL, KEY_VAL);
+
+        assertEquals(KEY_VAL, grid(0).cache(CACHE_NAME_CLIENT).get(KEY_VAL));
+        assertEquals(KEY_VAL, grid(1).cache(CACHE_NAME_CLIENT).get(KEY_VAL));
+        assertEquals(KEY_VAL, grid(2).cache(CACHE_NAME_CLIENT).get(KEY_VAL));
+
+        // DHT Destroy from client node. Cache should be removed from each node.
+
+        assertTrue(grid(2).configuration().isClientMode());
+
+        IgniteCache<Object, Object> cache = grid(2).cache(CACHE_NAME_CLIENT);
+
+        cache.destroy(); // Client node.
+
+        checkDestroyed(cache);
+    }
+
+    /**
+     * Test Double Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNearDoubleDestroy() throws Exception {
+        nearDestroy();
+
+        nearDestroy();
+    }
+
+    /**
+     * Test Near Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    private void nearDestroy() throws Exception {
+        grid(0).getOrCreateCache(getNearConfig());
+
+        grid(2).getOrCreateNearCache(CACHE_NAME_NEAR, new NearCacheConfiguration());
+
+        assertNull(grid(0).cache(CACHE_NAME_NEAR).get(KEY_VAL));
+        assertNull(grid(2).cache(CACHE_NAME_NEAR).get(KEY_VAL));
+
+        grid(2).cache(CACHE_NAME_NEAR).put(KEY_VAL, KEY_VAL);
+        grid(0).cache(CACHE_NAME_NEAR).put(KEY_VAL, "near-test");
+
+        assertEquals("near-test", grid(2).cache(CACHE_NAME_NEAR).localPeek(KEY_VAL));
+
+        // Near cache destroy. Cache should be removed from each node.
+
+        IgniteCache<Object, Object> cache = grid(2).cache(CACHE_NAME_NEAR);
+
+        cache.destroy();
+
+        checkDestroyed(cache);
+    }
+
+    /**
+     * Test Double Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLocalDoubleDestroy() throws Exception {
+        localDestroy();
+
+        localDestroy();
+    }
+
+    /**
+     * Test Local Destroy.
+     *
+     * @throws Exception If failed.
+     */
+    private void localDestroy() throws Exception {
+        grid(0).getOrCreateCache(getLocalConfig());
+
+        assert grid(0).cache(CACHE_NAME_LOC).get(KEY_VAL) == null;
+        assert grid(1).cache(CACHE_NAME_LOC).get(KEY_VAL) == null;
+
+        grid(0).cache(CACHE_NAME_LOC).put(KEY_VAL, KEY_VAL + 0);
+        grid(1).cache(CACHE_NAME_LOC).put(KEY_VAL, KEY_VAL + 1);
+
+        assert grid(0).cache(CACHE_NAME_LOC).get(KEY_VAL).equals(KEY_VAL + 0);
+        assert grid(1).cache(CACHE_NAME_LOC).get(KEY_VAL).equals(KEY_VAL + 1);
+
+        grid(0).cache(CACHE_NAME_LOC).destroy();
+
+        assertNull(grid(0).cache(CACHE_NAME_LOC));
+    }
+
+    /**
+     * Test Dht close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDhtClose() throws Exception {
+        IgniteCache<Integer, Integer> dhtCache0 = grid(0).getOrCreateCache(getDhtConfig());
+
+        final Integer key = primaryKey(dhtCache0);
+
+        assertNull(dhtCache0.get(key));
+
+        dhtCache0.put(key, key);
+
+        assertEquals(key, dhtCache0.get(key));
+
+        // DHT Close. No-op.
+
+        IgniteCache<Integer, Integer> dhtCache1 = grid(1).cache(CACHE_NAME_DHT);
+        IgniteCache<Integer, Integer> dhtCache2 = grid(2).cache(CACHE_NAME_DHT);
+
+        dhtCache0.close();
+
+        try {
+            dhtCache0.get(key);// Not affected, but can not be taken.
+
+            fail();
+        }
+        catch (IllegalStateException ignored) {
+            // No-op
+        }
+
+        assertEquals(key, dhtCache1.get(key)); // Not affected.
+        assertEquals(key, dhtCache2.get(key));// Not affected.
+
+        // DHT Creation after closed.
+
+        IgniteCache<Integer, Integer> dhtCache0New = grid(0).cache(CACHE_NAME_DHT);
+
+        assertNotSame(dhtCache0, dhtCache0New);
+
+        assertEquals(key, dhtCache0New.get(key)); // Not affected, can be taken since cache reopened.
+
+        dhtCache2.put(key, key + 1);
+
+        assertEquals((Object)(key + 1), dhtCache0New.get(key));
+
+        // Check close at last node.
+
+        stopAllGrids(true);
+
+        startGrid(0);
+
+        dhtCache0 = grid(0).getOrCreateCache(getDhtConfig());
+
+        assertNull(dhtCache0.get(key));
+
+        dhtCache0.put(key, key);
+
+        assertEquals(key, dhtCache0.get(key));
+
+        // Closing last node.
+        dhtCache0.close();
+
+        try {
+            dhtCache0.get(key);// Can not be taken.
+
+            fail();
+        }
+        catch (IllegalStateException ignored) {
+            // No-op
+        }
+
+        // Reopening cache.
+        dhtCache0 = grid(0).cache(CACHE_NAME_DHT);
+
+        assertEquals(key, dhtCache0.get(key)); // Entry not loosed.
+    }
+
+    /**
+     * Test Dht close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDhtCloseWithTry() throws Exception {
+        String curVal = null;
+
+        for (int i = 0; i < 3; i++) {
+            try (IgniteCache<String, String> cache0 = grid(0).getOrCreateCache(getDhtConfig())) {
+                IgniteCache<String, String> cache1 = grid(1).cache(CACHE_NAME_DHT);
+                IgniteCache<String, String> cache2 = grid(2).cache(CACHE_NAME_DHT);
+
+                if (i == 0) {
+                    assert cache0.get(KEY_VAL) == null;
+                    assert cache1.get(KEY_VAL) == null;
+                    assert cache2.get(KEY_VAL) == null;
+                }
+                else {
+                    assert cache0.get(KEY_VAL).equals(curVal);
+                    assert cache1.get(KEY_VAL).equals(curVal);
+                    assert cache2.get(KEY_VAL).equals(curVal);
+                }
+
+                curVal = KEY_VAL + curVal;
+
+                cache0.put(KEY_VAL, curVal);
+
+                assert cache0.get(KEY_VAL).equals(curVal);
+                assert cache1.get(KEY_VAL).equals(curVal);
+                assert cache2.get(KEY_VAL).equals(curVal);
+            }
+        }
+    }
+
+    /**
+     * Test Client close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientClose() throws Exception {
+        IgniteCache<String, String> cache0 = grid(0).getOrCreateCache(getClientConfig());
+
+        assert cache0.get(KEY_VAL) == null;
+
+        cache0.put(KEY_VAL, KEY_VAL);
+
+        assert cache0.get(KEY_VAL).equals(KEY_VAL);
+
+        // DHT Close from client node. Should affect only client node.
+
+        IgniteCache<String, String> cache1 = grid(1).cache(CACHE_NAME_CLIENT);
+        IgniteCache<String, String> cache2 = grid(2).cache(CACHE_NAME_CLIENT);
+
+        assert cache2.get(KEY_VAL).equals(KEY_VAL);
+
+        cache2.close();// Client node.
+
+        assert cache0.get(KEY_VAL).equals(KEY_VAL);// Not affected.
+        assert cache1.get(KEY_VAL).equals(KEY_VAL);// Not affected.
+
+        try {
+            cache2.get(KEY_VAL);// Affected.
+
+            assert false;
+        }
+        catch (IllegalStateException ignored) {
+            // No-op
+        }
+
+        // DHT Creation from client node after closed.
+        IgniteCache<String, String> cache2New = grid(2).cache(CACHE_NAME_CLIENT);
+
+        assertNotSame(cache2, cache2New);
+
+        assert cache2New.get(KEY_VAL).equals(KEY_VAL);
+
+        cache0.put(KEY_VAL, KEY_VAL + "recreated");
+
+        assert cache0.get(KEY_VAL).equals(KEY_VAL + "recreated");
+        assert cache1.get(KEY_VAL).equals(KEY_VAL + "recreated");
+        assert cache2New.get(KEY_VAL).equals(KEY_VAL + "recreated");
+    }
+
+    /**
+     * Test Client close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testClientCloseWithTry() throws Exception {
+        String curVal = null;
+
+        for (int i = 0; i < 3; i++) {
+            try (IgniteCache<String, String> cache2 = grid(2).getOrCreateCache(getClientConfig())) {
+                IgniteCache<String, String> cache0 = grid(0).cache(CACHE_NAME_CLIENT);
+                IgniteCache<String, String> cache1 = grid(1).cache(CACHE_NAME_CLIENT);
+
+                if (i == 0) {
+                    assert cache0.get(KEY_VAL) == null;
+                    assert cache1.get(KEY_VAL) == null;
+                    assert cache2.get(KEY_VAL) == null;
+                }
+                else {
+                    assert cache0.get(KEY_VAL).equals(curVal);
+                    assert cache1.get(KEY_VAL).equals(curVal);
+                    assert cache2.get(KEY_VAL).equals(curVal);
+                }
+
+                curVal = KEY_VAL + curVal;
+
+                cache2.put(KEY_VAL, curVal);
+
+                assert cache0.get(KEY_VAL).equals(curVal);
+                assert cache1.get(KEY_VAL).equals(curVal);
+                assert cache2.get(KEY_VAL).equals(curVal);
+            }
+
+            awaitPartitionMapExchange();
+        }
+    }
+
+    /**
+     * Test Near close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNearClose() throws Exception {
+        IgniteCache<String, String> cache0 = grid(0).getOrCreateCache(getNearConfig());
+
+        // GridDhtTxPrepareRequest requests to Client node will be counted.
+        CountingTxRequestsToClientNodeTcpCommunicationSpi.nodeFilter = grid(2).context().localNodeId();
+
+        // Near Close from client node.
+
+        IgniteCache<String, String> cache1 = grid(1).cache(CACHE_NAME_NEAR);
+        IgniteCache<String, String> cache2 = grid(2).createNearCache(CACHE_NAME_NEAR, new NearCacheConfiguration());
+
+        assert cache2.get(KEY_VAL) == null;
+
+        // Subscribing to events.
+        cache2.put(KEY_VAL, KEY_VAL);
+
+        CountingTxRequestsToClientNodeTcpCommunicationSpi.cnt.set(0);
+
+        cache0.put(KEY_VAL, "near-test");
+
+        U.sleep(1000);
+
+        //Ensure near cache was automatically updated.
+        assert CountingTxRequestsToClientNodeTcpCommunicationSpi.cnt.get() != 0;
+
+        assert cache2.localPeek(KEY_VAL).equals("near-test");
+
+        cache2.close();
+
+        CountingTxRequestsToClientNodeTcpCommunicationSpi.cnt.set(0);
+
+        // Should not produce messages to client node.
+        cache0.put(KEY_VAL, KEY_VAL + 0);
+
+        U.sleep(1000);
+
+        // Ensure near cache was NOT automatically updated.
+        assert CountingTxRequestsToClientNodeTcpCommunicationSpi.cnt.get() == 0;
+
+        assert cache0.get(KEY_VAL).equals(KEY_VAL + 0);// Not affected.
+        assert cache1.get(KEY_VAL).equals(KEY_VAL + 0);// Not affected.
+
+        try {
+            cache2.get(KEY_VAL);// Affected.
+
+            assert false;
+        }
+        catch (IllegalArgumentException | IllegalStateException ignored) {
+            // No-op
+        }
+
+        // Near Creation from client node after closed.
+
+        IgniteCache<String, String> cache2New = grid(2).createNearCache(CACHE_NAME_NEAR, new NearCacheConfiguration());
+
+        assertNotSame(cache2, cache2New);
+
+        // Subscribing to events.
+        cache2New.put(KEY_VAL, KEY_VAL);
+
+        assert cache2New.localPeek(KEY_VAL).equals(KEY_VAL);
+
+        cache0.put(KEY_VAL, KEY_VAL + "recreated");
+
+        assert cache0.get(KEY_VAL).equals(KEY_VAL + "recreated");
+        assert cache1.get(KEY_VAL).equals(KEY_VAL + "recreated");
+        assert cache2New.localPeek(KEY_VAL).equals(KEY_VAL + "recreated");
+    }
+
+    /**
+     * Test Near close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNearCloseWithTry() throws Exception {
+        String curVal = null;
+
+        grid(0).getOrCreateCache(getNearConfig());
+
+        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+        for (int i = 0; i < 3; i++) {
+            try (IgniteCache<String, String> cache2 = grid(2).getOrCreateNearCache(CACHE_NAME_NEAR, nearCfg)) {
+                IgniteCache<String, String> cache0 = grid(0).cache(CACHE_NAME_NEAR);
+                IgniteCache<String, String> cache1 = grid(1).cache(CACHE_NAME_NEAR);
+
+                assert cache2.localPeek(KEY_VAL) == null;
+
+                assert cache0.get(KEY_VAL) == null || cache0.get(KEY_VAL).equals(curVal);
+                assert cache1.get(KEY_VAL) == null || cache1.get(KEY_VAL).equals(curVal);
+                assert cache2.get(KEY_VAL) == null || cache2.get(KEY_VAL).equals(curVal);
+
+                curVal = KEY_VAL + curVal;
+
+                cache2.put(KEY_VAL, curVal);
+
+                assert cache2.localPeek(KEY_VAL).equals(curVal);
+
+                assert cache0.get(KEY_VAL).equals(curVal);
+                assert cache1.get(KEY_VAL).equals(curVal);
+                assert cache2.get(KEY_VAL).equals(curVal);
+            }
+        }
+    }
+
+    /**
+     * Test Local close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLocalClose() throws Exception {
+        grid(0).getOrCreateCache(getLocalConfig());
+
+        assert grid(0).cache(CACHE_NAME_LOC).get(KEY_VAL) == null;
+        assert grid(1).cache(CACHE_NAME_LOC).get(KEY_VAL) == null;
+
+        grid(0).cache(CACHE_NAME_LOC).put(KEY_VAL, KEY_VAL + 0);
+        grid(1).cache(CACHE_NAME_LOC).put(KEY_VAL, KEY_VAL + 1);
+
+        assert grid(0).cache(CACHE_NAME_LOC).get(KEY_VAL).equals(KEY_VAL + 0);
+        assert grid(1).cache(CACHE_NAME_LOC).get(KEY_VAL).equals(KEY_VAL + 1);
+
+        // Local close. Same as Local destroy.
+
+        IgniteCache<Object, Object> cache = grid(1).cache(CACHE_NAME_LOC);
+
+        cache.close();
+
+        checkUsageFails(cache);
+
+        assertNull(grid(1).cache(CACHE_NAME_LOC));
+
+        // Local creation after closed.
+
+        grid(0).getOrCreateCache(getLocalConfig());
+
+        grid(0).cache(CACHE_NAME_LOC).put(KEY_VAL, KEY_VAL + "recreated0");
+        grid(1).cache(CACHE_NAME_LOC).put(KEY_VAL, KEY_VAL + "recreated1");
+        grid(2).cache(CACHE_NAME_LOC).put(KEY_VAL, KEY_VAL + "recreated2");
+
+        assert grid(0).cache(CACHE_NAME_LOC).get(KEY_VAL).equals(KEY_VAL + "recreated0");
+        assert grid(1).cache(CACHE_NAME_LOC).get(KEY_VAL).equals(KEY_VAL + "recreated1");
+        assert grid(2).cache(CACHE_NAME_LOC).get(KEY_VAL).equals(KEY_VAL + "recreated2");
+    }
+
+    /**
+     * Test Local close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLocalCloseWithTry() throws Exception {
+        String curVal = null;
+
+        for (int i = 0; i < 3; i++) {
+            try (IgniteCache<String, String> cache2 = grid(2).getOrCreateCache(getLocalConfig())) {
+                IgniteCache<String, String> cache0 = grid(0).cache(CACHE_NAME_LOC);
+                IgniteCache<String, String> cache1 = grid(1).cache(CACHE_NAME_LOC);
+
+                assert cache0.get(KEY_VAL) == null;
+                assert cache1.get(KEY_VAL) == null;
+                assert cache2.get(KEY_VAL) == null;
+
+                curVal = KEY_VAL + curVal;
+
+                cache0.put(KEY_VAL, curVal + 1);
+                cache1.put(KEY_VAL, curVal + 2);
+                cache2.put(KEY_VAL, curVal + 3);
+
+                assert cache0.get(KEY_VAL).equals(curVal + 1);
+                assert cache1.get(KEY_VAL).equals(curVal + 2);
+                assert cache2.get(KEY_VAL).equals(curVal + 3);
+            }
+        }
+    }
+
+    /**
+     * Tests concurrent close.
+     *
+     * @throws Exception If failed.
+     */
+    public void testConcurrentCloseSetWithTry() throws Exception {
+        final AtomicInteger a1 = new AtomicInteger();
+        final AtomicInteger a2 = new AtomicInteger();
+        final AtomicInteger a3 = new AtomicInteger();
+        final AtomicInteger a4 = new AtomicInteger();
+
+        Thread t1 = new Thread(new Runnable() {
+            @Override public void run() {
+                Thread.currentThread().setName("test-thread-1");
+
+                closeWithTry(a1, 0);
+            }
+        });
+        Thread t2 = new Thread(new Runnable() {
+            @Override public void run() {
+                Thread.currentThread().setName("test-thread-2");
+
+                closeWithTry(a2, 0);
+            }
+        });
+        Thread t3 = new Thread(new Runnable() {
+            @Override public void run() {
+                Thread.currentThread().setName("test-thread-3");
+
+                closeWithTry(a3, 2);
+            }
+        });
+        Thread t4 = new Thread(new Runnable() {
+            @Override public void run() {
+                Thread.currentThread().setName("test-thread-4");
+
+                closeWithTry(a4, 2);
+            }
+        });
+
+        IgniteCache<Object, Object> cache = grid(0).getOrCreateCache(getDhtConfig());
+
+        cache.close();
+
+        t1.start();
+        t2.start();
+        t3.start();
+        t4.start();
+
+        try {
+            U.sleep(1000);
+        }
+        finally {
+            stop = true;
+        }
+
+        t1.join();
+        t2.join();
+        t3.join();
+        t4.join();
+
+        assert a1.get() > 1;
+        assert a2.get() > 1;
+        assert a3.get() > 1;
+        assert a4.get() > 1;
+
+        checkUsageFails(cache);
+    }
+
+    /**
+     * @param a AtomicInteger.
+     * @param node Node.
+     */
+    public void closeWithTry(AtomicInteger a, int node) {
+        while (!stop) {
+            try (IgniteCache<String, String> cache = grid(node).getOrCreateCache(getDhtConfig())) {
+                a.incrementAndGet();
+
+                assert cache.get(KEY_VAL) == null || cache.get(KEY_VAL).equals(KEY_VAL);
+
+                cache.put(KEY_VAL, KEY_VAL);
+
+                assert cache.get(KEY_VAL).equals(KEY_VAL);
+            }
+        }
+    }
+
+    /**
+     * Tests start -> destroy -> start -> close using CacheManager.
+     */
+    public void testTckStyleCreateDestroyClose() {
+        CacheManager mgr = Caching.getCachingProvider().getCacheManager();
+
+        String cacheName = "cache";
+
+        mgr.createCache(cacheName, new MutableConfiguration<Integer, String>().setTypes(Integer.class, String.class));
+
+        mgr.destroyCache(cacheName);
+
+        Cache<Integer, String> cache = mgr.createCache(cacheName,
+            new MutableConfiguration<Integer, String>().setTypes(Integer.class, String.class));
+
+        cache.close();
+
+        cache.close();
+
+        try {
+            cache.get(1);
+
+            fail();
+        }
+        catch (IllegalStateException e) {
+            // No-op;
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @throws Exception If failed.
+     */
+    private void checkDestroyed(IgniteCache<Object, Object> cache) throws Exception {
+        checkUsageFails(cache);
+
+        awaitPartitionMapExchange();
+
+        String cacheName = cache.getName();
+
+        for (int i = 0; i < 3; i++)
+            assertNull("Unexpected cache for node: " + i, grid(i).cache(cacheName));
+    }
+
+    /**
+     * @param cache Cache.
+     * @throws Exception If failed.
+     */
+    private void checkUsageFails(IgniteCache<Object, Object> cache) throws Exception {
+        try {
+            cache.get(0);
+
+            fail();
+        }
+        catch (IllegalStateException e) {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java
index 82667d9..2d52933 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java
@@ -163,7 +163,7 @@ public abstract class CacheStoreUsageMultinodeDynamicStartAbstractTest extends C
             cache = srv.cache(null);
 
             if (cache != null)
-                cache.close();
+                cache.destroy();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridProjectionForCachesOnDaemonNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridProjectionForCachesOnDaemonNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridProjectionForCachesOnDaemonNodeSelfTest.java
index b9acd99..e640f82 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridProjectionForCachesOnDaemonNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridProjectionForCachesOnDaemonNodeSelfTest.java
@@ -88,7 +88,7 @@ public class GridProjectionForCachesOnDaemonNodeSelfTest extends GridCommonAbstr
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        ignite.cache(null).close();
+        ignite.cache(null).destroy();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index cd19703..d1f8016 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -175,7 +175,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
         GridTestUtils.runMultiThreaded(new Callable<Object>() {
             @Override public Object call() throws Exception {
-                futs.add(kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME));
+                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME));
 
                 return null;
             }
@@ -237,7 +237,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             @Override public Object call() throws Exception {
                 IgniteEx kernal = grid(ThreadLocalRandom.current().nextInt(nodeCount()));
 
-                futs.add(kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME));
+                futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME));
 
                 return null;
             }
@@ -300,7 +300,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         for (int g = 0; g < nodeCount(); g++)
             caches[g] = grid(g).cache(DYNAMIC_CACHE_NAME);
 
-        kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME).get();
+        kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME).get();
 
         for (int g = 0; g < nodeCount(); g++) {
             final IgniteKernal kernal0 = (IgniteKernal) grid(g);
@@ -353,7 +353,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             }
 
             // Undeploy cache.
-            kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME).get();
+            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME).get();
 
             startGrid(nodeCount() + 1);
 
@@ -430,7 +430,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                     }, IllegalArgumentException.class, null);
             }
 
-            kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME).get();
+            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME).get();
 
             stopGrid(nodeCount() + 1);
             stopGrid(nodeCount());
@@ -483,7 +483,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
             GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    IgniteKernal ignite = (IgniteKernal) grid(nodeCount());
+                    IgniteKernal ignite = (IgniteKernal)grid(nodeCount());
 
                     return ignite.getCache(DYNAMIC_CACHE_NAME);
                 }
@@ -497,7 +497,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount() + 1; g++)
                 assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1"));
 
-            kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME).get();
+            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME).get();
         }
         finally {
             stopGrid(nodeCount());
@@ -539,7 +539,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount() + 1; g++)
                 assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1"));
 
-            kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME).get();
+            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME).get();
         }
         finally {
             stopGrid(nodeCount());
@@ -585,7 +585,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount() + 1; g++)
                 assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1"));
 
-            kernal.context().cache().dynamicStopCache(DYNAMIC_CACHE_NAME).get();
+            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME).get();
         }
         finally {
             stopGrid(nodeCount());
@@ -638,10 +638,15 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             ignite(i).events().localListen(lsnrs[i], EventType.EVTS_CACHE_LIFECYCLE);
         }
 
-        try (IgniteCache<Object, Object> ignored = ignite(0).createCache(cfg)) {
+        IgniteCache<Object, Object> cache = ignite(0).createCache(cfg);
+
+        try {
             for (CountDownLatch start : starts)
                 start.await();
         }
+        finally {
+            cache.destroy();
+        }
 
         for (CountDownLatch stop : stops)
             stop.await();
@@ -665,28 +670,29 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             ccfg.setCacheMode(CacheMode.PARTITIONED);
             ccfg.setNodeFilter(NODE_FILTER);
 
-            try (IgniteCache cache = ig.createCache(ccfg, new NearCacheConfiguration())) {
-                assertNotNull(cache);
+            IgniteCache cache = ig.createCache(ccfg, new NearCacheConfiguration());
+            assertNotNull(cache);
 
-                GridCacheAdapter<Object, Object> cacheAdapter =
-                    ((IgniteKernal)ig).internalCache(DYNAMIC_CACHE_NAME);
+            GridCacheAdapter<Object, Object> cacheAdapter =
+                ((IgniteKernal)ig).internalCache(DYNAMIC_CACHE_NAME);
 
-                assertNotNull(cacheAdapter);
-                assertFalse(cacheAdapter.context().affinityNode());
-                assertTrue(cacheAdapter.context().isNear());
+            assertNotNull(cacheAdapter);
+            assertFalse(cacheAdapter.context().affinityNode());
+            assertTrue(cacheAdapter.context().isNear());
 
-                try {
-                    IgniteEx grid = startGrid(nodeCount() + 1);
+            try {
+                IgniteEx grid = startGrid(nodeCount() + 1);
 
-                    // Check that new node sees near node.
-                    GridDiscoveryManager disco = grid.context().discovery();
+                // Check that new node sees near node.
+                GridDiscoveryManager disco = grid.context().discovery();
 
-                    assertTrue(disco.cacheNearNode(disco.node(ig.cluster().localNode().id()),
-                        DYNAMIC_CACHE_NAME));
-                }
-                finally {
-                    stopGrid(nodeCount() + 1);
-                }
+                assertTrue(disco.cacheNearNode(disco.node(ig.cluster().localNode().id()),
+                    DYNAMIC_CACHE_NAME));
+            }
+            finally {
+                cache.destroy();
+
+                stopGrid(nodeCount() + 1);
             }
         }
         finally {
@@ -955,14 +961,14 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
         cfg.setNodeFilter(F.not(NODE_FILTER));
 
-        try (IgniteCache<Object, Object> ignored = ignite(0).createCache(cfg)) {
+        IgniteCache<Object, Object> cache = ignite(0).createCache(cfg);
 
-            final CountDownLatch[] latches = new CountDownLatch[nodeCount()];
+        final CountDownLatch[] latches = new CountDownLatch[nodeCount()];
 
-            IgnitePredicate[] lsnrs = new IgnitePredicate[nodeCount()];
+        IgnitePredicate[] lsnrs = new IgnitePredicate[nodeCount()];
 
-            for (int i = 0; i < nodeCount(); i++) {
-                final int idx = i;
+        for (int i = 0; i < nodeCount(); i++) {
+            final int idx = i;
 
                 latches[i] = new CountDownLatch(1);
                 lsnrs[i] = new IgnitePredicate<CacheEvent>() {
@@ -971,29 +977,30 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                             case EventType.EVT_CACHE_NODES_LEFT:
                                 latches[idx].countDown();
 
-                                break;
+                            break;
 
-                            default:
-                                assert false;
-                        }
+                        default:
+                            assert false;
+                    }
 
-                        assertEquals(DYNAMIC_CACHE_NAME, e.cacheName());
+                    assertEquals(DYNAMIC_CACHE_NAME, e.cacheName());
 
-                        return true;
-                    }
-                };
+                    return true;
+                }
+            };
 
-                ignite(i).events().localListen(lsnrs[i], EventType.EVTS_CACHE_LIFECYCLE);
-            }
+            ignite(i).events().localListen(lsnrs[i], EventType.EVTS_CACHE_LIFECYCLE);
+        }
 
-            stopGrid(nodeCount());
+        stopGrid(nodeCount());
 
-            for (CountDownLatch latch : latches)
-                latch.await();
+        for (CountDownLatch latch : latches)
+            latch.await();
 
-            for (int i = 0; i < nodeCount(); i++)
-                ignite(i).events().stopLocalListen(lsnrs[i]);
-        }
+        for (int i = 0; i < nodeCount(); i++)
+            ignite(i).events().stopLocalListen(lsnrs[i]);
+
+        cache.destroy();
     }
 
     /**
@@ -1007,7 +1014,9 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         try {
             CacheConfiguration cfg = new CacheConfiguration(DYNAMIC_CACHE_NAME);
 
-            try (IgniteCache cache = ignite(0).createCache(cfg)) {
+            IgniteCache cache = ignite(0).createCache(cfg);
+
+            try {
                 for (int i = 0; i < 100; i++) {
                     assertFalse(ignite(0).affinity(DYNAMIC_CACHE_NAME).mapKeyToPrimaryAndBackups(i)
                         .contains(dNode.cluster().localNode()));
@@ -1015,6 +1024,9 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                     cache.put(i, i);
                 }
             }
+            finally {
+                cache.destroy();
+            }
         }
         finally {
             stopGrid(nodeCount());
@@ -1027,23 +1039,25 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAwaitPartitionMapExchange() throws Exception {
-        try (IgniteCache ignored = grid(0).getOrCreateCache(new CacheConfiguration(DYNAMIC_CACHE_NAME))) {
-            awaitPartitionMapExchange();
+        IgniteCache cache = grid(0).getOrCreateCache(new CacheConfiguration(DYNAMIC_CACHE_NAME));
 
-            startGrid(nodeCount());
+        awaitPartitionMapExchange();
 
-            awaitPartitionMapExchange();
+        startGrid(nodeCount());
 
-            startGrid(nodeCount() + 1);
+        awaitPartitionMapExchange();
 
-            awaitPartitionMapExchange();
+        startGrid(nodeCount() + 1);
 
-            stopGrid(nodeCount() + 1);
+        awaitPartitionMapExchange();
 
-            awaitPartitionMapExchange();
+        stopGrid(nodeCount() + 1);
 
-            stopGrid(nodeCount());
-        }
+        awaitPartitionMapExchange();
+
+        stopGrid(nodeCount());
+
+        cache.destroy();
     }
 
     /**
@@ -1084,9 +1098,11 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 if (iter % 10 == 0)
                     log.info("Cache start/stop iteration: " + iter);
 
-                try (IgniteCache<Object, Object> cache = ignite1.getOrCreateCache("cache-" + iter)) {
-                    assertNotNull(cache);
-                }
+                IgniteCache<Object, Object> cache = ignite1.getOrCreateCache("cache-" + iter);
+
+                assertNotNull(cache);
+
+                cache.destroy();
 
                 iter++;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
index d60a0c3..5a51a1b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -498,7 +498,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
         spi1.reset();
         spi2.reset();
 
-        assertNull(((IgniteKernal)ignite2).context().cache().context().cache().internalCache("cache1"));
+        assertNull(((IgniteKernal)ignite2).context().cache().context().cache().internalCache(CACHE_NAME1));
 
         if (nearCache)
             ignite2.getOrCreateNearCache(CACHE_NAME1, new NearCacheConfiguration<>());
@@ -507,7 +507,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
         waitForTopologyUpdate(3, new AffinityTopologyVersion(3, 1));
 
-        GridCacheAdapter cache = ((IgniteKernal)ignite2).context().cache().context().cache().internalCache("cache1");
+        GridCacheAdapter cache = ((IgniteKernal)ignite2).context().cache().context().cache().internalCache(CACHE_NAME1);
 
         assertNotNull(cache);
         assertEquals(nearCache, cache.context().isNear());
@@ -533,6 +533,29 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
         spi1.reset();
         spi2.reset();
 
+        AffinityTopologyVersion topVer;
+
+        if (!srvNode) {
+            log.info("Close client cache: " + CACHE_NAME1);
+
+            ignite2.cache(CACHE_NAME1).close();
+
+            assertNull(((IgniteKernal)ignite2).context().cache().context().cache().internalCache(CACHE_NAME1));
+
+            waitForTopologyUpdate(3, new AffinityTopologyVersion(3, 2));
+
+            assertEquals(0, spi0.partitionsSingleMessages());
+            assertEquals(0, spi0.partitionsFullMessages());
+            assertEquals(0, spi1.partitionsSingleMessages());
+            assertEquals(0, spi1.partitionsFullMessages());
+            assertEquals(0, spi2.partitionsSingleMessages());
+            assertEquals(0, spi2.partitionsFullMessages());
+
+            topVer = new AffinityTopologyVersion(3, 3);
+        }
+        else
+            topVer = new AffinityTopologyVersion(3, 2);
+
         final String CACHE_NAME2 = "cache2";
 
         ccfg = new CacheConfiguration();
@@ -541,7 +564,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
         ignite2.createCache(ccfg);
 
-        waitForTopologyUpdate(3, new AffinityTopologyVersion(3, 2));
+        waitForTopologyUpdate(3, topVer);
 
         assertEquals(0, spi0.partitionsSingleMessages());
         assertEquals(2, spi0.partitionsFullMessages());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java
index 3d44600..f4b0d2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java
@@ -96,7 +96,7 @@ public class CacheLocalOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         if (cache != null)
-            cache.close();
+            cache.destroy();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
index 12b6458..470ac79 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
@@ -76,15 +76,17 @@ public class DataStreamerMultinodeCreateCacheTest extends GridCommonAbstractTest
                 while (System.currentTimeMillis() < stopTime) {
                     String cacheName = "cache-" + threadIdx + "-" + (iter % 10);
 
-                    try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName)) {
-                        try (IgniteDataStreamer<Object, Object> stmr = ignite.dataStreamer(cacheName)) {
-                            ((DataStreamerImpl<Object, Object>)stmr).maxRemapCount(0);
+                    IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName);
 
-                            for (int i = 0; i < 1000; i++)
-                                stmr.addData(i, i);
-                        }
+                    try (IgniteDataStreamer<Object, Object> stmr = ignite.dataStreamer(cacheName)) {
+                        ((DataStreamerImpl<Object, Object>)stmr).maxRemapCount(0);
+
+                        for (int i = 0; i < 1000; i++)
+                            stmr.addData(i, i);
                     }
 
+                    cache.destroy();
+
                     iter++;
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 12d2b05..bde3a72 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -131,6 +131,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheRemoveAllSelfTest.class);
 
+        suite.addTestSuite(CacheStopAndDestroySelfTest.class);
+
         suite.addTestSuite(CacheOffheapMapEntrySelfTest.class);
 
         suite.addTestSuite(CacheJdbcStoreSessionListenerSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3fba883/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java
index eea3a9b..701668b0 100644
--- a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java
@@ -109,9 +109,9 @@ public class CacheConfigurationP2PTestClient {
             if (cnt != 600)
                 throw new Exception("Unexpected query result: " + cnt);
 
-            cache1.close();
+            cache1.destroy();
 
-            cache2.close();
+            cache2.destroy();
         }
     }
 }