You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/06/06 11:57:23 UTC

[24/32] ignite git commit: IGNITE-3247: IGFS: Optimized non-stream client operations.

IGNITE-3247: IGFS:  Optimized non-stream client operations.


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

Branch: refs/heads/ignite-3212
Commit: fdc3aa6e6f1f163351ef1200e18faed7db96d218
Parents: 14c80a1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sun Jun 5 21:00:53 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 6 09:47:07 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/igfs/IgfsPath.java   |  26 +++-
 .../org/apache/ignite/igfs/IgfsPathSummary.java |  32 ++++-
 .../processors/igfs/IgfsBlockLocationImpl.java  |  87 ++++++++++--
 .../internal/processors/igfs/IgfsFileImpl.java  |  34 ++++-
 .../internal/processors/igfs/IgfsImpl.java      |  63 ++++++++-
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../internal/processors/igfs/IgfsUtils.java     |  34 +++++
 .../igfs/client/IgfsClientAbstractCallable.java |  19 ++-
 .../igfs/client/IgfsClientAffinityCallable.java |  95 +++++++++++++
 .../igfs/client/IgfsClientDeleteCallable.java   |  77 ++++++++++
 .../igfs/client/IgfsClientExistsCallable.java   |  58 ++++++++
 .../igfs/client/IgfsClientInfoCallable.java     |  59 ++++++++
 .../client/IgfsClientListFilesCallable.java     |  61 ++++++++
 .../client/IgfsClientListPathsCallable.java     |  60 ++++++++
 .../igfs/client/IgfsClientMkdirsCallable.java   |  82 +++++++++++
 .../igfs/client/IgfsClientRenameCallable.java   |  80 +++++++++++
 .../igfs/client/IgfsClientSetTimesCallable.java |  87 ++++++++++++
 .../igfs/client/IgfsClientSizeCallable.java     |  59 ++++++++
 .../igfs/client/IgfsClientSummaryCallable.java  |  59 ++++++++
 .../igfs/client/IgfsClientUpdateCallable.java   |  81 +++++++++++
 .../ignite/internal/util/IgniteUtils.java       |  40 ++++++
 .../processors/igfs/IgfsAbstractSelfTest.java   |  35 ++++-
 .../igfs/IgfsClientCacheSelfTest.java           | 139 -------------------
 .../igfs/IgfsDualAsyncClientSelfTest.java       |  28 ++++
 .../igfs/IgfsDualSyncClientSelfTest.java        |  28 ++++
 .../processors/igfs/IgfsOneClientNodeTest.java  |   5 +-
 .../igfs/IgfsPrimaryClientSelfTest.java         |  30 ++++
 ...PrimaryRelaxedConsistencyClientSelfTest.java |  28 ++++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  11 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   2 +-
 30 files changed, 1324 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
index f1ceb2e..c705274 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
@@ -268,16 +268,32 @@ public final class IgfsPath implements Comparable<IgfsPath>, Externalizable, Bin
 
     /** {@inheritDoc} */
     @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
-        BinaryRawWriter out = writer.rawWriter();
-
-        out.writeString(path);
+        writeRawBinary(writer.rawWriter());
     }
 
     /** {@inheritDoc} */
     @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
-        BinaryRawReader in = reader.rawReader();
+        readRawBinary(reader.rawReader());
+    }
+
+    /**
+     * Write raw binary.
+     *
+     * @param writer Raw writer.
+     * @throws BinaryObjectException If failed.
+     */
+    public void writeRawBinary(BinaryRawWriter writer) throws BinaryObjectException {
+        writer.writeString(path);
+    }
 
-        path = in.readString();
+    /**
+     * Read raw binary.
+     *
+     * @param reader Raw reader.
+     * @throws BinaryObjectException If failed.
+     */
+    public void readRawBinary(BinaryRawReader reader) throws BinaryObjectException {
+        path = reader.readString();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java
index e84e128..7d60b95 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java
@@ -21,12 +21,20 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * Path summary: total files count, total directories count, total length.
  */
-public class IgfsPathSummary implements Externalizable {
+public class IgfsPathSummary implements Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -134,6 +142,28 @@ public class IgfsPathSummary implements Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeInt(filesCnt);
+        rawWriter.writeInt(dirCnt);
+        rawWriter.writeLong(totalLen);
+
+        IgfsUtils.writePath(rawWriter, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        filesCnt = rawReader.readInt();
+        dirCnt = rawReader.readInt();
+        totalLen = rawReader.readLong();
+
+        path = IgfsUtils.readPath(rawReader);
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsPathSummary.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockLocationImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockLocationImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockLocationImpl.java
index 0ec31ba..2d4a0af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockLocationImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsBlockLocationImpl.java
@@ -27,6 +27,12 @@ import java.util.Collection;
 import java.util.LinkedHashSet;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -37,7 +43,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 /**
  * File block location in the grid.
  */
-public class IgfsBlockLocationImpl implements IgfsBlockLocation, Externalizable {
+public class IgfsBlockLocationImpl implements IgfsBlockLocation, Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -155,13 +161,7 @@ public class IgfsBlockLocationImpl implements IgfsBlockLocation, Externalizable
         return S.toString(IgfsBlockLocationImpl.class, this);
     }
 
-    /**
-     * Writes this object to data output. Note that this is not externalizable
-     * interface because we want to eliminate any marshaller.
-     *
-     * @param out Data output to write.
-     * @throws IOException If write failed.
-     */
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         assert names != null;
         assert hosts != null;
@@ -189,13 +189,7 @@ public class IgfsBlockLocationImpl implements IgfsBlockLocation, Externalizable
             out.writeUTF(host);
     }
 
-    /**
-     * Reads object from data input. Note we do not use externalizable interface
-     * to eliminate marshaller.
-     *
-     * @param in Data input.
-     * @throws IOException If read failed.
-     */
+    /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException {
         start = in.readLong();
         len = in.readLong();
@@ -226,6 +220,69 @@ public class IgfsBlockLocationImpl implements IgfsBlockLocation, Externalizable
             hosts.add(in.readUTF());
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        assert names != null;
+        assert hosts != null;
+
+        rawWriter.writeLong(start);
+        rawWriter.writeLong(len);
+
+        rawWriter.writeBoolean(nodeIds != null);
+
+        if (nodeIds != null) {
+            rawWriter.writeInt(nodeIds.size());
+
+            for (UUID nodeId : nodeIds)
+                U.writeUuid(rawWriter, nodeId);
+        }
+
+        rawWriter.writeInt(names.size());
+
+        for (String name : names)
+            rawWriter.writeString(name);
+
+        rawWriter.writeInt(hosts.size());
+
+        for (String host : hosts)
+            rawWriter.writeString(host);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        start = rawReader.readLong();
+        len = rawReader.readLong();
+
+        int size;
+
+        if (rawReader.readBoolean()) {
+            size = rawReader.readInt();
+
+            nodeIds = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++)
+                nodeIds.add(U.readUuid(rawReader));
+        }
+
+        size = rawReader.readInt();
+
+        names = new ArrayList<>(size);
+
+        for (int i = 0; i < size; i++)
+            names.add(rawReader.readString());
+
+        size = rawReader.readInt();
+
+        hosts = new ArrayList<>(size);
+
+        for (int i = 0; i < size; i++)
+            hosts.add(rawReader.readString());
+    }
+
     /**
      * Converts collection of rich nodes to block location data.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
index be8d0fc..9f79f42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
@@ -17,6 +17,12 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -35,7 +41,7 @@ import java.util.Map;
 /**
  * File or directory information.
  */
-public final class IgfsFileImpl implements IgfsFile, Externalizable {
+public final class IgfsFileImpl implements IgfsFile, Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -229,6 +235,32 @@ public final class IgfsFileImpl implements IgfsFile, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        IgfsUtils.writePath(rawWriter, path);
+        rawWriter.writeInt(blockSize);
+        rawWriter.writeLong(grpBlockSize);
+        rawWriter.writeLong(len);
+        IgfsUtils.writeProperties(rawWriter, props);
+        rawWriter.writeLong(accessTime);
+        rawWriter.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        path = IgfsUtils.readPath(rawReader);
+        blockSize = rawReader.readInt();
+        grpBlockSize = rawReader.readLong();
+        len = rawReader.readLong();
+        props = IgfsUtils.readProperties(rawReader);
+        accessTime = rawReader.readLong();
+        modificationTime = rawReader.readLong();
+    }
+
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return path.hashCode();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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 e9ddf89..d9e4ba4 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
@@ -55,6 +55,18 @@ import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 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;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientInfoCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientListFilesCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientListPathsCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientMkdirsCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientRenameCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientSetTimesCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientSizeCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientSummaryCallable;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientUpdateCallable;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
@@ -530,6 +542,9 @@ public final class IgfsImpl implements IgfsEx {
     @Override public boolean exists(final IgfsPath path) {
         A.notNull(path, "path");
 
+        if (meta.isClient())
+            return meta.runClientTask(new IgfsClientExistsCallable(cfg.getName(), path));
+
         return safeOp(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
                 if (log.isDebugEnabled())
@@ -575,6 +590,9 @@ public final class IgfsImpl implements IgfsEx {
     @Override @Nullable public IgfsFile info(final IgfsPath path) {
         A.notNull(path, "path");
 
+        if (meta.isClient())
+            return meta.runClientTask(new IgfsClientInfoCallable(cfg.getName(), path));
+
         return safeOp(new Callable<IgfsFile>() {
             @Override public IgfsFile call() throws Exception {
                 if (log.isDebugEnabled())
@@ -591,6 +609,9 @@ public final class IgfsImpl implements IgfsEx {
     @Override public IgfsPathSummary summary(final IgfsPath path) {
         A.notNull(path, "path");
 
+        if (meta.isClient())
+            return meta.runClientTask(new IgfsClientSummaryCallable(cfg.getName(), path));
+
         return safeOp(new Callable<IgfsPathSummary>() {
             @Override public IgfsPathSummary call() throws Exception {
                 if (log.isDebugEnabled())
@@ -616,6 +637,9 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(props, "props");
         A.ensure(!props.isEmpty(), "!props.isEmpty()");
 
+        if (meta.isClient())
+            return meta.runClientTask(new IgfsClientUpdateCallable(cfg.getName(), path, props));
+
         return safeOp(new Callable<IgfsFile>() {
             @Override public IgfsFile call() throws Exception {
                 if (log.isDebugEnabled())
@@ -662,6 +686,12 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(src, "src");
         A.notNull(dest, "dest");
 
+        if (meta.isClient()) {
+            meta.runClientTask(new IgfsClientRenameCallable(cfg.getName(), src, dest));
+
+            return;
+        }
+
         safeOp(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 if (log.isDebugEnabled())
@@ -708,6 +738,9 @@ public final class IgfsImpl implements IgfsEx {
     @Override public boolean delete(final IgfsPath path, final boolean recursive) {
         A.notNull(path, "path");
 
+        if (meta.isClient())
+            return meta.runClientTask(new IgfsClientDeleteCallable(cfg.getName(), path, recursive));
+
         return safeOp(new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
                 if (log.isDebugEnabled())
@@ -761,6 +794,12 @@ public final class IgfsImpl implements IgfsEx {
     @Override public void mkdirs(final IgfsPath path, @Nullable final Map<String, String> props)  {
         A.notNull(path, "path");
 
+        if (meta.isClient()) {
+            meta.runClientTask(new IgfsClientMkdirsCallable(cfg.getName(), path, props));
+
+            return ;
+        }
+
         safeOp(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 if (log.isDebugEnabled())
@@ -790,6 +829,9 @@ public final class IgfsImpl implements IgfsEx {
     @Override public Collection<IgfsPath> listPaths(final IgfsPath path) {
         A.notNull(path, "path");
 
+        if (meta.isClient())
+            meta.runClientTask(new IgfsClientListPathsCallable(cfg.getName(), path));
+
         return safeOp(new Callable<Collection<IgfsPath>>() {
             @Override public Collection<IgfsPath> call() throws Exception {
                 if (log.isDebugEnabled())
@@ -840,6 +882,9 @@ public final class IgfsImpl implements IgfsEx {
     @Override public Collection<IgfsFile> listFiles(final IgfsPath path) {
         A.notNull(path, "path");
 
+        if (meta.isClient())
+            meta.runClientTask(new IgfsClientListFilesCallable(cfg.getName(), path));
+
         return safeOp(new Callable<Collection<IgfsFile>>() {
             @Override public Collection<IgfsFile> call() throws Exception {
                 if (log.isDebugEnabled())
@@ -1146,11 +1191,17 @@ public final class IgfsImpl implements IgfsEx {
     @Override public void setTimes(final IgfsPath path, final long accessTime, final long modificationTime) {
         A.notNull(path, "path");
 
+        if (accessTime == -1 && modificationTime == -1)
+            return;
+
+        if (meta.isClient()) {
+            meta.runClientTask(new IgfsClientSetTimesCallable(cfg.getName(), path, accessTime, modificationTime));
+
+            return;
+        }
+
         safeOp(new Callable<Void>() {
             @Override public Void call() throws Exception {
-                if (accessTime == -1 && modificationTime == -1)
-                    return null;
-
                 FileDescriptor desc = getFileDescriptor(path);
 
                 if (desc == null) {
@@ -1197,6 +1248,9 @@ public final class IgfsImpl implements IgfsEx {
         A.ensure(start >= 0, "start >= 0");
         A.ensure(len >= 0, "len >= 0");
 
+        if (meta.isClient())
+            return meta.runClientTask(new IgfsClientAffinityCallable(cfg.getName(), path, start, len, maxLen));
+
         return safeOp(new Callable<Collection<IgfsBlockLocation>>() {
             @Override public Collection<IgfsBlockLocation> call() throws Exception {
                 if (log.isDebugEnabled())
@@ -1278,6 +1332,9 @@ public final class IgfsImpl implements IgfsEx {
     @Override public long size(final IgfsPath path) {
         A.notNull(path, "path");
 
+        if (meta.isClient())
+            return meta.runClientTask(new IgfsClientSizeCallable(cfg.getName(), path));
+
         return safeOp(new Callable<Long>() {
             @Override public Long call() throws Exception {
                 IgniteUuid nextId = meta.fileId(path);

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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 8569305..d46a381 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
@@ -248,7 +248,7 @@ public class IgfsMetaManager extends IgfsManager {
             return clientCompute().call(task);
         }
         catch (ClusterTopologyException e) {
-            throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes left." , e);
+            throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 340fb97..d7aae5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -702,6 +702,40 @@ public class IgfsUtils {
     }
 
     /**
+     * Write IGFS path.
+     *
+     * @param writer Writer.
+     * @param path Path.
+     */
+    public static void writePath(BinaryRawWriter writer, @Nullable IgfsPath path) {
+        if (path != null) {
+            writer.writeBoolean(true);
+
+            path.writeRawBinary(writer);
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /**
+     * Read IGFS path.
+     *
+     * @param reader Reader.
+     * @return Path.
+     */
+    @Nullable public static IgfsPath readPath(BinaryRawReader reader) {
+        if (reader.readBoolean()) {
+            IgfsPath path = new IgfsPath();
+
+            path.readRawBinary(reader);
+
+            return path;
+        }
+        else
+            return null;
+    }
+
+    /**
      * Parses the TRASH file name to extract the original path.
      *
      * @param name The TRASH short (entry) name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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 b83ed13..d9c3456 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
@@ -24,8 +24,10 @@ import org.apache.ignite.binary.BinaryRawWriter;
 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.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
@@ -40,6 +42,9 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
     /** IGFS name. */
     protected String igfsName;
 
+    /** Path for operation. */
+    protected IgfsPath path;
+
     /** Injected instance. */
     @IgniteInstanceResource
     private transient Ignite ignite;
@@ -55,9 +60,11 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
      * Constructor.
      *
      * @param igfsName IGFS name.
+     * @param path Path.
      */
-    protected IgfsClientAbstractCallable(@Nullable String igfsName) {
+    protected IgfsClientAbstractCallable(@Nullable String igfsName, @Nullable IgfsPath path) {
         this.igfsName = igfsName;
+        this.path = path;
     }
 
     /** {@inheritDoc} */
@@ -83,6 +90,7 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
         BinaryRawWriter rawWriter = writer.rawWriter();
 
         rawWriter.writeString(igfsName);
+        IgfsUtils.writePath(rawWriter, path);
 
         writeBinary0(rawWriter);
     }
@@ -92,6 +100,7 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
         BinaryRawReader rawReader = reader.rawReader();
 
         igfsName = rawReader.readString();
+        path = IgfsUtils.readPath(rawReader);
 
         readBinary0(rawReader);
     }
@@ -101,12 +110,16 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
      *
      * @param rawWriter Raw writer.
      */
-    protected abstract void writeBinary0(BinaryRawWriter rawWriter);
+    protected void writeBinary0(BinaryRawWriter rawWriter) {
+        // No-op.
+    }
 
     /**
      * Read binary.
      *
      * @param rawReader Raw reader.
      */
-    protected abstract void readBinary0(BinaryRawReader rawReader);
+    protected void readBinary0(BinaryRawReader rawReader) {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..1668f36
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
@@ -0,0 +1,95 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.igfs.IgfsBlockLocation;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collection;
+
+/**
+ * IGFS client affinity callable.
+ */
+public class IgfsClientAffinityCallable extends IgfsClientAbstractCallable<Collection<IgfsBlockLocation>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Start. */
+    private long start;
+
+    /** Length. */
+    private long len;
+
+    /** Maximum length. */
+    private long maxLen;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientAffinityCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS 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);
+
+        this.start = start;
+        this.len = len;
+        this.maxLen = maxLen;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Collection<IgfsBlockLocation> call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().affinity(path, start, len, maxLen);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
+        writer.writeLong(start);
+        writer.writeLong(len);
+        writer.writeLong(maxLen);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
+        start = reader.readLong();
+        len = reader.readLong();
+        maxLen = reader.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientAffinityCallable.class, this);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..c1b8be8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
@@ -0,0 +1,77 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS client delete callable.
+ */
+public class IgfsClientDeleteCallable extends IgfsClientAbstractCallable<Boolean> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Recursion flag. */
+    private boolean recursive;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientDeleteCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     * @param recursive Recursive flag.
+     */
+    public IgfsClientDeleteCallable(@Nullable String igfsName, IgfsPath path, boolean recursive) {
+        super(igfsName, path);
+
+        this.recursive = recursive;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Boolean call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().delete(path, recursive);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
+        writer.writeBoolean(recursive);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
+        recursive = reader.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientDeleteCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..04b63d8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
@@ -0,0 +1,58 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS client exists callable.
+ */
+public class IgfsClientExistsCallable extends IgfsClientAbstractCallable<Boolean> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientExistsCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     */
+    public IgfsClientExistsCallable(@Nullable String igfsName, IgfsPath path) {
+        super(igfsName, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Boolean call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().exists(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientExistsCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..f97c3c4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
@@ -0,0 +1,59 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS client info callable.
+ */
+public class IgfsClientInfoCallable extends IgfsClientAbstractCallable<IgfsFile> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientInfoCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     */
+    public IgfsClientInfoCallable(@Nullable String igfsName, IgfsPath path) {
+        super(igfsName, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgfsFile call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().info(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientInfoCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..325e714
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
@@ -0,0 +1,61 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collection;
+
+/**
+ * IGFS client list files callable.
+ */
+public class IgfsClientListFilesCallable extends IgfsClientAbstractCallable<Collection<IgfsFile>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientListFilesCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     */
+    public IgfsClientListFilesCallable(@Nullable String igfsName, IgfsPath path) {
+        super(igfsName, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Collection<IgfsFile> call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().listFiles(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientListFilesCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..78b4c84
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
@@ -0,0 +1,60 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collection;
+
+/**
+ * IGFS client list paths callable.
+ */
+public class IgfsClientListPathsCallable extends IgfsClientAbstractCallable<Collection<IgfsPath>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientListPathsCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     */
+    public IgfsClientListPathsCallable(@Nullable String igfsName, IgfsPath path) {
+        super(igfsName, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Collection<IgfsPath> call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().listPaths(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientListPathsCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..944da6f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
@@ -0,0 +1,82 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Map;
+
+/**
+ * IGFS client mkdirs callable.
+ */
+public class IgfsClientMkdirsCallable extends IgfsClientAbstractCallable<Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Properties. */
+    private Map<String, String> props;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientMkdirsCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     * @param props Properties.
+     */
+    public IgfsClientMkdirsCallable(@Nullable String igfsName, IgfsPath path, @Nullable Map<String, String> props) {
+        super(igfsName, path);
+
+        this.props = props;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Void call0(IgfsContext ctx) throws Exception {
+        ctx.igfs().mkdirs(path, props);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
+        IgfsUtils.writeProperties(writer, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
+        props = IgfsUtils.readProperties(reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientMkdirsCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..55afb83
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
@@ -0,0 +1,80 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS client rename callable.
+ */
+public class IgfsClientRenameCallable extends IgfsClientAbstractCallable<Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Destination path. */
+    private IgfsPath destPath;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientRenameCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param srcPath Source path.
+     * @param destPath Destination path.
+     */
+    public IgfsClientRenameCallable(@Nullable String igfsName, IgfsPath srcPath, IgfsPath destPath) {
+        super(igfsName, srcPath);
+
+        this.destPath = destPath;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Void call0(IgfsContext ctx) throws Exception {
+        ctx.igfs().rename(path, destPath);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
+        IgfsUtils.writePath(writer, destPath);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
+        destPath = IgfsUtils.readPath(reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientRenameCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..277effc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
@@ -0,0 +1,87 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS client set times callable.
+ */
+public class IgfsClientSetTimesCallable extends IgfsClientAbstractCallable<Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Access time. */
+    private long accessTime;
+
+    /** Modification time. */
+    private long modificationTime;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientSetTimesCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     * @param accessTime Access time.
+     * @param modificationTime Modification time.
+     */
+    public IgfsClientSetTimesCallable(@Nullable String igfsName, IgfsPath path, long accessTime,
+        long modificationTime) {
+        super(igfsName, path);
+
+        this.accessTime = accessTime;
+        this.modificationTime = modificationTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Void call0(IgfsContext ctx) throws Exception {
+        ctx.igfs().setTimes(path, accessTime, modificationTime);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
+        writer.writeLong(accessTime);
+        writer.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
+        accessTime = reader.readLong();
+        modificationTime = reader.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientSetTimesCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..474a940
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
@@ -0,0 +1,59 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS client size callable.
+ */
+public class IgfsClientSizeCallable extends IgfsClientAbstractCallable<Long> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientSizeCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     */
+    public IgfsClientSizeCallable(@Nullable String igfsName, IgfsPath path) {
+        super(igfsName, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Long call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().size(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientSizeCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..7e29029
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
@@ -0,0 +1,59 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.igfs.IgfsPathSummary;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS client summary callable.
+ */
+public class IgfsClientSummaryCallable extends IgfsClientAbstractCallable<IgfsPathSummary> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientSummaryCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     */
+    public IgfsClientSummaryCallable(@Nullable String igfsName, IgfsPath path) {
+        super(igfsName, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgfsPathSummary call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().summary(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientSummaryCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/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
new file mode 100644
index 0000000..4acf4eb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
@@ -0,0 +1,81 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Map;
+
+/**
+ * IGFS client update callable.
+ */
+public class IgfsClientUpdateCallable extends IgfsClientAbstractCallable<IgfsFile> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Properties. */
+    private Map<String, String> props;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientUpdateCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param path Path.
+     * @param props Properties.
+     */
+    public IgfsClientUpdateCallable(@Nullable String igfsName, IgfsPath path, @Nullable Map<String, String> props) {
+        super(igfsName, path);
+
+        this.props = props;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgfsFile call0(IgfsContext ctx) throws Exception {
+        return ctx.igfs().update(path, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
+        IgfsUtils.writeProperties(writer, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
+        props = IgfsUtils.readProperties(reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientUpdateCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index cbefd7d..1147124 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -149,6 +149,8 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.cluster.ClusterGroupEmptyException;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
@@ -4723,6 +4725,44 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Writes UUID to binary writer.
+     *
+     * @param out Output Binary writer.
+     * @param uid UUID to write.
+     * @throws IOException If write failed.
+     */
+    public static void writeUuid(BinaryRawWriter out, UUID uid) {
+        // Write null flag.
+        if (uid != null) {
+            out.writeBoolean(true);
+
+            out.writeLong(uid.getMostSignificantBits());
+            out.writeLong(uid.getLeastSignificantBits());
+        }
+        else
+            out.writeBoolean(false);
+    }
+
+    /**
+     * Reads UUID from binary reader.
+     *
+     * @param in Binary reader.
+     * @return Read UUID.
+     * @throws IOException If read failed.
+     */
+    @Nullable public static UUID readUuid(BinaryRawReader in) {
+        // If UUID is not null.
+        if (in.readBoolean()) {
+            long most = in.readLong();
+            long least = in.readLong();
+
+            return new UUID(most, least);
+        }
+        else
+            return null;
+    }
+
+    /**
      * Writes {@link org.apache.ignite.lang.IgniteUuid} to output stream. This method is meant to be used by
      * implementations of {@link Externalizable} interface.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index d67092f..48d6d98 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -21,6 +21,7 @@ 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.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
@@ -190,6 +191,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /** Memory mode. */
     protected final CacheMemoryMode memoryMode;
 
+    /** IP finder for primary topology. */
+    protected final TcpDiscoveryVmIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** IP finder for secondary topology. */
+    protected final TcpDiscoveryVmIpFinder secondaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+
     /** Ignite nodes of cluster, excluding the secondary file system node, if any. */
     protected Ignite[] nodes;
 
@@ -237,6 +244,13 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * @return Client flag.
+     */
+    protected boolean client() {
+        return false;
+    }
+
+    /**
      * @return Use optimized marshaller flag.
      */
     protected boolean useOptimizedMarshaller() {
@@ -269,8 +283,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         igfsSecondaryFileSystem = createSecondaryFileSystemStack();
 
-        TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
         nodes = new Ignite[nodeCount()];
 
         for (int i = 0; i < nodes.length; i++) {
@@ -280,6 +292,21 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         }
 
         igfs = (IgfsImpl) nodes[0].fileSystem("igfs");
+
+        if (client()) {
+            // Start client.
+            Ignition.setClientMode(true);
+
+            try {
+                Ignite ignite = startGridWithIgfs("ignite-client", "igfs", mode, igfsSecondaryFileSystem,
+                    PRIMARY_REST_CFG, primaryIpFinder);
+
+                igfs = (IgfsImpl) ignite.fileSystem("igfs");
+            }
+            finally {
+                Ignition.setClientMode(false);
+            }
+        }
     }
 
     /**
@@ -290,7 +317,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      */
     protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
         Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
-            SECONDARY_REST_CFG, new TcpDiscoveryVmIpFinder(true));
+            SECONDARY_REST_CFG, secondaryIpFinder);
 
         IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary");
 
@@ -343,6 +370,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
 
         dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setNearConfiguration(null);
         dataCacheCfg.setCacheMode(PARTITIONED);
         dataCacheCfg.setNearConfiguration(null);
         dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
@@ -355,6 +383,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
 
         metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setNearConfiguration(null);
         metaCacheCfg.setCacheMode(REPLICATED);
         metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         metaCacheCfg.setAtomicityMode(TRANSACTIONAL);

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
deleted file mode 100644
index 8e8eac1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
+++ /dev/null
@@ -1,139 +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 org.apache.ignite.Ignite;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.FileSystemConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
-import org.apache.ignite.igfs.IgfsMode;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
-import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-
-/**
- * Test for igfs with nodes in client mode (see {@link IgniteConfiguration#setClientMode(boolean)}.
- */
-public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Meta-information cache name. */
-    private static final String META_CACHE_NAME = "meta";
-
-    /** Data cache name. */
-    private static final String DATA_CACHE_NAME = null;
-
-    /**
-     * Constructor.
-     */
-    public IgfsClientCacheSelfTest() {
-        super(IgfsMode.PRIMARY);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        igfsSecondaryFileSystem = createSecondaryFileSystemStack();
-
-        Ignite ignitePrimary = G.start(getConfiguration(getTestGridName(1)));
-
-        igfs = (IgfsImpl) ignitePrimary.fileSystem("igfs");
-    }
-
-    /**{@inheritDoc} */
-    protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
-        Ignite igniteSecondary = G.start(getConfiguration(getTestGridName(0)));
-
-        IgfsEx secondaryIgfsImpl = (IgfsEx)igniteSecondary.fileSystem("igfs");
-
-        igfsSecondary = new IgfsExUniversalFileSystemAdapter(secondaryIgfsImpl);
-
-        return secondaryIgfsImpl.asSecondary();
-    }
-
-    /**
-     *
-     * @param gridName Grid name.
-     * @return Ignite configuration.
-     * @throws Exception If failed.
-     */
-    protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setCacheConfiguration(
-            cacheConfiguration(META_CACHE_NAME),
-            cacheConfiguration(DATA_CACHE_NAME)
-        );
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        if (!gridName.equals(getTestGridName(0))) {
-            cfg.setClientMode(true);
-
-            disco.setForceServerMode(true);
-        }
-
-        cfg.setDiscoverySpi(disco);
-
-        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
-
-        igfsCfg.setMetaCacheName(META_CACHE_NAME);
-        igfsCfg.setDataCacheName(DATA_CACHE_NAME);
-        igfsCfg.setName("igfs");
-
-        cfg.setFileSystemConfiguration(igfsCfg);
-
-        return cfg;
-    }
-
-    /**
-     * @param cacheName Cache name.
-     * @return Cache configuration.
-     */
-    protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName(cacheName);
-
-        cacheCfg.setNearConfiguration(null);
-
-        if (META_CACHE_NAME.equals(cacheName))
-            cacheCfg.setCacheMode(REPLICATED);
-        else {
-            cacheCfg.setCacheMode(PARTITIONED);
-
-            cacheCfg.setBackups(0);
-            cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        }
-
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        return cacheCfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java
new file mode 100644
index 0000000..e453346
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAsyncClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * Tests for DUAL_ASYNC mode and client cache.
+ */
+public class IgfsDualAsyncClientSelfTest extends IgfsDualAsyncSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java
new file mode 100644
index 0000000..bb3da32
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualSyncClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * Tests for DUAL_SYNC mode and client cache.
+ */
+public class IgfsDualSyncClientSelfTest extends IgfsDualSyncSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
index 25d75a2..3bfd372 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
@@ -101,7 +101,6 @@ public class IgfsOneClientNodeTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStartIgfs() throws Exception {
-
         final IgfsImpl igfs = (IgfsImpl) grid(0).fileSystem("igfs");
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
@@ -109,7 +108,7 @@ public class IgfsOneClientNodeTest extends GridCommonAbstractTest {
                 IgfsAbstractSelfTest.create(igfs, new IgfsPath[]{new IgfsPath("/dir")}, null);
                 return null;
             }
-        }, IgfsException.class, "Cache server nodes not found.");
+        }, IgfsException.class, "Failed to execute operation because there are no IGFS metadata nodes.");
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -119,7 +118,7 @@ public class IgfsOneClientNodeTest extends GridCommonAbstractTest {
 
                 return null;
             }
-        }, IgfsException.class, "Cache server nodes not found.");
+        }, IgfsException.class, "Failed to execute operation because there are no IGFS metadata nodes.");
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
new file mode 100644
index 0000000..12fd93c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryClientSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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 org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Test for IGFS with nodes in client mode (see {@link IgniteConfiguration#setClientMode(boolean)}.
+ */
+public class IgfsPrimaryClientSelfTest extends IgfsPrimarySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyClientSelfTest.java
new file mode 100644
index 0000000..54c742d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * Tests for PRIMARY mode and relaxed consistency model with client cache.
+ */
+public class IgfsPrimaryRelaxedConsistencyClientSelfTest extends IgfsPrimaryRelaxedConsistencySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdc3aa6e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index 0047edb..3e80614 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -27,7 +27,9 @@ import org.apache.ignite.internal.processors.igfs.IgfsBackupsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsBlockMessageSystemPoolStarvationSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsCachePerBlockLruEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsCacheSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsClientCacheSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncClientSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsDualSyncClientSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryClientSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDataManagerSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsDualSyncSelfTest;
@@ -43,6 +45,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsOneClientNodeTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapTieredSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapValuesSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOptimziedMarshallerSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedClientSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedConsistencySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimarySelfTest;
@@ -103,7 +106,11 @@ public class IgniteIgfsTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgfsModesSelfTest.class));
         suite.addTest(new TestSuite(IgfsMetricsSelfTest.class));
 
-        suite.addTest(new TestSuite(IgfsClientCacheSelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryClientSelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryRelaxedClientSelfTest.class));
+        suite.addTest(new TestSuite(IgfsDualSyncClientSelfTest.class));
+        suite.addTest(new TestSuite(IgfsDualAsyncClientSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsOneClientNodeTest.class));
 
         suite.addTest(new TestSuite(IgfsModeResolverSelfTest.class));