You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/18 10:02:35 UTC

[3/7] ignite git commit: IGNITE-3920: Removed IgfsPaths and secondary file system propagation to client Hadoop file systems. This closes #1077. This closes #1107.

IGNITE-3920: Removed IgfsPaths and secondary file system propagation to client Hadoop file systems. This closes #1077. This closes #1107.


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

Branch: refs/heads/ignite-4985
Commit: feba95348391938aa7bb32499c647103b6a0a16f
Parents: 227599f
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Tue Apr 18 12:54:53 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Apr 18 12:54:53 2017 +0300

----------------------------------------------------------------------
 .../igfs/common/IgfsControlResponse.java        |  28 +
 .../internal/igfs/common/IgfsIpcCommand.java    |   5 +-
 .../ignite/internal/igfs/common/IgfsLogger.java |  60 +-
 .../internal/igfs/common/IgfsMarshaller.java    |  12 +
 .../igfs/common/IgfsModeResolverRequest.java    |  35 +
 .../processors/hadoop/HadoopPayloadAware.java   |  28 -
 .../internal/processors/igfs/IgfsAsyncImpl.java |   5 -
 .../ignite/internal/processors/igfs/IgfsEx.java |   7 -
 .../processors/igfs/IgfsHandshakeResponse.java  |  22 +-
 .../internal/processors/igfs/IgfsImpl.java      |  23 +-
 .../processors/igfs/IgfsIpcHandler.java         |  20 +-
 .../processors/igfs/IgfsModeResolver.java       |  91 ++-
 .../internal/processors/igfs/IgfsPaths.java     | 152 ----
 .../resources/META-INF/classnames.properties    |   1 -
 ...fsLocalSecondaryFileSystemProxySelfTest.java |   1 -
 .../internal/processors/igfs/IgfsMock.java      |   7 -
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   9 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 698 ++++---------------
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    | 481 +++----------
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |  11 +-
 .../hadoop/impl/igfs/HadoopIgfsEx.java          |   8 +
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      |  23 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |  12 +
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  14 +
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |  14 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  10 +
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |  32 +-
 ...condaryFileSystemInitializationSelfTest.java | 213 ------
 .../testsuites/IgniteHadoopTestSuite.java       |   3 -
 29 files changed, 530 insertions(+), 1495 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java
index 595f3c4..f1219c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsBlockLocationImpl;
 import org.apache.ignite.internal.processors.igfs.IgfsFileImpl;
 import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse;
 import org.apache.ignite.internal.processors.igfs.IgfsInputStreamDescriptor;
+import org.apache.ignite.internal.processors.igfs.IgfsModeResolver;
 import org.apache.ignite.internal.processors.igfs.IgfsStatus;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -115,6 +116,9 @@ public class IgfsControlResponse extends IgfsMessage {
     /** Response is a path summary. */
     public static final int RES_TYPE_IGFS_PATH_SUMMARY = 12;
 
+    /** Response is a path summary. */
+    public static final int RES_TYPE_MODE_RESOLVER = 13;
+
     /** Message header size. */
     public static final int RES_HEADER_SIZE = 9;
 
@@ -257,6 +261,15 @@ public class IgfsControlResponse extends IgfsMessage {
     }
 
     /**
+     * @param res Status response.
+     */
+    public void modeResolver(IgfsModeResolver res) {
+        resType = RES_TYPE_MODE_RESOLVER;
+
+        this.res = res;
+    }
+
+    /**
      * @param len Response length.
      */
     public void length(int len) {
@@ -425,6 +438,7 @@ public class IgfsControlResponse extends IgfsMessage {
             case RES_TYPE_IGFS_FILE:
             case RES_TYPE_IGFS_STREAM_DESCRIPTOR:
             case RES_TYPE_HANDSHAKE:
+            case RES_TYPE_MODE_RESOLVER:
             case RES_TYPE_STATUS: {
                 out.writeBoolean(res != null);
 
@@ -630,6 +644,20 @@ public class IgfsControlResponse extends IgfsMessage {
                 break;
             }
 
+            case RES_TYPE_MODE_RESOLVER: {
+                boolean hasVal = in.readBoolean();
+
+                if (hasVal) {
+                    IgfsModeResolver msg = new IgfsModeResolver();
+
+                    msg.readExternal(in);
+
+                    res = msg;
+                }
+
+                break;
+            }
+
             case RES_TYPE_BYTE_ARRAY:
                 assert false : "Response type of byte array should never be processed by marshaller.";
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsIpcCommand.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsIpcCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsIpcCommand.java
index e39319b..9af7a78 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsIpcCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsIpcCommand.java
@@ -82,7 +82,10 @@ public enum IgfsIpcCommand {
     WRITE_BLOCK,
 
     /** Server response. */
-    CONTROL_RESPONSE;
+    CONTROL_RESPONSE,
+
+    /** Mode resolver request */
+    MODE_RESOLVER;
 
     /** All values */
     private static final List<IgfsIpcCommand> ALL = Arrays.asList(values());

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsLogger.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsLogger.java
index 4a0d99b..0882832 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsLogger.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsLogger.java
@@ -31,7 +31,6 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -168,6 +167,7 @@ public final class IgfsLogger {
      * Get logger instance for the given endpoint.
      *
      * @param endpoint Endpoint.
+     * @param igfsName IGFS name.
      * @param dir Path.
      * @param batchSize Batch size.
      *
@@ -264,12 +264,11 @@ public final class IgfsLogger {
      *
      * @param streamId Stream ID.
      * @param path Path.
-     * @param mode Mode.
      * @param bufSize Buffer size.
      * @param dataLen Data length.
      */
-    public void logOpen(long streamId, IgfsPath path, IgfsMode mode, int bufSize, long dataLen) {
-        addEntry(new Entry(TYPE_OPEN_IN, path.toString(), mode, streamId, bufSize, dataLen, null, null, null, null,
+    public void logOpen(long streamId, IgfsPath path, int bufSize, long dataLen) {
+        addEntry(new Entry(TYPE_OPEN_IN, path.toString(), streamId, bufSize, dataLen, null, null, null, null,
             null, null, null, null, null, null, null, null, null, null));
     }
 
@@ -278,15 +277,14 @@ public final class IgfsLogger {
      *
      * @param streamId Stream ID.
      * @param path Path.
-     * @param mode Mode.
      * @param overwrite Overwrite flag.
      * @param bufSize Buffer size.
      * @param replication Replication factor.
      * @param blockSize Block size.
      */
-    public void logCreate(long streamId, IgfsPath path, IgfsMode mode, boolean overwrite, int bufSize,
+    public void logCreate(long streamId, IgfsPath path, boolean overwrite, int bufSize,
         int replication, long blockSize) {
-        addEntry(new Entry(TYPE_OPEN_OUT, path.toString(), mode, streamId, bufSize, null, false, overwrite, replication,
+        addEntry(new Entry(TYPE_OPEN_OUT, path.toString(), streamId, bufSize, null, false, overwrite, replication,
             blockSize, null, null, null, null, null, null, null, null, null, null));
     }
 
@@ -295,11 +293,10 @@ public final class IgfsLogger {
      *
      * @param streamId Stream ID.
      * @param path Path.
-     * @param mode Mode.
      * @param bufSize Buffer size.
      */
-    public void logAppend(long streamId, IgfsPath path, IgfsMode mode, int bufSize) {
-        addEntry(new Entry(TYPE_OPEN_OUT, path.toString(), mode, streamId, bufSize, null, true, null, null, null, null,
+    public void logAppend(long streamId, IgfsPath path, int bufSize) {
+        addEntry(new Entry(TYPE_OPEN_OUT, path.toString(), streamId, bufSize, null, true, null, null, null, null,
             null, null, null, null, null, null, null, null, null));
     }
 
@@ -311,7 +308,7 @@ public final class IgfsLogger {
      * @param readLen Read bytes count.
      */
     public void logRandomRead(long streamId, long pos, int readLen) {
-        addEntry(new Entry(TYPE_RANDOM_READ, null, null, streamId, null, null, null, null, null, null, pos, readLen,
+        addEntry(new Entry(TYPE_RANDOM_READ, null, streamId, null, null, null, null, null, null, pos, readLen,
             null, null, null, null, null, null, null, null));
     }
 
@@ -322,7 +319,7 @@ public final class IgfsLogger {
      * @param pos Position.
      */
     public void logSeek(long streamId, long pos) {
-        addEntry(new Entry(TYPE_SEEK, null, null, streamId, null, null, null, null, null, null, pos, null, null, null,
+        addEntry(new Entry(TYPE_SEEK, null, streamId, null, null, null, null, null, null, pos, null, null, null,
             null, null, null, null, null, null));
     }
 
@@ -333,7 +330,7 @@ public final class IgfsLogger {
      * @param skipCnt Skip bytes count.
      */
     public void logSkip(long streamId, long skipCnt) {
-        addEntry(new Entry(TYPE_SKIP, null, null, streamId, null, null, null, null, null, null, null, null, skipCnt,
+        addEntry(new Entry(TYPE_SKIP, null, streamId, null, null, null, null, null, null, null, null, skipCnt,
             null, null, null, null, null, null, null));
     }
 
@@ -344,7 +341,7 @@ public final class IgfsLogger {
      * @param readLimit Read limit.
      */
     public void logMark(long streamId, long readLimit) {
-        addEntry(new Entry(TYPE_MARK, null, null, streamId, null, null, null, null, null, null, null, null, null,
+        addEntry(new Entry(TYPE_MARK, null, streamId, null, null, null, null, null, null, null, null, null,
             readLimit, null, null, null, null, null, null));
     }
 
@@ -354,7 +351,7 @@ public final class IgfsLogger {
      * @param streamId Stream ID.
      */
     public void logReset(long streamId) {
-        addEntry(new Entry(TYPE_RESET, null, null, streamId, null, null, null, null, null, null, null, null, null, null,
+        addEntry(new Entry(TYPE_RESET, null, streamId, null, null, null, null, null, null, null, null, null, null,
             null, null, null, null, null, null));
     }
 
@@ -367,7 +364,7 @@ public final class IgfsLogger {
      * @param total Total bytes read.
      */
     public void logCloseIn(long streamId, long userTime, long readTime, long total) {
-        addEntry(new Entry(TYPE_CLOSE_IN, null, null, streamId, null, null, null, null, null, null, null, null, null,
+        addEntry(new Entry(TYPE_CLOSE_IN, null, streamId, null, null, null, null, null, null, null, null, null,
             null, userTime, readTime, total ,null, null, null));
     }
 
@@ -380,7 +377,7 @@ public final class IgfsLogger {
      * @param total Total bytes read.
      */
     public void logCloseOut(long streamId, long userTime, long writeTime, long total) {
-        addEntry(new Entry(TYPE_CLOSE_OUT, null, null, streamId, null, null, null, null, null, null, null, null, null,
+        addEntry(new Entry(TYPE_CLOSE_OUT, null, streamId, null, null, null, null, null, null, null, null, null,
             null, userTime, writeTime, total, null, null, null));
     }
 
@@ -388,10 +385,9 @@ public final class IgfsLogger {
      * Log directory creation event.
      *
      * @param path Path.
-     * @param mode Mode.
      */
-    public void logMakeDirectory(IgfsPath path, IgfsMode mode) {
-        addEntry(new Entry(TYPE_DIR_MAKE, path.toString(), mode, null, null, null, null, null, null, null, null, null,
+    public void logMakeDirectory(IgfsPath path) {
+        addEntry(new Entry(TYPE_DIR_MAKE, path.toString(), null, null, null, null, null, null, null, null, null,
             null, null, null, null, null, null, null, null));
     }
 
@@ -399,11 +395,10 @@ public final class IgfsLogger {
      * Log directory listing event.
      *
      * @param path Path.
-     * @param mode Mode.
      * @param files Files.
      */
-    public void logListDirectory(IgfsPath path, IgfsMode mode, String[] files) {
-        addEntry(new Entry(TYPE_DIR_LIST, path.toString(), mode, null, null, null, null, null, null, null, null, null,
+    public void logListDirectory(IgfsPath path, String[] files) {
+        addEntry(new Entry(TYPE_DIR_LIST, path.toString(), null, null, null, null, null, null, null, null, null,
             null, null, null, null, null, null, null, files));
     }
 
@@ -411,11 +406,10 @@ public final class IgfsLogger {
      * Log rename event.
      *
      * @param path Path.
-     * @param mode Mode.
      * @param destPath Destination path.
      */
-    public void logRename(IgfsPath path, IgfsMode mode, IgfsPath destPath) {
-        addEntry(new Entry(TYPE_RENAME, path.toString(), mode, null, null, null, null, null, null, null, null, null,
+    public void logRename(IgfsPath path, IgfsPath destPath) {
+        addEntry(new Entry(TYPE_RENAME, path.toString(), null, null, null, null, null, null, null, null, null,
             null, null, null, null, null, destPath.toString(), null, null));
     }
 
@@ -423,11 +417,10 @@ public final class IgfsLogger {
      * Log delete event.
      *
      * @param path Path.
-     * @param mode Mode.
      * @param recursive Recursive flag.
      */
-    public void logDelete(IgfsPath path, IgfsMode mode, boolean recursive) {
-        addEntry(new Entry(TYPE_DELETE, path.toString(), mode, null, null, null, null, null, null, null, null, null,
+    public void logDelete(IgfsPath path, boolean recursive) {
+        addEntry(new Entry(TYPE_DELETE, path.toString(), null, null, null, null, null, null, null, null, null,
             null, null, null, null, null, null, recursive, null));
     }
 
@@ -512,9 +505,6 @@ public final class IgfsLogger {
         /** File/dir path. */
         private final String path;
 
-        /** Path mode. */
-        private IgfsMode mode;
-
         /** Stream ID. */
         private final long streamId;
 
@@ -571,7 +561,6 @@ public final class IgfsLogger {
          *
          * @param type Event type.
          * @param path Path.
-         * @param mode Path mode.
          * @param streamId Stream ID.
          * @param bufSize Buffer size.
          * @param dataLen Data length.
@@ -590,7 +579,7 @@ public final class IgfsLogger {
          * @param recursive Recursive flag.
          * @param list Listed directories.
          */
-        Entry(int type, String path, IgfsMode mode, Long streamId, Integer bufSize, Long dataLen, Boolean append,
+        Entry(int type, String path, Long streamId, Integer bufSize, Long dataLen, Boolean append,
             Boolean overwrite, Integer replication, Long blockSize, Long pos, Integer readLen, Long skipCnt,
             Long readLimit, Long userTime, Long sysTime, Long total, String destPath, Boolean recursive,
             String[] list) {
@@ -599,7 +588,6 @@ public final class IgfsLogger {
 
             this.type = type;
             this.path = path;
-            this.mode = mode;
             this.streamId = streamId != null ? streamId : -1;
             this.bufSize = bufSize != null ? bufSize : -1;
             this.dataLen = dataLen != null ? dataLen : -1;
@@ -679,7 +667,7 @@ public final class IgfsLogger {
             SB res = new SB();
 
             res.a(ts).a(DELIM_FIELD).a(threadId).a(DELIM_FIELD).a(pid).a(DELIM_FIELD).a(type).a(DELIM_FIELD)
-                .a(string(path)).a(DELIM_FIELD).a(string(mode)).a(DELIM_FIELD).a(string(streamId)).a(DELIM_FIELD)
+                .a(string(path)).a(DELIM_FIELD).a(DELIM_FIELD).a(string(streamId)).a(DELIM_FIELD)
                 .a(string(bufSize)).a(DELIM_FIELD).a(string(dataLen)).a(DELIM_FIELD).a(string(append)).a(DELIM_FIELD)
                 .a(string(overwrite)).a(DELIM_FIELD).a(string(replication)).a(DELIM_FIELD).a(string(blockSize))
                 .a(DELIM_FIELD).a(string(pos)).a(DELIM_FIELD).a(string(readLen)).a(DELIM_FIELD).a(string(skipCnt))

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
index 4645437..d9b1256 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
@@ -182,6 +182,12 @@ public class IgfsMarshaller {
                     break;
                 }
 
+                case MODE_RESOLVER: {
+                    out.write(hdr);
+
+                    break;
+                }
+
                 default: {
                     assert false : "Invalid command: " + msg.command();
 
@@ -299,6 +305,12 @@ public class IgfsMarshaller {
                     break;
                 }
 
+                case MODE_RESOLVER: {
+                    msg = new IgfsModeResolverRequest();
+
+                    break;
+                }
+
                 default: {
                     assert false : "Invalid command: " + cmd;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsModeResolverRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsModeResolverRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsModeResolverRequest.java
new file mode 100644
index 0000000..3e78a9a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsModeResolverRequest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.igfs.common;
+
+import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.MODE_RESOLVER;
+
+/**
+ * Handshake request.
+ */
+public class IgfsModeResolverRequest extends IgfsMessage {
+    /** {@inheritDoc} */
+    @Override public IgfsIpcCommand command() {
+        return MODE_RESOLVER;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void command(IgfsIpcCommand cmd) {
+        // No-op.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopPayloadAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopPayloadAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopPayloadAware.java
deleted file mode 100644
index 9b79729..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopPayloadAware.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.hadoop;
-
-/**
- * Gets payload for Hadoop secondary file system.
- */
-public interface HadoopPayloadAware {
-    /**
-     * @return Payload.
-     */
-    public Object getPayload();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
index 528c18f..c4e2fe7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
@@ -154,11 +154,6 @@ public class IgfsAsyncImpl extends AsyncSupportAdapter<IgniteFileSystem> impleme
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsPaths proxyPaths() {
-        return igfs.proxyPaths();
-    }
-
-    /** {@inheritDoc} */
     @Override public IgfsInputStream open(IgfsPath path, int bufSize,
         int seqReadsBeforePrefetch) {
         return igfs.open(path, bufSize, seqReadsBeforePrefetch);

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
index c869695..296db47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
@@ -42,13 +42,6 @@ public interface IgfsEx extends IgniteFileSystem {
     public IgfsContext context();
 
     /**
-     * Get handshake message.
-     *
-     * @return Handshake message.
-     */
-    public IgfsPaths proxyPaths();
-
-    /**
      * Gets global space counters.
      *
      * @return Tuple in which first component is used space on all nodes,

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHandshakeResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHandshakeResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHandshakeResponse.java
index 1ba98ac..c447376 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHandshakeResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHandshakeResponse.java
@@ -33,9 +33,6 @@ public class IgfsHandshakeResponse implements Externalizable {
     /** IGFS name. */
     private String igfsName;
 
-    /** SECONDARY paths. */
-    private IgfsPaths paths;
-
     /** Server block size. */
     private long blockSize;
 
@@ -52,14 +49,10 @@ public class IgfsHandshakeResponse implements Externalizable {
     /**
      * Constructor.
      *
-     * @param paths Secondary paths.
      * @param blockSize Server default block size.
      */
-    public IgfsHandshakeResponse(String igfsName, IgfsPaths paths, long blockSize, Boolean sampling) {
-        assert paths != null;
-
+    public IgfsHandshakeResponse(String igfsName, long blockSize, Boolean sampling) {
         this.igfsName = igfsName;
-        this.paths = paths;
         this.blockSize = blockSize;
         this.sampling = sampling;
     }
@@ -72,13 +65,6 @@ public class IgfsHandshakeResponse implements Externalizable {
     }
 
     /**
-     * @return SECONDARY paths configured on server.
-     */
-    public IgfsPaths secondaryPaths() {
-        return paths;
-    }
-
-    /**
      * @return Server default block size.
      */
     public long blockSize() {
@@ -96,8 +82,6 @@ public class IgfsHandshakeResponse implements Externalizable {
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeString(out, igfsName);
 
-        paths.writeExternal(out);
-
         out.writeLong(blockSize);
 
         if (sampling != null) {
@@ -112,10 +96,6 @@ public class IgfsHandshakeResponse implements Externalizable {
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         igfsName = U.readString(in);
 
-        paths = new IgfsPaths();
-
-        paths.readExternal(in);
-
         blockSize = in.readLong();
 
         if (in.readBoolean())

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/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 a946c71..1c0fbc2 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
@@ -67,7 +67,6 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadabl
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
-import org.apache.ignite.internal.processors.hadoop.HadoopPayloadAware;
 import org.apache.ignite.internal.processors.igfs.client.IgfsClientAffinityCallable;
 import org.apache.ignite.internal.processors.igfs.client.IgfsClientDeleteCallable;
 import org.apache.ignite.internal.processors.igfs.client.IgfsClientExistsCallable;
@@ -120,9 +119,6 @@ public final class IgfsImpl implements IgfsEx {
     /** Default directory metadata. */
     static final Map<String, String> DFLT_DIR_META = F.asMap(IgfsUtils.PROP_PERMISSION, PERMISSION_DFLT_VAL);
 
-    /** Handshake message. */
-    private final IgfsPaths secondaryPaths;
-
     /** Cache based structure (meta data) manager. */
     private IgfsMetaManager meta;
 
@@ -256,13 +252,6 @@ public final class IgfsImpl implements IgfsEx {
 
         modeRslvr = new IgfsModeResolver(dfltMode, modes);
 
-        Object secondaryFsPayload = null;
-
-        if (secondaryFs instanceof HadoopPayloadAware)
-            secondaryFsPayload = ((HadoopPayloadAware) secondaryFs).getPayload();
-
-        secondaryPaths = new IgfsPaths(secondaryFsPayload, dfltMode, modeRslvr.modesOrdered());
-
         // Check whether IGFS LRU eviction policy is set on data cache.
         String dataCacheName = igfsCtx.configuration().getDataCacheConfiguration().getName();
 
@@ -415,7 +404,7 @@ public final class IgfsImpl implements IgfsEx {
     /**
      * @return Mode resolver.
      */
-    IgfsModeResolver modeResolver() {
+    public IgfsModeResolver modeResolver() {
         return modeRslvr;
     }
 
@@ -430,11 +419,6 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsPaths proxyPaths() {
-        return secondaryPaths;
-    }
-
-    /** {@inheritDoc} */
     @Override public String clientLogDirectory() {
         return logDir;
     }
@@ -639,10 +623,9 @@ public final class IgfsImpl implements IgfsEx {
                     default:
                         assert mode == PROXY : "Unknown mode: " + mode;
 
-                        IgfsFile file = secondaryFs.update(path, props);
+                        IgfsFile status = secondaryFs.update(path, props);
 
-                        if (file != null)
-                            return new IgfsFileImpl(file, data.groupBlockSize());
+                        return status != null ? new IgfsFileImpl(status, data.groupBlockSize()) : null;
                 }
 
                 return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
index efa39fa..203f383 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
@@ -245,12 +245,28 @@ class IgfsIpcHandler implements IgfsServerHandler {
             case WRITE_BLOCK:
                 return processStreamControlRequest(ses, cmd, msg, in);
 
+            case MODE_RESOLVER:
+                return processModeResolver();
+
             default:
                 throw new IgniteCheckedException("Unsupported IPC command: " + cmd);
         }
     }
 
     /**
+     * Process mode resolver request.
+     *
+     * @return Status response.
+     */
+    private IgfsMessage processModeResolver() {
+        IgfsControlResponse res = new IgfsControlResponse();
+
+        res.modeResolver(((IgfsImpl)igfs).modeResolver());
+
+        return res;
+    }
+
+    /**
      * Processes handshake request.
      *
      * @param req Handshake request.
@@ -266,8 +282,8 @@ class IgfsIpcHandler implements IgfsServerHandler {
 
         igfs.clientLogDirectory(req.logDirectory());
 
-        IgfsHandshakeResponse handshake = new IgfsHandshakeResponse(igfs.name(), igfs.proxyPaths(),
-            igfs.groupBlockSize(), igfs.globalSampling());
+        IgfsHandshakeResponse handshake = new IgfsHandshakeResponse(igfs.name(), igfs.groupBlockSize(),
+            igfs.globalSampling());
 
         res.handshake(handshake);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/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 d4e248d..33d6a8c 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,6 +17,10 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -28,17 +32,21 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
-public class IgfsModeResolver {
+public class IgfsModeResolver implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Maximum size of map with cached path modes. */
     private static final int MAX_PATH_CACHE = 1000;
 
     /** Default mode. */
-    private final IgfsMode dfltMode;
+    private IgfsMode dfltMode;
 
     /** Modes for particular paths. Ordered from longest to shortest. */
     private List<T2<IgfsPath, IgfsMode>> modes;
@@ -47,13 +55,21 @@ public class IgfsModeResolver {
     private Map<IgfsPath, IgfsMode> modesCache;
 
     /** Set to store parent dual paths that have primary children. */
-    private final Set<IgfsPath> dualParentsWithPrimaryChildren;
+    private Set<IgfsPath> dualParentsWithPrimaryChildren;
+
+    /**
+     * Empty constructor required by {@link Externalizable}.
+     */
+    public IgfsModeResolver() {
+        // No-op.
+    }
 
     /**
      * Constructor
      *
      * @param dfltMode Default IGFS mode.
      * @param modes List of configured modes. The order is significant as modes are added in order of occurrence.
+     * @throws IgniteCheckedException On error.
      */
     public IgfsModeResolver(IgfsMode dfltMode, @Nullable ArrayList<T2<IgfsPath, IgfsMode>> modes)
             throws IgniteCheckedException {
@@ -61,7 +77,7 @@ public class IgfsModeResolver {
 
         this.dfltMode = dfltMode;
 
-        this.dualParentsWithPrimaryChildren = new HashSet<>();
+        dualParentsWithPrimaryChildren = new HashSet<>();
 
         this.modes = IgfsUtils.preparePathModes(dfltMode, modes, dualParentsWithPrimaryChildren);
 
@@ -104,14 +120,6 @@ public class IgfsModeResolver {
     }
 
     /**
-     * @return Copy of properly ordered modes prefixes
-     *  or {@code null} if no modes set.
-     */
-    @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.
@@ -120,4 +128,63 @@ public class IgfsModeResolver {
     public boolean hasPrimaryChild(IgfsPath path) {
         return dualParentsWithPrimaryChildren.contains(path);
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeEnum(out, dfltMode);
+
+        if (modes != null) {
+            out.writeBoolean(true);
+            out.writeInt(modes.size());
+
+            for (T2<IgfsPath, IgfsMode> pathMode : modes) {
+                assert pathMode.getKey() != null;
+
+                pathMode.getKey().writeExternal(out);
+
+                U.writeEnum(out, pathMode.getValue());
+            }
+        }
+        else
+            out.writeBoolean(false);
+
+        if (!F.isEmpty(dualParentsWithPrimaryChildren)) {
+            out.writeBoolean(true);
+            out.writeInt(dualParentsWithPrimaryChildren.size());
+
+            for (IgfsPath p : dualParentsWithPrimaryChildren)
+                p.writeExternal(out);
+        }
+        else
+            out.writeBoolean(false);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        dfltMode = IgfsMode.fromOrdinal(in.readByte());
+
+        if (in.readBoolean()) {
+            int size = in.readInt();
+
+            modes = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++) {
+                IgfsPath path = IgfsUtils.readPath(in);
+
+                modes.add(new T2<>(path, IgfsMode.fromOrdinal(in.readByte())));
+            }
+
+            modesCache = new GridBoundedConcurrentLinkedHashMap<>(MAX_PATH_CACHE);
+        }
+
+        dualParentsWithPrimaryChildren = new HashSet<>();
+
+        if (in.readBoolean()) {
+            int size = in.readInt();
+
+            for (int i = 0; i < size; i++)
+                dualParentsWithPrimaryChildren.add(IgfsUtils.readPath(in));
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/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
deleted file mode 100644
index 9752411..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.igfs;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.ArrayList;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.igfs.IgfsMode;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.jdk.JdkMarshaller;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Description of path modes.
- */
-public class IgfsPaths implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private byte[] payloadBytes;
-
-    /** Default IGFS mode. */
-    private IgfsMode dfltMode;
-
-    /** Path modes. */
-    private ArrayList<T2<IgfsPath, IgfsMode>> pathModes;
-
-    /**
-     * Empty constructor required by {@link Externalizable}.
-     */
-    public IgfsPaths() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param payload Payload.
-     * @param dfltMode Default IGFS mode.
-     * @param pathModes Path modes.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable ArrayList<T2<IgfsPath, IgfsMode>> pathModes)
-        throws IgniteCheckedException {
-        this.dfltMode = dfltMode;
-        this.pathModes = pathModes;
-
-        if (payload == null)
-            payloadBytes = null;
-        else {
-            ByteArrayOutputStream out = new ByteArrayOutputStream();
-
-            U.marshal(new JdkMarshaller(), payload, out);
-
-            payloadBytes = out.toByteArray();
-        }
-    }
-
-    /**
-     * @return Default IGFS mode.
-     */
-    public IgfsMode defaultMode() {
-        return dfltMode;
-    }
-
-    /**
-     * @return Path modes.
-     */
-    @Nullable public ArrayList<T2<IgfsPath, IgfsMode>> pathModes() {
-        return pathModes;
-    }
-
-    /**
-     * @return Payload.
-     *
-     * @throws IgniteCheckedException If failed to deserialize the payload.
-     */
-    @Nullable public Object getPayload(ClassLoader clsLdr) throws IgniteCheckedException {
-        if (payloadBytes == null)
-            return null;
-        else {
-            ByteArrayInputStream in = new ByteArrayInputStream(payloadBytes);
-
-            return U.unmarshal(new JdkMarshaller(), in, clsLdr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeByteArray(out, payloadBytes);
-
-        U.writeEnum(out, dfltMode);
-
-        if (pathModes != null) {
-            out.writeBoolean(true);
-            out.writeInt(pathModes.size());
-
-            for (T2<IgfsPath, IgfsMode> pathMode : pathModes) {
-                assert pathMode.getKey() != null;
-
-                pathMode.getKey().writeExternal(out);
-
-                U.writeEnum(out, pathMode.getValue());
-            }
-        }
-        else
-            out.writeBoolean(false);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        payloadBytes = U.readByteArray(in);
-
-        dfltMode = IgfsMode.fromOrdinal(in.readByte());
-
-        if (in.readBoolean()) {
-            int size = in.readInt();
-
-            pathModes = new ArrayList<>(size);
-
-            for (int i = 0; i < size; i++) {
-                IgfsPath path = IgfsUtils.readPath(in);
-
-                pathModes.add(new T2<>(path, IgfsMode.fromOrdinal(in.readByte())));
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index e5f80dd..58679ea 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1154,7 +1154,6 @@ org.apache.ignite.internal.processors.igfs.IgfsMetaManager$2
 org.apache.ignite.internal.processors.igfs.IgfsMetaManager$3
 org.apache.ignite.internal.processors.igfs.IgfsMetricsAdapter
 org.apache.ignite.internal.processors.igfs.IgfsNodePredicate
-org.apache.ignite.internal.processors.igfs.IgfsPaths
 org.apache.ignite.internal.processors.igfs.IgfsProcessor$1
 org.apache.ignite.internal.processors.igfs.IgfsSamplingKey
 org.apache.ignite.internal.processors.igfs.IgfsServer$ClientWorker$1

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java
index e7e3ac8..5b0a8df 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java
@@ -255,7 +255,6 @@ public class IgfsLocalSecondaryFileSystemProxySelfTest extends IgfsProxySelfTest
 
             len -= 1024 * 2;
             start += 1024;
-            System.out.println("+++ ");
         }
 
         len = igfs.info(filePath).length();

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java
index 833ab4c..9a950ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java
@@ -69,13 +69,6 @@ public class IgfsMock implements IgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsPaths proxyPaths() {
-        throwUnsupported();
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public IgfsInputStream open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch)
         throws IgniteException {
         throwUnsupported();

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
index ee71503..82f641f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
@@ -30,7 +30,6 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
-import org.apache.ignite.internal.processors.hadoop.HadoopPayloadAware;
 import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils;
 import org.apache.ignite.internal.processors.hadoop.delegate.HadoopIgfsSecondaryFileSystemDelegate;
 import org.apache.ignite.lang.IgniteOutClosure;
@@ -48,8 +47,7 @@ import java.util.concurrent.Callable;
  * <p>
  * Target {@code FileSystem}'s are created on per-user basis using passed {@link HadoopFileSystemFactory}.
  */
-public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, LifecycleAware,
-    HadoopPayloadAware {
+public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, LifecycleAware {
     /** The default user name. It is used if no user context is set. */
     private String dfltUsrName;
 
@@ -287,9 +285,4 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
         if (target != null)
             target.stop();
     }
-
-    /** {@inheritDoc} */
-    @Override public HadoopFileSystemFactory getPayload() {
-        return factory;
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/feba9534/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 866fc1a..efb1a45 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
@@ -17,6 +17,18 @@
 
 package org.apache.ignite.hadoop.fs.v1;
 
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
@@ -30,55 +42,28 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.configuration.FileSystemConfiguration;
-import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory;
-import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsFile;
-import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathSummary;
 import org.apache.ignite.internal.igfs.common.IgfsLogger;
-import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils;
-import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate;
 import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInputStream;
 import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsOutputStream;
-import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsProxyInputStream;
-import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsProxyOutputStream;
 import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsStreamDelegate;
 import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsWrapper;
 import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse;
 import org.apache.ignite.internal.processors.igfs.IgfsModeResolver;
-import org.apache.ignite.internal.processors.igfs.IgfsPaths;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_BATCH_SIZE;
 import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_DIR;
-import static org.apache.ignite.igfs.IgfsMode.PROXY;
 import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_COLOCATED_WRITES;
 import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_BATCH_SIZE;
 import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_DIR;
@@ -125,15 +110,9 @@ import static org.apache.ignite.internal.processors.igfs.IgfsEx.IGFS_SCHEME;
  * and {@code config/hadoop/default-config.xml} configuration files in Ignite installation.
  */
 public class IgniteHadoopFileSystem extends FileSystem {
-    /** Internal property to indicate management connection. */
-    public static final String IGFS_MANAGEMENT = "fs.igfs.management.connection";
-
     /** Empty array of file block locations. */
     private static final BlockLocation[] EMPTY_BLOCK_LOCATIONS = new BlockLocation[0];
 
-    /** Empty array of file statuses. */
-    public static final FileStatus[] EMPTY_FILE_STATUS = new FileStatus[0];
-
     /** Ensures that close routine is invoked at most once. */
     private final AtomicBoolean closeGuard = new AtomicBoolean();
 
@@ -156,21 +135,9 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /** Client logger. */
     private IgfsLogger clientLog;
 
-    /** Secondary URI string. */
-    private URI secondaryUri;
-
     /** The user name this file system was created on behalf of. */
     private String user;
 
-    /** IGFS mode resolver. */
-    private IgfsModeResolver modeRslvr;
-
-    /** The secondary file system factory. */
-    private HadoopFileSystemFactoryDelegate factory;
-
-    /** Management connection flag. */
-    private boolean mgmt;
-
     /** Whether custom sequential reads before prefetch value is provided. */
     private boolean seqReadsBeforePrefetchOverride;
 
@@ -214,6 +181,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /**
      * Gets non-null user name as per the Hadoop file system viewpoint.
      * @return the user name, never null.
+     * @throws IOException On error.
      */
     public static String getFsHadoopUser() throws IOException {
         UserGroupInformation currUgi = UserGroupInformation.getCurrentUser();
@@ -253,8 +221,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             setConf(cfg);
 
-            mgmt = cfg.getBoolean(IGFS_MANAGEMENT, false);
-
             if (!IGFS_SCHEME.equals(name.getScheme()))
                 throw new IOException("Illegal file system URI [expected=" + IGFS_SCHEME +
                     "://[name]/[optional_path], actual=" + name + ']');
@@ -293,8 +259,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             igfsGrpBlockSize = handshake.blockSize();
 
-            IgfsPaths paths = handshake.secondaryPaths();
-
             // Initialize client logger.
             Boolean logEnabled = parameter(cfg, PARAM_IGFS_LOG_ENABLED, uriAuthority, false);
 
@@ -310,61 +274,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
             else
                 clientLog = IgfsLogger.disabledLogger();
 
-            try {
-                modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
-            }
-            catch (IgniteCheckedException ice) {
-                throw new IOException(ice);
-            }
-
-            boolean initSecondary = paths.defaultMode() == PROXY;
-
-            if (!initSecondary && paths.pathModes() != null && !paths.pathModes().isEmpty()) {
-                for (T2<IgfsPath, IgfsMode> pathMode : paths.pathModes()) {
-                    IgfsMode mode = pathMode.getValue();
-
-                    if (mode == PROXY) {
-                        initSecondary = true;
-
-                        break;
-                    }
-                }
-            }
-
-            if (initSecondary) {
-                try {
-                    HadoopFileSystemFactory factory0 =
-                        (HadoopFileSystemFactory)paths.getPayload(getClass().getClassLoader());
-
-                    factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0);
-                }
-                catch (IgniteCheckedException e) {
-                    throw new IOException("Failed to get secondary file system factory.", e);
-                }
-
-                if (factory == null)
-                    throw new IOException("Failed to get secondary file system factory (did you set " +
-                        IgniteHadoopIgfsSecondaryFileSystem.class.getName() + " as \"secondaryFIleSystem\" in " +
-                        FileSystemConfiguration.class.getName() + "?)");
-
-                factory.start();
-
-                try {
-                    FileSystem secFs = (FileSystem)factory.get(user);
-
-                    secondaryUri = secFs.getUri();
-
-                    A.ensure(secondaryUri != null, "Secondary file system uri should not be null.");
-                }
-                catch (IOException e) {
-                    if (!mgmt)
-                        throw new IOException("Failed to connect to the secondary file system: " + secondaryUri, e);
-                    else
-                        LOG.warn("Visor failed to create secondary file system (operations on paths with PROXY mode " +
-                            "will have no effect): " + e.getMessage());
-                }
-            }
-
             // set working directory to the home directory of the current Fs user:
             setWorkingDirectory(null);
         }
@@ -426,9 +335,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
         if (clientLog.isLogEnabled())
             clientLog.close();
 
-        if (factory != null)
-            factory.stop();
-
         // Reset initialized resources.
         uri = null;
         rmtClient = null;
@@ -441,23 +347,9 @@ public class IgniteHadoopFileSystem extends FileSystem {
         try {
             A.notNull(p, "p");
 
-            if (mode(p) == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
+            IgfsPath path = convert(p);
 
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    // No-op for management connection.
-                    return;
-                }
-
-                secondaryFs.setTimes(toSecondary(p), mtime, atime);
-            }
-            else {
-                IgfsPath path = convert(p);
-
-                rmtClient.setTimes(path, atime, mtime);
-            }
+            rmtClient.setTimes(path, atime, mtime);
         }
         finally {
             leaveBusy();
@@ -471,19 +363,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
         try {
             A.notNull(p, "p");
 
-            if (mode(p) == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
-
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    // No-op for management connection.
-                    return;
-                }
-
-                secondaryFs.setPermission(toSecondary(p), perm);
-            }
-            else if (rmtClient.update(convert(p), permission(perm)) == null) {
+            if (rmtClient.update(convert(p), permission(perm)) == null) {
                 throw new IOException("Failed to set file permission (file not found?)" +
                     " [path=" + p + ", perm=" + perm + ']');
             }
@@ -502,19 +382,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
         enterBusy();
 
         try {
-            if (mode(p) == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
-
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    // No-op for management connection.
-                    return;
-                }
-
-                secondaryFs.setOwner(toSecondary(p), username, grpName);
-            }
-            else if (rmtClient.update(convert(p), F.asMap(IgfsUtils.PROP_USER_NAME, username,
+            if (rmtClient.update(convert(p), F.asMap(IgfsUtils.PROP_USER_NAME, username,
                 IgfsUtils.PROP_GROUP_NAME, grpName)) == null) {
                 throw new IOException("Failed to set file permission (file not found?)" +
                     " [path=" + p + ", userName=" + username + ", groupName=" + grpName + ']');
@@ -533,58 +401,29 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
         try {
             IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (mode == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
-
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to open file (secondary file system is not initialized): " + f);
-                }
-
-                FSDataInputStream is = secondaryFs.open(toSecondary(f), bufSize);
-
-                if (clientLog.isLogEnabled()) {
-                    // At this point we do not know file size, so we perform additional request to remote FS to get it.
-                    FileStatus status = secondaryFs.getFileStatus(toSecondary(f));
 
-                    long size = status != null ? status.getLen() : -1;
+            HadoopIgfsStreamDelegate stream = seqReadsBeforePrefetchOverride ?
+                rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
 
-                    long logId = IgfsLogger.nextId();
+            long logId = -1;
 
-                    clientLog.logOpen(logId, path, PROXY, bufSize, size);
+            if (clientLog.isLogEnabled()) {
+                logId = IgfsLogger.nextId();
 
-                    return new FSDataInputStream(new HadoopIgfsProxyInputStream(is, clientLog, logId));
-                }
-                else
-                    return is;
+                clientLog.logOpen(logId, path, bufSize, stream.length());
             }
-            else {
-                HadoopIgfsStreamDelegate stream = seqReadsBeforePrefetchOverride ?
-                    rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
-
-                long logId = -1;
 
-                if (clientLog.isLogEnabled()) {
-                    logId = IgfsLogger.nextId();
-
-                    clientLog.logOpen(logId, path, mode, bufSize, stream.length());
-                }
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
-                        ", bufSize=" + bufSize + ']');
+            if (LOG.isDebugEnabled())
+                LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
+                    ", bufSize=" + bufSize + ']');
 
-                HadoopIgfsInputStream igfsIn = new HadoopIgfsInputStream(stream, stream.length(),
-                    bufSize, LOG, clientLog, logId);
+            HadoopIgfsInputStream igfsIn = new HadoopIgfsInputStream(stream, stream.length(),
+                bufSize, LOG, clientLog, logId);
 
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opened input stream [path=" + path + ", delegate=" + stream + ']');
+            if (LOG.isDebugEnabled())
+                LOG.debug("Opened input stream [path=" + path + ", delegate=" + stream + ']');
 
-                return new FSDataInputStream(igfsIn);
-            }
+            return new FSDataInputStream(igfsIn);
         }
         finally {
             leaveBusy();
@@ -603,70 +442,45 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
         try {
             IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
 
             if (LOG.isDebugEnabled())
                 LOG.debug("Opening output stream in create [thread=" + Thread.currentThread().getName() + "path=" +
                     path + ", overwrite=" + overwrite + ", bufSize=" + bufSize + ']');
 
-            if (mode == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
+            Map<String,String> propMap = permission(perm);
 
-                if (secondaryFs == null) {
-                    assert mgmt;
+            propMap.put(IgfsUtils.PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites));
 
-                    throw new IOException("Failed to create file (secondary file system is not initialized): " + f);
-                }
+            // Create stream and close it in the 'finally' section if any sequential operation failed.
+            HadoopIgfsStreamDelegate stream = rmtClient.create(path, overwrite, colocateFileWrites,
+                replication, blockSize, propMap);
 
-                FSDataOutputStream os =
-                    secondaryFs.create(toSecondary(f), perm, overwrite, bufSize, replication, blockSize, progress);
+            assert stream != null;
 
-                if (clientLog.isLogEnabled()) {
-                    long logId = IgfsLogger.nextId();
+            long logId = -1;
 
-                    clientLog.logCreate(logId, path, PROXY, overwrite, bufSize, replication, blockSize);
+            if (clientLog.isLogEnabled()) {
+                logId = IgfsLogger.nextId();
 
-                    return new FSDataOutputStream(new HadoopIgfsProxyOutputStream(os, clientLog, logId));
-                }
-                else
-                    return os;
+                clientLog.logCreate(logId, path, overwrite, bufSize, replication, blockSize);
             }
-            else {
-                Map<String,String> propMap = permission(perm);
-
-                propMap.put(IgfsUtils.PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites));
-
-                // Create stream and close it in the 'finally' section if any sequential operation failed.
-                HadoopIgfsStreamDelegate stream = rmtClient.create(path, overwrite, colocateFileWrites,
-                    replication, blockSize, propMap);
-
-                assert stream != null;
-
-                long logId = -1;
-
-                if (clientLog.isLogEnabled()) {
-                    logId = IgfsLogger.nextId();
 
-                    clientLog.logCreate(logId, path, mode, overwrite, bufSize, replication, blockSize);
-                }
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
+            if (LOG.isDebugEnabled())
+                LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
 
-                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
-                    logId);
+            HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
+                logId);
 
-                bufSize = Math.max(64 * 1024, bufSize);
+            bufSize = Math.max(64 * 1024, bufSize);
 
-                out = new BufferedOutputStream(igfsOut, bufSize);
+            out = new BufferedOutputStream(igfsOut, bufSize);
 
-                FSDataOutputStream res = new FSDataOutputStream(out, null, 0);
+            FSDataOutputStream res = new FSDataOutputStream(out, null, 0);
 
-                // Mark stream created successfully.
-                out = null;
+            // Mark stream created successfully.
+            out = null;
 
-                return res;
-            }
+            return res;
         }
         finally {
             // Close if failed during stream creation.
@@ -686,58 +500,34 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
         try {
             IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
 
             if (LOG.isDebugEnabled())
                 LOG.debug("Opening output stream in append [thread=" + Thread.currentThread().getName() +
                     ", path=" + path + ", bufSize=" + bufSize + ']');
 
-            if (mode == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
-
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to append file (secondary file system is not initialized): " + f);
-                }
+            HadoopIgfsStreamDelegate stream = rmtClient.append(path, false, null);
 
-                FSDataOutputStream os = secondaryFs.append(toSecondary(f), bufSize, progress);
+            assert stream != null;
 
-                if (clientLog.isLogEnabled()) {
-                    long logId = IgfsLogger.nextId();
+            long logId = -1;
 
-                    clientLog.logAppend(logId, path, PROXY, bufSize); // Don't have stream ID.
+            if (clientLog.isLogEnabled()) {
+                logId = IgfsLogger.nextId();
 
-                    return new FSDataOutputStream(new HadoopIgfsProxyOutputStream(os, clientLog, logId));
-                }
-                else
-                    return os;
+                clientLog.logAppend(logId, path, bufSize);
             }
-            else {
-                HadoopIgfsStreamDelegate stream = rmtClient.append(path, false, null);
-
-                assert stream != null;
-
-                long logId = -1;
-
-                if (clientLog.isLogEnabled()) {
-                    logId = IgfsLogger.nextId();
 
-                    clientLog.logAppend(logId, path, mode, bufSize);
-                }
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
+            if (LOG.isDebugEnabled())
+                LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
 
-                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
-                    logId);
+            HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
+                logId);
 
-                bufSize = Math.max(64 * 1024, bufSize);
+            bufSize = Math.max(64 * 1024, bufSize);
 
-                BufferedOutputStream out = new BufferedOutputStream(igfsOut, bufSize);
+            BufferedOutputStream out = new BufferedOutputStream(igfsOut, bufSize);
 
-                return new FSDataOutputStream(out, null, 0);
-            }
+            return new FSDataOutputStream(out, null, 0);
         }
         finally {
             leaveBusy();
@@ -755,39 +545,22 @@ public class IgniteHadoopFileSystem extends FileSystem {
         try {
             IgfsPath srcPath = convert(src);
             IgfsPath dstPath = convert(dst);
-            IgfsMode mode = mode(srcPath);
-
-            if (mode == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
 
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return false;
-                }
+            if (clientLog.isLogEnabled())
+                clientLog.logRename(srcPath, dstPath);
 
-                if (clientLog.isLogEnabled())
-                    clientLog.logRename(srcPath, PROXY, dstPath);
-
-                return secondaryFs.rename(toSecondary(src), toSecondary(dst));
+            try {
+                rmtClient.rename(srcPath, dstPath);
             }
-            else {
-                if (clientLog.isLogEnabled())
-                    clientLog.logRename(srcPath, mode, dstPath);
+            catch (IOException ioe) {
+                // Log the exception before rethrowing since it may be ignored:
+                LOG.warn("Failed to rename [srcPath=" + srcPath + ", dstPath=" + dstPath + ']',
+                    ioe);
 
-                try {
-                    rmtClient.rename(srcPath, dstPath);
-                }
-                catch (IOException ioe) {
-                    // Log the exception before rethrowing since it may be ignored:
-                    LOG.warn("Failed to rename [srcPath=" + srcPath + ", dstPath=" + dstPath + ", mode=" + mode + ']',
-                        ioe);
-
-                    throw ioe;
-                }
-
-                return true;
+                throw ioe;
             }
+
+            return true;
         }
         catch (IOException e) {
             // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
@@ -817,31 +590,14 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
         try {
             IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (mode == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
 
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return false;
-                }
+            // Will throw exception if delete failed.
+            boolean res = rmtClient.delete(path, recursive);
 
-                if (clientLog.isLogEnabled())
-                    clientLog.logDelete(path, PROXY, recursive);
+            if (clientLog.isLogEnabled())
+                clientLog.logDelete(path, recursive);
 
-                return secondaryFs.delete(toSecondary(f), recursive);
-            }
-            else {
-                // Will throw exception if delete failed.
-                boolean res = rmtClient.delete(path, recursive);
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logDelete(path, mode, recursive);
-
-                return res;
-            }
+            return res;
         }
         catch (IOException e) {
             // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
@@ -864,60 +620,29 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
         try {
             IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
 
-            if (mode == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
+            Collection<IgfsFile> list = rmtClient.listFiles(path);
 
-                if (secondaryFs == null) {
-                    assert mgmt;
+            if (list == null)
+                throw new FileNotFoundException("File " + f + " does not exist.");
 
-                    return EMPTY_FILE_STATUS;
-                }
-
-                FileStatus[] arr = secondaryFs.listStatus(toSecondary(f));
+            List<IgfsFile> files = new ArrayList<>(list);
 
-                if (arr == null)
-                    throw new FileNotFoundException("File " + f + " does not exist.");
+            FileStatus[] arr = new FileStatus[files.size()];
 
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = toPrimary(arr[i]);
+            for (int i = 0; i < arr.length; i++)
+                arr[i] = convert(files.get(i));
 
-                if (clientLog.isLogEnabled()) {
-                    String[] fileArr = new String[arr.length];
-
-                    for (int i = 0; i < arr.length; i++)
-                        fileArr[i] = arr[i].getPath().toString();
-
-                    clientLog.logListDirectory(path, PROXY, fileArr);
-                }
-
-                return arr;
-            }
-            else {
-                Collection<IgfsFile> list = rmtClient.listFiles(path);
-
-                if (list == null)
-                    throw new FileNotFoundException("File " + f + " does not exist.");
-
-                List<IgfsFile> files = new ArrayList<>(list);
-
-                FileStatus[] arr = new FileStatus[files.size()];
+            if (clientLog.isLogEnabled()) {
+                String[] fileArr = new String[arr.length];
 
                 for (int i = 0; i < arr.length; i++)
-                    arr[i] = convert(files.get(i));
-
-                if (clientLog.isLogEnabled()) {
-                    String[] fileArr = new String[arr.length];
+                    fileArr[i] = arr[i].getPath().toString();
 
-                    for (int i = 0; i < arr.length; i++)
-                        fileArr[i] = arr[i].getPath().toString();
-
-                    clientLog.logListDirectory(path, mode, fileArr);
-                }
-
-                return arr;
+                clientLog.logListDirectory(path, fileArr);
             }
+
+            return arr;
         }
         finally {
             leaveBusy();
@@ -933,35 +658,17 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
     /** {@inheritDoc} */
     @Override public void setWorkingDirectory(Path newPath) {
-        try {
-            if (newPath == null) {
-                Path homeDir = getHomeDirectory();
+        if (newPath == null)
+            workingDir = getHomeDirectory();
+        else {
+            Path fixedNewPath = fixRelativePart(newPath);
 
-                FileSystem secondaryFs  = secondaryFileSystem();
+            String res = fixedNewPath.toUri().getPath();
 
-                if (secondaryFs != null)
-                    secondaryFs.setWorkingDirectory(toSecondary(homeDir));
+            if (!DFSUtil.isValidName(res))
+                throw new IllegalArgumentException("Invalid DFS directory name " + res);
 
-                workingDir = homeDir;
-            }
-            else {
-                Path fixedNewPath = fixRelativePart(newPath);
-
-                String res = fixedNewPath.toUri().getPath();
-
-                if (!DFSUtil.isValidName(res))
-                    throw new IllegalArgumentException("Invalid DFS directory name " + res);
-
-                FileSystem secondaryFs  = secondaryFileSystem();
-
-                if (secondaryFs != null)
-                    secondaryFs.setWorkingDirectory(toSecondary(fixedNewPath));
-
-                workingDir = fixedNewPath;
-            }
-        }
-        catch (IOException e) {
-            throw new RuntimeException("Failed to obtain secondary file system instance.", e);
+            workingDir = fixedNewPath;
         }
     }
 
@@ -979,30 +686,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
         try {
             IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (mode == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
 
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return false;
-                }
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logMakeDirectory(path, PROXY);
-
-                return secondaryFs.mkdirs(toSecondary(f), perm);
-            }
-            else {
-                boolean mkdirRes = rmtClient.mkdirs(path, permission(perm));
+            boolean mkdirRes = rmtClient.mkdirs(path, permission(perm));
 
-                if (clientLog.isLogEnabled())
-                    clientLog.logMakeDirectory(path, mode);
+            if (clientLog.isLogEnabled())
+                clientLog.logMakeDirectory(path);
 
-                return mkdirRes;
-            }
+            return mkdirRes;
         }
         catch (IOException e) {
             // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
@@ -1024,25 +714,12 @@ public class IgniteHadoopFileSystem extends FileSystem {
         enterBusy();
 
         try {
-            if (mode(f) == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
-
-                if (secondaryFs == null) {
-                    assert mgmt;
+            IgfsFile info = rmtClient.info(convert(f));
 
-                    throw new IOException("Failed to get file status (secondary file system is not initialized): " + f);
-                }
+            if (info == null)
+                throw new FileNotFoundException("File not found: " + f);
 
-                return toPrimary(secondaryFs.getFileStatus(toSecondary(f)));
-            }
-            else {
-                IgfsFile info = rmtClient.info(convert(f));
-
-                if (info == null)
-                    throw new FileNotFoundException("File not found: " + f);
-
-                return convert(info);
-            }
+            return convert(info);
         }
         finally {
             leaveBusy();
@@ -1056,24 +733,10 @@ public class IgniteHadoopFileSystem extends FileSystem {
         enterBusy();
 
         try {
-            if (mode(f) == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
+            IgfsPathSummary sum = rmtClient.contentSummary(convert(f));
 
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to get content summary (secondary file system is not initialized): " +
-                        f);
-                }
-
-                return secondaryFs.getContentSummary(toSecondary(f));
-            }
-            else {
-                IgfsPathSummary sum = rmtClient.contentSummary(convert(f));
-
-                return new ContentSummary(sum.totalLength(), sum.filesCount(), sum.directoriesCount(),
-                    -1, sum.totalLength(), rmtClient.fsStatus().spaceTotal());
-            }
+            return new ContentSummary(sum.totalLength(), sum.filesCount(), sum.directoriesCount(),
+                -1, sum.totalLength(), rmtClient.fsStatus().spaceTotal());
         }
         finally {
             leaveBusy();
@@ -1089,35 +752,20 @@ public class IgniteHadoopFileSystem extends FileSystem {
         try {
             IgfsPath path = convert(status.getPath());
 
-            if (mode(status.getPath()) == PROXY) {
-                final FileSystem secondaryFs = secondaryFileSystem();
-
-                if (secondaryFs == null) {
-                    assert mgmt;
+            long now = System.currentTimeMillis();
 
-                    return EMPTY_BLOCK_LOCATIONS;
-                }
-
-                Path secPath = toSecondary(status.getPath());
-
-                return secondaryFs.getFileBlockLocations(secondaryFs.getFileStatus(secPath), start, len);
-            }
-            else {
-                long now = System.currentTimeMillis();
+            List<IgfsBlockLocation> affinity = new ArrayList<>(rmtClient.affinity(path, start, len));
 
-                List<IgfsBlockLocation> affinity = new ArrayList<>(rmtClient.affinity(path, start, len));
+            BlockLocation[] arr = new BlockLocation[affinity.size()];
 
-                BlockLocation[] arr = new BlockLocation[affinity.size()];
+            for (int i = 0; i < arr.length; i++)
+                arr[i] = convert(affinity.get(i));
 
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = convert(affinity.get(i));
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Fetched file locations [path=" + path + ", fetchTime=" +
-                        (System.currentTimeMillis() - now) + ", locations=" + Arrays.asList(arr) + ']');
+            if (LOG.isDebugEnabled())
+                LOG.debug("Fetched file locations [path=" + path + ", fetchTime=" +
+                    (System.currentTimeMillis() - now) + ", locations=" + Arrays.asList(arr) + ']');
 
-                return arr;
-            }
+            return arr;
         }
         catch (FileNotFoundException ignored) {
             return EMPTY_BLOCK_LOCATIONS;
@@ -1134,92 +782,18 @@ public class IgniteHadoopFileSystem extends FileSystem {
     }
 
     /**
-     * Resolve path mode.
-     *
-     * @param path HDFS path.
-     * @return Path mode.
-     */
-    public IgfsMode mode(Path path) {
-        return mode(convert(path));
-    }
-
-    /**
-     * Resolve path mode.
-     *
-     * @param path IGFS path.
-     * @return Path mode.
-     */
-    public IgfsMode mode(IgfsPath path) {
-        return modeRslvr.resolveMode(path);
-    }
-
-    /**
-     * @return {@code true} If secondary file system is initialized.
-     */
-    public boolean hasSecondaryFileSystem() {
-        return factory != null;
-    }
-
-    /**
-     * Convert the given path to path acceptable by the primary file system.
-     *
-     * @param path Path.
-     * @return Primary file system path.
-     */
-    private Path toPrimary(Path path) {
-        return convertPath(path, uri);
-    }
-
-    /**
-     * Convert the given path to path acceptable by the secondary file system.
-     *
-     * @param path Path.
-     * @return Secondary file system path.
-     */
-    private Path toSecondary(Path path) {
-        assert factory != null;
-        assert secondaryUri != null;
-
-        return convertPath(path, secondaryUri);
-    }
-
-    /**
-     * Convert path using the given new URI.
-     *
-     * @param path Old path.
-     * @param newUri New URI.
-     * @return New path.
+     * @return Mode resolver.
+     * @throws IOException On error.
      */
-    private Path convertPath(Path path, URI newUri) {
-        assert newUri != null;
-
-        if (path != null) {
-            URI pathUri = path.toUri();
+    public IgfsModeResolver getModeResolver() throws IOException {
+        enterBusy();
 
-            try {
-                return new Path(new URI(pathUri.getScheme() != null ? newUri.getScheme() : null,
-                    pathUri.getAuthority() != null ? newUri.getAuthority() : null, pathUri.getPath(), null, null));
-            }
-            catch (URISyntaxException e) {
-                throw new IgniteException("Failed to construct secondary file system path from the primary file " +
-                    "system path: " + path, e);
-            }
+        try {
+            return rmtClient.modeResolver();
+        }
+        finally {
+            leaveBusy();
         }
-        else
-            return null;
-    }
-
-    /**
-     * Convert a file status obtained from the secondary file system to a status of the primary file system.
-     *
-     * @param status Secondary file system status.
-     * @return Primary file system status.
-     */
-    @SuppressWarnings("deprecation")
-    private FileStatus toPrimary(FileStatus status) {
-        return status != null ? new FileStatus(status.getLen(), status.isDir(), status.getReplication(),
-            status.getBlockSize(), status.getModificationTime(), status.getAccessTime(), status.getPermission(),
-            status.getOwner(), status.getGroup(), toPrimary(status.getPath())) : null;
     }
 
     /**
@@ -1352,16 +926,4 @@ public class IgniteHadoopFileSystem extends FileSystem {
     public String user() {
         return user;
     }
-
-    /**
-     * Gets cached or creates a {@link FileSystem}.
-     *
-     * @return The secondary file system.
-     */
-    private @Nullable FileSystem secondaryFileSystem() throws IOException{
-        if (factory == null)
-            return null;
-
-        return (FileSystem)factory.get(user);
-    }
 }
\ No newline at end of file