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/22 10:34:46 UTC

ignite git commit: IGNITE-2806: IGFS: Implemented relaxed consistency model.

Repository: ignite
Updated Branches:
  refs/heads/master c3b02dca7 -> 2694c3ce1


IGNITE-2806: IGFS: Implemented relaxed consistency model.


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

Branch: refs/heads/master
Commit: 2694c3ce1e2c55151dac741a5d162b1e73239e52
Parents: c3b02dc
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Mar 22 12:34:35 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Mar 22 12:34:35 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  | 49 +++++++++++++++++-
 .../processors/igfs/IgfsMetaManager.java        | 38 ++++++++------
 .../internal/processors/igfs/IgfsPathIds.java   | 52 ++++++++++++++------
 .../internal/processors/igfs/IgfsProcessor.java | 29 ++++++-----
 .../processors/igfs/IgfsAbstractSelfTest.java   | 11 +++++
 .../igfs/IgfsPrimaryRelaxedSelfTest.java        | 28 +++++++++++
 .../processors/igfs/IgfsProcessorSelfTest.java  | 17 ++++---
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  2 +
 8 files changed, 177 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2694c3ce/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/2694c3ce/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 d6c5995..a4212ba 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
@@ -136,8 +136,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 {
@@ -889,19 +901,19 @@ 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);
 
                 try (IgniteInternalTx tx = startTx()) {
                     // Obtain the locks.
                     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 + ']');
 
@@ -1103,7 +1115,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();
 
@@ -1114,7 +1126,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);
@@ -1589,16 +1601,14 @@ 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.
                     try (IgniteInternalTx tx = startTx()) {
                         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;
 
@@ -2907,14 +2917,14 @@ 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.
                     try (IgniteInternalTx tx = startTx()) {
                         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;
 
@@ -2998,7 +3008,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.
@@ -3017,7 +3027,7 @@ public class IgfsMetaManager extends IgfsManager {
                     try (IgniteInternalTx tx = startTx()) {
                         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/2694c3ce/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 e2fe58d..446495e 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 the last element is there. If this element is the last in the path, then
+            // existence of it's parent and link between them are checked as well.
+            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/2694c3ce/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..44f6e44 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,6 +43,17 @@ import org.apache.ignite.lang.IgniteClosure;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
+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.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -104,10 +105,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 +119,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/2694c3ce/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 5894fa2..ec3878c 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
@@ -225,6 +225,13 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * @return Relaxed consistency flag.
+     */
+    protected boolean relaxedConsistency() {
+        return false;
+    }
+
+    /**
      * Data chunk.
      *
      * @param len Length.
@@ -302,6 +309,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/2694c3ce/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..6691df5
--- /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 and relaxed consistency model.
+ */
+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/2694c3ce/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
index bcc2314..269706e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
@@ -35,6 +35,7 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -779,14 +780,18 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
         assert !igfs.exists(path(dirPath));
         assert !igfs.exists(path(filePath));
 
-        int metaSize = 0;
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                int metaSize = 0;
 
-        for (Object metaId : grid(0).cachex(igfs.configuration().getMetaCacheName()).keySet()) {
-            if (!IgfsUtils.isRootOrTrashId((IgniteUuid)metaId))
-                metaSize++;
-        }
+                for (Object metaId : grid(0).cachex(igfs.configuration().getMetaCacheName()).keySet()) {
+                    if (!IgfsUtils.isRootOrTrashId((IgniteUuid)metaId))
+                        metaSize++;
+                }
 
-        assert metaSize == 0;
+                return metaSize == 0;
+            }
+        }, 5000);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/2694c3ce/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..038cb54 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
@@ -40,6 +40,7 @@ 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.IgfsPrimaryRelaxedSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorValidationSelfTest;
@@ -67,6 +68,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));