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 2017/03/02 11:30:59 UTC

[22/50] [abbrv] ignite git commit: IGNITE-2356: Added high-availability feature to IGFS client. From now on it is possible to connect to IGFS through Ignite client. This way in case one server fails, client will still be able to work with other nodes. Th

IGNITE-2356: Added high-availability feature to IGFS client. From now on it is possible to connect to IGFS through Ignite client. This way in case one server fails, client will still be able to work with other nodes. This closes #1155. This closes #1251.


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

Branch: refs/heads/ignite-4565-ddl
Commit: 6bdff2c31ae9e754d38b2bc5504198c77b173be6
Parents: 4a78341
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Wed Feb 15 18:42:56 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Feb 15 18:42:56 2017 +0300

----------------------------------------------------------------------
 .../local/LocalIgfsSecondaryFileSystem.java     |   6 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  59 +++-
 .../internal/processors/igfs/IgfsImpl.java      |  30 +-
 .../processors/igfs/IgfsMetaManager.java        |  14 +-
 .../igfs/client/IgfsClientAbstractCallable.java |  23 +-
 .../igfs/client/IgfsClientAffinityCallable.java |   6 +-
 .../igfs/client/IgfsClientDeleteCallable.java   |   6 +-
 .../igfs/client/IgfsClientExistsCallable.java   |   5 +-
 .../igfs/client/IgfsClientInfoCallable.java     |   5 +-
 .../client/IgfsClientListFilesCallable.java     |   5 +-
 .../client/IgfsClientListPathsCallable.java     |   5 +-
 .../igfs/client/IgfsClientMkdirsCallable.java   |   6 +-
 .../igfs/client/IgfsClientRenameCallable.java   |   6 +-
 .../igfs/client/IgfsClientSetTimesCallable.java |   6 +-
 .../igfs/client/IgfsClientSizeCallable.java     |   5 +-
 .../igfs/client/IgfsClientSummaryCallable.java  |   5 +-
 .../igfs/client/IgfsClientUpdateCallable.java   |   6 +-
 .../meta/IgfsClientMetaIdsForPathCallable.java  |   5 +-
 .../meta/IgfsClientMetaInfoForPathCallable.java |   5 +-
 .../meta/IgfsClientMetaUnlockCallable.java      |   8 +-
 .../secondary/local/LocalFileSystemUtils.java   |  25 ++
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../testframework/junits/GridAbstractTest.java  |  21 +-
 .../junits/multijvm/IgniteNodeRunner.java       |  55 +++-
 .../junits/multijvm/IgniteProcessProxy.java     |  58 ++--
 modules/hadoop/pom.xml                          |   7 +
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      | 190 ++++++++++--
 .../hadoop/impl/igfs/HadoopIgfsUtils.java       |   6 +
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  99 +++---
 .../test/config/hadoop-fs-open-test/grid-0.xml  | 141 +++++++++
 .../test/config/hadoop-fs-open-test/grid-1.xml  | 141 +++++++++
 .../test/config/hadoop-fs-open-test/grid-2.xml  | 141 +++++++++
 .../test/config/igfs-cli-config-dual-async.xml  | 150 +++++++++
 .../test/config/igfs-cli-config-dual-sync.xml   | 148 +++++++++
 .../src/test/config/igfs-cli-config-primary.xml | 139 +++++++++
 .../src/test/config/igfs-cli-config-proxy.xml   | 148 +++++++++
 .../IgniteHadoopFileSystemAbstractSelfTest.java | 117 ++++---
 ...opFileSystemClientBasedAbstractSelfTest.java | 193 ++++++++++++
 ...pFileSystemClientBasedDualAsyncSelfTest.java |  38 +++
 ...opFileSystemClientBasedDualSyncSelfTest.java |  38 +++
 ...niteHadoopFileSystemClientBasedOpenTest.java | 304 +++++++++++++++++++
 ...oopFileSystemClientBasedPrimarySelfTest.java |  38 +++
 ...adoopFileSystemClientBasedProxySelfTest.java |  37 +++
 ...oopbackExternalToClientAbstractSelfTest.java |  61 ++++
 ...opbackExternalToClientDualAsyncSelfTest.java |  33 ++
 ...oopbackExternalToClientDualSyncSelfTest.java |  33 ++
 ...LoopbackExternalToClientPrimarySelfTest.java |  33 ++
 ...emLoopbackExternalToClientProxySelfTest.java |  33 ++
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |  33 --
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |  33 --
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |  33 --
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |  33 --
 ...emShmemExternalToClientAbstractSelfTest.java | 106 +++++++
 ...mShmemExternalToClientDualAsyncSelfTest.java |  33 ++
 ...emShmemExternalToClientDualSyncSelfTest.java |  33 ++
 ...temShmemExternalToClientPrimarySelfTest.java |  33 ++
 ...ystemShmemExternalToClientProxySelfTest.java |  33 ++
 .../testsuites/IgniteHadoopTestSuite.java       |  21 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |  17 +-
 59 files changed, 2700 insertions(+), 353 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
index 86f7387..209c6d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
@@ -366,13 +366,15 @@ public class LocalIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, Li
 
         Map<String, String> props = LocalFileSystemUtils.posixAttributesToMap(attrs);
 
+        BasicFileAttributes basicAttrs = LocalFileSystemUtils.basicAttributes(file);
+
         if (isDir) {
             return new LocalFileSystemIgfsFile(path, false, true, 0,
-                attrs.lastAccessTime().toMillis(), attrs.lastModifiedTime().toMillis(), 0, props);
+                basicAttrs.lastAccessTime().toMillis(), basicAttrs.lastModifiedTime().toMillis(), 0, props);
         }
         else {
             return new LocalFileSystemIgfsFile(path, file.isFile(), false, 0,
-                attrs.lastAccessTime().toMillis(), attrs.lastModifiedTime().toMillis(), file.length(), props);
+                basicAttrs.lastAccessTime().toMillis(), basicAttrs.lastModifiedTime().toMillis(), file.length(), props);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 42ff739..05d19c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -45,6 +45,7 @@ import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteState;
@@ -71,6 +72,7 @@ import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
 import org.apache.ignite.internal.util.typedef.CA;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -173,7 +175,7 @@ public class IgnitionEx {
     /** */
     private static ThreadLocal<Boolean> clientMode = new ThreadLocal<>();
 
-    /**
+    /*
      * Checks runtime version to be 1.7.x or 1.8.x.
      * This will load pretty much first so we must do these checks here.
      */
@@ -499,7 +501,8 @@ public class IgnitionEx {
 
         U.warn(null, "Default Spring XML file not found (is IGNITE_HOME set?): " + DFLT_CFG);
 
-        return start0(new GridStartContext(new IgniteConfiguration(), null, springCtx), true).grid();
+        return start0(new GridStartContext(new IgniteConfiguration(), null, springCtx), true)
+            .get1().grid();
     }
 
     /**
@@ -512,7 +515,7 @@ public class IgnitionEx {
      *      also if named grid has already been started.
      */
     public static Ignite start(IgniteConfiguration cfg) throws IgniteCheckedException {
-        return start(cfg, null, true);
+        return start(cfg, null, true).get1();
     }
 
     /**
@@ -521,14 +524,35 @@ public class IgnitionEx {
      *
      * @param cfg Grid configuration. This cannot be {@code null}.
      * failIfStarted Throw or not an exception if grid is already started.
-     * @return Started grid.
+     * @param failIfStarted When flag is {@code true} and grid with specified name has been already started
+     *      the exception is thrown. Otherwise the existing instance of the grid is returned.
+     * @return Started grid or existing grid.
      * @throws IgniteCheckedException If grid could not be started. This exception will be thrown
      *      also if named grid has already been started.
      */
     public static Ignite start(IgniteConfiguration cfg, boolean failIfStarted) throws IgniteCheckedException {
-        return start(cfg, null, failIfStarted);
+        return start(cfg, null, failIfStarted).get1();
+    }
+
+    /**
+     * Gets or starts new grid instance if it hasn't been started yet.
+     *
+     * @param cfg Grid configuration. This cannot be {@code null}.
+     * @return Tuple with: grid instance and flag to indicate the instance is started by this call.
+     *      So, when the new ignite instance is started the flag is {@code true}. If an existing instance is returned
+     *      the flag is {@code false}.
+     * @throws IgniteException If grid could not be started.
+     */
+    public static T2<Ignite, Boolean> getOrStart(IgniteConfiguration cfg) throws IgniteException {
+        try {
+            return start(cfg, null, false);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
+
     /**
      * Starts grid with given configuration. Note that this method will throw and exception if grid with the name
      * provided in given configuration is already started.
@@ -545,7 +569,7 @@ public class IgnitionEx {
     public static Ignite start(IgniteConfiguration cfg, @Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
         A.notNull(cfg, "cfg");
 
-        return start0(new GridStartContext(cfg, null, springCtx), true).grid();
+        return start0(new GridStartContext(cfg, null, springCtx), true).get1().grid();
     }
 
     /**
@@ -558,14 +582,18 @@ public class IgnitionEx {
      *      If provided, this context can be injected into grid tasks and grid jobs using
      *      {@link SpringApplicationContextResource @SpringApplicationContextResource} annotation.
      * @param failIfStarted Throw or not an exception if grid is already started.
-     * @return Started grid.
+     * @return Tuple with: grid instance and flag to indicate the instance is started by this call.
+     *      So, when the new ignite instance is started the flag is {@code true}. If an existing instance is returned
+     *      the flag is {@code false}.
      * @throws IgniteCheckedException If grid could not be started. This exception will be thrown
      *      also if named grid has already been started.
      */
-    public static Ignite start(IgniteConfiguration cfg, @Nullable GridSpringResourceContext springCtx, boolean failIfStarted) throws IgniteCheckedException {
+    public static T2<Ignite, Boolean> start(IgniteConfiguration cfg, @Nullable GridSpringResourceContext springCtx, boolean failIfStarted) throws IgniteCheckedException {
         A.notNull(cfg, "cfg");
 
-        return start0(new GridStartContext(cfg, null, springCtx), failIfStarted).grid();
+        T2<IgniteNamedInstance, Boolean> res = start0(new GridStartContext(cfg, null, springCtx), failIfStarted);
+
+        return new T2<>((Ignite)res.get1().grid(), res.get2());
     }
 
     /**
@@ -961,7 +989,8 @@ public class IgnitionEx {
 
                 // Use either user defined context or our one.
                 IgniteNamedInstance grid = start0(
-                    new GridStartContext(cfg, springCfgUrl, springCtx == null ? cfgMap.get2() : springCtx), true);
+                    new GridStartContext(cfg, springCfgUrl, springCtx == null
+                        ? cfgMap.get2() : springCtx), true).get1();
 
                 // Add it if it was not stopped during startup.
                 if (grid != null)
@@ -993,10 +1022,12 @@ public class IgnitionEx {
      *
      * @param startCtx Start context.
      * @param failIfStarted Throw or not an exception if grid is already started.
-     * @return Started grid.
+     * @return Tuple with: grid instance and flag to indicate the instance is started by this call.
+     *      So, when the new ignite instance is started the flag is {@code true}. If an existing instance is returned
+     *      the flag is {@code false}.
      * @throws IgniteCheckedException If grid could not be started.
      */
-    private static IgniteNamedInstance start0(GridStartContext startCtx, boolean failIfStarted ) throws IgniteCheckedException {
+    private static T2<IgniteNamedInstance, Boolean> start0(GridStartContext startCtx, boolean failIfStarted ) throws IgniteCheckedException {
         assert startCtx != null;
 
         String name = startCtx.config().getGridName();
@@ -1027,7 +1058,7 @@ public class IgnitionEx {
                     throw new IgniteCheckedException("Ignite instance with this name has already been started: " + name);
             }
             else
-                return old;
+                return new T2<>(old, false);
 
         if (startCtx.config().getWarmupClosure() != null)
             startCtx.config().getWarmupClosure().apply(startCtx.config());
@@ -1069,7 +1100,7 @@ public class IgnitionEx {
         if (grid == null)
             throw new IgniteCheckedException("Failed to start grid with provided configuration.");
 
-        return grid;
+        return new T2<>(grid, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/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 59674f8..56da121 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
@@ -59,6 +59,7 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathIsDirectoryException;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
 import org.apache.ignite.igfs.IgfsPathSummary;
+import org.apache.ignite.igfs.IgfsUserContext;
 import org.apache.ignite.igfs.mapreduce.IgfsRecordResolver;
 import org.apache.ignite.igfs.mapreduce.IgfsTask;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
@@ -494,7 +495,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient())
-            return meta.runClientTask(new IgfsClientExistsCallable(cfg.getName(), path));
+            return meta.runClientTask(new IgfsClientExistsCallable(cfg.getName(), IgfsUserContext.currentUser(), path));
 
         return safeOp(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
@@ -544,7 +545,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient())
-            return meta.runClientTask(new IgfsClientInfoCallable(cfg.getName(), path));
+            return meta.runClientTask(new IgfsClientInfoCallable(cfg.getName(), IgfsUserContext.currentUser(), path));
 
         return safeOp(new Callable<IgfsFile>() {
             @Override public IgfsFile call() throws Exception {
@@ -570,7 +571,8 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient())
-            return meta.runClientTask(new IgfsClientSummaryCallable(cfg.getName(), path));
+            return meta.runClientTask(new IgfsClientSummaryCallable(cfg.getName(), IgfsUserContext.currentUser(),
+                path));
 
         return safeOp(new Callable<IgfsPathSummary>() {
             @Override public IgfsPathSummary call() throws Exception {
@@ -589,7 +591,8 @@ public final class IgfsImpl implements IgfsEx {
         A.ensure(!props.isEmpty(), "!props.isEmpty()");
 
         if (meta.isClient())
-            return meta.runClientTask(new IgfsClientUpdateCallable(cfg.getName(), path, props));
+            return meta.runClientTask(new IgfsClientUpdateCallable(cfg.getName(), IgfsUserContext.currentUser(),
+                path, props));
 
         return safeOp(new Callable<IgfsFile>() {
             @Override public IgfsFile call() throws Exception {
@@ -652,7 +655,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(dest, "dest");
 
         if (meta.isClient()) {
-            meta.runClientTask(new IgfsClientRenameCallable(cfg.getName(), src, dest));
+            meta.runClientTask(new IgfsClientRenameCallable(cfg.getName(), IgfsUserContext.currentUser(), src, dest));
 
             return;
         }
@@ -712,7 +715,8 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient())
-            return meta.runClientTask(new IgfsClientDeleteCallable(cfg.getName(), path, recursive));
+            return meta.runClientTask(new IgfsClientDeleteCallable(cfg.getName(), IgfsUserContext.currentUser(),
+                path, recursive));
 
         return safeOp(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
@@ -754,7 +758,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient()) {
-            meta.runClientTask(new IgfsClientMkdirsCallable(cfg.getName(), path, props));
+            meta.runClientTask(new IgfsClientMkdirsCallable(cfg.getName(), IgfsUserContext.currentUser(), path, props));
 
             return ;
         }
@@ -797,7 +801,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient())
-            meta.runClientTask(new IgfsClientListPathsCallable(cfg.getName(), path));
+            meta.runClientTask(new IgfsClientListPathsCallable(cfg.getName(), IgfsUserContext.currentUser(), path));
 
         return safeOp(new Callable<Collection<IgfsPath>>() {
             @Override public Collection<IgfsPath> call() throws Exception {
@@ -846,7 +850,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient())
-            meta.runClientTask(new IgfsClientListFilesCallable(cfg.getName(), path));
+            meta.runClientTask(new IgfsClientListFilesCallable(cfg.getName(), IgfsUserContext.currentUser(), path));
 
         return safeOp(new Callable<Collection<IgfsFile>>() {
             @Override public Collection<IgfsFile> call() throws Exception {
@@ -1223,7 +1227,8 @@ public final class IgfsImpl implements IgfsEx {
             return;
 
         if (meta.isClient()) {
-            meta.runClientTask(new IgfsClientSetTimesCallable(cfg.getName(), path, accessTime, modificationTime));
+            meta.runClientTask(new IgfsClientSetTimesCallable(cfg.getName(), IgfsUserContext.currentUser(), path,
+                accessTime, modificationTime));
 
             return;
         }
@@ -1257,7 +1262,8 @@ public final class IgfsImpl implements IgfsEx {
         A.ensure(len >= 0, "len >= 0");
 
         if (meta.isClient())
-            return meta.runClientTask(new IgfsClientAffinityCallable(cfg.getName(), path, start, len, maxLen));
+            return meta.runClientTask(new IgfsClientAffinityCallable(cfg.getName(), IgfsUserContext.currentUser(),
+                path, start, len, maxLen));
 
         return safeOp(new Callable<Collection<IgfsBlockLocation>>() {
             @Override public Collection<IgfsBlockLocation> call() throws Exception {
@@ -1343,7 +1349,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         if (meta.isClient())
-            return meta.runClientTask(new IgfsClientSizeCallable(cfg.getName(), path));
+            return meta.runClientTask(new IgfsClientSizeCallable(cfg.getName(), IgfsUserContext.currentUser(), path));
 
         return safeOp(new Callable<Long>() {
             @Override public Long call() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/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 4f579e8..95d73a8 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
@@ -38,6 +38,7 @@ import org.apache.ignite.igfs.IgfsPathAlreadyExistsException;
 import org.apache.ignite.igfs.IgfsPathIsDirectoryException;
 import org.apache.ignite.igfs.IgfsPathIsNotDirectoryException;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
+import org.apache.ignite.igfs.IgfsUserContext;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
 import org.apache.ignite.internal.IgniteEx;
@@ -423,7 +424,8 @@ public class IgfsMetaManager extends IgfsManager {
 
         // Get IDs.
         if (client) {
-            List<IgniteUuid> ids = runClientTask(new IgfsClientMetaIdsForPathCallable(cfg.getName(), path));
+            List<IgniteUuid> ids = runClientTask(new IgfsClientMetaIdsForPathCallable(cfg.getName(),
+                IgfsUserContext.currentUser(), path));
 
             return new IgfsPathIds(path, parts, ids.toArray(new IgniteUuid[ids.size()]));
         }
@@ -667,8 +669,8 @@ public class IgfsMetaManager extends IgfsManager {
         throws IgniteCheckedException {
 
         if(client) {
-            runClientTask(new IgfsClientMetaUnlockCallable(cfg.getName(), fileId, lockId, modificationTime,
-                updateSpace, space, affRange));
+            runClientTask(new IgfsClientMetaUnlockCallable(cfg.getName(), IgfsUserContext.currentUser(), fileId,
+                lockId, modificationTime, updateSpace, space, affRange));
 
             return;
         }
@@ -2073,7 +2075,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     @Nullable public IgfsEntryInfo infoForPath(IgfsPath path) throws IgniteCheckedException {
-        return client ? runClientTask(new IgfsClientMetaInfoForPathCallable(cfg.getName(), path)) : info(fileId(path));
+        return client ? runClientTask(new IgfsClientMetaInfoForPathCallable(cfg.getName(),
+            IgfsUserContext.currentUser(), path)) : info(fileId(path));
     }
 
     /**
@@ -2084,7 +2087,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     public List<IgniteUuid> idsForPath(IgfsPath path) throws IgniteCheckedException {
-        return client ? runClientTask(new IgfsClientMetaIdsForPathCallable(cfg.getName(), path)) : fileIds(path);
+        return client ? runClientTask(new IgfsClientMetaIdsForPathCallable(cfg.getName(),
+            IgfsUserContext.currentUser(), path)) : fileIds(path);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
index d9c3456..7a4b5b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.igfs.client;
 
+import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryRawReader;
@@ -25,6 +26,7 @@ import org.apache.ignite.binary.BinaryReader;
 import org.apache.ignite.binary.BinaryWriter;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.igfs.IgfsUserContext;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
@@ -45,7 +47,11 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
     /** Path for operation. */
     protected IgfsPath path;
 
+    /** User name. */
+    protected String user;
+
     /** Injected instance. */
+    @SuppressWarnings("UnusedDeclaration")
     @IgniteInstanceResource
     private transient Ignite ignite;
 
@@ -60,20 +66,29 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    protected IgfsClientAbstractCallable(@Nullable String igfsName, @Nullable IgfsPath path) {
+    protected IgfsClientAbstractCallable(@Nullable String igfsName, @Nullable String user, @Nullable IgfsPath path) {
         this.igfsName = igfsName;
         this.path = path;
+        this.user = user;
     }
 
     /** {@inheritDoc} */
     @Override public final T call() throws Exception {
         assert ignite != null;
 
-        IgfsEx igfs = (IgfsEx)ignite.fileSystem(igfsName);
+        final IgfsEx igfs = (IgfsEx)ignite.fileSystem(igfsName);
 
-        return call0(igfs.context());
+        if (user != null) {
+            return IgfsUserContext.doAs(user, new Callable<T>() {
+                @Override public T call() throws Exception {
+                    return call0(igfs.context());
+                }
+            });
+        } else
+            return call0(igfs.context());
     }
 
     /**
@@ -90,6 +105,7 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
         BinaryRawWriter rawWriter = writer.rawWriter();
 
         rawWriter.writeString(igfsName);
+        rawWriter.writeString(user);
         IgfsUtils.writePath(rawWriter, path);
 
         writeBinary0(rawWriter);
@@ -100,6 +116,7 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
         BinaryRawReader rawReader = reader.rawReader();
 
         igfsName = rawReader.readString();
+        user = rawReader.readString();
         path = IgfsUtils.readPath(rawReader);
 
         readBinary0(rawReader);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
index 1668f36..790d57e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
@@ -55,13 +55,15 @@ public class IgfsClientAffinityCallable extends IgfsClientAbstractCallable<Colle
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      * @param start Start.
      * @param len Length.
      * @param maxLen Maximum length.
      */
-    public IgfsClientAffinityCallable(@Nullable String igfsName, IgfsPath path, long start, long len, long maxLen) {
-        super(igfsName, path);
+    public IgfsClientAffinityCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path, long start,
+        long len, long maxLen) {
+        super(igfsName, user, path);
 
         this.start = start;
         this.len = len;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
index c1b8be8..349bed1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
@@ -46,11 +46,13 @@ public class IgfsClientDeleteCallable extends IgfsClientAbstractCallable<Boolean
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      * @param recursive Recursive flag.
      */
-    public IgfsClientDeleteCallable(@Nullable String igfsName, IgfsPath path, boolean recursive) {
-        super(igfsName, path);
+    public IgfsClientDeleteCallable(@Nullable String igfsName, @Nullable String user,
+        IgfsPath path, boolean recursive) {
+        super(igfsName, user, path);
 
         this.recursive = recursive;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
index 04b63d8..d8a0858 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
@@ -40,10 +40,11 @@ public class IgfsClientExistsCallable extends IgfsClientAbstractCallable<Boolean
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientExistsCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientExistsCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
index f97c3c4..fef6869 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
@@ -41,10 +41,11 @@ public class IgfsClientInfoCallable extends IgfsClientAbstractCallable<IgfsFile>
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientInfoCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientInfoCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
index 325e714..4414884 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
@@ -43,10 +43,11 @@ public class IgfsClientListFilesCallable extends IgfsClientAbstractCallable<Coll
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientListFilesCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientListFilesCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
index 78b4c84..4adf134 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
@@ -42,10 +42,11 @@ public class IgfsClientListPathsCallable extends IgfsClientAbstractCallable<Coll
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientListPathsCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientListPathsCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
index 944da6f..4c50be3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
@@ -49,11 +49,13 @@ public class IgfsClientMkdirsCallable extends IgfsClientAbstractCallable<Void> {
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      * @param props Properties.
      */
-    public IgfsClientMkdirsCallable(@Nullable String igfsName, IgfsPath path, @Nullable Map<String, String> props) {
-        super(igfsName, path);
+    public IgfsClientMkdirsCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path,
+        @Nullable Map<String, String> props) {
+        super(igfsName, user, path);
 
         this.props = props;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
index 55afb83..3743127 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
@@ -47,11 +47,13 @@ public class IgfsClientRenameCallable extends IgfsClientAbstractCallable<Void> {
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param srcPath Source path.
      * @param destPath Destination path.
      */
-    public IgfsClientRenameCallable(@Nullable String igfsName, IgfsPath srcPath, IgfsPath destPath) {
-        super(igfsName, srcPath);
+    public IgfsClientRenameCallable(@Nullable String igfsName, @Nullable String user,
+        IgfsPath srcPath, IgfsPath destPath) {
+        super(igfsName, user, srcPath);
 
         this.destPath = destPath;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
index 277effc..0914c3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
@@ -49,13 +49,15 @@ public class IgfsClientSetTimesCallable extends IgfsClientAbstractCallable<Void>
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      * @param accessTime Access time.
      * @param modificationTime Modification time.
      */
-    public IgfsClientSetTimesCallable(@Nullable String igfsName, IgfsPath path, long accessTime,
+    public IgfsClientSetTimesCallable(@Nullable String igfsName, @Nullable String user,
+        IgfsPath path, long accessTime,
         long modificationTime) {
-        super(igfsName, path);
+        super(igfsName, user, path);
 
         this.accessTime = accessTime;
         this.modificationTime = modificationTime;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
index 928d65d..5d345f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
@@ -40,10 +40,11 @@ public class IgfsClientSizeCallable extends IgfsClientAbstractCallable<Long> {
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientSizeCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientSizeCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
index 7e29029..ab940bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
@@ -41,10 +41,11 @@ public class IgfsClientSummaryCallable extends IgfsClientAbstractCallable<IgfsPa
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientSummaryCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientSummaryCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
index 4acf4eb..9ae6e22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
@@ -50,11 +50,13 @@ public class IgfsClientUpdateCallable extends IgfsClientAbstractCallable<IgfsFil
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      * @param props Properties.
      */
-    public IgfsClientUpdateCallable(@Nullable String igfsName, IgfsPath path, @Nullable Map<String, String> props) {
-        super(igfsName, path);
+    public IgfsClientUpdateCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path,
+        @Nullable Map<String, String> props) {
+        super(igfsName, user, path);
 
         this.props = props;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
index 7b3d142..00fc087 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
@@ -45,10 +45,11 @@ public class IgfsClientMetaIdsForPathCallable extends IgfsClientAbstractCallable
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientMetaIdsForPathCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientMetaIdsForPathCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
index cb31663..4c56adb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
@@ -43,10 +43,11 @@ public class IgfsClientMetaInfoForPathCallable extends IgfsClientAbstractCallabl
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param path Path.
      */
-    public IgfsClientMetaInfoForPathCallable(@Nullable String igfsName, IgfsPath path) {
-        super(igfsName, path);
+    public IgfsClientMetaInfoForPathCallable(@Nullable String igfsName, @Nullable String user, IgfsPath path) {
+        super(igfsName, user, path);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java
index a2f4ab7..26c3166 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java
@@ -64,6 +64,7 @@ public class IgfsClientMetaUnlockCallable extends IgfsClientAbstractCallable<Voi
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param user IGFS user name.
      * @param fileId File ID.
      * @param lockId Lock ID.
      * @param modificationTime Modification time to write to file info.
@@ -71,9 +72,10 @@ public class IgfsClientMetaUnlockCallable extends IgfsClientAbstractCallable<Voi
      * @param space Space.
      * @param affRange Affinity range.
      */
-    public IgfsClientMetaUnlockCallable(@Nullable String igfsName, IgniteUuid fileId, IgniteUuid lockId,
-        long modificationTime, boolean updateSpace, long space, final IgfsFileAffinityRange affRange) {
-        super(igfsName, null);
+    public IgfsClientMetaUnlockCallable(@Nullable String igfsName, @Nullable String user, IgniteUuid fileId,
+        IgniteUuid lockId, long modificationTime, boolean updateSpace, long space,
+        final IgfsFileAffinityRange affRange) {
+        super(igfsName, user, null);
 
         this.fileId = fileId;
         this.lockId = lockId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java
index 59383c5..397b709 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.igfs.secondary.local;
 
+import java.nio.file.attribute.BasicFileAttributeView;
+import java.nio.file.attribute.BasicFileAttributes;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -91,6 +93,7 @@ public class LocalFileSystemUtils {
      * Get POSIX attributes for file.
      *
      * @param file File.
+     * @return PosixFileAttributes.
      */
     @Nullable public static PosixFileAttributes posixAttributes(File file) {
         PosixFileAttributes attrs = null;
@@ -109,6 +112,28 @@ public class LocalFileSystemUtils {
     }
 
     /**
+     * Get POSIX attributes for file.
+     *
+     * @param file File.
+     * @return BasicFileAttributes.
+     */
+    @Nullable public static BasicFileAttributes basicAttributes(File file) {
+        BasicFileAttributes attrs = null;
+
+        try {
+            BasicFileAttributeView view = Files.getFileAttributeView(file.toPath(), BasicFileAttributeView.class);
+
+            if (view != null)
+                attrs = view.readAttributes();
+        }
+        catch (IOException e) {
+            throw new IgfsException("Failed to read basic file attributes: " + file.getAbsolutePath(), e);
+        }
+
+        return attrs;
+    }
+
+    /**
      * Convert POSIX attributes to property map.
      *
      * @param attrs Attributes view.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
index 7605a73..cffac65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
@@ -659,7 +659,7 @@ public class GridToStringBuilder {
      * @param arr Array object.
      * @return String representation of an array.
      */
-    private static String arrayToString(Class arrType, Object arr) {
+    public static String arrayToString(Class arrType, Object arr) {
         if (arrType.equals(byte[].class))
             return Arrays.toString((byte[])arr);
         if (arrType.equals(boolean[].class))

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 6619810..a37a3eb 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -817,19 +817,37 @@ public abstract class GridAbstractTest extends TestCase {
      * Starts new grid at another JVM with given name.
      *
      * @param gridName Grid name.
+     * @param cfg Ignite configuration.
      * @param ctx Spring context.
      * @return Started grid.
      * @throws Exception If failed.
      */
     protected Ignite startRemoteGrid(String gridName, IgniteConfiguration cfg, GridSpringResourceContext ctx)
         throws Exception {
+        return startRemoteGrid(gridName, cfg, ctx, grid(0), true);
+    }
+
+    /**
+     * Starts new grid at another JVM with given name.
+     *
+     * @param gridName Grid name.
+     * @param cfg Ignite configuration.
+     * @param ctx Spring context.
+     * @param locNode Local node.
+     * @param resetDiscovery Reset DiscoverySpi.
+     * @return Started grid.
+     * @throws Exception If failed.
+     */
+    protected Ignite startRemoteGrid(String gridName, IgniteConfiguration cfg, GridSpringResourceContext ctx,
+        IgniteEx locNode, boolean resetDiscovery)
+        throws Exception {
         if (ctx != null)
             throw new UnsupportedOperationException("Starting of grid at another jvm by context doesn't supported.");
 
         if (cfg == null)
             cfg = optimize(getConfiguration(gridName));
 
-        return new IgniteProcessProxy(cfg, log, grid(0));
+        return new IgniteProcessProxy(cfg, log, locNode, resetDiscovery);
     }
 
     /**
@@ -837,6 +855,7 @@ public abstract class GridAbstractTest extends TestCase {
      *
      * @param cfg Configuration.
      * @return Optimized configuration (by modifying passed in one).
+     * @throws IgniteCheckedException On error.
      */
     protected IgniteConfiguration optimize(IgniteConfiguration cfg) throws IgniteCheckedException {
         // TODO: IGNITE-605: propose another way to avoid network overhead in tests.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
index 7d1a37d..3b56b66 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
@@ -96,21 +96,41 @@ public class IgniteNodeRunner {
      * @throws IOException If failed.
      * @see #readCfgFromFileAndDeleteFile(String)
      */
-    public static String storeToFile(IgniteConfiguration cfg) throws IOException {
+    public static String storeToFile(IgniteConfiguration cfg, boolean resetDiscovery) throws IOException {
         String fileName = IGNITE_CONFIGURATION_FILE + cfg.getNodeId();
 
+        storeToFile(cfg, fileName, true, resetDiscovery);
+
+        return fileName;
+    }
+
+    /**
+     * Stores {@link IgniteConfiguration} to file as xml.
+     *
+     * @param cfg Ignite Configuration.
+     * @param fileName A name of file where the configuration was stored.
+     * @param resetMarshaller Reset marshaller configuration to default.
+     * @param resetDiscovery Reset discovery configuration to default.
+     * @throws IOException If failed.
+     * @see #readCfgFromFileAndDeleteFile(String)
+     */
+    public static void storeToFile(IgniteConfiguration cfg, String fileName,
+        boolean resetMarshaller,
+        boolean resetDiscovery) throws IOException {
         try(OutputStream out = new BufferedOutputStream(new FileOutputStream(fileName))) {
             IgniteConfiguration cfg0 = new IgniteConfiguration(cfg);
 
+            if (resetMarshaller)
+                cfg0.setMarshaller(null);
+
+            if (resetDiscovery)
+                cfg0.setDiscoverySpi(null);
+
             cfg0.setMBeanServer(null);
-            cfg0.setMarshaller(null);
-            cfg0.setDiscoverySpi(null);
             cfg0.setGridLogger(null);
 
             new XStream().toXML(cfg0, out);
         }
-
-        return fileName;
     }
 
     /**
@@ -119,22 +139,27 @@ public class IgniteNodeRunner {
      * @param fileName File name.
      * @return Readed configuration.
      * @throws IOException If failed.
-     * @see #storeToFile(IgniteConfiguration)
+     * @see #storeToFile(IgniteConfiguration, boolean)
+     * @throws IgniteCheckedException On error.
      */
     private static IgniteConfiguration readCfgFromFileAndDeleteFile(String fileName)
         throws IOException, IgniteCheckedException {
         try(BufferedReader cfgReader = new BufferedReader(new FileReader(fileName))) {
             IgniteConfiguration cfg = (IgniteConfiguration)new XStream().fromXML(cfgReader);
 
-            Marshaller marsh = IgniteTestResources.getMarshaller();
+            if (cfg.getMarshaller() == null) {
+                Marshaller marsh = IgniteTestResources.getMarshaller();
 
-            cfg.setMarshaller(marsh);
+                cfg.setMarshaller(marsh);
+            }
 
-            X.println("Configured marshaller class: " + marsh.getClass().getName());
+            X.println("Configured marshaller class: " + cfg.getMarshaller().getClass().getName());
 
-            TcpDiscoverySpi disco = new TcpDiscoverySpi();
-            disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
-            cfg.setDiscoverySpi(disco);
+            if (cfg.getDiscoverySpi() == null) {
+                TcpDiscoverySpi disco = new TcpDiscoverySpi();
+                disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
+                cfg.setDiscoverySpi(disco);
+            }
 
             return cfg;
         }
@@ -161,9 +186,9 @@ public class IgniteNodeRunner {
                 MonitoredVm vm = monitoredHost.getMonitoredVm(new VmIdentifier("//" + jvmId + "?mode=r"), 0);
 
                 if (IgniteNodeRunner.class.getName().equals(MonitoredVmUtil.mainClass(vm, true))) {
-                    Process killProc = U.isWindows() ?
-                        Runtime.getRuntime().exec(new String[] {"taskkill", "/pid", jvmId.toString(), "/f", "/t"}) :
-                        Runtime.getRuntime().exec(new String[] {"kill", "-9", jvmId.toString()});
+                    Process killProc = Runtime.getRuntime().exec(U.isWindows() ?
+                        new String[] {"taskkill", "/pid", jvmId.toString(), "/f", "/t"} :
+                        new String[] {"kill", "-9", jvmId.toString()});
 
                     killProc.waitFor();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 9bb5205..f6d938c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -113,14 +113,27 @@ public class IgniteProcessProxy implements IgniteEx {
      * @param cfg Configuration.
      * @param log Logger.
      * @param locJvmGrid Local JVM grid.
+     * @throws Exception On error.
      */
     public IgniteProcessProxy(IgniteConfiguration cfg, IgniteLogger log, Ignite locJvmGrid)
         throws Exception {
+        this(cfg, log, locJvmGrid, true);
+    }
+
+    /**
+     * @param cfg Configuration.
+     * @param log Logger.
+     * @param locJvmGrid Local JVM grid.
+     * @param resetDiscovery Reset DiscoverySpi at the configuration.
+     * @throws Exception On error.
+     */
+    public IgniteProcessProxy(IgniteConfiguration cfg, IgniteLogger log, Ignite locJvmGrid, boolean resetDiscovery)
+        throws Exception {
         this.cfg = cfg;
         this.locJvmGrid = locJvmGrid;
         this.log = log.getLogger("jvm-" + id.toString().substring(0, id.toString().indexOf('-')));
 
-        String cfgFileName = IgniteNodeRunner.storeToFile(cfg.setNodeId(id));
+        String cfgFileName = IgniteNodeRunner.storeToFile(cfg.setNodeId(id), resetDiscovery);
 
         Collection<String> filteredJvmArgs = new ArrayList<>();
 
@@ -138,7 +151,8 @@ public class IgniteProcessProxy implements IgniteEx {
 
         final CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
 
-        locJvmGrid.events().localListen(new NodeStartedListener(id, rmtNodeStartedLatch), EventType.EVT_NODE_JOINED);
+        if (locJvmGrid != null)
+            locJvmGrid.events().localListen(new NodeStartedListener(id, rmtNodeStartedLatch), EventType.EVT_NODE_JOINED);
 
         proc = GridJavaProcess.exec(
             IgniteNodeRunner.class.getCanonicalName(),
@@ -156,7 +170,8 @@ public class IgniteProcessProxy implements IgniteEx {
             System.getProperty("surefire.test.class.path")
         );
 
-        assert rmtNodeStartedLatch.await(30, TimeUnit.SECONDS): "Remote node has not joined [id=" + id + ']';
+        if (locJvmGrid != null)
+            assert rmtNodeStartedLatch.await(30, TimeUnit.SECONDS): "Remote node has not joined [id=" + id + ']';
 
         IgniteProcessProxy prevVal = gridProxies.putIfAbsent(cfg.getGridName(), this);
 
@@ -611,27 +626,29 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public void close() throws IgniteException {
-        final CountDownLatch rmtNodeStoppedLatch = new CountDownLatch(1);
+        if (locJvmGrid != null) {
+            final CountDownLatch rmtNodeStoppedLatch = new CountDownLatch(1);
 
-        locJvmGrid.events().localListen(new IgnitePredicateX<Event>() {
-            @Override public boolean applyx(Event e) {
-                if (((DiscoveryEvent)e).eventNode().id().equals(id)) {
-                    rmtNodeStoppedLatch.countDown();
+            locJvmGrid.events().localListen(new IgnitePredicateX<Event>() {
+                @Override public boolean applyx(Event e) {
+                    if (((DiscoveryEvent)e).eventNode().id().equals(id)) {
+                        rmtNodeStoppedLatch.countDown();
 
-                    return false;
-                }
+                        return false;
+                    }
 
-                return true;
-            }
-        }, EventType.EVT_NODE_LEFT, EventType.EVT_NODE_FAILED);
+                    return true;
+                }
+            }, EventType.EVT_NODE_LEFT, EventType.EVT_NODE_FAILED);
 
-        compute().run(new StopGridTask(localJvmGrid().name(), true));
+            compute().run(new StopGridTask(localJvmGrid().name(), true));
 
-        try {
-            assert U.await(rmtNodeStoppedLatch, 15, TimeUnit.SECONDS) : "NodeId=" + id;
-        }
-        catch (IgniteInterruptedCheckedException e) {
-            throw new IgniteException(e);
+            try {
+                assert U.await(rmtNodeStoppedLatch, 15, TimeUnit.SECONDS) : "NodeId=" + id;
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                throw new IgniteException(e);
+            }
         }
 
         try {
@@ -658,6 +675,9 @@ public class IgniteProcessProxy implements IgniteEx {
      * @return {@link IgniteCompute} instance to communicate with remote node.
      */
     public IgniteCompute remoteCompute() {
+        if (locJvmGrid == null)
+            return null;
+
         ClusterGroup grp = locJvmGrid.cluster().forNodeId(id);
 
         if (grp.nodes().isEmpty())

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index e5373ab..5432052 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -42,6 +42,13 @@
         </dependency>
 
         <dependency>
+            <groupId>com.thoughtworks.xstream</groupId>
+            <artifactId>xstream</artifactId>
+            <version>1.4.8</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-log4j</artifactId>
             <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java
index f9f98ed..ed882c7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java
@@ -17,9 +17,20 @@
 
 package org.apache.ignite.internal.processors.hadoop.impl.igfs;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
 import org.apache.commons.logging.Log;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsInputStream;
@@ -28,25 +39,32 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathSummary;
 import org.apache.ignite.igfs.IgfsUserContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse;
 import org.apache.ignite.internal.processors.igfs.IgfsStatus;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteOutClosure;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
+import static org.apache.ignite.IgniteState.STARTED;
 
 /**
  * Communication with grid in the same process.
  */
 public class HadoopIgfsInProc implements HadoopIgfsEx {
+    /** Ignite client reference counters (node name, reference count). */
+    private static final Map<String, Integer> REF_CTRS = new HashMap<>();
+
+    /** Reference count monitor. */
+    private static final Object REF_CTR_MUX = new Object();
+
     /** Target IGFS. */
     private final IgfsEx igfs;
 
@@ -68,15 +86,122 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
      *
      * @param igfs Target IGFS.
      * @param log Log.
+     * @param userName User name.
      */
-    public HadoopIgfsInProc(IgfsEx igfs, Log log, String userName) throws IgniteCheckedException {
-        this.user = IgfsUtils.fixUserName(userName);
-
+    private HadoopIgfsInProc(IgfsEx igfs, Log log, String userName) {
         this.igfs = igfs;
 
         this.log = log;
 
         bufSize = igfs.configuration().getBlockSize() * 2;
+
+        user = IgfsUtils.fixUserName(userName);
+    }
+
+    /**
+     * Creates instance of the HadoopIgfsInProcWithIgniteRefsCount by IGFS name.
+     *
+     * @param igfsName Target IGFS name.
+     * @param log Log.
+     * @param userName User name.
+     * @return HadoopIgfsInProcWithIgniteRefsCount instance. {@code null} if the IGFS not fount in the current VM.
+     */
+    public static HadoopIgfsInProc create(String igfsName, Log log, String userName) {
+        synchronized (REF_CTR_MUX) {
+            for (Ignite ignite : Ignition.allGrids()) {
+                HadoopIgfsInProc delegate = create0(ignite, igfsName, log, userName);
+
+                if (delegate != null)
+                    return delegate;
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * Creates instance of the HadoopIgfsInProcWithIgniteRefsCount by IGFS name, ignite client node is created
+     * if necessary.
+     *
+     * @param igniteCfgPath Path to ignite configuration.
+     * @param igfsName Target IGFS name.
+     * @param log Log.
+     * @param userName User name.
+     * @return HadoopIgfsInProcWithIgniteRefsCount instance. {@code null} if the IGFS not fount in the current VM.
+     * @throws IgniteCheckedException On error.
+     */
+    public static HadoopIgfsInProc create(String igniteCfgPath, String igfsName, Log log, String userName)
+        throws IgniteCheckedException {
+        IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> cfgPair =
+            IgnitionEx.loadConfiguration(igniteCfgPath);
+
+        IgniteConfiguration cfg = cfgPair.get1();
+
+        cfg.setClientMode(true);
+
+        String nodeName = cfg.getGridName();
+
+        synchronized (REF_CTR_MUX) {
+            T2<Ignite, Boolean> startRes = IgnitionEx.getOrStart(cfg);
+
+            boolean newNodeStarted = startRes.get2();
+
+            if (newNodeStarted) {
+                assert !REF_CTRS.containsKey(nodeName) : "The ignite instance already exists in the ref count map";
+
+                REF_CTRS.put(nodeName, 0);
+            }
+
+            HadoopIgfsInProc hadoop = create0(startRes.get1(), igfsName, log, userName);
+
+            if (hadoop == null) {
+                if (newNodeStarted) {
+                    REF_CTRS.remove(nodeName);
+
+                    Ignition.stop(nodeName, true);
+                }
+
+                throw new HadoopIgfsCommunicationException("Ignite client node doesn't have IGFS with the " +
+                    "given name: " + igfsName);
+            }
+
+            return hadoop;
+        }
+    }
+
+    /**
+     * Creates instance of the HadoopIgfsInProcWithIgniteRefsCount by IGFS name.
+     *
+     * @param ignite Ignite instance.
+     * @param igfsName Target IGFS name.
+     * @param log Log.
+     * @param userName User name.
+     * @return HadoopIgfsInProcWithIgniteRefsCount instance. {@code null} if the IGFS not found
+     *      in the specified ignite instance.
+     */
+    private static HadoopIgfsInProc create0(Ignite ignite, String igfsName, Log log, String userName) {
+        assert Thread.holdsLock(REF_CTR_MUX);
+        assert ignite != null;
+
+        if (Ignition.state(ignite.name()) == STARTED) {
+            try {
+                for (IgniteFileSystem fs : ignite.fileSystems()) {
+                    if (F.eq(fs.name(), igfsName)) {
+                        Integer ctr = REF_CTRS.get(ignite.name());
+
+                        if (ctr != null)
+                            REF_CTRS.put(ignite.name(), ctr + 1);
+
+                        return new HadoopIgfsInProc((IgfsEx)fs, log, userName);
+                    }
+                }
+            }
+            catch (IllegalStateException ignore) {
+                // May happen if the grid state has changed:
+            }
+        }
+
+        return null;
     }
 
     /** {@inheritDoc} */
@@ -91,8 +216,10 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
          });
     }
 
-    /** {@inheritDoc} */
-    @Override public void close(boolean force) {
+    /**
+     * Call onClose for all listeners.
+     */
+    private void notifyListenersOnClose() {
         // Perform cleanup.
         for (HadoopIgfsStreamEventListener lsnr : lsnrs.values()) {
             try {
@@ -106,6 +233,29 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
+    @Override public void close(boolean force) {
+        notifyListenersOnClose();
+
+        String gridName = igfs.context().kernalContext().grid().name();
+
+        synchronized (REF_CTR_MUX) {
+            Integer cnt = REF_CTRS.get(gridName);
+
+            if (cnt != null) {
+                // The node was created by this HadoopIgfsWrapper.
+                // The node must be stopped when there are not opened filesystems that are used one.
+                if (cnt > 1)
+                    REF_CTRS.put(gridName, cnt - 1);
+                else {
+                    REF_CTRS.remove(gridName);
+
+                    G.stop(gridName, false);
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public IgfsFile info(final IgfsPath path) throws IgniteCheckedException {
         try {
             return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsFile>() {
@@ -123,7 +273,8 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFile update(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
+    @Override public IgfsFile update(final IgfsPath path, final Map<String, String> props) throws
+        IgniteCheckedException {
         try {
             return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsFile>() {
                 @Override public IgfsFile apply() {
@@ -140,7 +291,8 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean setTimes(final IgfsPath path, final long accessTime, final long modificationTime) throws IgniteCheckedException {
+    @Override public Boolean setTimes(final IgfsPath path, final long accessTime, final long modificationTime)
+        throws IgniteCheckedException {
         try {
             IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
                 @Override public Void apply() {
@@ -253,7 +405,8 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
+    @Override public Boolean mkdirs(final IgfsPath path, final Map<String, String> props)
+        throws IgniteCheckedException {
         try {
             IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
                 @Override public Void apply() {
@@ -269,7 +422,7 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
             throw new IgniteCheckedException(e);
         }
         catch (IllegalStateException ignored) {
-            throw new HadoopIgfsCommunicationException("Failed to create directory because Grid is stopping: " +
+            throw new HadoopIgfsCommunicationException("Failed to findIgfsAndCreate directory because Grid is stopping: " +
                 path);
         }
     }
@@ -350,8 +503,9 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate create(final IgfsPath path, final boolean overwrite, final boolean colocate,
-        final int replication, final long blockSize, final @Nullable Map<String, String> props) throws IgniteCheckedException {
+    @Override public HadoopIgfsStreamDelegate create(final IgfsPath path, final boolean overwrite,
+        final boolean colocate, final int replication, final long blockSize, final @Nullable Map<String, String> props)
+        throws IgniteCheckedException {
         try {
             return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
                 @Override public HadoopIgfsStreamDelegate apply() {
@@ -366,7 +520,7 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
             throw new IgniteCheckedException(e);
         }
         catch (IllegalStateException ignored) {
-            throw new HadoopIgfsCommunicationException("Failed to create file because Grid is stopping: " + path);
+            throw new HadoopIgfsCommunicationException("Failed to findIgfsAndCreate file because Grid is stopping: " + path);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java
index ee7756e..3349cbf 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java
@@ -48,6 +48,12 @@ public class HadoopIgfsUtils {
     /** Parameter name for endpoint no local TCP flag. */
     public static final String PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP = "fs.igfs.%s.endpoint.no_local_tcp";
 
+    /** Parameter name for endpoint no remote TCP flag. */
+    public static final String PARAM_IGFS_ENDPOINT_NO_REMOTE_TCP = "fs.igfs.%s.endpoint.no_remote_tcp";
+
+    /** Parameter name for endpoint path to Ignite client configuration. */
+    public static final String PARAM_IGFS_ENDPOINT_IGNITE_CFG_PATH = "fs.igfs.%s.config_path";
+
     /**
      * Get string parameter.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/6bdff2c3/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java
index aaf79de..6fa5d60 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java
@@ -17,19 +17,20 @@
 
 package org.apache.ignite.internal.processors.hadoop.impl.igfs;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteFileSystem;
-import org.apache.ignite.IgniteIllegalStateException;
-import org.apache.ignite.Ignition;
 import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathSummary;
 import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint;
-import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse;
 import org.apache.ignite.internal.processors.igfs.IgfsStatus;
 import org.apache.ignite.internal.util.typedef.F;
@@ -38,18 +39,12 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.ignite.IgniteState.STARTED;
 import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint.LOCALHOST;
+import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_IGNITE_CFG_PATH;
 import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED;
 import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM;
 import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP;
+import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_REMOTE_TCP;
 import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.parameter;
 
 /**
@@ -82,9 +77,9 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
      *
      * @param authority Authority (connection string).
      * @param logDir Log directory for server.
-     * @param user User name.
      * @param conf Configuration.
      * @param log Current logger.
+     * @param user User name.
      * @throws IOException On error.
      */
     public HadoopIgfsWrapper(String authority, String logDir, Configuration conf, Log log, String user)
@@ -358,6 +353,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
         // These fields will contain possible exceptions from shmem and TCP endpoints.
         Exception errShmem = null;
         Exception errTcp = null;
+        Exception errClient = null;
 
         // 1. If delegate is set, return it immediately.
         Delegate curDelegate = delegateRef.get();
@@ -369,28 +365,37 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
         boolean skipInProc = parameter(conf, PARAM_IGFS_ENDPOINT_NO_EMBED, authority, false);
 
         if (!skipInProc) {
-            IgfsEx igfs = getIgfsEx(endpoint.igfs());
+            HadoopIgfsInProc hadoop = HadoopIgfsInProc.create(endpoint.igfs(), log, userName);
+
+            if (hadoop != null)
+                curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
+        }
+
+        // 3. Try Ignite client
+        String igniteCliCfgPath = parameter(conf, PARAM_IGFS_ENDPOINT_IGNITE_CFG_PATH, authority, null);
 
-            if (igfs != null) {
-                HadoopIgfsEx hadoop = null;
+        if (curDelegate == null && !F.isEmpty(igniteCliCfgPath)) {
+            HadoopIgfsInProc hadoop = null;
 
-                try {
-                    hadoop = new HadoopIgfsInProc(igfs, log, userName);
+            try {
+                hadoop = HadoopIgfsInProc.create(igniteCliCfgPath, endpoint.igfs(), log, userName);
 
+                if (hadoop != null)
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    if (e instanceof HadoopIgfsCommunicationException)
-                        if (hadoop != null)
-                            hadoop.close(true);
+            }
+            catch (Exception e) {
+                if (hadoop != null)
+                    hadoop.close(true);
 
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to connect to in-process IGFS, fallback to IPC mode.", e);
-                }
+                if (log.isDebugEnabled())
+                    log.debug("Failed to connect to IGFS using Ignite client [host=" + endpoint.host() +
+                        ", port=" + endpoint.port() + ", igniteCfg=" + igniteCliCfgPath + ']', e);
+
+                errClient = e;
             }
         }
 
-        // 3. Try connecting using shmem.
+        // 4. Try connecting using shmem.
         boolean skipLocShmem = parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority, false);
 
         if (curDelegate == null && !skipLocShmem && !U.isWindows()) {
@@ -412,7 +417,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             }
         }
 
-        // 4. Try local TCP connection.
+        // 5. Try local TCP connection.
         boolean skipLocTcp = parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP, authority, false);
 
         if (curDelegate == null && !skipLocTcp) {
@@ -436,8 +441,10 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             }
         }
 
-        // 5. Try remote TCP connection.
-        if (curDelegate == null && (skipLocTcp || !F.eq(LOCALHOST, endpoint.host()))) {
+        // 6. Try remote TCP connection.
+        boolean skipRmtTcp = parameter(conf, PARAM_IGFS_ENDPOINT_NO_REMOTE_TCP, authority, false);
+
+        if (curDelegate == null && !skipRmtTcp && (skipLocTcp || !F.eq(LOCALHOST, endpoint.host()))) {
             HadoopIgfsEx hadoop = null;
 
             try {
@@ -470,7 +477,11 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             if (errShmem != null)
                 errMsg.a("[type=SHMEM, port=" + endpoint.port() + ", err=" + errShmem + "], ");
 
-            errMsg.a("[type=TCP, host=" + endpoint.host() + ", port=" + endpoint.port() + ", err=" + errTcp + "]] ");
+            if (errTcp != null)
+                errMsg.a("[type=TCP, host=" + endpoint.host() + ", port=" + endpoint.port() + ", err=" + errTcp + "]] ");
+
+            if (errClient != null)
+                errMsg.a("[type=CLIENT, cfg=" + igniteCliCfgPath + ", err=" + errClient + "]] ");
 
             errMsg.a("(ensure that IGFS is running and have IPC endpoint enabled; ensure that " +
                 "ignite-shmem-1.0.0.jar is in Hadoop classpath if you use shared memory endpoint).");
@@ -532,28 +543,4 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 hadoop.close(force);
         }
     }
-
-    /**
-     * Helper method to find Igfs of the given name in the given Ignite instance.
-     *
-     * @param igfsName The name of Igfs.
-     * @return The file system instance, or null if not found.
-     */
-    private static IgfsEx getIgfsEx(@Nullable String igfsName) {
-        for (Ignite ignite : Ignition.allGrids()) {
-            if (Ignition.state(ignite.name()) == STARTED) {
-                try {
-                    for (IgniteFileSystem fs : ignite.fileSystems()) {
-                        if (F.eq(fs.name(), igfsName))
-                            return (IgfsEx)fs;
-                    }
-                }
-                catch (IgniteIllegalStateException ignore) {
-                    // May happen if the grid state has changed:
-                }
-            }
-        }
-
-        return null;
-    }
 }
\ No newline at end of file