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 2016/12/02 09:25:14 UTC

[08/19] ignite git commit: ignite-4088 Added methods to create/destroy multiple caches. This closes #1174.

ignite-4088 Added methods to create/destroy multiple caches. This closes #1174.

(cherry picked from commit f445e7b)


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

Branch: refs/heads/master
Commit: 56998e704e9a67760c70481c10c56e72c0a866bb
Parents: 9c6824b
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Fri Oct 28 16:27:34 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 24 18:00:03 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  77 ++++-
 .../apache/ignite/internal/IgniteKernal.java    |  81 ++++++
 .../processors/cache/GridCacheProcessor.java    | 291 ++++++++++++++-----
 .../cache/IgniteDynamicCacheStartSelfTest.java  | 217 ++++++++++++--
 .../processors/igfs/IgfsIgniteMock.java         |  19 ++
 .../ignite/testframework/junits/IgniteMock.java |  15 +
 .../junits/multijvm/IgniteProcessProxy.java     |  15 +
 .../org/apache/ignite/IgniteSpringBean.java     |  21 ++
 8 files changed, 616 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index bd21468..0de08d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -20,6 +20,7 @@ package org.apache.ignite;
 import java.util.Collection;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
+import javax.cache.CacheException;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterGroup;
@@ -220,8 +221,24 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheCfg Cache configuration to use.
      * @return Instance of started cache.
+     * @throws CacheException If a cache with the same name already exists or other error occurs.
      */
-    public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg);
+    public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg) throws CacheException;
+
+    /**
+     * Dynamically starts new caches with the given cache configurations.
+     * <p>
+     * If local node is an affinity node, this method will return the instance of started caches.
+     * Otherwise, it will create a client caches on local node.
+     * <p>
+     * If for one of configurations a cache with the same name already exists in the grid, an exception will be thrown regardless
+     * whether the given configuration matches the configuration of the existing cache or not.
+     *
+     * @param cacheCfgs Collection of cache configuration to use.
+     * @return Collection of instances of started caches.
+     * @throws CacheException If one of created caches exists or other error occurs.
+     */
+    public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) throws CacheException;
 
     /**
      * Dynamically starts new cache using template configuration.
@@ -233,8 +250,9 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheName Cache name.
      * @return Instance of started cache.
+     * @throws CacheException If a cache with the same name already exists or other error occurs.
      */
-    public <K, V> IgniteCache<K, V> createCache(String cacheName);
+    public <K, V> IgniteCache<K, V> createCache(String cacheName) throws CacheException;
 
     /**
      * Gets existing cache with the given name or creates new one with the given configuration.
@@ -245,23 +263,39 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheCfg Cache configuration to use.
      * @return Existing or newly created cache.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg);
+    public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) throws CacheException;
 
     /**
      * Gets existing cache with the given name or creates new one using template configuration.
      *
      * @param cacheName Cache name.
      * @return Existing or newly created cache.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName);
+    public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) throws CacheException;
+
+    /**
+     * Gets existing caches with the given name or created one with the given configuration.
+     * <p>
+     * If a cache with the same name already exist, this method will not check that the given
+     * configuration matches the configuration of existing cache and will return an instance
+     * of the existing cache.
+     *
+     * @param cacheCfgs Collection of cache configuration to use.
+     * @return Collection of existing or newly created caches.
+     * @throws CacheException If error occurs.
+     */
+    public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) throws CacheException;
 
     /**
      * Adds cache configuration template.
      *
      * @param cacheCfg Cache configuration template.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg);
+    public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) throws CacheException;
 
     /**
      * Dynamically starts new cache with the given cache configuration.
@@ -275,10 +309,11 @@ public interface Ignite extends AutoCloseable {
      * @param cacheCfg Cache configuration to use.
      * @param nearCfg Near cache configuration to use on local node in case it is not an
      *      affinity node.
+     * @throws CacheException If a cache with the same name already exists or other error occurs.
      * @return Instance of started cache.
      */
     public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg,
-        NearCacheConfiguration<K, V> nearCfg);
+        NearCacheConfiguration<K, V> nearCfg) throws CacheException;
 
     /**
      * Gets existing cache with the given cache configuration or creates one if it does not exist.
@@ -293,9 +328,10 @@ public interface Ignite extends AutoCloseable {
      * @param cacheCfg Cache configuration.
      * @param nearCfg Near cache configuration for client.
      * @return {@code IgniteCache} instance.
+     * @throws CacheException If error occurs.
      */
     public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg,
-        NearCacheConfiguration<K, V> nearCfg);
+        NearCacheConfiguration<K, V> nearCfg) throws CacheException;
 
     /**
      * Starts a near cache on local node if cache was previously started with one of the
@@ -305,8 +341,10 @@ public interface Ignite extends AutoCloseable {
      * @param cacheName Cache name.
      * @param nearCfg Near cache configuration.
      * @return Cache instance.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg);
+    public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg)
+        throws CacheException;
 
     /**
      * Gets existing near cache with the given name or creates a new one.
@@ -314,15 +352,26 @@ public interface Ignite extends AutoCloseable {
      * @param cacheName Cache name.
      * @param nearCfg Near configuration.
      * @return {@code IgniteCache} instance.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg);
+    public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg)
+        throws CacheException;
 
     /**
      * Stops dynamically started cache.
      *
      * @param cacheName Cache name to stop.
+     * @throws CacheException If error occurs.
+     */
+    public void destroyCache(String cacheName) throws CacheException;
+
+    /**
+     * Stops dynamically started caches.
+     *
+     * @param cacheNames Collection of cache names to stop.
+     * @throws CacheException If error occurs.
      */
-    public void destroyCache(String cacheName);
+    public void destroyCaches(Collection<String> cacheNames) throws CacheException;
 
     /**
      * Gets an instance of {@link IgniteCache} API. {@code IgniteCache} is a fully-compatible
@@ -330,8 +379,9 @@ public interface Ignite extends AutoCloseable {
      *
      * @param name Cache name.
      * @return Instance of the cache for the specified name.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> cache(@Nullable String name);
+    public <K, V> IgniteCache<K, V> cache(@Nullable String name) throws CacheException;
 
     /**
      * Gets the collection of names of currently available caches.
@@ -357,8 +407,9 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheName Cache name ({@code null} for default cache).
      * @return Data streamer.
+     * @throws IllegalStateException If node is stopping.
      */
-    public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName);
+    public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName) throws IllegalStateException;
 
     /**
      * Gets an instance of IGFS (Ignite In-Memory File System). If one is not
@@ -372,7 +423,7 @@ public interface Ignite extends AutoCloseable {
      * @return IGFS instance.
      * @throws IllegalArgumentException If IGFS with such name is not configured.
      */
-    public IgniteFileSystem fileSystem(String name);
+    public IgniteFileSystem fileSystem(String name) throws IllegalArgumentException;
 
     /**
      * Gets all instances of IGFS (Ignite In-Memory File System).

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index ef9c651..c536506 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2541,6 +2541,33 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        A.notNull(cacheCfgs, "cacheCfgs");
+
+        guard();
+
+        try {
+            ctx.cache().dynamicStartCaches(cacheCfgs,
+                true,
+                true).get();
+
+            List<IgniteCache> createdCaches = new ArrayList<>(cacheCfgs.size());
+
+            for (CacheConfiguration cacheCfg : cacheCfgs)
+                createdCaches.add(ctx.cache().publicJCache(cacheCfg.getName()));
+
+            return createdCaches;
+        }
+        catch (IgniteCheckedException e) {
+            throw CU.convertToCacheException(e);
+        }
+        finally {
+            unguard();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         guard();
@@ -2585,6 +2612,32 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        A.notNull(cacheCfgs, "cacheCfgs");
+
+        guard();
+
+        try {
+            ctx.cache().dynamicStartCaches(cacheCfgs,
+                false,
+                true).get();
+
+            List<IgniteCache> createdCaches = new ArrayList<>(cacheCfgs.size());
+
+            for (CacheConfiguration cacheCfg : cacheCfgs)
+                createdCaches.add(ctx.cache().publicJCache(cacheCfg.getName()));
+
+            return createdCaches;
+        }
+        catch (IgniteCheckedException e) {
+            throw CU.convertToCacheException(e);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(
         CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg
@@ -2745,6 +2798,18 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        IgniteInternalFuture stopFut = destroyCachesAsync(cacheNames, true);
+
+        try {
+            stopFut.get();
+        }
+        catch (IgniteCheckedException e) {
+            throw CU.convertToCacheException(e);
+        }
+    }
+
     /**
      * @param cacheName Cache name.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
@@ -2761,6 +2826,22 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * @param cacheNames Collection of cache names.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Ignite future.
+     */
+    public IgniteInternalFuture<?> destroyCachesAsync(Collection<String> cacheNames, boolean checkThreadTx) {
+        guard();
+
+        try {
+            return ctx.cache().dynamicDestroyCaches(cacheNames, checkThreadTx);
+        }
+        finally {
+            unguard();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
         guard();

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 5e777fd..0e0d769 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -2284,99 +2284,92 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (checkThreadTx)
             checkEmptyTransactions();
 
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
-
-        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
-
-        req.failIfExists(failIfExists);
-
-        if (ccfg != null) {
-            try {
-                cloneCheckSerializable(ccfg);
-            }
-            catch (IgniteCheckedException e) {
-                return new GridFinishedFuture<>(e);
-            }
-
-            if (desc != null) {
-                if (failIfExists) {
-                    return new GridFinishedFuture<>(new CacheExistsException("Failed to start cache " +
-                        "(a cache with the same name is already started): " + cacheName));
-                }
-                else {
-                    CacheConfiguration descCfg = desc.cacheConfiguration();
-
-                    // Check if we were asked to start a near cache.
-                    if (nearCfg != null) {
-                        if (CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter())) {
-                            // If we are on a data node and near cache was enabled, return success, else - fail.
-                            if (descCfg.getNearConfiguration() != null)
-                                return new GridFinishedFuture<>();
-                            else
-                                return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start near " +
-                                    "cache (local node is an affinity node for cache): " + cacheName));
-                        }
-                        else
-                            // If local node has near cache, return success.
-                            req.clientStartOnly(true);
-                    }
-                    else
-                        req.clientStartOnly(true);
+        try {
+            DynamicCacheChangeRequest req = prepareCacheChangeRequest(
+                ccfg,
+                cacheName,
+                nearCfg,
+                cacheType,
+                failIfExists,
+                failIfNotStarted);
 
-                    req.deploymentId(desc.deploymentId());
+            if (req != null)
+                return F.first(initiateCacheChanges(F.asList(req), failIfExists));
+            else
+                return new GridFinishedFuture<>();
+        }
+        catch (Exception e) {
+            return new GridFinishedFuture<>(e);
+        }
+    }
 
-                    req.startCacheConfiguration(descCfg);
-                }
-            }
-            else {
-                req.deploymentId(IgniteUuid.randomUuid());
+    /**
+     * Dynamically starts multiple caches.
+     *
+     * @param ccfgList Collection of cache configuration.
+     * @param failIfExists Fail if exists flag.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Future that will be completed when all caches are deployed.
+     */
+    public IgniteInternalFuture<?> dynamicStartCaches(
+        Collection<CacheConfiguration> ccfgList,
+        boolean failIfExists,
+        boolean checkThreadTx
+    ) {
+        return dynamicStartCaches(ccfgList, CacheType.USER, failIfExists, checkThreadTx);
+    }
 
-                try {
-                    CacheConfiguration cfg = new CacheConfiguration(ccfg);
+    /**
+     * Dynamically starts multiple caches.
+     *
+     * @param ccfgList Collection of cache configuration.
+     * @param cacheType Cache type.
+     * @param failIfExists Fail if exists flag.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Future that will be completed when all caches are deployed.
+     */
+    private IgniteInternalFuture<?> dynamicStartCaches(
+        Collection<CacheConfiguration> ccfgList,
+        CacheType cacheType,
+        boolean failIfExists,
+        boolean checkThreadTx
+    ) {
+        if (checkThreadTx)
+            checkEmptyTransactions();
 
-                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
+        List<DynamicCacheChangeRequest> reqList = new ArrayList<>(ccfgList.size());
 
-                    initialize(false, cfg, cacheObjCtx);
+        try {
+            for (CacheConfiguration ccfg : ccfgList) {
+                DynamicCacheChangeRequest req = prepareCacheChangeRequest(
+                    ccfg,
+                    ccfg.getName(),
+                    null,
+                    cacheType,
+                    failIfExists,
+                    true
+                );
 
-                    req.startCacheConfiguration(cfg);
-                }
-                catch (IgniteCheckedException e) {
-                    return new GridFinishedFuture(e);
-                }
+                if (req != null)
+                    reqList.add(req);
             }
         }
-        else {
-            req.clientStartOnly(true);
-
-            if (desc != null)
-                ccfg = desc.cacheConfiguration();
-
-            if (ccfg == null) {
-                if (failIfNotStarted)
-                    return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " +
-                        "(a cache with the given name is not started): " + cacheName));
-                else
-                    return new GridFinishedFuture<>();
-            }
-
-            req.deploymentId(desc.deploymentId());
-            req.startCacheConfiguration(ccfg);
+        catch (Exception e) {
+            return new GridFinishedFuture<>(e);
         }
 
-        // Fail cache with swap enabled creation on grid without swap space SPI.
-        if (ccfg.isSwapEnabled())
-            for (ClusterNode n : ctx.discovery().allNodes())
-                if (!GridCacheUtils.clientNode(n) && !GridCacheUtils.isSwapEnabled(n))
-                    return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start cache " +
-                        cacheName + " with swap enabled: Remote Node with ID " + n.id().toString().toUpperCase() +
-                        " has not swap SPI configured"));
+        if (!reqList.isEmpty()) {
+            GridCompoundFuture<?, ?> compoundFut = new GridCompoundFuture<>();
 
-        if (nearCfg != null)
-            req.nearCacheConfiguration(nearCfg);
+            for (DynamicCacheStartFuture fut : initiateCacheChanges(reqList, failIfExists))
+                compoundFut.add((IgniteInternalFuture)fut);
 
-        req.cacheType(cacheType);
+            compoundFut.markInitialized();
 
-        return F.first(initiateCacheChanges(F.asList(req), failIfExists));
+            return compoundFut;
+        }
+        else
+            return new GridFinishedFuture<>();
     }
 
     /**
@@ -2396,6 +2389,35 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param cacheNames Collection of cache names to destroy.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Future that will be completed when cache is destroyed.
+     */
+    public IgniteInternalFuture<?> dynamicDestroyCaches(Collection<String> cacheNames, boolean checkThreadTx) {
+        if (checkThreadTx)
+            checkEmptyTransactions();
+
+        List<DynamicCacheChangeRequest> reqs = new ArrayList<>(cacheNames.size());
+
+        for (String cacheName : cacheNames) {
+            DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
+
+            t.stop(true);
+
+            reqs.add(t);
+        }
+
+        GridCompoundFuture<?, ?> compoundFut = new GridCompoundFuture<>();
+
+        for (DynamicCacheStartFuture fut : initiateCacheChanges(reqs, false))
+            compoundFut.add((IgniteInternalFuture)fut);
+
+        compoundFut.markInitialized();
+
+        return compoundFut;
+    }
+
+    /**
      * @param cacheName Cache name to close.
      * @return Future that will be completed when cache is closed.
      */
@@ -2416,6 +2438,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param reqs Requests.
+     * @param failIfExists Fail if exists flag.
      * @return Collection of futures.
      */
     @SuppressWarnings("TypeMayBeWeakened")
@@ -3608,6 +3631,114 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Prepares DynamicCacheChangeRequest for cache creation.
+     *
+     * @param ccfg Cache configuration
+     * @param cacheName Cache name
+     * @param nearCfg Near cache configuration
+     * @param cacheType Cache type
+     * @param failIfExists Fail if exists flag.
+     * @param failIfNotStarted If {@code true} fails if cache is not started.
+     * @return Request or {@code null} if cache already exists.
+     * @throws IgniteCheckedException if some of pre-checks failed
+     * @throws CacheExistsException if cache exists and failIfExists flag is {@code true}
+     */
+    private DynamicCacheChangeRequest prepareCacheChangeRequest(
+        @Nullable CacheConfiguration ccfg,
+        String cacheName,
+        @Nullable NearCacheConfiguration nearCfg,
+        CacheType cacheType,
+        boolean failIfExists,
+        boolean failIfNotStarted
+    ) throws IgniteCheckedException {
+        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
+
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
+
+        req.failIfExists(failIfExists);
+
+        if (ccfg != null) {
+            cloneCheckSerializable(ccfg);
+
+            if (desc != null) {
+                if (failIfExists) {
+                    throw new CacheExistsException("Failed to start cache " +
+                        "(a cache with the same name is already started): " + cacheName);
+                }
+                else {
+                    CacheConfiguration descCfg = desc.cacheConfiguration();
+
+                    // Check if we were asked to start a near cache.
+                    if (nearCfg != null) {
+                        if (CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter())) {
+                            // If we are on a data node and near cache was enabled, return success, else - fail.
+                            if (descCfg.getNearConfiguration() != null)
+                                return null;
+                            else
+                                throw new IgniteCheckedException("Failed to start near " +
+                                    "cache (local node is an affinity node for cache): " + cacheName);
+                        }
+                        else
+                            // If local node has near cache, return success.
+                            req.clientStartOnly(true);
+                    }
+                    else
+                        req.clientStartOnly(true);
+
+                    req.deploymentId(desc.deploymentId());
+
+                    req.startCacheConfiguration(descCfg);
+                }
+            }
+            else {
+                req.deploymentId(IgniteUuid.randomUuid());
+
+                CacheConfiguration cfg = new CacheConfiguration(ccfg);
+
+                CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
+
+                initialize(false, cfg, cacheObjCtx);
+
+                req.startCacheConfiguration(cfg);
+            }
+        }
+        else {
+            req.clientStartOnly(true);
+
+            if (desc != null)
+                ccfg = desc.cacheConfiguration();
+
+            if (ccfg == null) {
+                if (failIfNotStarted) {
+                    throw new CacheExistsException("Failed to start client cache " +
+                        "(a cache with the given name is not started): " + cacheName);
+                }
+                else
+                    return null;
+            }
+
+            req.deploymentId(desc.deploymentId());
+            req.startCacheConfiguration(ccfg);
+        }
+
+        // Fail cache with swap enabled creation on grid without swap space SPI.
+        if (ccfg.isSwapEnabled())
+            for (ClusterNode n : ctx.discovery().allNodes())
+                if (!GridCacheUtils.clientNode(n) && !GridCacheUtils.isSwapEnabled(n)) {
+                    throw new IgniteCheckedException("Failed to start cache " +
+                        cacheName + " with swap enabled: Remote Node with ID " + n.id().toString().toUpperCase() +
+                        " has not swap SPI configured");
+                }
+
+        if (nearCfg != null)
+            req.nearCacheConfiguration(nearCfg);
+
+        req.cacheType(cacheType);
+
+        return req;
+    }
+
+    /**
      * @param obj Object to clone.
      * @return Object copy.
      * @throws IgniteCheckedException If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/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 c9cd750..48e06ee 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
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.CountDownLatch;
@@ -181,7 +183,8 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 info("Succeeded: " + System.identityHashCode(fut));
 
                 succeeded++;
-            } catch (IgniteCheckedException e) {
+            }
+            catch (IgniteCheckedException e) {
                 info(e.getMessage());
 
                 failed++;
@@ -246,7 +249,8 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 info("Succeeded: " + System.identityHashCode(fut));
 
                 succeeded++;
-            } catch (IgniteCheckedException e) {
+            }
+            catch (IgniteCheckedException e) {
                 info(e.getMessage());
 
                 failed++;
@@ -289,6 +293,20 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopCachesSimpleTransactional() throws Exception {
+        checkStartStopCachesSimple(CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopCachesSimpleAtomic() throws Exception {
+        checkStartStopCachesSimple(CacheAtomicityMode.ATOMIC);
+    }
+
+    /**
      * @param mode Cache atomicity mode.
      * @throws Exception If failed.
      */
@@ -325,10 +343,10 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         for (int g = 0; g < nodeCount(); g++)
             caches[g] = grid(g).cache(DYNAMIC_CACHE_NAME);
 
-        kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+        kernal.destroyCache(DYNAMIC_CACHE_NAME);
 
         for (int g = 0; g < nodeCount(); g++) {
-            final IgniteKernal kernal0 = (IgniteKernal) grid(g);
+            final IgniteKernal kernal0 = (IgniteKernal)grid(g);
 
             final int idx = g;
 
@@ -346,6 +364,87 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param mode Cache atomicity mode.
+     * @throws Exception If failed.
+     */
+    private void checkStartStopCachesSimple(CacheAtomicityMode mode) throws Exception {
+        final IgniteEx kernal = grid(0);
+        final int cacheCnt = 3;
+
+        List<CacheConfiguration> ccfgList = new ArrayList<>();
+
+        for (int i = 0; i < cacheCnt; i++) {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+            ccfg.setAtomicityMode(mode);
+            ccfg.setName(DYNAMIC_CACHE_NAME + Integer.toString(i));
+
+            ccfgList.add(ccfg);
+        }
+
+        kernal.createCaches(ccfgList);
+
+        for (int g = 0; g < nodeCount(); g++) {
+            IgniteEx kernal0 = grid(g);
+
+            for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
+                f.get();
+
+            info("Getting cache for node: " + g);
+
+            for (int i = 0; i < cacheCnt; i++)
+                assertNotNull(grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)));
+        }
+
+        for (int i = 0; i < cacheCnt; i++)
+            grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)).put(Integer.toString(i), Integer.toString(i));
+
+        for (int g = 0; g < nodeCount(); g++) {
+            for (int i = 0; i < cacheCnt; i++) {
+                assertEquals(
+                    Integer.toString(i),
+                    grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)).get(Integer.toString(i))
+                );
+            }
+        }
+
+        // Grab caches before stop.
+        final IgniteCache[] caches = new IgniteCache[nodeCount() * cacheCnt];
+
+        for (int g = 0; g < nodeCount(); g++) {
+            for (int i = 0; i < cacheCnt; i++)
+                caches[g * nodeCount() + i] = grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i));
+        }
+
+        List<String> namesToDestroy = new ArrayList<>();
+
+        for (int i = 0; i < cacheCnt; i++)
+            namesToDestroy.add(DYNAMIC_CACHE_NAME + Integer.toString(i));
+
+        kernal.destroyCaches(namesToDestroy);
+
+        for (int g = 0; g < nodeCount(); g++) {
+            final IgniteKernal kernal0 = (IgniteKernal)grid(g);
+
+            for (int i = 0; i < cacheCnt; i++) {
+                final int idx = g * nodeCount() + i;
+                final int expVal = i;
+
+                for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
+                    f.get();
+
+                assertNull(kernal0.cache(DYNAMIC_CACHE_NAME));
+
+                GridTestUtils.assertThrows(log, new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        return caches[idx].get(Integer.toString(expVal));
+                    }
+                }, IllegalStateException.class, null);
+            }
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testStartStopCacheAddNode() throws Exception {
@@ -378,13 +477,13 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             }
 
             // Undeploy cache.
-            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
 
             startGrid(nodeCount() + 1);
 
             // Check that cache is not deployed on new node after undeploy.
             for (int g = 0; g < nodeCount() + 2; g++) {
-                final IgniteKernal kernal0 = (IgniteKernal) grid(g);
+                final IgniteKernal kernal0 = (IgniteKernal)grid(g);
 
                 for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                     f.get();
@@ -431,16 +530,16 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount(); g++) {
                 for (int i = 0; i < 100; i++) {
                     assertFalse(grid(g).affinity(DYNAMIC_CACHE_NAME).mapKeyToPrimaryAndBackups(i)
-                            .contains(grid(nodeCount()).cluster().localNode()));
+                        .contains(grid(nodeCount()).cluster().localNode()));
 
                     assertFalse(grid(g).affinity(DYNAMIC_CACHE_NAME).mapKeyToPrimaryAndBackups(i)
-                            .contains(grid(nodeCount() + 1).cluster().localNode()));
+                        .contains(grid(nodeCount() + 1).cluster().localNode()));
                 }
             }
 
             // Check that cache is not deployed on new node after undeploy.
             for (int g = 0; g < nodeCount() + 2; g++) {
-                final IgniteKernal kernal0 = (IgniteKernal) grid(g);
+                final IgniteKernal kernal0 = (IgniteKernal)grid(g);
 
                 for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                     f.get();
@@ -455,7 +554,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                     }, IllegalArgumentException.class, null);
             }
 
-            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
 
             stopGrid(nodeCount() + 1);
             stopGrid(nodeCount());
@@ -489,6 +588,36 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testFailWhenOneOfConfiguredCacheExists() throws Exception {
+        GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                final Ignite kernal = grid(0);
+
+                CacheConfiguration ccfgDynamic = new CacheConfiguration();
+                ccfgDynamic.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+
+                ccfgDynamic.setName(DYNAMIC_CACHE_NAME);
+
+                ccfgDynamic.setNodeFilter(NODE_FILTER);
+
+                CacheConfiguration ccfgStatic = new CacheConfiguration();
+                ccfgStatic.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+
+                // Cache is already configured, should fail.
+                ccfgStatic.setName(STATIC_CACHE_NAME);
+
+                ccfgStatic.setNodeFilter(NODE_FILTER);
+
+                return kernal.createCaches(F.asList(ccfgDynamic, ccfgStatic));
+            }
+        }, CacheExistsException.class, null);
+
+        assertNull(grid(0).cache(DYNAMIC_CACHE_NAME));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testClientCache() throws Exception {
         try {
             testAttribute = false;
@@ -522,7 +651,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().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
         }
         finally {
             stopGrid(nodeCount());
@@ -547,7 +676,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
             ccfg.setNodeFilter(NODE_FILTER);
 
-            final IgniteKernal started = (IgniteKernal) grid(nodeCount());
+            final IgniteKernal started = (IgniteKernal)grid(nodeCount());
 
             started.createCache(ccfg);
 
@@ -564,14 +693,13 @@ 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().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
         }
         finally {
             stopGrid(nodeCount());
         }
     }
 
-
     /**
      * @throws Exception If failed.
      */
@@ -610,7 +738,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().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
         }
         finally {
             stopGrid(nodeCount());
@@ -760,7 +888,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 nearGrid.getOrCreateNearCache(DYNAMIC_CACHE_NAME, new NearCacheConfiguration());
 
                 GridCacheContext<Object, Object> nCtx = ((IgniteKernal)nearGrid)
-                        .internalCache(DYNAMIC_CACHE_NAME).context();
+                    .internalCache(DYNAMIC_CACHE_NAME).context();
 
                 assertTrue(nCtx.isNear());
                 assertFalse(nCtx.affinityNode());
@@ -771,11 +899,12 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 clientGrid.getOrCreateCache(cfg);
 
                 GridCacheContext<Object, Object> cCtx = ((IgniteKernal)clientGrid)
-                        .internalCache(DYNAMIC_CACHE_NAME).context();
+                    .internalCache(DYNAMIC_CACHE_NAME).context();
 
                 assertFalse(cCtx.isNear());
                 assertFalse(cCtx.affinityNode());
-            } finally {
+            }
+            finally {
                 stopGrid(nodeCount() + 1);
                 stopGrid(nodeCount());
             }
@@ -785,6 +914,40 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /** {@inheritDoc} */
+    public void testGetOrCreateCollection() throws Exception {
+        final int cacheCnt = 3;
+
+        try {
+            final Collection<CacheConfiguration> ccfgs = new ArrayList<>();
+
+            for (int i = 0; i < cacheCnt; i++) {
+                final CacheConfiguration cfg = new CacheConfiguration();
+
+                cfg.setName(DYNAMIC_CACHE_NAME + Integer.toString(i));
+                cfg.setNodeFilter(NODE_FILTER);
+
+                ccfgs.add(cfg);
+
+                grid(0).getOrCreateCaches(ccfgs);
+            }
+
+            for (int i = 0; i < cacheCnt; i++) {
+                assertNotNull(grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)));
+
+                IgniteCache<Object, Object> jcache = grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i));
+
+                jcache.put(Integer.toString(i), Integer.toString(i));
+
+                assertEquals(jcache.get(Integer.toString(i)), Integer.toString(i));
+            }
+        }
+        finally {
+            for (int i = 0; i < cacheCnt; i++)
+                grid(0).destroyCache(DYNAMIC_CACHE_NAME + Integer.toString(i));
+        }
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -813,7 +976,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             assertNull(err.get());
 
             for (int i = 0; i < nodeCount(); i++) {
-                GridCacheContext<Object, Object> ctx = ((IgniteKernal) ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
+                GridCacheContext<Object, Object> ctx = ((IgniteKernal)ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
                     .context();
 
                 assertTrue(ctx.affinityNode());
@@ -906,7 +1069,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 assertNull(err.get());
 
                 for (int i = 0; i < nodeCount(); i++) {
-                    GridCacheContext<Object, Object> ctx = ((IgniteKernal) ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
+                    GridCacheContext<Object, Object> ctx = ((IgniteKernal)ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
                         .context();
 
                     assertTrue(ctx.affinityNode());
@@ -914,7 +1077,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 }
 
                 for (int i = 0; i < clientCnt; i++) {
-                    GridCacheContext<Object, Object> ctx = ((IgniteKernal) ignite(nodeCount() + i))
+                    GridCacheContext<Object, Object> ctx = ((IgniteKernal)ignite(nodeCount() + i))
                         .internalCache(DYNAMIC_CACHE_NAME).context();
 
                     assertFalse(ctx.affinityNode());
@@ -995,12 +1158,12 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < nodeCount(); i++) {
             final int idx = i;
 
-                latches[i] = new CountDownLatch(1);
-                lsnrs[i] = new IgnitePredicate<CacheEvent>() {
-                    @Override public boolean apply(CacheEvent e) {
-                        switch (e.type()) {
-                            case EventType.EVT_CACHE_NODES_LEFT:
-                                latches[idx].countDown();
+            latches[i] = new CountDownLatch(1);
+            lsnrs[i] = new IgnitePredicate<CacheEvent>() {
+                @Override public boolean apply(CacheEvent e) {
+                    switch (e.type()) {
+                        case EventType.EVT_CACHE_NODES_LEFT:
+                            latches[idx].countDown();
 
                             break;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
index c9f77cd..1b779c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
@@ -291,6 +291,13 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throwUnsupported();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         throwUnsupported();
 
@@ -312,6 +319,13 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throwUnsupported();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
         throwUnsupported();
     }
@@ -354,6 +368,11 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        throwUnsupported();
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable String name) {
         throwUnsupported();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index b559897..5722fa3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -230,6 +230,11 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
         return null;
     }
@@ -261,6 +266,11 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         return null;
     }
@@ -276,6 +286,11 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 2598bc5..21fc28d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -426,6 +426,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
@@ -441,6 +446,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
@@ -477,6 +487,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable final String name) {
         return new IgniteCacheProcessProxy<>(name, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
index 7f79c0e..0a4bf20 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -302,6 +302,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        checkIgnite();
+
+        return g.createCaches(cacheCfgs);
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg, NearCacheConfiguration<K, V> nearCfg) {
         checkIgnite();
 
@@ -330,6 +337,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        checkIgnite();
+
+        return g.getOrCreateCaches(cacheCfgs);
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         checkIgnite();
 
@@ -351,6 +365,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        checkIgnite();
+
+        g.destroyCaches(cacheNames);
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
         checkIgnite();