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/15 14:18:22 UTC

[06/15] ignite git commit: IGNITE-3343: IGFS: Secondary file system is not queried for statuses during MKDIRS and CREATE operations. This closes #896.

IGNITE-3343: IGFS: Secondary file system is not queried for statuses during MKDIRS and CREATE operations. This closes #896.


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

Branch: refs/heads/ignite-3161
Commit: dc81069ba9ebb88bc11cf6917e8733cc1f6de2fb
Parents: 9ddf9d8
Author: Ivan Veselovskiy <iv...@gridgain.com>
Authored: Tue Aug 2 11:11:24 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 15 12:30:53 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteFileSystem.java     |  3 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |  4 +-
 .../processors/igfs/IgfsCreateResult.java       |  2 +-
 .../internal/processors/igfs/IgfsImpl.java      | 63 ++++++-----------
 .../processors/igfs/IgfsMetaManager.java        | 74 +++++---------------
 .../processors/igfs/IgfsModeResolver.java       | 31 ++++++--
 .../internal/processors/igfs/IgfsPaths.java     |  6 +-
 .../IgfsSecondaryFileSystemCreateContext.java   |  3 +
 .../internal/processors/igfs/IgfsUtils.java     | 12 +++-
 .../processors/igfs/IgfsAbstractSelfTest.java   | 34 ++++++++-
 .../igfs/IgfsDualAbstractSelfTest.java          | 13 ++++
 .../igfs/IgfsModeResolverSelfTest.java          | 51 ++++++++++++--
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  7 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  7 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  1 -
 .../hadoop/HadoopAbstractMapReduceTest.java     | 13 +++-
 16 files changed, 196 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
index bbc5bea..f9aeb8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
@@ -362,8 +362,7 @@ public interface IgniteFileSystem extends IgniteAsyncSupport {
      * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values
      * will be removed from the stored properties or ignored if they don't exist in the file info.
      * <p>
-     * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated
-     * to the secondary file system:
+     * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes with Hadoop secondary file system only the following properties will be updated:
      * <ul>
      * <li>{@code usrName} - file owner name;</li>
      * <li>{@code grpName} - file owner group;</li>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
index 3f124eb..4d9d255 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
@@ -43,8 +43,8 @@ public interface IgfsSecondaryFileSystem {
      * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values
      * will be removed from the stored properties or ignored if they don't exist in the file info.
      * <p>
-     * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated
-     * to the secondary file system:
+     * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes with Hadoop secondary file system only the
+     * following properties will be updated on the secondary file system:
      * <ul>
      * <li>{@code usrName} - file owner name;</li>
      * <li>{@code grpName} - file owner group;</li>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java
index 0b09e02..eb1bc9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java
@@ -23,7 +23,7 @@ import org.jetbrains.annotations.Nullable;
 import java.io.OutputStream;
 
 /**
- * IGFS file create result.
+ * IGFS file create or append result.
  */
 public class IgfsCreateResult {
     /** File info in the primary file system. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/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 0d1ffc1..e1f8e61 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.Set;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -248,7 +249,7 @@ public final class IgfsImpl implements IgfsEx {
             }
         }
 
-        modeRslvr = new IgfsModeResolver(dfltMode, IgfsUtils.preparePathModes(dfltMode, modes));
+        modeRslvr = new IgfsModeResolver(dfltMode, modes);
 
         Object secondaryFsPayload = null;
 
@@ -816,12 +817,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) {
@@ -846,7 +849,7 @@ public final class IgfsImpl implements IgfsEx {
 
                 IgfsMode mode = resolveMode(path);
 
-                Collection<IgfsFile> files = new HashSet<>();
+                Set<IgfsFile> files = new HashSet<>();
 
                 if (IgfsUtils.isDualMode(mode)) {
                     assert secondaryFs != null;
@@ -859,6 +862,9 @@ public final class IgfsImpl implements IgfsEx {
 
                             files.add(impl);
                         }
+
+                        if (!modeRslvr.hasPrimaryChild(path))
+                            return files;
                     }
                     catch (Exception e) {
                         U.error(log, "List files in DUAL mode failed [path=" + path + ']', e);
@@ -1345,29 +1351,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) {
@@ -1519,20 +1502,16 @@ public final class IgfsImpl implements IgfsEx {
 
             case DUAL_SYNC:
             case DUAL_ASYNC:
-                info = meta.infoForPath(path);
-
-                if (info == null) {
-                    try {
-                        IgfsFile status = secondaryFs.info(path);
+                try {
+                    IgfsFile status = secondaryFs.info(path);
 
-                        if (status != null)
-                            return new IgfsFileImpl(status, data.groupBlockSize());
-                    }
-                    catch (Exception e) {
-                        U.error(log, "File info operation in DUAL mode failed [path=" + path + ']', e);
+                    if (status != null)
+                        return new IgfsFileImpl(status, data.groupBlockSize());
+                }
+                catch (Exception e) {
+                    U.error(log, "File info operation in DUAL mode failed [path=" + path + ']', e);
 
-                        throw e;
-                    }
+                    throw e;
                 }
 
                 break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/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 d891b38..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
@@ -3051,29 +3051,19 @@ public class IgfsMetaManager extends IgfsManager {
                             if (secondaryCtx != null) {
                                 secondaryOut = secondaryCtx.create();
 
-                                IgfsFile secondaryFile = secondaryCtx.info();
-
-                                if (secondaryFile == null)
-                                    throw fsException("Failed to open output stream to the file created in " +
-                                        "the secondary file system because it no longer exists: " + path);
-                                else if (secondaryFile.isDirectory())
-                                    throw fsException("Failed to open output stream to the file created in " +
-                                        "the secondary file system because the path points to a directory: " + path);
-
-                                newAccessTime = secondaryFile.accessTime();
-                                newModificationTime = secondaryFile.modificationTime();
-                                newProps = secondaryFile.properties();
-                                newLen = secondaryFile.length();
-                                newBlockSize = secondaryFile.blockSize();
+                                newAccessTime = 0L;
+                                newModificationTime = 0L;
+                                newProps = null;
                             }
                             else {
                                 newAccessTime = System.currentTimeMillis();
                                 newModificationTime = newAccessTime;
                                 newProps = fileProps;
-                                newLen = 0L;
-                                newBlockSize = blockSize;
                             }
 
+                            newLen = 0L;
+                            newBlockSize = blockSize;
+
                             IgfsEntryInfo newInfo = invokeAndGet(overwriteId,
                                 new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps,
                                     newBlockSize, affKey, newLockId, evictExclude, newLen));
@@ -3254,18 +3244,9 @@ public class IgfsMetaManager extends IgfsManager {
             Map<String, String> props;
 
             if (secondaryCtx != null) {
-                IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(lastCreatedPath);
-
-                if (secondaryInfo == null)
-                    throw new IgfsException("Failed to perform operation because secondary file system path was " +
-                        "modified concurrently: " + lastCreatedPath);
-                else if (secondaryInfo.isFile())
-                    throw new IgfsException("Failed to perform operation because secondary file system entity is " +
-                        "not directory: " + lastCreatedPath);
-
-                accessTime = secondaryInfo.accessTime();
-                modificationTime = secondaryInfo.modificationTime();
-                props = secondaryInfo.properties();
+                accessTime = 0L;
+                modificationTime = 0L;
+                props = null;
             }
             else {
                 accessTime = curTime;
@@ -3293,18 +3274,9 @@ public class IgfsMetaManager extends IgfsManager {
             Map<String, String> props;
 
             if (secondaryCtx != null) {
-                IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(pathIds.path());
-
-                if (secondaryInfo == null)
-                    throw new IgfsException("Failed to perform operation because secondary file system path was " +
-                        "modified concurrnetly: " + pathIds.path());
-                else if (secondaryInfo.isFile())
-                    throw new IgfsException("Failed to perform operation because secondary file system entity is " +
-                        "not directory: " + lastCreatedPath);
-
-                accessTime = secondaryInfo.accessTime();
-                modificationTime = secondaryInfo.modificationTime();
-                props = secondaryInfo.properties();
+                accessTime = 0L;
+                modificationTime = 0L;
+                props = null;
             }
             else {
                 accessTime = curTime;
@@ -3322,29 +3294,19 @@ public class IgfsMetaManager extends IgfsManager {
             int newBlockSize;
 
             if (secondaryCtx != null) {
-                IgfsFile secondaryFile = secondaryCtx.info();
-
-                if (secondaryFile == null)
-                    throw fsException("Failed to open output stream to the file created in " +
-                        "the secondary file system because it no longer exists: " + pathIds.path());
-                else if (secondaryFile.isDirectory())
-                    throw fsException("Failed to open output stream to the file created in " +
-                        "the secondary file system because the path points to a directory: " + pathIds.path());
-
-                newAccessTime = secondaryFile.accessTime();
-                newModificationTime = secondaryFile.modificationTime();
-                newProps = secondaryFile.properties();
-                newLen = secondaryFile.length();
-                newBlockSize = secondaryFile.blockSize();
+                newAccessTime = 0L;
+                newModificationTime = 0L;
+                newProps = null;
             }
             else {
                 newAccessTime = curTime;
                 newModificationTime = curTime;
                 newProps = fileProps;
-                newLen = 0L;
-                newBlockSize = blockSize;
             }
 
+            newLen = 0L;
+            newBlockSize = blockSize;
+
             procMap.put(curId, new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps,
                 newBlockSize, affKey, createFileLockId(false), evictExclude, newLen));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/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..33b835f 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
@@ -17,9 +17,12 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.util.Collections;
+import java.util.ArrayList;
+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 ArrayList<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);
@@ -94,10 +103,20 @@ public class IgfsModeResolver {
     }
 
     /**
-     * @return Unmodifiable copy of properly ordered modes prefixes
+     * @return Copy of properly ordered modes prefixes
      *  or {@code null} if no modes set.
      */
-    @Nullable public List<T2<IgfsPath, IgfsMode>> modesOrdered() {
-        return modes != null ? Collections.unmodifiableList(modes) : null;
+    @Nullable public ArrayList<T2<IgfsPath, IgfsMode>> modesOrdered() {
+        return modes != null ? new ArrayList<>(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/dc81069b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
index 4a79259..2bdb23b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
@@ -48,7 +48,7 @@ public class IgfsPaths implements Externalizable {
     private IgfsMode dfltMode;
 
     /** Path modes. */
-    private List<T2<IgfsPath, IgfsMode>> pathModes;
+    private ArrayList<T2<IgfsPath, IgfsMode>> pathModes;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -65,7 +65,7 @@ public class IgfsPaths implements Externalizable {
      * @param pathModes Path modes.
      * @throws IgniteCheckedException If failed.
      */
-    public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable List<T2<IgfsPath, IgfsMode>> pathModes)
+    public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable ArrayList<T2<IgfsPath, IgfsMode>> pathModes)
         throws IgniteCheckedException {
         this.dfltMode = dfltMode;
         this.pathModes = pathModes;
@@ -91,7 +91,7 @@ public class IgfsPaths implements Externalizable {
     /**
      * @return Path modes.
      */
-    @Nullable public List<T2<IgfsPath, IgfsMode>> pathModes() {
+    @Nullable public ArrayList<T2<IgfsPath, IgfsMode>> pathModes() {
         return pathModes;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
index 1c0efd6..788efe3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
@@ -28,6 +28,7 @@ import java.util.Map;
 
 /**
  * Context for secondary file system create request.
+ * Note that it is never used for dual mode append operation.
  */
 public class IgfsSecondaryFileSystemCreateContext {
     /** File system. */
@@ -68,6 +69,8 @@ public class IgfsSecondaryFileSystemCreateContext {
      */
     public IgfsSecondaryFileSystemCreateContext(IgfsSecondaryFileSystem fs, IgfsPath path, boolean overwrite,
         boolean simpleCreate, @Nullable Map<String, String> props, short replication, long blockSize, int bufSize) {
+        assert fs != null;
+
         this.fs = fs;
         this.path = path;
         this.overwrite = overwrite;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/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..b9788f8 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,11 +821,13 @@ 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;
 
@@ -857,6 +861,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/dc81069b/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 384da95..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");
@@ -1071,17 +1080,36 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testRootPropertiesPersistAfterFormat() throws Exception {
-        igfs.update(new IgfsPath("/"), Collections.singletonMap("foo", "moo"));
+        if (dual && !(igfsSecondaryFileSystem instanceof IgfsSecondaryFileSystemImpl)) {
+            // In case of Hadoop dual mode only user name, group name, and permission properties are updated,
+            // an arbitrary named property is just ignored:
+            checkRootPropertyUpdate("foo", "moo", null);
+            checkRootPropertyUpdate(IgfsUtils.PROP_PERMISSION, "0777", "0777");
+        }
+        else {
+            checkRootPropertyUpdate("foo", "moo", "moo");
+            checkRootPropertyUpdate(IgfsUtils.PROP_PERMISSION, "0777", "0777");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception
+     */
+    private void checkRootPropertyUpdate(String prop, String setVal, String expGetVal) throws Exception {
+        final IgfsPath rootPath = new IgfsPath("/");
+
+        igfs.update(rootPath, Collections.singletonMap(prop, setVal));
 
         igfs.format();
 
-        IgfsFile file = igfs.info(new IgfsPath("/"));
+        IgfsFile file = igfs.info(rootPath);
 
         assert file != null;
 
         Map<String,String> props = file.properties();
 
-        assertEquals("moo", props.get("foo"));
+        assertEquals(expGetVal, props.get(prop));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/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..92e1178 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,13 @@ 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.
+
+        // All the child paths of "/ignite/" must be visible in listings:
         assert igfs.listFiles(gg).size() == 3;
         assert igfs.listPaths(gg).size() == 3;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/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..bd7e413 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
@@ -17,8 +17,11 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.ArrayList;
 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;
@@ -38,8 +41,9 @@ public class IgfsModeResolverSelfTest extends TestCase {
 
     /** {@inheritDoc} */
     @Override protected void setUp() throws Exception {
-        reslvr = new IgfsModeResolver(DUAL_SYNC, Arrays.asList(new T2<>(new IgfsPath("/a/b/c/d"), PROXY), new T2<>
-            (new IgfsPath("/a/P/"), PRIMARY), new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC)));
+        reslvr = new IgfsModeResolver(DUAL_SYNC, new ArrayList<>(Arrays.asList(new T2<>(
+            new IgfsPath("/a/b/c/d"), PROXY), new T2<>(new IgfsPath("/a/P/"), PRIMARY), new T2<>(new IgfsPath("/a/b/"),
+            DUAL_ASYNC))));
     }
 
     /**
@@ -90,7 +94,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 +106,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 +122,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 +135,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 +144,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/dc81069b/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/dc81069b/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;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
index 083ee54..f793ec3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
@@ -48,7 +48,6 @@ import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc81069b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java
index ef886e4..3731213 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java
@@ -41,6 +41,7 @@ import org.apache.ignite.configuration.HadoopConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter;
 import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
+import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
 import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
@@ -123,8 +124,16 @@ public class HadoopAbstractMapReduceTest extends HadoopAbstractWordCountTest {
      * @param p The path.
      * @return The owner.
      */
-    private static String getOwner(IgfsEx i, IgfsPath p) {
-        return i.info(p).property(IgfsUtils.PROP_USER_NAME);
+    private static String getOwner(final IgfsEx i, final IgfsPath p) {
+        return IgfsUserContext.doAs(USER, new IgniteOutClosure<String>() {
+            @Override public String apply() {
+                IgfsFile f = i.info(p);
+
+                assert f != null;
+
+                return f.property(IgfsUtils.PROP_USER_NAME);
+            }
+        });
     }
 
     /**