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/08/02 07:41:30 UTC

[07/14] ignite git commit: IGNITE-3343: fixed listings when a primary path is an immediate child of secondary.

IGNITE-3343: fixed listings when a primary path is an immediate child of secondary.


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

Branch: refs/heads/ignite-3343
Commit: ace2e7154086c781851c89de00fb7aa5d0171e5b
Parents: ae19bfe
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Jul 27 21:16:19 2016 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Wed Jul 27 21:16:19 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsImpl.java      | 67 ++++++--------------
 .../processors/igfs/IgfsMetaManager.java        | 30 ++++-----
 .../processors/igfs/IgfsModeResolver.java       | 23 ++++++-
 .../internal/processors/igfs/IgfsUtils.java     | 14 +++-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  9 +++
 .../igfs/IgfsDualAbstractSelfTest.java          | 12 ++++
 .../igfs/IgfsModeResolverSelfTest.java          | 45 +++++++++++--
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  7 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  7 +-
 9 files changed, 138 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/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 3779e68..0c7427c 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
@@ -248,7 +248,7 @@ public final class IgfsImpl implements IgfsEx {
             }
         }
 
-        modeRslvr = new IgfsModeResolver(dfltMode, IgfsUtils.preparePathModes(dfltMode, modes));
+        modeRslvr = new IgfsModeResolver(dfltMode, modes);
 
         Object secondaryFsPayload = null;
 
@@ -809,12 +809,14 @@ public final class IgfsImpl implements IgfsEx {
                     }
                 }
 
-                IgniteUuid fileId = meta.fileId(path);
+                if (!IgfsUtils.isDualMode(mode) || modeRslvr.hasPrimaryChild(path)) {
+                    IgniteUuid fileId = meta.fileId(path);
 
-                if (fileId != null)
-                    files.addAll(meta.directoryListing(fileId).keySet());
-                else if (mode == PRIMARY)
-                    throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
+                    if (fileId != null)
+                        files.addAll(meta.directoryListing(fileId).keySet());
+                    else if (mode == PRIMARY)
+                        throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
+                }
 
                 return F.viewReadOnly(files, new C1<String, IgfsPath>() {
                     @Override public IgfsPath apply(String e) {
@@ -853,7 +855,8 @@ public final class IgfsImpl implements IgfsEx {
                             files.add(impl);
                         }
 
-                        return files;
+                        if (!modeRslvr.hasPrimaryChild(path))
+                            return files;
                     }
                     catch (Exception e) {
                         U.error(log, "List files in DUAL mode failed [path=" + path + ']', e);
@@ -871,8 +874,7 @@ public final class IgfsImpl implements IgfsEx {
                     if (info != null) {
                         if (info.isFile())
                             // If this is a file, return its description.
-                            return Collections.<IgfsFile>singleton(new IgfsFileImpl(path, info,
-                                data.groupBlockSize()));
+                            return Collections.<IgfsFile>singleton(new IgfsFileImpl(path, info, data.groupBlockSize()));
 
                         // Perform the listing.
                         for (Map.Entry<String, IgfsListingEntry> e : info.listing().entrySet()) {
@@ -930,8 +932,8 @@ public final class IgfsImpl implements IgfsEx {
 
                     IgfsSecondaryInputStreamDescriptor desc = meta.openDual(secondaryFs, path, bufSize0);
 
-                    IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, desc.info(),
-                        cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, desc.reader(), metrics);
+                    IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, desc.info(), cfg
+                        .getPrefetchBlocks(), seqReadsBeforePrefetch, desc.reader(), metrics);
 
                     IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ);
 
@@ -947,8 +949,8 @@ public final class IgfsImpl implements IgfsEx {
                     throw new IgfsPathIsDirectoryException("Failed to open file (not a file): " + path);
 
                 // Input stream to read data from grid cache with separate blocks.
-                IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, info,
-                    cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, null, metrics);
+                IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, info, cfg
+                    .getPrefetchBlocks(), seqReadsBeforePrefetch, null, metrics);
 
                 IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ);
 
@@ -1022,24 +1024,16 @@ public final class IgfsImpl implements IgfsEx {
                 IgfsSecondaryFileSystemCreateContext secondaryCtx = null;
 
                 if (mode != PRIMARY)
-                    secondaryCtx = new IgfsSecondaryFileSystemCreateContext(secondaryFs, path, overwrite, simpleCreate,
-                        fileProps, (short)replication, groupBlockSize(), bufSize);
+                    secondaryCtx = new IgfsSecondaryFileSystemCreateContext(secondaryFs, path, overwrite,
+                        simpleCreate, fileProps, (short)replication, groupBlockSize(), bufSize);
 
                 // Await for async ops completion if in DUAL mode.
                 if (mode != PRIMARY)
                     await(path);
 
                 // Perform create.
-                IgfsCreateResult res = meta.create(
-                    path,
-                    dirProps,
-                    overwrite,
-                    cfg.getBlockSize(),
-                    affKey,
-                    evictExclude(path, mode == PRIMARY),
-                    fileProps,
-                    secondaryCtx
-                );
+                IgfsCreateResult res = meta.create(path, dirProps, overwrite, cfg.getBlockSize(), affKey,
+                    evictExclude(path, mode == PRIMARY), fileProps, secondaryCtx);
 
                 assert res != null;
 
@@ -1340,29 +1334,6 @@ public final class IgfsImpl implements IgfsEx {
         return fut;
     }
 
-    /**
-     * Get file descriptor for specified path.
-     *
-     * @param path Path to file.
-     * @return Detailed file descriptor or {@code null}, if file does not exist.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable private FileDescriptor getFileDescriptor(IgfsPath path) throws IgniteCheckedException {
-        assert path != null;
-
-        List<IgniteUuid> ids = meta.idsForPath(path);
-
-        IgfsEntryInfo fileInfo = meta.info(ids.get(ids.size() - 1));
-
-        if (fileInfo == null)
-            return null; // File does not exist.
-
-        // Resolve parent ID for removed file.
-        IgniteUuid parentId = ids.size() >= 2 ? ids.get(ids.size() - 2) : null;
-
-        return new FileDescriptor(parentId, path.name(), fileInfo.id(), fileInfo.isFile());
-    }
-
     /** {@inheritDoc} */
     @Override public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
         Collection<IgfsPath> paths, @Nullable T arg) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/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 c0f152c..0d14a3a 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
@@ -114,12 +114,6 @@ public class IgfsMetaManager extends IgfsManager {
         }
     };
 
-    /** Used for access and modification time for meta cache in dual modes. */
-    private static final long UNKNOWN_TIME = -1L;
-
-    /** Unknown file properties. */
-    private static final Map<String, String> UNKNOWN_PROPERTIES = null;
-
     /** IGFS configuration. */
     private FileSystemConfiguration cfg;
 
@@ -3057,9 +3051,9 @@ public class IgfsMetaManager extends IgfsManager {
                             if (secondaryCtx != null) {
                                 secondaryOut = secondaryCtx.create();
 
-                                newAccessTime = UNKNOWN_TIME;
-                                newModificationTime = UNKNOWN_TIME;
-                                newProps = UNKNOWN_PROPERTIES;
+                                newAccessTime = 0L;
+                                newModificationTime = 0L;
+                                newProps = null;
                             }
                             else {
                                 newAccessTime = System.currentTimeMillis();
@@ -3250,9 +3244,9 @@ public class IgfsMetaManager extends IgfsManager {
             Map<String, String> props;
 
             if (secondaryCtx != null) {
-                accessTime = UNKNOWN_TIME;
-                modificationTime = UNKNOWN_TIME;
-                props = UNKNOWN_PROPERTIES;
+                accessTime = 0L;
+                modificationTime = 0L;
+                props = null;
             }
             else {
                 accessTime = curTime;
@@ -3280,9 +3274,9 @@ public class IgfsMetaManager extends IgfsManager {
             Map<String, String> props;
 
             if (secondaryCtx != null) {
-                accessTime = UNKNOWN_TIME;
-                modificationTime = UNKNOWN_TIME;
-                props = UNKNOWN_PROPERTIES;
+                accessTime = 0L;
+                modificationTime = 0L;
+                props = null;
             }
             else {
                 accessTime = curTime;
@@ -3300,9 +3294,9 @@ public class IgfsMetaManager extends IgfsManager {
             int newBlockSize;
 
             if (secondaryCtx != null) {
-                newAccessTime = UNKNOWN_TIME;
-                newModificationTime = UNKNOWN_TIME;
-                newProps = UNKNOWN_PROPERTIES;
+                newAccessTime = 0L;
+                newModificationTime = 0L;
+                newProps = null;
             }
             else {
                 newAccessTime = curTime;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java
index 907051f..978c725 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java
@@ -18,8 +18,11 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
@@ -42,18 +45,24 @@ public class IgfsModeResolver {
     /** Cached modes per path. */
     private Map<IgfsPath, IgfsMode> modesCache;
 
+    /** Set to store parent dual paths that have primary children. */
+    private final Set<IgfsPath> dualParentsWithPrimaryChildren;
+
     /**
      * Constructor
      *
      * @param dfltMode Default IGFS mode.
      * @param modes List of configured modes. The order is significant as modes are added in order of occurrence.
      */
-    public IgfsModeResolver(IgfsMode dfltMode, @Nullable List<T2<IgfsPath, IgfsMode>> modes) {
+    public IgfsModeResolver(IgfsMode dfltMode, @Nullable List<T2<IgfsPath, IgfsMode>> modes)
+            throws IgniteCheckedException {
         assert dfltMode != null;
 
         this.dfltMode = dfltMode;
 
-        this.modes = modes;
+        this.dualParentsWithPrimaryChildren = new HashSet<>();
+
+        this.modes = IgfsUtils.preparePathModes(dfltMode, modes, dualParentsWithPrimaryChildren);
 
         if (modes != null)
             modesCache = new GridBoundedConcurrentLinkedHashMap<>(MAX_PATH_CACHE);
@@ -100,4 +109,14 @@ public class IgfsModeResolver {
     @Nullable public List<T2<IgfsPath, IgfsMode>> modesOrdered() {
         return modes != null ? Collections.unmodifiableList(modes) : null;
     }
+
+    /**
+     * Answers if the given path has an immediate child of PRIMARY mode.
+     *
+     * @param path The path to query.
+     * @return If the given path has an immediate child of PRIMARY mode.
+     */
+    public boolean hasPrimaryChild(IgfsPath path) {
+        return dualParentsWithPrimaryChildren.contains(path);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/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 03b7611..c582b2e 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,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
@@ -66,6 +67,7 @@ import java.util.concurrent.ThreadLocalRandom;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT;
 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;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -819,14 +821,18 @@ public class IgfsUtils {
      *
      * @param dfltMode The root mode. Must always be not null.
      * @param modes The subdirectory modes.
+     * @param dualParentsContainingPrimaryChildren The set to store parents into.
      * @return Descending list of filtered and checked modes.
-     * @throws IgniteCheckedException On error or
+     * @throws IgniteCheckedException On error.
      */
     public static ArrayList<T2<IgfsPath, IgfsMode>> preparePathModes(final IgfsMode dfltMode,
-        @Nullable List<T2<IgfsPath, IgfsMode>> modes) throws IgniteCheckedException {
+        @Nullable List<T2<IgfsPath, IgfsMode>> modes, Set<IgfsPath> dualParentsContainingPrimaryChildren)
+        throws IgniteCheckedException {
         if (modes == null)
             return null;
 
+        modes = new ArrayList<>(modes); // modes collection may be immutable.
+
         // Sort by depth, shallow first.
         Collections.sort(modes, new Comparator<Map.Entry<IgfsPath, IgfsMode>>() {
             @Override public int compare(Map.Entry<IgfsPath, IgfsMode> o1, Map.Entry<IgfsPath, IgfsMode> o2) {
@@ -857,6 +863,10 @@ public class IgfsUtils {
                     // Add to the 1st position (deep first).
                     resModes.add(0, mode);
 
+                    // Store primary paths inside dual paths in separate collection:
+                    if (mode.getValue() == PRIMARY)
+                        dualParentsContainingPrimaryChildren.add(mode.getKey().parent());
+
                     break;
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/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 9f783aa..fd3f9b6 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
@@ -245,6 +245,13 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * @return Relaxed consistency flag.
+     */
+    protected boolean initializeDefaultPathModes() {
+        return false;
+    }
+
+    /**
      * @return Client flag.
      */
     protected boolean client() {
@@ -369,6 +376,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
         igfsCfg.setRelaxedConsistency(relaxedConsistency());
 
+        igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes());
+
         CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
 
         dataCacheCfg.setName("dataCache");

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
index 124bec6..252c119 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
@@ -55,6 +55,12 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
     }
 
+    /** {@inheritDoc} */
+    @Override protected boolean initializeDefaultPathModes() {
+        // Enable default modes in order to test various modes.
+        return true;
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -68,6 +74,12 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         for (IgfsPath p : paths)
             assert igfs.exists(p);
 
+        assert igfs.modeResolver().resolveMode(gg) == mode;
+        assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "sync")) == IgfsMode.DUAL_SYNC;
+        assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "async")) == IgfsMode.DUAL_ASYNC;
+        assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "primary")) == IgfsMode.PRIMARY;
+        assert !igfsSecondary.exists("/ignite/primary"); // PRIMARY mode path must exist in upper level fs only.
+
         assert igfs.listFiles(gg).size() == 3;
         assert igfs.listPaths(gg).size() == 3;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java
index f3bb516..5117edb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import junit.framework.TestCase;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.igfs.IgfsMode;
@@ -90,7 +92,7 @@ public class IgfsModeResolverSelfTest extends TestCase {
         try {
             IgfsUtils.preparePathModes(DUAL_SYNC, Arrays.asList(
                 new T2<>(new IgfsPath("/a/"), PRIMARY),
-                new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC)));
+                new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC)), new HashSet<IgfsPath>());
 
             fail("IgniteCheckedException expected");
         }
@@ -102,7 +104,8 @@ public class IgfsModeResolverSelfTest extends TestCase {
         for (IgfsMode m: IgfsMode.values()) {
             if (m != IgfsMode.PRIMARY) {
                 try {
-                    IgfsUtils.preparePathModes(PRIMARY, Arrays.asList(new T2<>(new IgfsPath("/a/"), DUAL_ASYNC)));
+                    IgfsUtils.preparePathModes(PRIMARY, Arrays.asList(new T2<>(new IgfsPath("/a/"), DUAL_ASYNC)),
+                        new HashSet<IgfsPath>());
 
                     fail("IgniteCheckedException expected");
                 }
@@ -117,7 +120,7 @@ public class IgfsModeResolverSelfTest extends TestCase {
             new T2<>(new IgfsPath("/a"), PRIMARY),
             new T2<>(new IgfsPath("/c/d/"), PRIMARY),
             new T2<>(new IgfsPath("/c/d/e/f"), PRIMARY)
-        ));
+        ), new HashSet<IgfsPath>());
         assertNotNull(modes);
         assertEquals(2, modes.size());
         assertEquals(modes, Arrays.asList(
@@ -130,7 +133,7 @@ public class IgfsModeResolverSelfTest extends TestCase {
             new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC),
             new T2<>(new IgfsPath("/a/b/c"), DUAL_SYNC),
             new T2<>(new IgfsPath("/a/b/c/d"), DUAL_ASYNC)
-        ));
+        ), new HashSet<IgfsPath>());
         assertNotNull(modes);
         assertEquals(modes.size(), 3);
         assertEquals(modes, Arrays.asList(
@@ -139,4 +142,38 @@ public class IgfsModeResolverSelfTest extends TestCase {
             new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC)
         ));
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDualParentsWithPrimaryChild() throws Exception {
+        Set<IgfsPath> set = new HashSet<>();
+
+        IgfsUtils.preparePathModes(DUAL_SYNC, Arrays.asList(
+            new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC),
+            new T2<>(new IgfsPath("/a/b/c"), PRIMARY),
+            new T2<>(new IgfsPath("/a/b/x/y"), PRIMARY),
+            new T2<>(new IgfsPath("/a/b/x/z"), PRIMARY),
+            new T2<>(new IgfsPath("/m"), PRIMARY)
+        ), set);
+        assertEquals(set, new HashSet<IgfsPath>() {{
+            add(new IgfsPath("/a/b"));
+            add(new IgfsPath("/a/b/x"));
+            add(new IgfsPath("/"));
+        }});
+
+        set = new HashSet<>();
+
+        IgfsUtils.preparePathModes(DUAL_ASYNC, Arrays.asList(
+            new T2<>(new IgfsPath("/a/b/x/y/z"), PRIMARY),
+            new T2<>(new IgfsPath("/a/b/c"), PRIMARY),
+            new T2<>(new IgfsPath("/a/k"), PRIMARY),
+            new T2<>(new IgfsPath("/a/z"), PRIMARY)
+        ), set);
+        assertEquals(set, new HashSet<IgfsPath>() {{
+            add(new IgfsPath("/a/b"));
+            add(new IgfsPath("/a"));
+            add(new IgfsPath("/a/b/x/y"));
+        }});
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index 83991aa..a06129e 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -309,7 +309,12 @@ public class IgniteHadoopFileSystem extends FileSystem {
             else
                 clientLog = IgfsLogger.disabledLogger();
 
-            modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
+            try {
+                modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
+            }
+            catch (IgniteCheckedException ice) {
+                throw new IOException(ice);
+            }
 
             boolean initSecondary = paths.defaultMode() == PROXY;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ace2e715/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
index cc02d05..bd8ed2d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -318,7 +318,12 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
             else
                 clientLog = IgfsLogger.disabledLogger();
 
-            modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
+            try {
+                modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
+            }
+            catch (IgniteCheckedException ice) {
+                throw new IOException(ice);
+            }
 
             boolean initSecondary = paths.defaultMode() == PROXY;