You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/02/21 10:55:55 UTC

[3/3] ignite git commit: IGNITE-3625: IGFS: "meta" and "data" caches are now configured inside IGFS bean. This closes #923. This closes #972.

IGNITE-3625: IGFS: "meta" and "data" caches are now configured inside IGFS bean. This closes #923. This closes #972.


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

Branch: refs/heads/ignite-2.0
Commit: 3ca46d1a1bebf940f916ef47eaae48c3a181433d
Parents: f55370b
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Tue Feb 21 13:55:30 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Feb 21 13:55:30 2017 +0300

----------------------------------------------------------------------
 config/hadoop/default-config.xml                |  29 --
 examples/config/filesystem/example-igfs.xml     |  27 --
 .../JettyRestProcessorAbstractSelfTest.java     |  23 --
 .../configuration/FileSystemConfiguration.java  |  75 +++--
 .../org/apache/ignite/internal/IgnitionEx.java  |   9 +-
 .../processors/cache/GridCacheAdapter.java      |   3 +-
 .../processors/cache/GridCacheProcessor.java    |   5 +-
 .../processors/igfs/IgfsDataManager.java        |  11 +-
 .../internal/processors/igfs/IgfsImpl.java      |   2 +-
 .../processors/igfs/IgfsMetaManager.java        |  13 +-
 .../internal/processors/igfs/IgfsProcessor.java | 130 +-------
 .../internal/processors/igfs/IgfsUtils.java     | 215 +++++++++++--
 .../visor/node/VisorIgfsConfiguration.java      |   4 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   6 +-
 modules/core/src/test/config/igfs-loopback.xml  |  27 --
 modules/core/src/test/config/igfs-shmem.xml     |  27 --
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |  51 ++-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |  17 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |   3 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          |  17 +-
 .../igfs/IgfsBackupFailoverSelfTest.java        |   7 +-
 ...lockMessageSystemPoolStarvationSelfTest.java |  17 +-
 ...sCachePerBlockLruEvictionPolicySelfTest.java |  18 +-
 .../processors/igfs/IgfsCacheSelfTest.java      |  19 +-
 .../igfs/IgfsDataManagerSelfTest.java           |  33 +-
 .../processors/igfs/IgfsMaxSizeSelfTest.java    |   6 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   6 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |  16 +-
 .../processors/igfs/IgfsModesSelfTest.java      |  51 ++-
 .../processors/igfs/IgfsOneClientNodeTest.java  |  13 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |  25 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   | 320 ++++---------------
 ...gfsSecondaryFileSystemInjectionSelfTest.java |   8 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |  17 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |  27 +-
 .../processors/igfs/IgfsStartCacheTest.java     |  28 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |  46 +--
 .../processors/igfs/IgfsTaskSelfTest.java       |   8 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |   8 +-
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |  30 +-
 .../test/config/hadoop-fs-open-test/grid-0.xml  |  57 ++--
 .../test/config/hadoop-fs-open-test/grid-1.xml  |  57 ++--
 .../test/config/hadoop-fs-open-test/grid-2.xml  |  57 ++--
 .../test/config/igfs-cli-config-dual-async.xml  |  57 ++--
 .../test/config/igfs-cli-config-dual-sync.xml   |  57 ++--
 .../src/test/config/igfs-cli-config-primary.xml |  57 ++--
 .../src/test/config/igfs-cli-config-proxy.xml   |  57 ++--
 .../impl/HadoopAbstractMapReduceTest.java       |   6 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |  18 +-
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   8 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  52 +--
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   8 +-
 ...oopSecondaryFileSystemConfigurationTest.java |  69 ++--
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |   6 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  67 ++--
 ...niteHadoopFileSystemClientBasedOpenTest.java |   3 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |  56 ++--
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   8 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |  47 +--
 ...niteHadoopFileSystemLoggerStateSelfTest.java |  22 +-
 ...condaryFileSystemInitializationSelfTest.java |  30 +-
 .../org/apache/ignite/spring/sprint-exclude.xml |   2 -
 62 files changed, 889 insertions(+), 1309 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/config/hadoop/default-config.xml
----------------------------------------------------------------------
diff --git a/config/hadoop/default-config.xml b/config/hadoop/default-config.xml
index d8d3636..83d3679 100644
--- a/config/hadoop/default-config.xml
+++ b/config/hadoop/default-config.xml
@@ -54,31 +54,6 @@
     -->
     <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <!--
-            Configure caches where IGFS will store data.
-        -->
-        <property name="cacheConfiguration">
-            <list>
-                <!--
-                    Configure metadata cache where file system structure will be stored. It must be TRANSACTIONAL,
-                    and must have backups to maintain file system consistency in case of node crash.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-meta"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
-
-                <!--
-                    Configure data cache where file's data will be stored.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-data"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
-            </list>
-        </property>
-
-        <!--
             This port will be used by Apache Hadoop client to connect to Ignite node as if it was a job tracker.
         -->
         <property name="connectorConfiguration">
@@ -96,10 +71,6 @@
                     <!-- IGFS name you will use to access IGFS through Hadoop API. -->
                     <property name="name" value="igfs"/>
 
-                    <!-- Caches with these names must be configured. -->
-                    <property name="metaCacheName" value="igfs-meta"/>
-                    <property name="dataCacheName" value="igfs-data"/>
-
                     <!-- Configure TCP endpoint for communication with the file system instance. -->
                     <property name="ipcEndpointConfiguration">
                         <bean class="org.apache.ignite.igfs.IgfsIpcEndpointConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/examples/config/filesystem/example-igfs.xml
----------------------------------------------------------------------
diff --git a/examples/config/filesystem/example-igfs.xml b/examples/config/filesystem/example-igfs.xml
index a72ddce..2236821 100644
--- a/examples/config/filesystem/example-igfs.xml
+++ b/examples/config/filesystem/example-igfs.xml
@@ -61,8 +61,6 @@
             <list>
                 <bean class="org.apache.ignite.configuration.FileSystemConfiguration">
                     <property name="name" value="igfs"/>
-                    <property name="metaCacheName" value="igfs-meta"/>
-                    <property name="dataCacheName" value="igfs-data"/>
 
                     <!-- Must correlate with cache affinity mapper. -->
                     <property name="blockSize" value="#{128 * 1024}"/>
@@ -97,31 +95,6 @@
             </list>
         </property>
 
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-data"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-                    <property name="backups" value="0"/>
-                    <property name="affinityMapper">
-                        <bean class="org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper">
-                            <!-- Haw many blocks in row will be stored on the same node. -->
-                            <constructor-arg value="512"/>
-                        </bean>
-                    </property>
-                </bean>
-
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-meta"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-                </bean>
-            </list>
-        </property>
-
         <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index 8d3ab74..84071ea 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -2274,32 +2274,9 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        CacheConfiguration cacheIgfs_data = new CacheConfiguration();
-
-        cacheIgfs_data.setName("igfs-data");
-        cacheIgfs_data.setCacheMode(CacheMode.PARTITIONED);
-        cacheIgfs_data.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        cacheIgfs_data.setBackups(0);
-
-        cacheIgfs_data.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-
-        cacheIgfs_data.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(512));
-
-        CacheConfiguration cacheIgfs_meta = new CacheConfiguration();
-
-        cacheIgfs_meta.setName("igfs-meta");
-        cacheIgfs_meta.setCacheMode(CacheMode.REPLICATED);
-        cacheIgfs_meta.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-
-        cacheIgfs_meta.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-
-        cfg.setCacheConfiguration(cfg.getCacheConfiguration()[0], cacheIgfs_data, cacheIgfs_meta);
-
         FileSystemConfiguration igfs = new FileSystemConfiguration();
 
         igfs.setName("igfs");
-        igfs.setDataCacheName("igfs-data");
-        igfs.setMetaCacheName("igfs-meta");
 
         igfs.setIpcEndpointConfiguration(new IgfsIpcEndpointConfiguration());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
index e665e84..1a8d576 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
@@ -98,12 +98,6 @@ public class FileSystemConfiguration {
     /** IGFS instance name. */
     private String name;
 
-    /** Cache name to store IGFS meta information. */
-    private String metaCacheName;
-
-    /** Cache name to store file's data blocks. */
-    private String dataCacheName;
-
     /** File's data block size (bytes). */
     private int blockSize = DFLT_BLOCK_SIZE;
 
@@ -185,6 +179,12 @@ public class FileSystemConfiguration {
     /** Update file length on flush flag. */
     private boolean updateFileLenOnFlush = DFLT_UPDATE_FILE_LEN_ON_FLUSH;
 
+    /** Meta cache config. */
+    private CacheConfiguration metaCacheCfg;
+
+    /** Data cache config. */
+    private CacheConfiguration dataCacheCfg;
+
     /**
      * Constructs default configuration.
      */
@@ -206,7 +206,7 @@ public class FileSystemConfiguration {
         blockSize = cfg.getBlockSize();
         bufSize = cfg.getStreamBufferSize();
         colocateMeta = cfg.isColocateMetadata();
-        dataCacheName = cfg.getDataCacheName();
+        dataCacheCfg = cfg.getDataCacheConfiguration();
         dfltMode = cfg.getDefaultMode();
         dualModeMaxPendingPutsSize = cfg.getDualModeMaxPendingPutsSize();
         dualModePutExec = cfg.getDualModePutExecutorService();
@@ -222,7 +222,7 @@ public class FileSystemConfiguration {
         ipcEndpointEnabled = cfg.isIpcEndpointEnabled();
         maxSpace = cfg.getMaxSpaceSize();
         maxTaskRangeLen = cfg.getMaximumTaskRangeLength();
-        metaCacheName = cfg.getMetaCacheName();
+        metaCacheCfg = cfg.getMetaCacheConfiguration();
         mgmtPort = cfg.getManagementPort();
         name = cfg.getName();
         pathModes = cfg.getPathModes();
@@ -255,40 +255,55 @@ public class FileSystemConfiguration {
     }
 
     /**
-     * Cache name to store IGFS meta information. If {@code null}, then instance
-     * with default meta-cache name will be used.
+     * Cache config to store IGFS meta information.
      *
-     * @return Cache name to store IGFS meta information.
+     * @return Cache configuration object.
      */
-    @Nullable public String getMetaCacheName() {
-        return metaCacheName;
+    @Nullable public CacheConfiguration getMetaCacheConfiguration() {
+        return metaCacheCfg;
     }
 
     /**
-     * Sets cache name to store IGFS meta information.
+     * Cache config to store IGFS meta information. If {@code null}, then default config for
+     * meta-cache will be used.
+     *
+     * Default configuration for the meta cache is:
+     * <ul>
+     *     <li>atomicityMode = TRANSACTIONAL</li>
+     *     <li>cacheMode = PARTITIONED</li>
+     *     <li>backups = 1</li>
+     * </ul>
      *
-     * @param metaCacheName Cache name to store IGFS meta information.
+     * @param metaCacheCfg Cache configuration object.
      */
-    public void setMetaCacheName(String metaCacheName) {
-        this.metaCacheName = metaCacheName;
+    public void setMetaCacheConfiguration(CacheConfiguration metaCacheCfg) {
+        this.metaCacheCfg = metaCacheCfg;
     }
 
     /**
-     * Cache name to store IGFS data.
+     * Cache config to store IGFS data.
      *
-     * @return Cache name to store IGFS data.
+     * @return Cache configuration object.
      */
-    @Nullable public String getDataCacheName() {
-        return dataCacheName;
+    @Nullable public CacheConfiguration getDataCacheConfiguration() {
+        return dataCacheCfg;
     }
 
     /**
-     * Sets cache name to store IGFS data.
+     * Cache config to store IGFS data. If {@code null}, then default config for
+     * data cache will be used.
+     *
+     * Default configuration for the data cache is:
+     * <ul>
+     *     <<li>atomicityMode = TRANSACTIONAL</li>
+     *     <li>cacheMode = PARTITIONED</li>
+     *     <li>backups = 0</li>
+     * </ul>
      *
-     * @param dataCacheName Cache name to store IGFS data.
+     * @param dataCacheCfg Cache configuration object.
      */
-    public void setDataCacheName(String dataCacheName) {
-        this.dataCacheName = dataCacheName;
+    public void setDataCacheConfiguration(CacheConfiguration dataCacheCfg) {
+        this.dataCacheCfg = dataCacheCfg;
     }
 
     /**
@@ -878,11 +893,11 @@ public class FileSystemConfiguration {
      * transaction with keys owned only by a single node.
      * <p>
      * IGFS stores information about file system structure (metadata) inside a transactional cache configured through
-     * {@link #getMetaCacheName()} property. Metadata updates caused by operations on IGFS usually require several
-     * internal keys to be updated. As IGFS metadata cache usually operates in {@link CacheMode#REPLICATED} mode,
-     * meaning that all nodes have all metadata locally, it makes sense to give a hint to Ignite to co-locate
-     * ownership of all metadata keys on a single node. This will decrease amount of network trips required to update
-     * metadata and hence could improve performance.
+     * {@link #getMetaCacheConfiguration()} property. Metadata updates caused by operations on IGFS usually require
+     * several internal keys to be updated. As IGFS metadata cache usually operates
+     * in {@link CacheMode#REPLICATED} mode, meaning that all nodes have all metadata locally, it makes sense to give
+     * a hint to Ignite to co-locate ownership of all metadata keys on a single node.
+     * This will decrease amount of network trips required to update metadata and hence could improve performance.
      * <p>
      * This property should be disabled if you see excessive CPU and network load on a single node, which
      * degrades performance and cannot be explained by business logic of your application.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 05d19c0..9fe2dba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -2126,17 +2126,20 @@ public class IgnitionEx {
                         throw new IgniteCheckedException("Cache name cannot be \"" + CU.UTILITY_CACHE_NAME +
                             "\" because it is reserved for internal purposes.");
 
+                    if (IgfsUtils.matchIgfsCacheName(ccfg.getName()))
+                        throw new IgniteCheckedException(
+                            "Cache name cannot start with \""+ IgfsUtils.IGFS_CACHE_PREFIX
+                                + "\" because it is reserved for IGFS internal purposes.");
+
                     cacheCfgs.add(ccfg);
                 }
             }
 
             cfg.setCacheConfiguration(cacheCfgs.toArray(new CacheConfiguration[cacheCfgs.size()]));
 
-            // Iterate over IGFS caches and prepare their configurations if needed.
             assert cfg.getCacheConfiguration() != null;
 
-            for (CacheConfiguration ccfg : cfg.getCacheConfiguration())
-                IgfsUtils.prepareCacheConfiguration(cfg, ccfg);
+            IgfsUtils.prepareCacheConfigurations(cfg);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 4507c53..285ba11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -97,6 +97,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
@@ -338,7 +339,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         if (igfsCfgs != null) {
             for (FileSystemConfiguration igfsCfg : igfsCfgs) {
-                if (F.eq(ctx.name(), igfsCfg.getDataCacheName())) {
+                if (F.eq(ctx.name(), igfsCfg.getDataCacheConfiguration().getName())) {
                     if (!ctx.isNear()) {
                         igfsDataCache = true;
                         igfsDataCacheSize = new LongAdder8();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/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 17f9c2a..b1644d1 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
@@ -99,6 +99,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTransactio
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
@@ -726,8 +727,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         if (igfsCfgs != null) {
             for (FileSystemConfiguration igfsCfg : igfsCfgs) {
-                internalCaches.add(maskNull(igfsCfg.getMetaCacheName()));
-                internalCaches.add(maskNull(igfsCfg.getDataCacheName()));
+                internalCaches.add(maskNull(igfsCfg.getMetaCacheConfiguration().getName()));
+                internalCaches.add(maskNull(igfsCfg.getDataCacheConfiguration().getName()));
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index d6297b9..dc845ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -130,6 +130,9 @@ public class IgfsDataManager extends IgfsManager {
     /** Async file delete worker. */
     private AsyncDeleteWorker delWorker;
 
+    /** Async file delete worker. */
+    private String dataCacheName;
+
     /** On-going remote reads futures. */
     private final ConcurrentHashMap8<IgfsBlockKey, IgniteInternalFuture<byte[]>> rmtReadFuts =
         new ConcurrentHashMap8<>();
@@ -180,19 +183,21 @@ public class IgfsDataManager extends IgfsManager {
 
         delWorker = new AsyncDeleteWorker(igfsCtx.kernalContext().gridName(),
             "igfs-" + igfsName + "-delete-worker", log);
+
+        dataCacheName = igfsCtx.configuration().getDataCacheConfiguration().getName();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected void onKernalStart0() throws IgniteCheckedException {
-        dataCachePrj = igfsCtx.kernalContext().cache().getOrStartCache(igfsCtx.configuration().getDataCacheName());
+        dataCachePrj = igfsCtx.kernalContext().cache().getOrStartCache(dataCacheName);
 
         assert dataCachePrj != null;
 
         dataCache = (IgniteInternalCache)dataCachePrj;
 
         AffinityKeyMapper mapper = igfsCtx.kernalContext().cache()
-            .internalCache(igfsCtx.configuration().getDataCacheName()).configuration().getAffinityMapper();
+            .internalCache(dataCacheName).configuration().getAffinityMapper();
 
         grpSize = mapper instanceof IgfsGroupDataBlocksKeyMapper ?
             ((IgfsGroupDataBlocksKeyMapper)mapper).getGroupSize() : 1;
@@ -201,7 +206,7 @@ public class IgfsDataManager extends IgfsManager {
 
         assert grpBlockSize != 0;
 
-        igfsCtx.kernalContext().cache().internalCache(igfsCtx.configuration().getDataCacheName()).preloader()
+        igfsCtx.kernalContext().cache().internalCache(dataCacheName).preloader()
             .startFuture().listen(new CI1<IgniteInternalFuture<Object>>() {
             @Override public void apply(IgniteInternalFuture<Object> f) {
                 dataCacheStartLatch.countDown();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 56da121..7165f31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -263,7 +263,7 @@ public final class IgfsImpl implements IgfsEx {
         secondaryPaths = new IgfsPaths(secondaryFsPayload, dfltMode, modeRslvr.modesOrdered());
 
         // Check whether IGFS LRU eviction policy is set on data cache.
-        String dataCacheName = igfsCtx.configuration().getDataCacheName();
+        String dataCacheName = igfsCtx.configuration().getDataCacheConfiguration().getName();
 
         for (CacheConfiguration cacheCfg : igfsCtx.kernalContext().config().getCacheConfiguration()) {
             if (F.eq(dataCacheName, cacheCfg.getName())) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 95d73a8..35d0776 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -155,6 +155,9 @@ public class IgfsMetaManager extends IgfsManager {
     /** Compute facade for client tasks. */
     private IgniteCompute cliCompute;
 
+    /** Compute facade for client tasks. */
+    private String metaCacheName;
+
     /**
      * Constructor.
      *
@@ -189,16 +192,18 @@ public class IgfsMetaManager extends IgfsManager {
         sampling = new IgfsSamplingKey(cfg.getName());
 
         log = igfsCtx.kernalContext().log(IgfsMetaManager.class);
+
+        metaCacheName = cfg.getMetaCacheConfiguration().getName();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("RedundantCast")
     @Override protected void onKernalStart0() throws IgniteCheckedException {
-        metaCache = igfsCtx.kernalContext().cache().getOrStartCache(cfg.getMetaCacheName());
+        metaCache = igfsCtx.kernalContext().cache().getOrStartCache(metaCacheName);
 
         assert metaCache != null;
 
-        igfsCtx.kernalContext().cache().internalCache(cfg.getMetaCacheName()).preloader().startFuture()
+        igfsCtx.kernalContext().cache().internalCache(metaCacheName).preloader().startFuture()
             .listen(new CI1<IgniteInternalFuture<Object>>() {
                 @Override public void apply(IgniteInternalFuture<Object> f) {
                     metaCacheStartLatch.countDown();
@@ -267,7 +272,7 @@ public class IgfsMetaManager extends IgfsManager {
     <T> T runClientTask(IgniteUuid affinityFileId, IgfsClientAbstractCallable<T> task) {
         try {
             return (cfg.isColocateMetadata()) ?
-                clientCompute().affinityCall(cfg.getMetaCacheName(), affinityFileId, task) :
+                clientCompute().affinityCall(metaCacheName, affinityFileId, task) :
                 clientCompute().call(task);
         }
         catch (ClusterTopologyException e) {
@@ -288,7 +293,7 @@ public class IgfsMetaManager extends IgfsManager {
         if (cliCompute0 == null) {
             IgniteEx ignite = igfsCtx.kernalContext().grid();
 
-            ClusterGroup cluster = ignite.cluster().forIgfsMetadataDataNodes(cfg.getName(), cfg.getMetaCacheName());
+            ClusterGroup cluster = ignite.cluster().forIgfsMetadataDataNodes(cfg.getName(), metaCacheName);
 
             cliCompute0 = ignite.compute(cluster);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
index 57bf70f..405016c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
@@ -26,14 +26,11 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
-import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
-import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.mapreduce.IgfsJob;
 import org.apache.ignite.igfs.mapreduce.IgfsRecordResolver;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteNodeAttributes;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.ipc.IpcServerEndpoint;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.F;
@@ -46,7 +43,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
@@ -55,8 +51,6 @@ import java.util.concurrent.ConcurrentMap;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.igfs.IgfsMode.PROXY;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
 
 /**
@@ -66,12 +60,6 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
     /** Null IGFS name. */
     private static final String NULL_NAME = UUID.randomUUID().toString();
 
-    /** Min available TCP port. */
-    private static final int MIN_TCP_PORT = 1;
-
-    /** Max available TCP port. */
-    private static final int MAX_TCP_PORT = 0xFFFF;
-
     /** Converts context to IGFS. */
     private static final IgniteClosure<IgfsContext,IgniteFileSystem> CTX_TO_IGFS = new C1<IgfsContext, IgniteFileSystem>() {
         @Override public IgniteFileSystem apply(IgfsContext igfsCtx) {
@@ -101,8 +89,6 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
         assert cfgs != null && cfgs.length > 0;
 
-        validateLocalIgfsConfigurations(cfgs);
-
         // Start IGFS instances.
         for (FileSystemConfiguration cfg : cfgs) {
             FileSystemConfiguration cfg0 = new FileSystemConfiguration(cfg);
@@ -111,9 +97,11 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
             CacheConfiguration[] cacheCfgs = igniteCfg.getCacheConfiguration();
 
+            String metaCacheName = cfg.getMetaCacheConfiguration().getName();
+
             if (cacheCfgs != null) {
                 for (CacheConfiguration cacheCfg : cacheCfgs) {
-                    if (F.eq(cacheCfg.getName(), cfg.getMetaCacheName())) {
+                    if (F.eq(cacheCfg.getName(), metaCacheName)) {
                         metaClient = false;
 
                         break;
@@ -162,7 +150,9 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
         assert igniteCfg.getFileSystemConfiguration() != null;
 
         for (FileSystemConfiguration igfsCfg : igniteCfg.getFileSystemConfiguration()) {
-            CacheConfiguration cacheCfg = cacheCfgs.get(igfsCfg.getDataCacheName());
+            String dataCacheName = igfsCfg.getDataCacheConfiguration().getName();
+
+            CacheConfiguration cacheCfg = cacheCfgs.get(dataCacheName);
 
             if (cacheCfg == null)
                 continue; // No cache for the given IGFS configuration.
@@ -178,8 +168,8 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
                 igfsCfg.getName(),
                 igfsCfg.getBlockSize(),
                 ((IgfsGroupDataBlocksKeyMapper)affMapper).getGroupSize(),
-                igfsCfg.getMetaCacheName(),
-                igfsCfg.getDataCacheName(),
+                igfsCfg.getMetaCacheConfiguration().getName(),
+                dataCacheName,
                 igfsCfg.getDefaultMode(),
                 igfsCfg.getPathModes(),
                 igfsCfg.isFragmentizerEnabled()));
@@ -288,110 +278,6 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
     }
 
     /**
-     * Validates local IGFS configurations. Compares attributes only for IGFSes with same name.
-     * @param cfgs IGFS configurations
-     * @throws IgniteCheckedException If any of IGFS configurations is invalid.
-     */
-    private void validateLocalIgfsConfigurations(FileSystemConfiguration[] cfgs) throws IgniteCheckedException {
-        Collection<String> cfgNames = new HashSet<>();
-
-        Collection<String> dataCacheNames = new HashSet<>();
-        Collection<String> metaCacheNames = new HashSet<>();
-
-        for (FileSystemConfiguration cfg : cfgs) {
-            String name = cfg.getName();
-
-            if (cfgNames.contains(name))
-                throw new IgniteCheckedException("Duplicate IGFS name found (check configuration and " +
-                    "assign unique name to each): " + name);
-
-            CacheConfiguration dataCacheCfg = config(cfg.getDataCacheName());
-            CacheConfiguration metaCacheCfg = config(cfg.getMetaCacheName());
-
-            if (dataCacheCfg == null)
-                throw new IgniteCheckedException("Data cache is not configured locally for IGFS: " + cfg);
-
-            if (GridQueryProcessor.isEnabled(dataCacheCfg))
-                throw new IgniteCheckedException("IGFS data cache cannot start with enabled query indexing.");
-
-            if (dataCacheCfg.getAtomicityMode() != TRANSACTIONAL && cfg.isFragmentizerEnabled())
-                throw new IgniteCheckedException("Data cache should be transactional: " + cfg.getDataCacheName() +
-                    " when fragmentizer is enabled");
-
-            if (metaCacheCfg == null)
-                throw new IgniteCheckedException("Metadata cache is not configured locally for IGFS: " + cfg);
-
-            if (GridQueryProcessor.isEnabled(metaCacheCfg))
-                throw new IgniteCheckedException("IGFS metadata cache cannot start with enabled query indexing.");
-
-            if (metaCacheCfg.getAtomicityMode() != TRANSACTIONAL)
-                throw new IgniteCheckedException("Meta cache should be transactional: " + cfg.getMetaCacheName());
-
-            if (F.eq(cfg.getDataCacheName(), cfg.getMetaCacheName()))
-                throw new IgniteCheckedException("Cannot use same cache as both data and meta cache: " + cfg.getName());
-
-            if (dataCacheNames.contains(cfg.getDataCacheName()))
-                throw new IgniteCheckedException("Data cache names should be different for different IGFS instances: "
-                    + cfg.getName());
-
-            if (metaCacheNames.contains(cfg.getMetaCacheName()))
-                throw new IgniteCheckedException("Meta cache names should be different for different IGFS instances: "
-                    + cfg.getName());
-
-            if (!(dataCacheCfg.getAffinityMapper() instanceof IgfsGroupDataBlocksKeyMapper))
-                throw new IgniteCheckedException("Invalid IGFS data cache configuration (key affinity mapper class should be " +
-                    IgfsGroupDataBlocksKeyMapper.class.getSimpleName() + "): " + cfg);
-
-            IgfsIpcEndpointConfiguration ipcCfg = cfg.getIpcEndpointConfiguration();
-
-            if (ipcCfg != null) {
-                final int tcpPort = ipcCfg.getPort();
-
-                if (!(tcpPort >= MIN_TCP_PORT && tcpPort <= MAX_TCP_PORT))
-                    throw new IgniteCheckedException("IGFS endpoint TCP port is out of range [" + MIN_TCP_PORT +
-                        ".." + MAX_TCP_PORT + "]: " + tcpPort);
-
-                if (ipcCfg.getThreadCount() <= 0)
-                    throw new IgniteCheckedException("IGFS endpoint thread count must be positive: " +
-                        ipcCfg.getThreadCount());
-            }
-
-            boolean secondary = cfg.getDefaultMode() == PROXY;
-
-            if (cfg.getPathModes() != null) {
-                for (Map.Entry<String, IgfsMode> mode : cfg.getPathModes().entrySet()) {
-                    if (mode.getValue() == PROXY)
-                        secondary = true;
-                }
-            }
-
-            if (secondary) {
-                // When working in any mode except of primary, secondary FS config must be provided.
-                assertParameter(cfg.getSecondaryFileSystem() != null,
-                    "secondaryFileSystem cannot be null when mode is not " + IgfsMode.PRIMARY);
-            }
-
-            dataCacheNames.add(cfg.getDataCacheName());
-            metaCacheNames.add(cfg.getMetaCacheName());
-
-            cfgNames.add(name);
-        }
-    }
-
-    /**
-     * @param cacheName Cache name.
-     * @return Configuration.
-     */
-    private CacheConfiguration config(String cacheName) {
-        for (CacheConfiguration ccfg : ctx.config().getCacheConfiguration()) {
-            if (F.eq(cacheName, ccfg.getName()))
-                return ccfg;
-        }
-
-        return null;
-    }
-
-    /**
      * Check IGFS config on remote node.
      *
      * @param rmtNode Remote node.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 6763d2a..ec12884 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -17,11 +17,16 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
@@ -31,6 +36,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
+import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.GridKernalContext;
@@ -38,6 +44,7 @@ import org.apache.ignite.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
@@ -65,6 +72,8 @@ import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC;
 import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
@@ -127,9 +136,27 @@ public class IgfsUtils {
     /** Flag: this is a file. */
     private static final byte FLAG_FILE = 0x2;
 
+    /** Filesystem cache prefix. */
+    public static final String IGFS_CACHE_PREFIX = "igfs-internal-";
+
+    /** Data cache suffix. */
+    public static final String DATA_CACHE_SUFFIX = "-data";
+
+    /** Meta cache suffix. */
+    public static final String META_CACHE_SUFFIX = "-meta";
+
     /** Maximum string length to be written at once. */
     private static final int MAX_STR_LEN = 0xFFFF / 4;
 
+    /** Min available TCP port. */
+    private static final int MIN_TCP_PORT = 1;
+
+    /** Max available TCP port. */
+    private static final int MAX_TCP_PORT = 0xFFFF;
+
+    /*
+     * Static initializer.
+     */
     static {
         TRASH_IDS = new IgniteUuid[TRASH_CONCURRENCY];
 
@@ -311,6 +338,7 @@ public class IgfsUtils {
     /**
      * Sends a series of event.
      *
+     * @param kernalCtx Kernal context.
      * @param path The path of the created file.
      * @param type The type of event to send.
      */
@@ -321,6 +349,7 @@ public class IgfsUtils {
     /**
      * Sends a series of event.
      *
+     * @param kernalCtx Kernal context.
      * @param path The path of the created file.
      * @param newPath New path.
      * @param type The type of event to send.
@@ -342,70 +371,189 @@ public class IgfsUtils {
     }
 
     /**
+     * @param cacheName Cache name.
+     * @return {@code True} in this is IGFS data or meta cache.
+     */
+    public static boolean matchIgfsCacheName(@Nullable String cacheName) {
+        return cacheName != null && cacheName.startsWith(IGFS_CACHE_PREFIX);
+    }
+
+    /**
      * @param cfg Grid configuration.
      * @param cacheName Cache name.
      * @return {@code True} in this is IGFS data or meta cache.
      */
     public static boolean isIgfsCache(IgniteConfiguration cfg, @Nullable String cacheName) {
+        return matchIgfsCacheName(cacheName);
+    }
+
+    /**
+     * Prepare cache configuration if this is IGFS meta or data cache.
+     *
+     * @param cfg Configuration.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static void prepareCacheConfigurations(IgniteConfiguration cfg) throws IgniteCheckedException {
         FileSystemConfiguration[] igfsCfgs = cfg.getFileSystemConfiguration();
+        List<CacheConfiguration> ccfgs = new ArrayList<>(Arrays.asList(cfg.getCacheConfiguration()));
 
         if (igfsCfgs != null) {
             for (FileSystemConfiguration igfsCfg : igfsCfgs) {
-                // IGFS config probably has not been validated yet => possible NPE, so we check for null.
-                if (igfsCfg != null) {
-                    if (F.eq(cacheName, igfsCfg.getDataCacheName()) || F.eq(cacheName, igfsCfg.getMetaCacheName()))
-                        return true;
+                if (igfsCfg == null)
+                    continue;
+
+                CacheConfiguration ccfgMeta = igfsCfg.getMetaCacheConfiguration();
+
+                if (ccfgMeta == null) {
+                    ccfgMeta = defaultMetaCacheConfig();
+
+                    igfsCfg.setMetaCacheConfiguration(ccfgMeta);
+                }
+
+                ccfgMeta.setName(IGFS_CACHE_PREFIX + igfsCfg.getName() + META_CACHE_SUFFIX);
+
+                ccfgs.add(ccfgMeta);
+
+                CacheConfiguration ccfgData = igfsCfg.getDataCacheConfiguration();
+
+                if (ccfgData == null) {
+                    ccfgData = defaultDataCacheConfig();
+
+                    igfsCfg.setDataCacheConfiguration(ccfgData);
                 }
+
+                ccfgData.setName(IGFS_CACHE_PREFIX + igfsCfg.getName() + DATA_CACHE_SUFFIX);
+
+                ccfgs.add(ccfgData);
+
+
+                // No copy-on-read.
+                ccfgMeta.setCopyOnRead(false);
+                ccfgData.setCopyOnRead(false);
+
+                // Always full-sync to maintain consistency.
+                ccfgMeta.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+                ccfgData.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+
+                // Set co-located affinity mapper if needed.
+                if (igfsCfg.isColocateMetadata() && ccfgMeta.getAffinityMapper() == null)
+                    ccfgMeta.setAffinityMapper(new IgfsColocatedMetadataAffinityKeyMapper());
+
+                // Set affinity mapper if needed.
+                if (ccfgData.getAffinityMapper() == null)
+                    ccfgData.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper());
             }
+
+            cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
         }
 
-        return false;
+        validateLocalIgfsConfigurations(cfg);
     }
 
     /**
-     * Prepare cache configuration if this is IGFS meta or data cache.
+     * Validates local IGFS configurations. Compares attributes only for IGFSes with same name.
      *
-     * @param cfg Configuration.
-     * @param ccfg Cache configuration.
+     * @param igniteCfg Ignite config.
+     * @throws IgniteCheckedException If any of IGFS configurations is invalid.
      */
-    public static void prepareCacheConfiguration(IgniteConfiguration cfg, CacheConfiguration ccfg) {
-        FileSystemConfiguration[] igfsCfgs = cfg.getFileSystemConfiguration();
+    private static void validateLocalIgfsConfigurations(IgniteConfiguration igniteCfg)
+        throws IgniteCheckedException {
 
-        if (igfsCfgs != null) {
-            for (FileSystemConfiguration igfsCfg : igfsCfgs) {
-                if (igfsCfg != null) {
-                    if (F.eq(ccfg.getName(), igfsCfg.getMetaCacheName())) {
-                        // No copy-on-read.
-                        ccfg.setCopyOnRead(false);
+        if (igniteCfg.getFileSystemConfiguration() == null || igniteCfg.getFileSystemConfiguration().length == 0)
+            return;
+
+        Collection<String> cfgNames = new HashSet<>();
 
-                        // Always full-sync to maintain consistency.
-                        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        for (FileSystemConfiguration cfg : igniteCfg.getFileSystemConfiguration()) {
+            String name = cfg.getName();
 
-                        // Set co-located affinity mapper if needed.
-                        if (igfsCfg.isColocateMetadata() && ccfg.getAffinityMapper() == null)
-                            ccfg.setAffinityMapper(new IgfsColocatedMetadataAffinityKeyMapper());
+            if (cfgNames.contains(name))
+                throw new IgniteCheckedException("Duplicate IGFS name found (check configuration and " +
+                    "assign unique name to each): " + name);
 
-                        return;
-                    }
+            CacheConfiguration ccfgData = cfg.getDataCacheConfiguration();
 
-                    if (F.eq(ccfg.getName(), igfsCfg.getDataCacheName())) {
-                        // No copy-on-read.
-                        ccfg.setCopyOnRead(false);
+            CacheConfiguration ccfgMeta = cfg.getMetaCacheConfiguration();
 
-                        // Always full-sync to maintain consistency.
-                        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+            if (GridQueryProcessor.isEnabled(ccfgData))
+                throw new IgniteCheckedException("IGFS data cache cannot start with enabled query indexing.");
 
-                        // Set affinity mapper if needed.
-                        if (ccfg.getAffinityMapper() == null)
-                            ccfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper());
+            if (GridQueryProcessor.isEnabled(ccfgMeta))
+                throw new IgniteCheckedException("IGFS metadata cache cannot start with enabled query indexing.");
 
-                        return;
-                    }
+            if (ccfgMeta.getAtomicityMode() != TRANSACTIONAL)
+                throw new IgniteCheckedException("IGFS metadata cache should be transactional: " + cfg.getName());
+
+            if (!(ccfgData.getAffinityMapper() instanceof IgfsGroupDataBlocksKeyMapper))
+                throw new IgniteCheckedException(
+                    "Invalid IGFS data cache configuration (key affinity mapper class should be " +
+                    IgfsGroupDataBlocksKeyMapper.class.getSimpleName() + "): " + cfg);
+
+            IgfsIpcEndpointConfiguration ipcCfg = cfg.getIpcEndpointConfiguration();
+
+            if (ipcCfg != null) {
+                final int tcpPort = ipcCfg.getPort();
+
+                if (!(tcpPort >= MIN_TCP_PORT && tcpPort <= MAX_TCP_PORT))
+                    throw new IgniteCheckedException("IGFS endpoint TCP port is out of range [" + MIN_TCP_PORT +
+                        ".." + MAX_TCP_PORT + "]: " + tcpPort);
+
+                if (ipcCfg.getThreadCount() <= 0)
+                    throw new IgniteCheckedException("IGFS endpoint thread count must be positive: " +
+                        ipcCfg.getThreadCount());
+            }
+
+            boolean secondary = cfg.getDefaultMode() == IgfsMode.PROXY;
+
+            if (cfg.getPathModes() != null) {
+                for (Map.Entry<String, IgfsMode> mode : cfg.getPathModes().entrySet()) {
+                    if (mode.getValue() == IgfsMode.PROXY)
+                        secondary = true;
                 }
             }
+
+            if (secondary && cfg.getSecondaryFileSystem() == null) {
+                // When working in any mode except of primary, secondary FS config must be provided.
+                throw new IgniteCheckedException("Grid configuration parameter invalid: " +
+                    "secondaryFileSystem cannot be null when mode is not " + IgfsMode.PRIMARY);
+            }
+
+            cfgNames.add(name);
         }
     }
 
+
+    /**
+     * @return Default IGFS cache configuration.
+     */
+    private static CacheConfiguration defaultCacheConfig() {
+        CacheConfiguration cfg = new CacheConfiguration();
+        cfg.setAtomicWriteOrderMode(CacheAtomicWriteOrderMode.PRIMARY);
+        cfg.setAtomicityMode(TRANSACTIONAL);
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheMode(CacheMode.PARTITIONED);
+
+        return cfg;
+    }
+
+    /**
+     * @return Default IGFS meta cache configuration.
+     */
+    private static CacheConfiguration defaultMetaCacheConfig() {
+        CacheConfiguration cfg = defaultCacheConfig();
+
+        cfg.setBackups(1);
+
+        return cfg;
+    }
+
+    /**
+     * @return Default IGFS data cache configuration.
+     */
+    private static CacheConfiguration defaultDataCacheConfig() {
+        return defaultCacheConfig();
+    }
+
     /**
      * Create empty directory with the given ID.
      *
@@ -532,6 +680,7 @@ public class IgfsUtils {
      *
      * @param in Reader.
      * @return Entry.
+     * @throws IOException If failed.
      */
     @Nullable public static IgfsListingEntry readListingEntry(DataInput in) throws IOException {
         if (in.readBoolean()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
index cb10c1c..7f8e3b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
@@ -130,8 +130,8 @@ public class VisorIgfsConfiguration implements Serializable, LessNamingBean {
         VisorIgfsConfiguration cfg = new VisorIgfsConfiguration();
 
         cfg.name = igfs.getName();
-        cfg.metaCacheName = igfs.getMetaCacheName();
-        cfg.dataCacheName = igfs.getDataCacheName();
+        cfg.metaCacheName = igfs.getMetaCacheConfiguration().getName();
+        cfg.dataCacheName = igfs.getDataCacheConfiguration().getName();
         cfg.blockSize = igfs.getBlockSize();
         cfg.prefetchBlocks = igfs.getPrefetchBlocks();
         cfg.streamBufSize = igfs.getStreamBufferSize();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index abe1364..48487ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -188,7 +188,8 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
                 if (proxyCache(cacheName))
                     continue;
 
-                if (arg.systemCaches() || !(isSystemCache(cacheName) || isIgfsCache(cfg, cacheName))) {
+                if (arg.systemCaches() ||
+                    !(isSystemCache(cacheName) || isIgfsCache(ignite.configuration(), cacheName))) {
                     long start0 = U.currentTimeMillis();
 
                     try {
@@ -227,7 +228,8 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
 
                 FileSystemConfiguration igfsCfg = igfs.configuration();
 
-                if (proxyCache(igfsCfg.getDataCacheName()) || proxyCache(igfsCfg.getMetaCacheName()))
+                if (proxyCache(igfsCfg.getDataCacheConfiguration().getName())
+                    || proxyCache(igfsCfg.getMetaCacheConfiguration().getName()))
                     continue;
 
                 try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/config/igfs-loopback.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/igfs-loopback.xml b/modules/core/src/test/config/igfs-loopback.xml
index dbbfb4e..5bfbdae 100644
--- a/modules/core/src/test/config/igfs-loopback.xml
+++ b/modules/core/src/test/config/igfs-loopback.xml
@@ -75,8 +75,6 @@
             <list>
                 <bean class="org.apache.ignite.configuration.FileSystemConfiguration">
                     <property name="name" value="igfs"/>
-                    <property name="metaCacheName" value="igfs-meta"/>
-                    <property name="dataCacheName" value="igfs-data"/>
 
                     <!-- Must correlate with cache affinity mapper. -->
                     <property name="blockSize" value="#{128 * 1024}"/>
@@ -97,31 +95,6 @@
             </list>
         </property>
 
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-data"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-                    <property name="backups" value="0"/>
-                    <property name="affinityMapper">
-                        <bean class="org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper">
-                            <!-- Haw many blocks in row will be stored on the same node. -->
-                            <constructor-arg value="512"/>
-                        </bean>
-                    </property>
-                </bean>
-
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-meta"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-                </bean>
-            </list>
-        </property>
-
         <!--
             Disable events.
         -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/config/igfs-shmem.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/igfs-shmem.xml b/modules/core/src/test/config/igfs-shmem.xml
index c4c8704..5876b41 100644
--- a/modules/core/src/test/config/igfs-shmem.xml
+++ b/modules/core/src/test/config/igfs-shmem.xml
@@ -75,8 +75,6 @@
             <list>
                 <bean class="org.apache.ignite.configuration.FileSystemConfiguration">
                     <property name="name" value="igfs"/>
-                    <property name="metaCacheName" value="igfs-meta"/>
-                    <property name="dataCacheName" value="igfs-data"/>
 
                     <!-- Must correlate with cache affinity mapper. -->
                     <property name="blockSize" value="#{128 * 1024}"/>
@@ -97,31 +95,6 @@
             </list>
         </property>
 
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-data"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-                    <property name="backups" value="0"/>
-                    <property name="affinityMapper">
-                        <bean class="org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper">
-                            <!-- Haw many blocks in row will be stored on the same node. -->
-                            <constructor-arg value="512"/>
-                        </bean>
-                    </property>
-                </bean>
-
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="igfs-meta"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-                </bean>
-            </list>
-        </property>
-
         <!--
             Disable events.
         -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
index 9c2c6e5..ea2be11 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
@@ -71,46 +71,33 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
     private IgnitePredicate<Event> lsnr;
 
     /**
-     * Gets cache configuration.
-     *
-     * @param gridName Grid name.
-     * @return Cache configuration.
+     * @return IGFS configuration for this test.
      */
-    @SuppressWarnings("deprecation")
-    protected CacheConfiguration[] getCacheConfiguration(String gridName) {
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName("dataCache");
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setNearConfiguration(null);
-        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg.setEvictionPolicy(null);
-        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        cacheCfg.setBackups(0);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+    protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setName("igfs");
+        igfsCfg.setBlockSize(512 * 1024); // Together with group blocks mapper will yield 64M per node groups.
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setNearConfiguration(null);
+        dataCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        dataCacheCfg.setEvictionPolicy(null);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
 
         CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
 
-        metaCacheCfg.setName("metaCache");
         metaCacheCfg.setCacheMode(REPLICATED);
         metaCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
         metaCacheCfg.setEvictionPolicy(null);
         metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
 
-        return new CacheConfiguration[] {cacheCfg, metaCacheCfg};
-    }
-
-    /**
-     * @return IGFS configuration for this test.
-     */
-    protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
-
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
-        igfsCfg.setName("igfs");
-
-        igfsCfg.setBlockSize(512 * 1024); // Together with group blocks mapper will yield 64M per node groups.
+        igfsCfg.setMetaCacheConfiguration(metaCacheCfg);
+        igfsCfg.setDataCacheConfiguration(dataCacheCfg);
 
         return igfsCfg;
     }
@@ -139,8 +126,6 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
 
         cfg.setFileSystemConfiguration(igfsCfg);
 
-        cfg.setCacheConfiguration(getCacheConfiguration(gridName));
-
         cfg.setHadoopConfiguration(null);
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
index d52a918..6536267 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
@@ -51,12 +51,6 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
     /** IGFS group size. */
     protected static final int IGFS_GROUP_SIZE = 32;
 
-    /** Metadata cache name. */
-    private static final String META_CACHE_NAME = "meta";
-
-    /** File data cache name. */
-    protected static final String DATA_CACHE_NAME = "data";
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -67,13 +61,9 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
-        cfg.setCacheConfiguration(metaConfiguration(), dataConfiguration());
-
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setName("igfs");
-        igfsCfg.setMetaCacheName(META_CACHE_NAME);
-        igfsCfg.setDataCacheName(DATA_CACHE_NAME);
         igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
 
         // Need to set this to avoid thread starvation.
@@ -82,6 +72,9 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setFragmentizerThrottlingBlockLength(16 * IGFS_BLOCK_SIZE);
         igfsCfg.setFragmentizerThrottlingDelay(10);
 
+        igfsCfg.setMetaCacheConfiguration(metaConfiguration());
+        igfsCfg.setDataCacheConfiguration(dataConfiguration());
+
         cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
@@ -95,8 +88,6 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
     protected CacheConfiguration metaConfiguration() {
         CacheConfiguration cfg = defaultCacheConfiguration();
 
-        cfg.setName(META_CACHE_NAME);
-
         cfg.setCacheMode(REPLICATED);
         cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cfg.setAtomicityMode(TRANSACTIONAL);
@@ -112,8 +103,6 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
     protected CacheConfiguration dataConfiguration() {
         CacheConfiguration cfg = defaultCacheConfiguration();
 
-        cfg.setName(DATA_CACHE_NAME);
-
         cfg.setCacheMode(PARTITIONED);
         cfg.setBackups(0);
         cfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(IGFS_GROUP_SIZE));

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
index 4e0f12b..5149493 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
@@ -244,7 +244,8 @@ public class IgfsFragmentizerSelfTest extends IgfsFragmentizerAbstractSelfTest {
                 for (int i = 0; i < NODE_CNT; i++) {
                     IgniteEx g = grid(i);
 
-                    GridCacheAdapter<Object, Object> cache = ((IgniteKernal)g).internalCache(DATA_CACHE_NAME);
+                    GridCacheAdapter<Object, Object> cache = ((IgniteKernal)g).internalCache(
+                        g.igfsx("igfs").configuration().getDataCacheConfiguration().getName());
 
                     assertTrue("Data cache is not empty [keys=" + cache.keySet() +
                         ", node=" + g.localNode().id() + ']', cache.isEmpty());

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java
index 374d3d3..90ac4b1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java
@@ -375,8 +375,6 @@ public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
         TcpDiscoveryIpFinder ipFinder) throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
         igfsCfg.setName(igfsName);
         igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
         igfsCfg.setDefaultMode(mode);
@@ -391,7 +389,6 @@ public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
 
         CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
 
-        dataCacheCfg.setName("dataCache");
         dataCacheCfg.setNearConfiguration(null);
         dataCacheCfg.setCacheMode(PARTITIONED);
         dataCacheCfg.setNearConfiguration(null);
@@ -404,12 +401,16 @@ public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
 
         CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
 
-        metaCacheCfg.setName("metaCache");
         metaCacheCfg.setNearConfiguration(null);
         metaCacheCfg.setCacheMode(REPLICATED);
         metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
 
+        prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
+
+        igfsCfg.setDataCacheConfiguration(dataCacheCfg);
+        igfsCfg.setMetaCacheConfiguration(metaCacheCfg);
+
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         if (useOptimizedMarshaller())
@@ -421,10 +422,7 @@ public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
 
         discoSpi.setIpFinder(ipFinder);
 
-        prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
-
         cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
         cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");
@@ -545,6 +543,7 @@ public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
     /**
      * Create the file in the given IGFS and write provided data chunks to it.
      *
+     * @param uni FS tests adaptor.
      * @param file File.
      * @param chunks Data chunks.
      * @throws IOException In case of IO exception.
@@ -942,7 +941,7 @@ public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
      * @return The data cache.
      */
     protected static GridCacheAdapter<IgfsBlockKey, byte[]> getDataCache(IgniteFileSystem igfs) {
-        String dataCacheName = igfs.configuration().getDataCacheName();
+        String dataCacheName = igfs.configuration().getDataCacheConfiguration().getName();
 
         IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
 
@@ -956,7 +955,7 @@ public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
      * @return The data cache.
      */
     protected static GridCacheAdapter<IgniteUuid, IgfsEntryInfo> getMetaCache(IgniteFileSystem igfs) {
-        String dataCacheName = igfs.configuration().getMetaCacheName();
+        String dataCacheName = igfs.configuration().getMetaCacheConfiguration().getName();
 
         IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
index 187aeeb..51b9ace 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
@@ -133,8 +133,6 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
         throws Exception {
         final FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
         igfsCfg.setName("igfs");
         igfsCfg.setBlockSize(igfsBlockSize);
         igfsCfg.setDefaultMode(mode);
@@ -142,7 +140,6 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
 
         CacheConfiguration<?,?> dataCacheCfg = defaultCacheConfiguration();
 
-        dataCacheCfg.setName("dataCache");
         dataCacheCfg.setCacheMode(PARTITIONED);
         dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(affGrpSize));
@@ -151,11 +148,13 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
 
         CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
 
-        metaCacheCfg.setName("metaCache");
         metaCacheCfg.setCacheMode(REPLICATED);
         metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
 
+        igfsCfg.setMetaCacheConfiguration(metaCacheCfg);
+        igfsCfg.setDataCacheConfiguration(dataCacheCfg);
+
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setGridName(gridName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
index 57c709b..fba78c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
@@ -57,12 +57,6 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
     /** Second node name. */
     private static final String NODE_2_NAME = "node2";
 
-    /** Data cache name. */
-    private static final String DATA_CACHE_NAME = "data";
-
-    /** Meta cache name. */
-    private static final String META_CACHE_NAME = "meta";
-
     /** Key in data caceh we will use to reproduce the issue. */
     private static final Integer DATA_KEY = 1;
 
@@ -212,13 +206,15 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
      * @throws Exception If failed.
      */
     private GridCacheAdapter dataCache(Ignite node) throws Exception  {
-        return ((IgniteKernal)node).internalCache(DATA_CACHE_NAME);
+        return ((IgniteKernal)node).internalCache(((IgniteKernal)node).igfsx(null).configuration()
+            .getDataCacheConfiguration().getName());
     }
 
     /**
      * Create node configuration.
      *
      * @param name Node name.
+     * @param ipFinder IpFinder.
      * @return Configuration.
      * @throws Exception If failed.
      */
@@ -226,7 +222,6 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
         // Data cache configuration.
         CacheConfiguration dataCcfg = new CacheConfiguration();
 
-        dataCcfg.setName(DATA_CACHE_NAME);
         dataCcfg.setCacheMode(CacheMode.REPLICATED);
         dataCcfg.setAtomicityMode(TRANSACTIONAL);
         dataCcfg.setWriteSynchronizationMode(FULL_SYNC);
@@ -236,7 +231,6 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
         // Meta cache configuration.
         CacheConfiguration metaCcfg = new CacheConfiguration();
 
-        metaCcfg.setName(META_CACHE_NAME);
         metaCcfg.setCacheMode(CacheMode.REPLICATED);
         metaCcfg.setAtomicityMode(TRANSACTIONAL);
         metaCcfg.setWriteSynchronizationMode(FULL_SYNC);
@@ -245,10 +239,10 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDefaultMode(IgfsMode.PRIMARY);
-        igfsCfg.setDataCacheName(DATA_CACHE_NAME);
-        igfsCfg.setMetaCacheName(META_CACHE_NAME);
         igfsCfg.setFragmentizerEnabled(false);
         igfsCfg.setBlockSize(1024);
+        igfsCfg.setDataCacheConfiguration(dataCcfg);
+        igfsCfg.setMetaCacheConfiguration(metaCcfg);
 
         // Ignite configuration.
         IgniteConfiguration cfg = getConfiguration(name);
@@ -258,7 +252,6 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
         discoSpi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(dataCcfg, metaCcfg);
         cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java
index 93171ba..1b8ab58 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java
@@ -98,8 +98,6 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
     private void startPrimary() throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
         igfsCfg.setName(IGFS_PRIMARY);
         igfsCfg.setBlockSize(512);
         igfsCfg.setDefaultMode(DUAL_SYNC);
@@ -115,7 +113,6 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
 
         CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
 
-        dataCacheCfg.setName("dataCache");
         dataCacheCfg.setCacheMode(PARTITIONED);
         dataCacheCfg.setNearConfiguration(null);
         dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
@@ -129,12 +126,14 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
 
         CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
 
-        metaCacheCfg.setName("metaCache");
         metaCacheCfg.setCacheMode(REPLICATED);
         metaCacheCfg.setNearConfiguration(null);
         metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
 
+        igfsCfg.setMetaCacheConfiguration(metaCacheCfg);
+        igfsCfg.setDataCacheConfiguration(dataCacheCfg);
+
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setGridName("grid-primary");
@@ -144,7 +143,6 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
         discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
 
         cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
         cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");
@@ -155,7 +153,7 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
         igfsPrimary = (IgfsImpl)g.fileSystem(IGFS_PRIMARY);
 
         dataCache = igfsPrimary.context().kernalContext().cache().internalCache(
-            igfsPrimary.context().configuration().getDataCacheName());
+            igfsPrimary.context().configuration().getDataCacheConfiguration().getName());
     }
 
     /**
@@ -166,8 +164,6 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
     private void startSecondary() throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
         igfsCfg.setName(IGFS_SECONDARY);
         igfsCfg.setBlockSize(512);
         igfsCfg.setDefaultMode(PRIMARY);
@@ -175,7 +171,6 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
 
         CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
 
-        dataCacheCfg.setName("dataCache");
         dataCacheCfg.setCacheMode(PARTITIONED);
         dataCacheCfg.setNearConfiguration(null);
         dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
@@ -185,12 +180,14 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
 
         CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
 
-        metaCacheCfg.setName("metaCache");
         metaCacheCfg.setCacheMode(REPLICATED);
         metaCacheCfg.setNearConfiguration(null);
         metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
 
+        igfsCfg.setMetaCacheConfiguration(metaCacheCfg);
+        igfsCfg.setDataCacheConfiguration(dataCacheCfg);
+
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setGridName("grid-secondary");
@@ -200,7 +197,6 @@ public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstra
         discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
 
         cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
         cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca46d1a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
index 81caa91..090c86a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
@@ -41,12 +41,6 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED;
  * Tests ensuring that IGFS data and meta caches are not "visible" through public API.
  */
 public class IgfsCacheSelfTest extends IgfsCommonAbstractTest {
-    /** Meta-information cache name. */
-    private static final String META_CACHE_NAME = "meta";
-
-    /** Data cache name. */
-    private static final String DATA_CACHE_NAME = null;
-
     /** Regular cache name. */
     private static final String CACHE_NAME = "cache";
 
@@ -54,8 +48,7 @@ public class IgfsCacheSelfTest extends IgfsCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setCacheConfiguration(cacheConfiguration(META_CACHE_NAME), cacheConfiguration(DATA_CACHE_NAME),
-            cacheConfiguration(CACHE_NAME));
+        cfg.setCacheConfiguration(cacheConfiguration(CACHE_NAME));
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
@@ -65,9 +58,9 @@ public class IgfsCacheSelfTest extends IgfsCommonAbstractTest {
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
-        igfsCfg.setMetaCacheName(META_CACHE_NAME);
-        igfsCfg.setDataCacheName(DATA_CACHE_NAME);
         igfsCfg.setName("igfs");
+        igfsCfg.setMetaCacheConfiguration(cacheConfiguration("meta"));
+        igfsCfg.setDataCacheConfiguration(cacheConfiguration("data"));
 
         cfg.setFileSystemConfiguration(igfsCfg);
 
@@ -80,7 +73,7 @@ public class IgfsCacheSelfTest extends IgfsCommonAbstractTest {
 
         cacheCfg.setName(cacheName);
 
-        if (META_CACHE_NAME.equals(cacheName))
+        if ("meta".equals(cacheName))
             cacheCfg.setCacheMode(REPLICATED);
         else {
             cacheCfg.setCacheMode(PARTITIONED);
@@ -128,7 +121,7 @@ public class IgfsCacheSelfTest extends IgfsCommonAbstractTest {
 
         GridTestUtils.assertThrows(log(), new Callable<Object>() {
             @Override public Object call() throws Exception {
-                g.cache(META_CACHE_NAME);
+                g.cache(((IgniteKernal)g).igfsx("igfs").configuration().getMetaCacheConfiguration().getName());
 
                 return null;
             }
@@ -136,7 +129,7 @@ public class IgfsCacheSelfTest extends IgfsCommonAbstractTest {
 
         GridTestUtils.assertThrows(log(), new Callable<Object>() {
             @Override public Object call() throws Exception {
-                g.cache(DATA_CACHE_NAME);
+                g.cache(((IgniteKernal)g).igfsx("igfs").configuration().getDataCacheConfiguration().getName());
 
                 return null;
             }