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/03/18 21:17:43 UTC

ignite git commit: IGNITE-2806: Implemented.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2806 [created] a5760d2d7


IGNITE-2806: Implemented.


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

Branch: refs/heads/ignite-2806
Commit: a5760d2d7c5411ad992c6b542d42956448b6bcaa
Parents: c506c44
Author: thatcoach <pp...@list.ru>
Authored: Fri Mar 18 23:17:20 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Fri Mar 18 23:17:20 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  | 49 +++++++++++++++++-
 .../processors/igfs/IgfsMetaManager.java        | 38 ++++++++------
 .../internal/processors/igfs/IgfsPathIds.java   | 52 ++++++++++++++------
 .../internal/processors/igfs/IgfsProcessor.java | 40 ++++++++-------
 .../processors/igfs/IgfsAbstractSelfTest.java   | 11 +++++
 .../igfs/IgfsPrimaryRelaxedSelfTest.java        | 28 +++++++++++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  | 30 +----------
 7 files changed, 171 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a5760d2d/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 99d364e..0d7f3cc 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
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.configuration;
 
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
@@ -27,6 +25,9 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
 /**
  * {@code IGFS} configuration. More than one file system can be configured within grid.
  * {@code IGFS} configuration is provided via {@link IgniteConfiguration#getFileSystemConfiguration()}
@@ -87,6 +88,9 @@ public class FileSystemConfiguration {
     /** Default value of metadata co-location flag. */
     public static boolean DFLT_COLOCATE_META = true;
 
+    /** Default value of relaxed consistency flag. */
+    public static boolean DFLT_RELAXED_CONSISTENCY = true;
+
     /** IGFS instance name. */
     private String name;
 
@@ -171,6 +175,9 @@ public class FileSystemConfiguration {
     /** Metadata co-location flag. */
     private boolean colocateMeta = DFLT_COLOCATE_META;
 
+    /** Relaxed consistency flag. */
+    private boolean relaxedConsistency = DFLT_RELAXED_CONSISTENCY;
+
     /**
      * Constructs default configuration.
      */
@@ -215,6 +222,7 @@ public class FileSystemConfiguration {
         perNodeBatchSize = cfg.getPerNodeBatchSize();
         perNodeParallelBatchCnt = cfg.getPerNodeParallelBatchCount();
         prefetchBlocks = cfg.getPrefetchBlocks();
+        relaxedConsistency = cfg.isRelaxedConsistency();
         seqReadsBeforePrefetch = cfg.getSequentialReadsBeforePrefetch();
         trashPurgeTimeout = cfg.getTrashPurgeTimeout();
     }
@@ -877,6 +885,43 @@ public class FileSystemConfiguration {
         this.colocateMeta = colocateMeta;
     }
 
+    /**
+     * Get relaxed consistency flag.
+     * <p>
+     * Concurrent file system operations might conflict with each other. E.g. {@code move("/a1/a2", "/b")} and
+     * {@code move("/b", "/a1")}. Hence, it is necessary to atomically verify that participating paths are still
+     * on their places to keep file system in consistent state in such cases. These checks are expensive in
+     * distributed environment.
+     * <p>
+     * Real applications, e.g. Hadoop jobs, rarely produce conflicting operations. So additional checks could be
+     * skipped in these scenarios without any negative effect on file system integrity. It significantly increases
+     * performance of file system operations.
+     * <p>
+     * If value of this flag is {@code true}, IGFS will skip expensive consistency checks. It is recommended to set
+     * this flag to {@code false} if your application has conflicting operations, or you do not how exactly users will
+     * use your system.
+     * <p>
+     * This property affects only {@link IgfsMode#PRIMARY} paths.
+     * <p>
+     * Defaults to {@link #DFLT_RELAXED_CONSISTENCY}.
+     *
+     * @return {@code True} if relaxed consistency is enabled.
+     */
+    public boolean isRelaxedConsistency() {
+        return relaxedConsistency;
+    }
+
+    /**
+     * Set relaxed consistency flag.
+     * <p>
+     * See {@link #isColocateMetadata()} for more information.
+     *
+     * @param relaxedConsistency Whether to use relaxed consistency optimization.
+     */
+    public void setRelaxedConsistency(boolean relaxedConsistency) {
+        this.relaxedConsistency = relaxedConsistency;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(FileSystemConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5760d2d/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 d66d9be..283a113 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
@@ -135,8 +135,20 @@ public class IgfsMetaManager extends IgfsManager {
     /** Busy lock. */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
+    /** Relaxed flag. */
+    private final boolean relaxed;
+
     /**
+     * Constructor.
      *
+     * @param relaxed Relaxed mode flag.
+     */
+    public IgfsMetaManager(boolean relaxed) {
+        this.relaxed = relaxed;
+    }
+
+    /**
+     * Await initialization.
      */
     void awaitInit() {
         try {
@@ -895,8 +907,8 @@ public class IgfsMetaManager extends IgfsManager {
                 // Lock participating IDs.
                 final Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                srcPathIds.addExistingIds(lockIds);
-                dstPathIds.addExistingIds(lockIds);
+                srcPathIds.addExistingIds(lockIds, relaxed);
+                dstPathIds.addExistingIds(lockIds, relaxed);
 
                 IgniteInternalTx tx = startTx();
 
@@ -905,11 +917,11 @@ public class IgfsMetaManager extends IgfsManager {
                     final Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                     // Verify integrity of source and destination paths.
-                    if (!srcPathIds.verifyIntegrity(lockInfos))
+                    if (!srcPathIds.verifyIntegrity(lockInfos, relaxed))
                         throw new IgfsPathNotFoundException("Failed to perform move because source directory " +
                             "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']');
 
-                    if (!dstPathIds.verifyIntegrity(lockInfos))
+                    if (!dstPathIds.verifyIntegrity(lockInfos, relaxed))
                         throw new IgfsPathNotFoundException("Failed to perform move because destination directory " +
                             "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']');
 
@@ -1153,7 +1165,7 @@ public class IgfsMetaManager extends IgfsManager {
                 // Prepare IDs to lock.
                 SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                pathIds.addExistingIds(allIds);
+                pathIds.addExistingIds(allIds, relaxed);
 
                 IgniteUuid trashId = IgfsUtils.randomTrashId();
 
@@ -1166,7 +1178,7 @@ public class IgfsMetaManager extends IgfsManager {
                     Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
 
                     // Ensure that all participants are still in place.
-                    if (!pathIds.verifyIntegrity(lockInfos))
+                    if (!pathIds.verifyIntegrity(lockInfos, relaxed))
                         return null;
 
                     IgfsEntryInfo victimInfo = lockInfos.get(victimId);
@@ -1673,18 +1685,16 @@ public class IgfsMetaManager extends IgfsManager {
                     // Prepare lock IDs. Essentially, they consist of two parts: existing IDs and potential new IDs.
                     Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                    pathIds.addExistingIds(lockIds);
+                    pathIds.addExistingIds(lockIds, relaxed);
                     pathIds.addSurrogateIds(lockIds);
 
-                    assert lockIds.size() == pathIds.count();
-
                     // Start TX.
                     IgniteInternalTx tx = startTx();
 
                     try {
                         final Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
-                        if (!pathIds.verifyIntegrity(lockInfos))
+                        if (!pathIds.verifyIntegrity(lockInfos, relaxed))
                             // Directory structure changed concurrently. So we simply re-try.
                             continue;
 
@@ -3405,7 +3415,7 @@ public class IgfsMetaManager extends IgfsManager {
                     // Prepare lock IDs.
                     Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                    pathIds.addExistingIds(lockIds);
+                    pathIds.addExistingIds(lockIds, relaxed);
                     pathIds.addSurrogateIds(lockIds);
 
                     // Start TX.
@@ -3414,7 +3424,7 @@ public class IgfsMetaManager extends IgfsManager {
                     try {
                         Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
-                        if (!pathIds.verifyIntegrity(lockInfos))
+                        if (!pathIds.verifyIntegrity(lockInfos, relaxed))
                             // Directory structure changed concurrently. So we simply re-try.
                             continue;
 
@@ -3502,7 +3512,7 @@ public class IgfsMetaManager extends IgfsManager {
                     // Prepare lock IDs.
                     Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                    pathIds.addExistingIds(lockIds);
+                    pathIds.addExistingIds(lockIds, relaxed);
                     pathIds.addSurrogateIds(lockIds);
 
                     // In overwrite mode we also lock ID of potential replacement as well as trash ID.
@@ -3523,7 +3533,7 @@ public class IgfsMetaManager extends IgfsManager {
                     try {
                         Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
-                        if (!pathIds.verifyIntegrity(lockInfos))
+                        if (!pathIds.verifyIntegrity(lockInfos, relaxed))
                             // Directory structure changed concurrently. So we simply re-try.
                             continue;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5760d2d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
index 2903239..1d447c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
@@ -221,11 +221,20 @@ public class IgfsPathIds {
      * Add existing IDs to provided collection.
      *
      * @param col Collection.
+     * @param relaxed Relaxed mode flag.
      */
     @SuppressWarnings("ManualArrayToCollectionCopy")
-    public void addExistingIds(Collection<IgniteUuid> col) {
-        for (int i = 0; i <= lastExistingIdx; i++)
-            col.add(ids[i]);
+    public void addExistingIds(Collection<IgniteUuid> col, boolean relaxed) {
+        if (relaxed) {
+            col.add(ids[lastExistingIdx]);
+
+            if (lastExistingIdx == ids.length - 1 && lastExistingIdx > 0)
+                col.add(ids[lastExistingIdx - 1]);
+        }
+        else {
+            for (int i = 0; i <= lastExistingIdx; i++)
+                col.add(ids[i]);
+        }
     }
 
     /**
@@ -265,24 +274,39 @@ public class IgfsPathIds {
      * Verify that observed paths are found in provided infos in the right order.
      *
      * @param infos Info.
+     * @param relaxed Whether to perform check in relaxed mode.
      * @return {@code True} if full integrity is preserved.
      */
-    public boolean verifyIntegrity(Map<IgniteUuid, IgfsEntryInfo> infos) {
-        for (int i = 0; i <= lastExistingIdx; i++) {
-            IgniteUuid curId = ids[i];
-            IgfsEntryInfo curInfo = infos.get(curId);
+    public boolean verifyIntegrity(Map<IgniteUuid, IgfsEntryInfo> infos, boolean relaxed) {
+        if (relaxed) {
+            // Relaxed mode ensures that last element is there and if it is not root - whether it's parent there
+            // and has link to child.
+            IgfsEntryInfo info = infos.get(ids[lastExistingIdx]);
 
-            // Check if required ID is there.
-            if (curInfo == null)
+            if (info == null)
                 return false;
 
-            // For non-leaf entry we check if child exists.
-            if (i < lastExistingIdx) {
-                String childName = parts[i + 1];
-                IgniteUuid childId = ids[i + 1];
+            if (lastExistingIdx == ids.length - 1 && lastExistingIdx > 0) {
+                IgfsEntryInfo parentInfo = infos.get(ids[lastExistingIdx - 1]);
+
+                if (parentInfo == null || !parentInfo.hasChild(parts[lastExistingIdx], ids[lastExistingIdx]))
+                    return false;
+            }
+        }
+        else {
+            // Strict mode ensures that all participants are in place and are still linked.
+            for (int i = 0; i <= lastExistingIdx; i++) {
+                IgfsEntryInfo info = infos.get(ids[i]);
 
-                if (!curInfo.hasChild(childName, childId))
+                // Check if required ID is there.
+                if (info == null)
                     return false;
+
+                // For non-leaf entry we check if child exists.
+                if (i < lastExistingIdx) {
+                    if (!info.hasChild(parts[i + 1], ids[i + 1]))
+                        return false;
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5760d2d/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 1b60252..5008c74 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
@@ -17,16 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-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;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.cache.affinity.AffinityKeyMapper;
@@ -53,12 +43,22 @@ import org.apache.ignite.lang.IgniteClosure;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-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.cache.CacheMemoryMode.OFFHEAP_VALUES;
-import static org.apache.ignite.igfs.IgfsMode.PROXY;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
+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;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.ignite.IgniteSystemProperties.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.IgniteNodeAttributes.*;
 
 /**
  * Fully operational Ignite file system processor.
@@ -104,10 +104,12 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
         // Start IGFS instances.
         for (FileSystemConfiguration cfg : cfgs) {
+            FileSystemConfiguration cfg0 = new FileSystemConfiguration(cfg);
+
             IgfsContext igfsCtx = new IgfsContext(
                 ctx,
-                new FileSystemConfiguration(cfg),
-                new IgfsMetaManager(),
+                cfg0,
+                new IgfsMetaManager(cfg0.isRelaxedConsistency()),
                 new IgfsDataManager(),
                 new IgfsServerManager(),
                 new IgfsFragmentizerManager());
@@ -116,7 +118,7 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
             for (IgfsManager mgr : igfsCtx.managers())
                 mgr.start(igfsCtx);
 
-            igfsCache.put(maskName(cfg.getName()), igfsCtx);
+            igfsCache.put(maskName(cfg0.getName()), igfsCtx);
         }
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5760d2d/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 edec572..350828a 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
@@ -228,6 +228,13 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * @return Relaxed consistency flag.
+     */
+    protected boolean relaxedConsistency() {
+        return false;
+    }
+
+    /**
      * Data chunk.
      *
      * @param len Length.
@@ -305,6 +312,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setSecondaryFileSystem(secondaryFs);
         igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS);
         igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
+        igfsCfg.setRelaxedConsistency(relaxedConsistency());
 
         CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
 
@@ -2368,6 +2376,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     private void checkDeadlocksRepeat(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl,
         int primaryLvlCnt, int renCnt, int delCnt,
         int updateCnt, int mkdirsCnt, int createCnt) throws Exception {
+        if (relaxedConsistency())
+            return;
+
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
                 checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt,

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5760d2d/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java
new file mode 100644
index 0000000..cf6cd14
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.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.
+ */
+public class IgfsPrimaryRelaxedSelfTest extends IgfsPrimarySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean relaxedConsistency() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5760d2d/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 489088c..88aea31 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
@@ -20,34 +20,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.igfs.IgfsFragmentizerSelfTest;
 import org.apache.ignite.igfs.IgfsFragmentizerTopologySelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsAttributesSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsBackupsDualAsyncSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsBackupsDualSyncSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsBackupsPrimarySelfTest;
-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.IgfsDataManagerSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsDualSyncSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsFileInfoSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsFileMapSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsGroupDataBlockKeyMapperHashSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsMetaManagerSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsMetricsSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsModeResolverSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsModesSelfTest;
-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.IgfsPrimarySelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsProcessorSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsProcessorValidationSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsSizeSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsStartCacheTest;
-import org.apache.ignite.internal.processors.igfs.IgfsStreamsSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsTaskSelfTest;
+import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.processors.igfs.split.IgfsByteDelimiterRecordResolverSelfTest;
 import org.apache.ignite.internal.processors.igfs.split.IgfsFixedLengthRecordResolverSelfTest;
 import org.apache.ignite.internal.processors.igfs.split.IgfsNewLineDelimiterRecordResolverSelfTest;
@@ -67,6 +40,7 @@ public class IgniteIgfsTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Ignite FS Test Suite For Platform Independent Tests");
 
         suite.addTest(new TestSuite(IgfsPrimarySelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryRelaxedSelfTest.class));
         suite.addTest(new TestSuite(IgfsPrimaryOffheapTieredSelfTest.class));
         suite.addTest(new TestSuite(IgfsPrimaryOffheapValuesSelfTest.class));
         suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class));