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 2016/06/05 16:17:21 UTC

[1/2] ignite git commit: WIP on tests.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3247 526ac2212 -> 14cf63fb3


WIP on tests.


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

Branch: refs/heads/ignite-3247
Commit: 7d9ceece656b29bdd3606bb2df9af5f8ce862f94
Parents: 526ac22
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sun Jun 5 19:09:29 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sun Jun 5 19:09:29 2016 +0300

----------------------------------------------------------------------
 .../igfs/client/IgfsClientMkdirsCallable.java   |   1 +
 .../igfs/client/IgfsClientUpdateCallable.java   |   1 +
 .../processors/igfs/IgfsAbstractSelfTest.java   |  18 ++-
 .../igfs/IgfsClientCacheSelfTest.java           | 139 -------------------
 .../igfs/IgfsPrimaryClientSelfTest.java         |  57 ++++++++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   4 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   2 +-
 7 files changed, 76 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7d9ceece/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
index 530353d..944da6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
@@ -53,6 +53,7 @@ public class IgfsClientMkdirsCallable extends IgfsClientAbstractCallable<Void> {
      * @param props Properties.
      */
     public IgfsClientMkdirsCallable(@Nullable String igfsName, IgfsPath path, @Nullable Map<String, String> props) {
+        super(igfsName, path);
 
         this.props = props;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d9ceece/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
index 0c75e07..4acf4eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
@@ -54,6 +54,7 @@ public class IgfsClientUpdateCallable extends IgfsClientAbstractCallable<IgfsFil
      * @param props Properties.
      */
     public IgfsClientUpdateCallable(@Nullable String igfsName, IgfsPath path, @Nullable Map<String, String> props) {
+        super(igfsName, path);
 
         this.props = props;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d9ceece/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 2f6c7bf..f628ea6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -55,6 +55,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
@@ -189,6 +190,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /** Memory mode. */
     protected final CacheMemoryMode memoryMode;
 
+    /** IP finder for primary topology. */
+    protected final TcpDiscoveryVmIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** IP finder for secondary topology. */
+    protected final TcpDiscoveryVmIpFinder secondaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+
     static {
         PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration();
 
@@ -258,7 +265,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         igfsSecondaryFileSystem = createSecondaryFileSystemStack();
 
-        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG);
+        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem,
+            PRIMARY_REST_CFG, primaryIpFinder);
 
         igfs = (IgfsImpl) ignite.fileSystem("igfs");
     }
@@ -271,7 +279,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      */
     protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
         Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
-            SECONDARY_REST_CFG);
+            SECONDARY_REST_CFG, secondaryIpFinder);
 
         IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary");
 
@@ -300,12 +308,14 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param mode IGFS mode.
      * @param secondaryFs Secondary file system (optional).
      * @param restCfg Rest configuration string (optional).
+     * @param ipFinder IP finder.
      * @return Started grid instance.
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
     protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
-        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg) throws Exception {
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg,
+        TcpDiscoveryIpFinder ipFinder) throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
@@ -347,7 +357,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+        discoSpi.setIpFinder(ipFinder);
 
         prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d9ceece/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
deleted file mode 100644
index 8e8eac1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.igfs;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-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.IgfsMode;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
-import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-
-/**
- * Test for igfs with nodes in client mode (see {@link IgniteConfiguration#setClientMode(boolean)}.
- */
-public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Meta-information cache name. */
-    private static final String META_CACHE_NAME = "meta";
-
-    /** Data cache name. */
-    private static final String DATA_CACHE_NAME = null;
-
-    /**
-     * Constructor.
-     */
-    public IgfsClientCacheSelfTest() {
-        super(IgfsMode.PRIMARY);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        igfsSecondaryFileSystem = createSecondaryFileSystemStack();
-
-        Ignite ignitePrimary = G.start(getConfiguration(getTestGridName(1)));
-
-        igfs = (IgfsImpl) ignitePrimary.fileSystem("igfs");
-    }
-
-    /**{@inheritDoc} */
-    protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
-        Ignite igniteSecondary = G.start(getConfiguration(getTestGridName(0)));
-
-        IgfsEx secondaryIgfsImpl = (IgfsEx)igniteSecondary.fileSystem("igfs");
-
-        igfsSecondary = new IgfsExUniversalFileSystemAdapter(secondaryIgfsImpl);
-
-        return secondaryIgfsImpl.asSecondary();
-    }
-
-    /**
-     *
-     * @param gridName Grid name.
-     * @return Ignite configuration.
-     * @throws Exception If failed.
-     */
-    protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setCacheConfiguration(
-            cacheConfiguration(META_CACHE_NAME),
-            cacheConfiguration(DATA_CACHE_NAME)
-        );
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        if (!gridName.equals(getTestGridName(0))) {
-            cfg.setClientMode(true);
-
-            disco.setForceServerMode(true);
-        }
-
-        cfg.setDiscoverySpi(disco);
-
-        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
-
-        igfsCfg.setMetaCacheName(META_CACHE_NAME);
-        igfsCfg.setDataCacheName(DATA_CACHE_NAME);
-        igfsCfg.setName("igfs");
-
-        cfg.setFileSystemConfiguration(igfsCfg);
-
-        return cfg;
-    }
-
-    /**
-     * @param cacheName Cache name.
-     * @return Cache configuration.
-     */
-    protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName(cacheName);
-
-        cacheCfg.setNearConfiguration(null);
-
-        if (META_CACHE_NAME.equals(cacheName))
-            cacheCfg.setCacheMode(REPLICATED);
-        else {
-            cacheCfg.setCacheMode(PARTITIONED);
-
-            cacheCfg.setBackups(0);
-            cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        }
-
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        return cacheCfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d9ceece/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
new file mode 100644
index 0000000..8eac994
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.igfs.IgfsMode;
+
+/**
+ * Test for igfs with nodes in client mode (see {@link IgniteConfiguration#setClientMode(boolean)}.
+ */
+public class IgfsPrimaryClientSelfTest extends IgfsAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgfsPrimaryClientSelfTest() {
+        super(IgfsMode.PRIMARY);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        igfsSecondaryFileSystem = createSecondaryFileSystemStack();
+
+        // Start server.
+        startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem, null, primaryIpFinder);
+
+        // Start client.
+
+        Ignition.setClientMode(true);
+
+        try {
+            Ignite cliIgnite = startGridWithIgfs("ignite_cli", "igfs", mode, igfsSecondaryFileSystem,
+                PRIMARY_REST_CFG, primaryIpFinder);
+
+            igfs = (IgfsImpl) cliIgnite.fileSystem("igfs");
+        }
+        finally {
+            Ignition.setClientMode(false);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d9ceece/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index 25c54e4..c083b89 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -27,7 +27,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsBackupsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsBlockMessageSystemPoolStarvationSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsCachePerBlockLruEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsCacheSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsClientCacheSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryClientSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDataManagerSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDualSyncSelfTest;
@@ -95,7 +95,7 @@ public class IgniteIgfsTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgfsModesSelfTest.class));
         suite.addTest(new TestSuite(IgfsMetricsSelfTest.class));
 
-        suite.addTest(new TestSuite(IgfsClientCacheSelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryClientSelfTest.class));
         suite.addTest(new TestSuite(IgfsOneClientNodeTest.class));
 
         suite.addTest(new TestSuite(IgfsModeResolverSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d9ceece/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
index 81f5743..c5f3425 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
@@ -110,7 +110,7 @@ public abstract class Hadoop1DualAbstractTest extends IgfsDualAbstractSelfTest {
      * @throws IOException On failure.
      */
     protected void startUnderlying() throws Exception {
-        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG);
+        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG, secondaryIpFinder);
     }
 
     /**


[2/2] ignite git commit: Added tests

Posted by vo...@apache.org.
Added tests


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

Branch: refs/heads/ignite-3247
Commit: 14cf63fb3a7860c5f8b0bbdd6a96226885971a0f
Parents: 7d9ceec
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sun Jun 5 19:17:15 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sun Jun 5 19:17:15 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsAbstractSelfTest.java   | 33 ++++++++++++++++--
 .../igfs/IgfsDualAsyncClientSelfTest.java       | 28 ++++++++++++++++
 .../igfs/IgfsDualSyncClientSelfTest.java        | 28 ++++++++++++++++
 .../igfs/IgfsPrimaryClientSelfTest.java         | 35 +++-----------------
 .../igfs/IgfsPrimaryRelaxedClientSelfTest.java  | 28 ++++++++++++++++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  7 ++++
 6 files changed, 125 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/14cf63fb/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index f628ea6..4ada318 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
@@ -240,6 +241,13 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * @return Client flag.
+     */
+    protected boolean client() {
+        return false;
+    }
+
+    /**
      * @return Use optimzied marshaller flag.
      */
     protected boolean useOptimizedMarshaller() {
@@ -265,10 +273,29 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         igfsSecondaryFileSystem = createSecondaryFileSystemStack();
 
-        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem,
-            PRIMARY_REST_CFG, primaryIpFinder);
+        if (client()) {
+            // Start server.
+            startGridWithIgfs("ignite_srv", "igfs", mode, igfsSecondaryFileSystem, null, primaryIpFinder);
+
+            // Start client.
+            Ignition.setClientMode(true);
+
+            try {
+                Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem,
+                    PRIMARY_REST_CFG, primaryIpFinder);
 
-        igfs = (IgfsImpl) ignite.fileSystem("igfs");
+                igfs = (IgfsImpl) ignite.fileSystem("igfs");
+            }
+            finally {
+                Ignition.setClientMode(false);
+            }
+        }
+        else {
+            Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem,
+                PRIMARY_REST_CFG, primaryIpFinder);
+
+            igfs = (IgfsImpl) ignite.fileSystem("igfs");
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/14cf63fb/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java
new file mode 100644
index 0000000..e453346
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * Tests for DUAL_ASYNC mode and client cache.
+ */
+public class IgfsDualAsyncClientSelfTest extends IgfsDualAsyncSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/14cf63fb/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java
new file mode 100644
index 0000000..bb3da32
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * Tests for DUAL_SYNC mode and client cache.
+ */
+public class IgfsDualSyncClientSelfTest extends IgfsDualSyncSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/14cf63fb/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
index 8eac994..12fd93c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
@@ -17,41 +17,14 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import org.apache.ignite.Ignite;
-import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.igfs.IgfsMode;
 
 /**
- * Test for igfs with nodes in client mode (see {@link IgniteConfiguration#setClientMode(boolean)}.
+ * Test for IGFS with nodes in client mode (see {@link IgniteConfiguration#setClientMode(boolean)}.
  */
-public class IgfsPrimaryClientSelfTest extends IgfsAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsPrimaryClientSelfTest() {
-        super(IgfsMode.PRIMARY);
-    }
-
+public class IgfsPrimaryClientSelfTest extends IgfsPrimarySelfTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        igfsSecondaryFileSystem = createSecondaryFileSystemStack();
-
-        // Start server.
-        startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem, null, primaryIpFinder);
-
-        // Start client.
-
-        Ignition.setClientMode(true);
-
-        try {
-            Ignite cliIgnite = startGridWithIgfs("ignite_cli", "igfs", mode, igfsSecondaryFileSystem,
-                PRIMARY_REST_CFG, primaryIpFinder);
-
-            igfs = (IgfsImpl) cliIgnite.fileSystem("igfs");
-        }
-        finally {
-            Ignition.setClientMode(false);
-        }
+    @Override protected boolean client() {
+        return true;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/14cf63fb/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.java
new file mode 100644
index 0000000..64bce8e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * Tests for PRIMARY mode and relaxed consistency model with client cache.
+ */
+public class IgfsPrimaryRelaxedClientSelfTest extends IgfsPrimaryRelaxedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/14cf63fb/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index c083b89..48eb665 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -27,6 +27,8 @@ import org.apache.ignite.internal.processors.igfs.IgfsBackupsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsBlockMessageSystemPoolStarvationSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsCachePerBlockLruEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsCacheSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncClientSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsDualSyncClientSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryClientSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDataManagerSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncSelfTest;
@@ -42,6 +44,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsOneClientNodeTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapTieredSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapValuesSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOptimziedMarshallerSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedClientSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorSelfTest;
@@ -96,6 +99,10 @@ public class IgniteIgfsTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgfsMetricsSelfTest.class));
 
         suite.addTest(new TestSuite(IgfsPrimaryClientSelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryRelaxedClientSelfTest.class));
+        suite.addTest(new TestSuite(IgfsDualSyncClientSelfTest.class));
+        suite.addTest(new TestSuite(IgfsDualAsyncClientSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsOneClientNodeTest.class));
 
         suite.addTest(new TestSuite(IgfsModeResolverSelfTest.class));