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/10 06:10:23 UTC

[01/24] ignite git commit: IGNITE-3228: Hadoop: workaround/fix for inefficient memory usage.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3038 c0e8c420e -> 645adfdc5


IGNITE-3228: Hadoop: workaround/fix for inefficient memory usage.


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

Branch: refs/heads/ignite-3038
Commit: 2490b0aef17595247f8a6ae29482bc900e2a9a8d
Parents: 52a2637
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Jun 2 09:11:09 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Jun 2 09:11:09 2016 +0300

----------------------------------------------------------------------
 .../shuffle/collections/HadoopMultimapBase.java | 90 +++++++++++++++++---
 1 file changed, 76 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2490b0ae/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
index e6995ca..7dcff3d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
@@ -30,7 +30,6 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.shuffle.streams.HadoopDataInStream;
 import org.apache.ignite.internal.processors.hadoop.shuffle.streams.HadoopDataOutStream;
 import org.apache.ignite.internal.processors.hadoop.shuffle.streams.HadoopOffheapBuffer;
-import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.jetbrains.annotations.Nullable;
 
@@ -48,7 +47,7 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
     protected final int pageSize;
 
     /** */
-    private final Collection<GridLongList> allPages = new ConcurrentLinkedQueue<>();
+    private final Collection<Page> allPages = new ConcurrentLinkedQueue<>();
 
     /**
      * @param jobInfo Job info.
@@ -64,11 +63,12 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
     }
 
     /**
-     * @param ptrs Page pointers.
+     * @param page Page.
      */
-    private void deallocate(GridLongList ptrs) {
-        while (!ptrs.isEmpty())
-            mem.release(ptrs.remove(), ptrs.remove());
+    private void deallocate(Page page) {
+        assert page != null;
+
+        mem.release(page.ptr, page.size);
     }
 
     /**
@@ -105,8 +105,8 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
 
     /** {@inheritDoc} */
     @Override public void close() {
-        for (GridLongList list : allPages)
-            deallocate(list);
+        for (Page page : allPages)
+            deallocate(page);
     }
 
     /**
@@ -190,8 +190,8 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
         /** */
         private long writeStart;
 
-        /** Size and pointer pairs list. */
-        private final GridLongList pages = new GridLongList(16);
+        /** Current page. */
+        private Page curPage;
 
         /**
          * @param ctx Task context.
@@ -222,11 +222,10 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
         private long allocateNextPage(long requestedSize) {
             int writtenSize = writtenSize();
 
-            long newPageSize = Math.max(writtenSize + requestedSize, pageSize);
+            long newPageSize = nextPageSize(writtenSize + requestedSize);
             long newPagePtr = mem.allocate(newPageSize);
 
-            pages.add(newPageSize);
-            pages.add(newPagePtr);
+            System.out.println("ALLOCATED: " + newPageSize);
 
             HadoopOffheapBuffer b = out.buffer();
 
@@ -240,10 +239,50 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
 
             writeStart = newPagePtr;
 
+            // At this point old page is not needed, so we release it.
+            Page oldPage = curPage;
+
+            curPage = new Page(newPagePtr, newPageSize);
+
+            if (oldPage != null)
+                allPages.add(oldPage);
+
             return b.move(requestedSize);
         }
 
         /**
+         * Get next page size.
+         *
+         * @param required Required amount of data.
+         * @return Next page size.
+         */
+        private long nextPageSize(long required) {
+            long pages = (required / pageSize) + 1;
+
+            long pagesPow2 = nextPowerOfTwo(pages);
+
+            return pagesPow2 * pageSize;
+        }
+
+        /**
+         * Get next power of two which greater or equal to the given number. Naive implementation.
+         *
+         * @param val Number
+         * @return Nearest pow2.
+         */
+        private long nextPowerOfTwo(long val) {
+            long res = 1;
+
+            while (res < val)
+                res = res << 1;
+
+            if (res < 0)
+                throw new IllegalArgumentException("Value is too big to find positive pow2: " + val);
+
+            return res;
+        }
+
+        /**
          * @return Fixed pointer.
          */
         private long fixAlignment() {
@@ -317,7 +356,8 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
 
         /** {@inheritDoc} */
         @Override public void close() throws IgniteCheckedException {
-            allPages.add(pages);
+            if (curPage != null)
+                allPages.add(curPage);
 
             keySer.close();
             valSer.close();
@@ -372,4 +412,26 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
             throw new UnsupportedOperationException();
         }
     }
+
+    /**
+     * Page.
+     */
+    private static class Page {
+        /** Pointer. */
+        private final long ptr;
+
+        /** Size. */
+        private final long size;
+
+        /**
+         * Constructor.
+         *
+         * @param ptr Pointer.
+         * @param size Size.
+         */
+        public Page(long ptr, long size) {
+            this.ptr = ptr;
+            this.size = size;
+        }
+    }
 }
\ No newline at end of file


[13/24] ignite git commit: IGNITE-3256: IGFS: Removed "exists" check during input stream close.

Posted by sb...@apache.org.
IGNITE-3256: IGFS: Removed "exists" check during input stream close.


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

Branch: refs/heads/ignite-3038
Commit: 1308b122f84af3c98656ac11d8bed606c6b4ecb5
Parents: e409b67
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 6 15:40:59 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 6 15:40:59 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/igfs/IgfsInputStreamImpl.java   | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1308b122/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
index de7071a..ca2f9f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
@@ -317,13 +317,9 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
                         pendingFutsLock.unlock();
                     }
                 }
-
-                // Safety to ensure no orphaned data blocks exist in case file was concurrently deleted.
-                if (!meta.exists(fileInfo.id()))
-                    data.delete(fileInfo);
             }
         }
-        catch (IgniteCheckedException e) {
+        catch (Exception e) {
             throw new IOException("File to close the file: " + path, e);
         }
         finally {


[17/24] ignite git commit: IGNITE-3259: Delete worker is not started on client nodes any more.

Posted by sb...@apache.org.
IGNITE-3259: Delete worker is not started on client nodes any more.


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

Branch: refs/heads/ignite-3038
Commit: 0176af13646a09541d65a10cf7ec0641c71e2ca7
Parents: 5254957
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 6 18:10:36 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 6 18:10:36 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 25 ++++++++++++++------
 1 file changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0176af13/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 1dd4c53..465116b 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
@@ -207,19 +207,20 @@ public class IgfsMetaManager extends IgfsManager {
         locNode = igfsCtx.kernalContext().discovery().localNode();
 
         // Start background delete worker.
-        delWorker = new IgfsDeleteWorker(igfsCtx);
+        if (!client) {
+            delWorker = new IgfsDeleteWorker(igfsCtx);
 
-        delWorker.start();
+            delWorker.start();
+        }
     }
 
     /** {@inheritDoc} */
     @Override protected void onKernalStop0(boolean cancel) {
         IgfsDeleteWorker delWorker0 = delWorker;
 
-        if (delWorker0 != null)
+        if (delWorker0 != null) {
             delWorker0.cancel();
 
-        if (delWorker0 != null) {
             try {
                 U.join(delWorker0);
             }
@@ -1136,7 +1137,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     tx.commit();
 
-                    delWorker.signal();
+                    signalDeleteWorker();
 
                     return newInfo.id();
                 }
@@ -1212,7 +1213,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     tx.commit();
 
-                    delWorker.signal();
+                    signalDeleteWorker();
 
                     return victimId;
                 }
@@ -2476,7 +2477,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 Boolean res = synchronizeAndExecute(task, fs, false, Collections.singleton(trashId), path);
 
-                delWorker.signal();
+                signalDeleteWorker();
 
                 return res;
             }
@@ -3341,4 +3342,14 @@ public class IgfsMetaManager extends IgfsManager {
         else
             IgfsUtils.sendEvents(igfsCtx.kernalContext(), leafPath, EventType.EVT_IGFS_DIR_CREATED);
     }
+
+    /**
+     * Signal delete worker thread.
+     */
+    private void signalDeleteWorker() {
+        IgfsDeleteWorker delWorker0 = delWorker;
+
+        if (delWorker0 != null)
+            delWorker0.signal();
+    }
 }
\ No newline at end of file


[14/24] ignite git commit: Merge remote-tracking branch 'upstream/gridgain-7.5.25' into gridgain-7.5.25

Posted by sb...@apache.org.
Merge remote-tracking branch 'upstream/gridgain-7.5.25' into gridgain-7.5.25


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

Branch: refs/heads/ignite-3038
Commit: 2c0d95f23abf710af8b277f8d3df06e6d35441fa
Parents: 1308b12 6cb9947
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 6 15:41:42 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 6 15:41:42 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryUtils.java     |  16 ++
 .../processors/cache/CacheObjectContext.java    |   3 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  78 +++++++-
 .../unsafe/GridOffheapSnapTreeSelfTest.java     |   2 +-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  23 +--
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |  17 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  | 195 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 8 files changed, 321 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[19/24] ignite git commit: GG-11197 optimize Visor response size and time in external mode.

Posted by sb...@apache.org.
GG-11197 optimize Visor response size and time in external mode.


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

Branch: refs/heads/ignite-3038
Commit: e4337d2b0154184c20621ac4953693d2e6ae1d6f
Parents: 065d2e7
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Jun 8 14:43:02 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Jun 8 14:43:02 2016 +0700

----------------------------------------------------------------------
 .../GridClientConnectionManagerAdapter.java     |  25 ++-
 .../connection/GridClientNioTcpConnection.java  |   3 +
 .../GridClientOptimizedMarshaller.java          |   4 +-
 .../GridClientZipOptimizedMarshaller.java       | 162 +++++++++++++++++++
 .../impl/GridTcpRouterNioListenerAdapter.java   |  11 +-
 .../message/GridClientHandshakeRequest.java     |   4 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |  19 ++-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  12 +-
 .../ignite/internal/util/nio/GridNioServer.java |  10 +-
 .../ignite/internal/visor/cache/VisorCache.java |  56 +------
 .../visor/cache/VisorCachePartition.java        |  89 ++++++++++
 .../visor/cache/VisorCachePartitions.java       |  88 ++++++++++
 .../visor/cache/VisorCachePartitionsTask.java   | 152 +++++++++++++++++
 .../internal/visor/cache/VisorCacheV3.java      |  68 +-------
 14 files changed, 564 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
index 1bea3cc..6ea7c22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -47,6 +47,8 @@ import org.apache.ignite.internal.client.GridClientProtocol;
 import org.apache.ignite.internal.client.GridServerUnreachableException;
 import org.apache.ignite.internal.client.impl.GridClientFutureAdapter;
 import org.apache.ignite.internal.client.impl.GridClientThreadFactory;
+import org.apache.ignite.internal.client.marshaller.GridClientMarshaller;
+import org.apache.ignite.internal.client.marshaller.optimized.GridClientZipOptimizedMarshaller;
 import org.apache.ignite.internal.client.util.GridClientStripedLock;
 import org.apache.ignite.internal.client.util.GridClientUtils;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeResponse;
@@ -460,9 +462,26 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
             GridClientConnection conn;
 
             if (cfg.getProtocol() == GridClientProtocol.TCP) {
-                conn = new GridClientNioTcpConnection(srv, clientId, addr, sslCtx, pingExecutor,
-                    cfg.getConnectTimeout(), cfg.getPingInterval(), cfg.getPingTimeout(),
-                    cfg.isTcpNoDelay(), cfg.getMarshaller(), marshId, top, cred, keepBinariesThreadLocal());
+                GridClientMarshaller marsh = cfg.getMarshaller();
+
+                try {
+                    conn = new GridClientNioTcpConnection(srv, clientId, addr, sslCtx, pingExecutor,
+                        cfg.getConnectTimeout(), cfg.getPingInterval(), cfg.getPingTimeout(),
+                        cfg.isTcpNoDelay(), marsh, marshId, top, cred, keepBinariesThreadLocal());
+                }
+                catch (GridClientException e) {
+                    if (marsh instanceof GridClientZipOptimizedMarshaller) {
+                        log.warning("Failed to connect with GridClientZipOptimizedMarshaller," +
+                            " trying to fallback to default marshaller: " + e);
+
+                        conn = new GridClientNioTcpConnection(srv, clientId, addr, sslCtx, pingExecutor,
+                            cfg.getConnectTimeout(), cfg.getPingInterval(), cfg.getPingTimeout(),
+                            cfg.isTcpNoDelay(), ((GridClientZipOptimizedMarshaller)marsh).defaultMarshaller(), marshId,
+                            top, cred, keepBinariesThreadLocal());
+                    }
+                    else
+                        throw e;
+                }
             }
             else
                 throw new GridServerUnreachableException("Failed to create client (protocol is not supported): " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index cfcb07f..8937504 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.client.impl.GridClientNodeMetricsAdapter;
 import org.apache.ignite.internal.client.marshaller.GridClientMarshaller;
 import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller;
 import org.apache.ignite.internal.client.marshaller.optimized.GridClientOptimizedMarshaller;
+import org.apache.ignite.internal.client.marshaller.optimized.GridClientZipOptimizedMarshaller;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientAuthenticationRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientCacheRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeRequest;
@@ -243,6 +244,8 @@ public class GridClientNioTcpConnection extends GridClientConnection {
             if (marshId != null)
                 req.marshallerId(marshId);
             // marsh != null.
+            else if (marsh instanceof GridClientZipOptimizedMarshaller)
+                req.marshallerId(GridClientZipOptimizedMarshaller.ID);
             else if (marsh instanceof GridClientOptimizedMarshaller)
                 req.marshallerId(GridClientOptimizedMarshaller.ID);
             else if (marsh instanceof GridClientJdkMarshaller)

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
index 4bc1dac..a112736 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
@@ -38,7 +38,7 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
     public static final byte ID = 1;
 
     /** Optimized marshaller. */
-    private final OptimizedMarshaller opMarsh;
+    protected final OptimizedMarshaller opMarsh;
 
     /**
      * Default constructor.
@@ -136,4 +136,4 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
             throw new UnsupportedOperationException();
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java
new file mode 100644
index 0000000..0a0d07c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java
@@ -0,0 +1,162 @@
+/*
+ * 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.client.marshaller.optimized;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+import java.util.zip.ZipOutputStream;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.client.marshaller.GridClientMarshaller;
+import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.plugin.PluginProvider;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wrapper, that adapts {@link OptimizedMarshaller} to {@link GridClientMarshaller} interface.
+ */
+public class GridClientZipOptimizedMarshaller extends GridClientOptimizedMarshaller {
+    /** ID. */
+    public static final byte ID = 3;
+
+    /** Default buffer size. */
+    private static final int DFLT_BUFFER_SIZE = 4096;
+
+    /** Default client marshaller to fallback. */
+    private final GridClientMarshaller dfltMarsh;
+
+    /**
+     * Constructor.
+     *
+     * @param dfltMarsh Marshaller to fallback to.
+     * @param plugins Plugins.
+     */
+    public GridClientZipOptimizedMarshaller(GridClientMarshaller dfltMarsh, @Nullable List<PluginProvider> plugins) {
+        super(plugins);
+
+        assert dfltMarsh!= null;
+
+        this.dfltMarsh = dfltMarsh;
+    }
+
+    public GridClientMarshaller defaultMarshaller() {
+        return dfltMarsh;
+    }
+
+    /**
+     * Zips bytes.
+     *
+     * @param input Input bytes.
+     * @return Zipped byte array.
+     * @throws IOException If failed.
+     */
+    public static byte[] zipBytes(byte[] input) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream(DFLT_BUFFER_SIZE);
+
+        try (ZipOutputStream zos = new ZipOutputStream(baos)) {
+            ZipEntry entry = new ZipEntry("");
+
+            try {
+                entry.setSize(input.length);
+
+                zos.putNextEntry(entry);
+                zos.write(input);
+            }
+            finally {
+                zos.closeEntry();
+            }
+        }
+
+        return baos.toByteArray();
+    }
+
+    /**
+     * Unzip bytes.
+     *
+     * @param input Zipped bytes.
+     * @return Unzipped byte array.
+     * @throws IOException
+     */
+    private static byte[] unzipBytes(byte[] input) throws IOException {
+        ByteArrayInputStream bais = new ByteArrayInputStream(input);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream(DFLT_BUFFER_SIZE);
+
+        try(ZipInputStream zis = new ZipInputStream(bais)) {
+            zis.getNextEntry();
+
+            byte[] buf = new byte[DFLT_BUFFER_SIZE];
+
+            int len = zis.read(buf);
+
+            while (len > 0) {
+                baos.write(buf, 0, len);
+
+                len = zis.read(buf);
+            }
+        }
+
+        return baos.toByteArray();
+    }
+    /** {@inheritDoc} */
+    @Override public ByteBuffer marshal(Object obj, int off) throws IOException {
+        try {
+            if (!(obj instanceof GridClientMessage))
+                throw new IOException("Message serialization of given type is not supported: " +
+                    obj.getClass().getName());
+
+            byte[] marshBytes = opMarsh.marshal(obj);
+
+            boolean zip = marshBytes.length > 512;
+
+            byte[] bytes = zip ? zipBytes(marshBytes) : marshBytes;
+
+            ByteBuffer buf = ByteBuffer.allocate(off + bytes.length + 1);
+
+            buf.position(off);
+            buf.put((byte)(zip ? 1 : 0));
+            buf.put(bytes);
+            buf.flip();
+
+            return buf;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unmarshal(byte[] bytes) throws IOException {
+        try {
+            boolean unzip = bytes[0] > 0;
+
+            byte[] marshBytes = Arrays.copyOfRange(bytes, 1, bytes.length);
+
+            return opMarsh.unmarshal(unzip ? unzipBytes(marshBytes) : marshBytes, null);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
index 02b63ad..6bcea09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.client.router.impl;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteLogger;
@@ -29,6 +30,7 @@ import org.apache.ignite.internal.client.GridClientFutureListener;
 import org.apache.ignite.internal.client.marshaller.GridClientMarshaller;
 import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller;
 import org.apache.ignite.internal.client.marshaller.optimized.GridClientOptimizedMarshaller;
+import org.apache.ignite.internal.client.marshaller.optimized.GridClientZipOptimizedMarshaller;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeRequest;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeResponse;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage;
@@ -39,6 +41,7 @@ import org.apache.ignite.internal.processors.rest.client.message.GridRouterRespo
 import org.apache.ignite.internal.util.nio.GridNioServerListener;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.PluginProvider;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.MARSHALLER;
@@ -78,7 +81,11 @@ public abstract class GridTcpRouterNioListenerAdapter implements GridNioServerLi
 
         marshMap = new HashMap<>();
 
-        marshMap.put(GridClientOptimizedMarshaller.ID, new GridClientOptimizedMarshaller(U.allPluginProviders()));
+        List<PluginProvider> providers = U.allPluginProviders();
+        GridClientOptimizedMarshaller optdMarsh = new GridClientOptimizedMarshaller(providers);
+
+        marshMap.put(GridClientOptimizedMarshaller.ID, optdMarsh);
+        marshMap.put(GridClientZipOptimizedMarshaller.ID, new GridClientZipOptimizedMarshaller(optdMarsh, providers));
         marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller());
 
         init();
@@ -213,4 +220,4 @@ public abstract class GridTcpRouterNioListenerAdapter implements GridNioServerLi
 
         return res;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequest.java
index 3790dd0..4e1ba91 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequest.java
@@ -66,7 +66,7 @@ public class GridClientHandshakeRequest extends GridClientAbstractMessage {
      * @param marshId Marshaller ID.
      */
     public void marshallerId(byte marshId) {
-        assert marshId >= 0 && marshId <= 2;
+        assert marshId >= 0 && marshId <= 3;
 
         this.marshId = marshId;
     }
@@ -104,4 +104,4 @@ public class GridClientHandshakeRequest extends GridClientAbstractMessage {
     @Override public String toString() {
         return getClass().getSimpleName() + " [arr=" + Arrays.toString(arr) + ']';
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
index bf177cf..2cfdb75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
@@ -250,14 +250,17 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
                             GridNioFuture<?> sf = ses.send(res);
 
                             // Check if send failed.
-                            if (sf.isDone())
-                                try {
-                                    sf.get();
-                                }
-                                catch (Exception e) {
-                                    U.error(log, "Failed to process client request [ses=" + ses + ", msg=" + msg + ']',
-                                        e);
+                            sf.listen(new CI1<IgniteInternalFuture<?>>() {
+                                @Override public void apply(IgniteInternalFuture<?> fut) {
+                                    try {
+                                        fut.get();
+                                    }
+                                    catch (IgniteCheckedException e) {
+                                        U.error(log, "Failed to process client request [ses=" + ses +
+                                            ", msg=" + msg + ']', e);
+                                    }
                                 }
+                            });
                         }
                     });
                 else
@@ -360,4 +363,4 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
     @Override public void onSessionIdleTimeout(GridNioSession ses) {
         ses.close();
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index 8e08481..0553b5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.client.marshaller.GridClientMarshaller;
 import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller;
 import org.apache.ignite.internal.client.marshaller.optimized.GridClientOptimizedMarshaller;
+import org.apache.ignite.internal.client.marshaller.optimized.GridClientZipOptimizedMarshaller;
 import org.apache.ignite.internal.client.ssl.GridSslContextFactory;
 import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage;
@@ -48,6 +49,7 @@ import org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.spi.IgnitePortProtocol;
 import org.jetbrains.annotations.Nullable;
 
@@ -169,8 +171,12 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
 
         Map<Byte, GridClientMarshaller> marshMap = new HashMap<>();
 
-        marshMap.put(GridClientOptimizedMarshaller.ID,
-            new GridClientOptimizedMarshaller(new ArrayList<>(ctx.plugins().allProviders())));
+        ArrayList<PluginProvider> providers = new ArrayList<>(ctx.plugins().allProviders());
+
+        GridClientOptimizedMarshaller optMarsh = new GridClientOptimizedMarshaller(providers);
+
+        marshMap.put(GridClientOptimizedMarshaller.ID, optMarsh);
+        marshMap.put(GridClientZipOptimizedMarshaller.ID, new GridClientZipOptimizedMarshaller(optMarsh, providers));
         marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller());
 
         lsnr.marshallers(marshMap);
@@ -290,4 +296,4 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
     @Override protected String getPortPropertyName() {
         return IgniteNodeAttributes.ATTR_REST_TCP_PORT;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 3ae8312..75cf776 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -1536,8 +1536,7 @@ public class GridNioServer<T> {
                     throw e;
                 }
                 catch (Exception e) {
-                    if (!closed)
-                        U.warn(log, "Failed to process selector key (will close): " + ses, e);
+                    U.warn(log, "Failed to process selector key (will close): " + ses, e);
 
                     close(ses, new GridNioException(e));
                 }
@@ -1558,9 +1557,10 @@ public class GridNioServer<T> {
                 try {
                     long writeTimeout0 = writeTimeout;
 
+                    boolean opWrite = key.isValid() && (key.interestOps() & SelectionKey.OP_WRITE) != 0;
+
                     // If we are writing and timeout passed.
-                    if (key.isValid() && (key.interestOps() & SelectionKey.OP_WRITE) != 0 &&
-                        now - ses.lastSendTime() > writeTimeout0) {
+                    if (opWrite && now - ses.lastSendTime() > writeTimeout0) {
                         filterChain.onSessionWriteTimeout(ses);
 
                         // Update timestamp to avoid multiple notifications within one timeout interval.
@@ -1571,7 +1571,7 @@ public class GridNioServer<T> {
 
                     long idleTimeout0 = idleTimeout;
 
-                    if (now - ses.lastReceiveTime() > idleTimeout0 && now - ses.lastSendScheduleTime() > idleTimeout0) {
+                    if (!opWrite && now - ses.lastReceiveTime() > idleTimeout0 && now - ses.lastSendScheduleTime() > idleTimeout0) {
                         filterChain.onSessionIdleTimeout(ses);
 
                         // Update timestamp to avoid multiple notifications within one timeout interval.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
index b5151c7..b77b795 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
@@ -18,24 +18,18 @@
 package org.apache.ignite.internal.visor.cache;
 
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Set;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
@@ -97,10 +91,10 @@ public class VisorCache implements Serializable {
     /** Number of partitions. */
     private int partitions;
 
-    /** Primary partitions IDs with sizes. */
+    /** @deprecated Needed only for backward compatibility. */
     private Collection<IgnitePair<Integer>> primaryPartitions;
 
-    /** Backup partitions IDs with sizes. */
+    /** @deprecated Needed only for backward compatibility. */
     private Collection<IgnitePair<Integer>> backupPartitions;
 
     /** Cache metrics. */
@@ -162,53 +156,11 @@ public class VisorCache implements Serializable {
 
                     partitionsMap = new GridDhtPartitionMap(map2.nodeId(), map2.updateSequence(), map2.map());
                 }
-
-                List<GridDhtLocalPartition> parts = top.localPartitions();
-
-                primaryPartitions = new ArrayList<>(parts.size());
-                backupPartitions = new ArrayList<>(parts.size());
-
-                for (GridDhtLocalPartition part : parts) {
-                    int p = part.id();
-
-                    int sz = part.size();
-
-                    // Pass -1 as topology version in order not to wait for topology version.
-                    if (part.primary(AffinityTopologyVersion.NONE))
-                        primaryPartitions.add(new IgnitePair<>(p, sz));
-                    else if (part.state() == GridDhtPartitionState.OWNING && part.backup(AffinityTopologyVersion.NONE))
-                        backupPartitions.add(new IgnitePair<>(p, sz));
-                }
-            }
-            else {
-                // Old way of collecting partitions info.
-                ClusterNode node = ignite.cluster().localNode();
-
-                int[] pp = ca.affinity().primaryPartitions(node);
-
-                primaryPartitions= new ArrayList<>(pp.length);
-
-                for (int p : pp) {
-                    Set set = ca.entrySet(p);
-
-                    primaryPartitions.add(new IgnitePair<>(p, set != null ? set.size() : 0));
-                }
-
-                int[] bp = ca.affinity().backupPartitions(node);
-
-                backupPartitions = new ArrayList<>(bp.length);
-
-                for (int p : bp) {
-                    Set set = ca.entrySet(p);
-
-                    backupPartitions.add(new IgnitePair<>(p, set != null ? set.size() : 0));
-                }
             }
         }
 
         size = ca.size();
         nearSize = ca.nearSize();
-
         dynamicDeploymentId = ca.context().dynamicDeploymentId();
         dhtSize = size - nearSize;
         primarySize = ca.primarySize();
@@ -401,14 +353,14 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @return Primary partitions IDs with sizes.
+     * @deprecated Needed only for backward compatibility.
      */
     public Collection<IgnitePair<Integer>> primaryPartitions() {
         return primaryPartitions;
     }
 
     /**
-     * @return Backup partitions IDs with sizes.
+     * @deprecated Needed only for backward compatibility.
      */
     public Collection<IgnitePair<Integer>> backupPartitions() {
         return backupPartitions;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java
new file mode 100644
index 0000000..5909c1a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java
@@ -0,0 +1,89 @@
+/*
+ * 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.visor.cache;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Data transfer object for information about keys in cache partition.
+ */
+public class VisorCachePartition implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private int part;
+
+    /** */
+    private int heap;
+
+    /** */
+    private long offheap;
+
+    /** */
+    private long swap;
+
+    /**
+     * Full constructor.
+     *
+     * @param part Partition id.
+     * @param heap Number of keys in heap.
+     * @param offheap Number of keys in offheap.
+     * @param swap Number of keys in swap.
+     */
+    public VisorCachePartition(int part, int heap, long offheap, long swap) {
+        this.part = part;
+        this.heap = heap;
+        this.offheap = offheap;
+        this.swap = swap;
+    }
+
+    /**
+     * @return Partition id.
+     */
+    public int partition() {
+        return part;
+    }
+
+    /**
+     * @return Number of keys in heap.
+     */
+    public int heap() {
+        return heap;
+    }
+
+    /**
+     * @return Number of keys in offheap.
+     */
+    public long offheap() {
+        return offheap;
+    }
+
+    /**
+     * @return Number of keys in swap.
+     */
+    public long swap() {
+        return swap;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCachePartition.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java
new file mode 100644
index 0000000..4634fa6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java
@@ -0,0 +1,88 @@
+/*
+ * 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.visor.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Data transfer object for information about cache partitions.
+ */
+public class VisorCachePartitions implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private List<VisorCachePartition> primary;
+
+    /** */
+    private List<VisorCachePartition> backup;
+
+    /**
+     * Default constructor.
+     */
+    public VisorCachePartitions() {
+        primary = new ArrayList<>();
+        backup = new ArrayList<>();
+    }
+
+    /**
+     * Add primary partition descriptor.
+     *
+     * @param part Partition id.
+     * @param heap Number of primary keys in heap.
+     * @param offheap Number of primary keys in offheap.
+     * @param swap Number of primary keys in swap.
+     */
+    public void addPrimary(int part, int heap, long offheap, long swap) {
+       primary.add(new VisorCachePartition(part, heap, offheap, swap));
+    }
+
+    /**
+     * Add backup partition descriptor.
+     *
+     * @param part Partition id.
+     * @param heap Number of backup keys in heap.
+     * @param offheap Number of backup keys in offheap.
+     * @param swap Number of backup keys in swap.
+     */
+    public void addBackup(int part, int heap, long offheap, long swap) {
+       backup.add(new VisorCachePartition(part, heap, offheap, swap));
+    }
+
+    /**
+     * @return Get list of primary partitions.
+     */
+    public List<VisorCachePartition> primary() {
+        return primary;
+    }
+
+    /**
+     * @return Get list of backup partitions.
+     */
+    public List<VisorCachePartition> backup() {
+        return backup;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCachePartitions.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java
new file mode 100644
index 0000000..80836c6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java
@@ -0,0 +1,152 @@
+/*
+ * 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.visor.cache;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorMultiNodeTask;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.visor.util.VisorTaskUtils.log;
+import static org.apache.ignite.internal.visor.util.VisorTaskUtils.escapeName;
+
+/**
+ * Task that collect keys distribution in partitions.
+ */
+@GridInternal
+public class VisorCachePartitionsTask extends VisorMultiNodeTask<String, Map<UUID, VisorCachePartitions>, VisorCachePartitions> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCachePartitionsJob job(String arg) {
+        return new VisorCachePartitionsJob(arg, debug);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override protected Map<UUID, VisorCachePartitions> reduce0(List<ComputeJobResult> results) {
+        Map<UUID, VisorCachePartitions> parts = new HashMap<>();
+
+        for (ComputeJobResult res : results) {
+            if (res.getException() != null)
+                throw res.getException();
+
+            parts.put(res.getNode().id(), (VisorCachePartitions)res.getData());
+        }
+
+        return parts;
+    }
+
+    /**
+     * Job that collect cache metrics from node.
+     */
+    private static class VisorCachePartitionsJob extends VisorJob<String, VisorCachePartitions> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with given argument.
+         *
+         * @param cacheName Cache name.
+         * @param debug Debug flag.
+         */
+        private VisorCachePartitionsJob(String cacheName, boolean debug) {
+            super(cacheName, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected VisorCachePartitions run(final String cacheName) throws IgniteException {
+            if (debug)
+                log(ignite.log(), "Collecting partitions for cache: " + escapeName(cacheName));
+
+            VisorCachePartitions parts = new VisorCachePartitions();
+
+            GridCacheAdapter ca = ignite.context().cache().internalCache(cacheName);
+
+            // Cache was not started.
+            if (ca == null || !ca.context().started())
+                return parts;
+
+            CacheConfiguration cfg = ca.configuration();
+
+            CacheMode mode = cfg.getCacheMode();
+
+            boolean partitioned = (mode == CacheMode.PARTITIONED || mode == CacheMode.REPLICATED)
+                && ca.context().affinityNode();
+
+            if (partitioned) {
+                GridCacheSwapManager swap = ca.context().swap();
+
+                GridDhtCacheAdapter dca = null;
+
+                if (ca instanceof GridNearCacheAdapter)
+                    dca = ((GridNearCacheAdapter)ca).dht();
+                else if (ca instanceof GridDhtCacheAdapter)
+                    dca = (GridDhtCacheAdapter)ca;
+
+                if (dca != null) {
+                    GridDhtPartitionTopology top = dca.topology();
+
+                    List<GridDhtLocalPartition> locParts = top.localPartitions();
+
+                    try {
+                        for (GridDhtLocalPartition part : locParts) {
+                            int p = part.id();
+
+                            int sz = part.size();
+
+                            // Pass -1 as topology version in order not to wait for topology version.
+                            if (part.primary(AffinityTopologyVersion.NONE))
+                                parts.addPrimary(p, sz, swap.offheapEntriesCount(p), swap.swapEntriesCount(p));
+                            else if (part.state() == GridDhtPartitionState.OWNING && part.backup(AffinityTopologyVersion.NONE))
+                                parts.addBackup(p, sz, swap.offheapEntriesCount(p), swap.swapEntriesCount(p));
+                        }
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IgniteException("Failed to collect keys distribution in partitions", e);
+                    }
+                }
+            }
+
+            return parts;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(VisorCachePartitionsJob.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4337d2b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
index bd9a3ce..fab37e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
@@ -17,90 +17,34 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheSwapManager;
 import org.apache.ignite.internal.util.lang.GridTuple3;
-import org.apache.ignite.internal.util.lang.IgnitePair;
 
 /**
  * Data transfer object for {@link IgniteCache}.
+ *
+ * @deprecated Needed only for backward compatibility.
  */
 public class VisorCacheV3 extends VisorCacheV2 {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Primary partitions IDs with offheap and swap entries count. */
+    /** @deprecated Needed only for backward compatibility. */
     private Collection<GridTuple3<Integer, Long, Long>> primaryPartsOffheapSwap;
 
-    /** Backup partitions IDs with offheap and swap entries count. */
+    /** @deprecated Needed only for backward compatibility. */
     private Collection<GridTuple3<Integer, Long, Long>> backupPartsOffheapSwap;
 
-    /** {@inheritDoc} */
-    @Override public VisorCache from(IgniteEx ignite, String cacheName, int sample) throws IgniteCheckedException {
-        VisorCache c = super.from(ignite, cacheName, sample);
-
-        if (c != null && c instanceof VisorCacheV3) {
-            VisorCacheV3 cacheV3 = (VisorCacheV3)c;
-
-            GridCacheAdapter ca = ignite.context().cache().internalCache(cacheName);
-
-            // Process only started caches.
-            if (ca != null && ca.context().started()) {
-                GridCacheSwapManager swap = ca.context().swap();
-
-                cacheV3.primaryPartsOffheapSwap = new ArrayList<>(c.primaryPartitions().size());
-
-                for (IgnitePair<Integer> part: c.primaryPartitions()) {
-                    int p = part.get1();
-
-                    cacheV3.primaryPartsOffheapSwap.add(new GridTuple3<>(p, swap.offheapEntriesCount(p), swap.swapEntriesCount(p)));
-                }
-
-                cacheV3.backupPartsOffheapSwap = new ArrayList<>(c.backupPartitions().size());
-
-                for (IgnitePair<Integer> part: c.backupPartitions()) {
-                    int p = part.get1();
-
-                    cacheV3.backupPartsOffheapSwap.add(new GridTuple3<>(p, swap.offheapEntriesCount(p), swap.swapEntriesCount(p)));
-                }
-            }
-        }
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected VisorCache initHistory(VisorCache c) {
-        super.initHistory(c);
-
-        if (c instanceof VisorCacheV3) {
-            ((VisorCacheV3)c).primaryPartsOffheapSwap = Collections.emptyList();
-            ((VisorCacheV3)c).backupPartsOffheapSwap = Collections.emptyList();
-        }
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override public VisorCache history() {
-        return initHistory(new VisorCacheV3());
-    }
-
     /**
-     * @return Collection with primary partitions IDs and offheap and swap entries count.
+     * @deprecated Needed only for backward compatibility.
      */
     public Collection<GridTuple3<Integer, Long, Long>> primaryPartitionsOffheapSwap() {
         return primaryPartsOffheapSwap;
     }
 
     /**
-     * @return Collection with backup partitions IDs and offheap and swap entries count.
+     * @deprecated Needed only for backward compatibility.
      */
     public Collection<GridTuple3<Integer, Long, Long>> backupPartitionsOffheapSwap() {
         return backupPartsOffheapSwap;


[02/24] ignite git commit: IGNITE-3228: Minor fix.

Posted by sb...@apache.org.
IGNITE-3228: Minor fix.


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

Branch: refs/heads/ignite-3038
Commit: 50a8ec24fcae6cb673a257789724dd5a2dbb734a
Parents: 2490b0a
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Jun 2 11:47:38 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Jun 2 11:47:38 2016 +0300

----------------------------------------------------------------------
 .../processors/hadoop/shuffle/collections/HadoopMultimapBase.java  | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/50a8ec24/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
index 7dcff3d..39b7c51 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
@@ -225,8 +225,6 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
             long newPageSize = nextPageSize(writtenSize + requestedSize);
             long newPagePtr = mem.allocate(newPageSize);
 
-            System.out.println("ALLOCATED: " + newPageSize);
-
             HadoopOffheapBuffer b = out.buffer();
 
             b.set(newPagePtr, newPageSize);


[15/24] ignite git commit: IGNITE-3257: IGFS: FileSystemConfiguration.DFLT_INIT_DFLT_PATH_MODES has been changed to "false".

Posted by sb...@apache.org.
IGNITE-3257: IGFS: FileSystemConfiguration.DFLT_INIT_DFLT_PATH_MODES has been changed to "false".


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

Branch: refs/heads/ignite-3038
Commit: 672b495a43905281cf10d7204b5ee411bc9706cc
Parents: 2c0d95f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 6 15:50:13 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 6 15:50:13 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/configuration/FileSystemConfiguration.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/672b495a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
index 518bbf6..625ba95 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
@@ -83,7 +83,7 @@ public class FileSystemConfiguration {
     public static final boolean DFLT_IPC_ENDPOINT_ENABLED = true;
 
     /** Default value of whether to initialize default path modes. */
-    public static final boolean DFLT_INIT_DFLT_PATH_MODES = true;
+    public static final boolean DFLT_INIT_DFLT_PATH_MODES = false;
 
     /** Default value of metadata co-location flag. */
     public static final boolean DFLT_COLOCATE_META = true;


[24/24] ignite git commit: ignite-3038

Posted by sb...@apache.org.
ignite-3038


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

Branch: refs/heads/ignite-3038
Commit: 645adfdc5c80834e33f54d640a014aa848b4e0a7
Parents: bcd4011
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 10 09:10:13 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 10 09:10:13 2016 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/MarshallerContextImpl.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/645adfdc/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 68a9d0a..b4c9607 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -37,7 +37,6 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
 import org.apache.ignite.internal.util.GridStripedLock;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;


[12/24] ignite git commit: Merge branch 'gridgain-7.5.25' of https://github.com/gridgain/apache-ignite into ignite-gg-11181

Posted by sb...@apache.org.
Merge branch 'gridgain-7.5.25' of https://github.com/gridgain/apache-ignite into ignite-gg-11181


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

Branch: refs/heads/ignite-3038
Commit: 6cb9947a96f7bfd92266a1b01fb5fef09d2269b2
Parents: a0c6ae5 9e9252c
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Jun 6 13:33:06 2016 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Jun 6 13:33:06 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryUtils.java     | 16 ++++
 .../processors/cache/CacheObjectContext.java    |  3 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java | 78 +++++++++++++++++++-
 3 files changed, 95 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b96afb2f/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 25c54e4..48eb665 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;
@@ -42,6 +44,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.IgfsPrimaryRelaxedSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorSelfTest;
@@ -95,7 +98,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));

http://git-wip-us.apache.org/repos/asf/ignite/blob/b96afb2f/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
index 81f5743..c5f3425 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
@@ -110,7 +110,7 @@ public abstract class Hadoop1DualAbstractTest extends IgfsDualAbstractSelfTest {
      * @throws IOException On failure.
      */
     protected void startUnderlying() throws Exception {
-        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG);
+        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG, secondaryIpFinder);
     }
 
     /**


[16/24] ignite git commit: IGNITE-3258: IGFS: Secondary file system input stream is opened only when it is really needed.

Posted by sb...@apache.org.
IGNITE-3258: IGFS: Secondary file system input stream is opened only when it is really needed.


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

Branch: refs/heads/ignite-3038
Commit: 525495742158de7b077078af372280943b52d87d
Parents: 672b495
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 6 17:50:58 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 6 17:50:58 2016 +0300

----------------------------------------------------------------------
 ...zySecondaryFileSystemPositionedReadable.java | 77 ++++++++++++++++++++
 .../processors/igfs/IgfsMetaManager.java        | 18 ++++-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  3 +
 .../processors/igfs/IgfsModesSelfTest.java      |  1 +
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |  1 +
 5 files changed, 98 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/52549574/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsLazySecondaryFileSystemPositionedReadable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsLazySecondaryFileSystemPositionedReadable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsLazySecondaryFileSystemPositionedReadable.java
new file mode 100644
index 0000000..0a57c34
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsLazySecondaryFileSystemPositionedReadable.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;
+
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.io.IOException;
+
+/**
+ * Lazy readable entity which is opened on demand.
+ */
+public class IgfsLazySecondaryFileSystemPositionedReadable implements IgfsSecondaryFileSystemPositionedReadable {
+    /** File system. */
+    private final IgfsSecondaryFileSystem fs;
+
+    /** Path. */
+    private final IgfsPath path;
+
+    /** Buffer size. */
+    private final int bufSize;
+
+    /** Target stream. */
+    private IgfsSecondaryFileSystemPositionedReadable target;
+
+    /**
+     * Constructor.
+     *
+     * @param fs File system.
+     * @param path Path.
+     * @param bufSize Buffer size.
+     */
+    public IgfsLazySecondaryFileSystemPositionedReadable(IgfsSecondaryFileSystem fs, IgfsPath path, int bufSize) {
+        assert fs != null;
+        assert path != null;
+
+        this.fs = fs;
+        this.path = path;
+        this.bufSize = bufSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(long pos, byte[] buf, int off, int len) throws IOException {
+        if (target == null)
+            target = fs.open(path, bufSize);
+
+        return target.read(pos, buf, off, len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        if (target != null)
+            target.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsLazySecondaryFileSystemPositionedReadable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/52549574/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 fa748f8..1dd4c53 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
@@ -2144,7 +2144,7 @@ public class IgfsMetaManager extends IgfsManager {
                         throw fsException(new IgfsPathIsDirectoryException("Failed to open file (not a file): " +
                             path));
 
-                    return new IgfsSecondaryInputStreamDescriptor(info, fs.open(path, bufSize));
+                    return new IgfsSecondaryInputStreamDescriptor(info, lazySecondaryReader(fs, path, bufSize));
                 }
 
                 // If failed, try synchronize.
@@ -2160,7 +2160,8 @@ public class IgfsMetaManager extends IgfsManager {
                                 throw fsException(new IgfsPathIsDirectoryException("Failed to open file " +
                                     "(not a file): " + path));
 
-                            return new IgfsSecondaryInputStreamDescriptor(infos.get(path), fs.open(path, bufSize));
+                            return new IgfsSecondaryInputStreamDescriptor(infos.get(path),
+                                lazySecondaryReader(fs, path, bufSize));
                         }
 
                         @Override public IgfsSecondaryInputStreamDescriptor onFailure(@Nullable Exception err)
@@ -2184,6 +2185,19 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Create lazy secondary file system reader.
+     *
+     * @param fs File system.
+     * @param path Path.
+     * @param bufSize Buffer size.
+     * @return Lazy reader.
+     */
+    private static IgfsLazySecondaryFileSystemPositionedReadable lazySecondaryReader(IgfsSecondaryFileSystem fs,
+        IgfsPath path, int bufSize) {
+        return new IgfsLazySecondaryFileSystemPositionedReadable(fs, path, bufSize);
+    }
+
+    /**
      * Synchronizes with secondary file system.
      *
      * @param fs File system.

http://git-wip-us.apache.org/repos/asf/ignite/blob/52549574/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 3fb7b91..76a038d 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
@@ -1082,6 +1082,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         createFile(igfs.asSecondary(), FILE, true, chunk);
 
         checkFileContent(igfs, FILE, chunk);
+
+        // Read again when the whole file is in memory.
+        checkFileContent(igfs, FILE, chunk);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/52549574/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
index df537bc..1e54f8c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
@@ -102,6 +102,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setMetaCacheName("replicated");
         igfsCfg.setName("igfs");
         igfsCfg.setBlockSize(512 * 1024);
+        igfsCfg.setInitializeDefaultPathModes(true);
 
         if (setNullMode)
             igfsCfg.setDefaultMode(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/52549574/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java
index 1d02f0f..e4c64ff 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java
@@ -226,6 +226,7 @@ public class HadoopFIleSystemFactorySelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setDefaultMode(dfltMode);
         igfsCfg.setIpcEndpointConfiguration(endpointCfg);
         igfsCfg.setSecondaryFileSystem(secondaryFs);
+        igfsCfg.setInitializeDefaultPathModes(true);
 
         CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
 


[18/24] ignite git commit: IGNITE-3260: IGFS: Delete messages are no longer passed.

Posted by sb...@apache.org.
IGNITE-3260: IGFS: Delete messages are no longer passed.


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

Branch: refs/heads/ignite-3038
Commit: 065d2e70c21418437eba5e725eaa8b1ebc3af6da
Parents: 0176af1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 6 18:12:42 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 6 18:12:42 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsAsyncImpl.java |   6 -
 .../processors/igfs/IgfsDataManager.java        |  61 ++---
 .../processors/igfs/IgfsDeleteWorker.java       |  42 ----
 .../ignite/internal/processors/igfs/IgfsEx.java |   9 -
 .../internal/processors/igfs/IgfsImpl.java      | 249 +++++--------------
 .../internal/processors/igfs/IgfsUtils.java     |   2 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |   2 -
 .../processors/igfs/IgfsSizeSelfTest.java       | 133 ----------
 .../HadoopDefaultMapReducePlannerSelfTest.java  |   6 -
 9 files changed, 83 insertions(+), 427 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
index 8653f90..7530557 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
@@ -33,7 +33,6 @@ import org.apache.ignite.igfs.mapreduce.IgfsRecordResolver;
 import org.apache.ignite.igfs.mapreduce.IgfsTask;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.AsyncSupportAdapter;
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
@@ -166,11 +165,6 @@ public class IgfsAsyncImpl extends AsyncSupportAdapter<IgniteFileSystem> impleme
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
-        return igfs.awaitDeletesAsync();
-    }
-
-    /** {@inheritDoc} */
     @Nullable @Override public String clientLogDirectory() {
         return igfs.clientLogDirectory();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index 16fbeb8..57a8c6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -33,7 +33,6 @@ import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
 import org.apache.ignite.igfs.IgfsOutOfSpaceException;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
-import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
@@ -1056,34 +1055,24 @@ public class IgfsDataManager extends IgfsManager {
     private void processPartialBlockWrite(IgniteUuid fileId, IgfsBlockKey colocatedKey, int startOff,
         byte[] data) throws IgniteCheckedException {
         if (dataCachePrj.igfsDataSpaceUsed() >= dataCachePrj.igfsDataSpaceMax()) {
-            try {
-                igfs.awaitDeletesAsync().get(trashPurgeTimeout);
-            }
-            catch (IgniteFutureTimeoutCheckedException ignore) {
-                // Ignore.
-            }
+            final WriteCompletionFuture completionFut = pendingWrites.get(fileId);
 
-            // Additional size check.
-            if (dataCachePrj.igfsDataSpaceUsed() >= dataCachePrj.igfsDataSpaceMax()) {
-                final WriteCompletionFuture completionFut = pendingWrites.get(fileId);
-
-                if (completionFut == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Missing completion future for file write request (most likely exception occurred " +
-                            "which will be thrown upon stream close) [fileId=" + fileId + ']');
+            if (completionFut == null) {
+                if (log.isDebugEnabled())
+                    log.debug("Missing completion future for file write request (most likely exception occurred " +
+                        "which will be thrown upon stream close) [fileId=" + fileId + ']');
 
-                    return;
-                }
+                return;
+            }
 
-                IgfsOutOfSpaceException e = new IgfsOutOfSpaceException("Failed to write data block " +
-                    "(IGFS maximum data size exceeded) [used=" + dataCachePrj.igfsDataSpaceUsed() +
-                    ", allowed=" + dataCachePrj.igfsDataSpaceMax() + ']');
+            IgfsOutOfSpaceException e = new IgfsOutOfSpaceException("Failed to write data block " +
+                "(IGFS maximum data size exceeded) [used=" + dataCachePrj.igfsDataSpaceUsed() +
+                ", allowed=" + dataCachePrj.igfsDataSpaceMax() + ']');
 
-                completionFut.onDone(new IgniteCheckedException("Failed to write data (not enough space on node): " +
-                    igfsCtx.kernalContext().localNodeId(), e));
+            completionFut.onDone(new IgniteCheckedException("Failed to write data (not enough space on node): " +
+                igfsCtx.kernalContext().localNodeId(), e));
 
-                return;
-            }
+            return;
         }
 
         // No affinity key present, just concat and return.
@@ -1225,26 +1214,10 @@ public class IgfsDataManager extends IgfsManager {
         assert !blocks.isEmpty();
 
         if (dataCachePrj.igfsDataSpaceUsed() >= dataCachePrj.igfsDataSpaceMax()) {
-            try {
-                try {
-                    igfs.awaitDeletesAsync().get(trashPurgeTimeout);
-                }
-                catch (IgniteFutureTimeoutCheckedException ignore) {
-                    // Ignore.
-                }
-
-                // Additional size check.
-                if (dataCachePrj.igfsDataSpaceUsed() >= dataCachePrj.igfsDataSpaceMax())
-                    return new GridFinishedFuture<Object>(
-                        new IgfsOutOfSpaceException("Failed to write data block (IGFS maximum data size " +
-                            "exceeded) [used=" + dataCachePrj.igfsDataSpaceUsed() +
-                            ", allowed=" + dataCachePrj.igfsDataSpaceMax() + ']'));
-
-            }
-            catch (IgniteCheckedException e) {
-                return new GridFinishedFuture<>(new IgniteCheckedException("Failed to store data " +
-                    "block due to unexpected exception.", e));
-            }
+            return new GridFinishedFuture<Object>(
+                new IgfsOutOfSpaceException("Failed to write data block (IGFS maximum data size " +
+                    "exceeded) [used=" + dataCachePrj.igfsDataSpaceUsed() +
+                    ", allowed=" + dataCachePrj.igfsDataSpaceMax() + ']'));
         }
 
         return dataCachePrj.putAllAsync(blocks);

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
index bae9354..310090d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
@@ -19,13 +19,10 @@ package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
-import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
@@ -37,8 +34,6 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
-import static org.apache.ignite.internal.GridTopic.TOPIC_IGFS;
-
 /**
  * IGFS worker for removal from the trash directory.
  */
@@ -49,9 +44,6 @@ public class IgfsDeleteWorker extends IgfsThread {
     /** How many files/folders to delete at once (i.e in a single transaction). */
     private static final int MAX_DELETE_BATCH = 100;
 
-    /** IGFS context. */
-    private final IgfsContext igfsCtx;
-
     /** Metadata manager. */
     private final IgfsMetaManager meta;
 
@@ -73,9 +65,6 @@ public class IgfsDeleteWorker extends IgfsThread {
     /** Cancellation flag. */
     private volatile boolean cancelled;
 
-    /** Message topic. */
-    private Object topic;
-
     /**
      * Constructor.
      *
@@ -84,15 +73,9 @@ public class IgfsDeleteWorker extends IgfsThread {
     IgfsDeleteWorker(IgfsContext igfsCtx) {
         super("igfs-delete-worker%" + igfsCtx.igfs().name() + "%" + igfsCtx.kernalContext().localNodeId() + "%");
 
-        this.igfsCtx = igfsCtx;
-
         meta = igfsCtx.meta();
         data = igfsCtx.data();
 
-        String igfsName = igfsCtx.igfs().name();
-
-        topic = F.isEmpty(igfsName) ? TOPIC_IGFS : TOPIC_IGFS.topic(igfsName);
-
         assert meta != null;
         assert data != null;
 
@@ -189,8 +172,6 @@ public class IgfsDeleteWorker extends IgfsThread {
                             if (log.isDebugEnabled())
                                 log.debug("Sending delete confirmation message [name=" + entry.getKey() +
                                     ", fileId=" + fileId + ']');
-
-                            sendDeleteMessage(new IgfsDeleteMessage(fileId));
                         }
                     }
                     else
@@ -201,8 +182,6 @@ public class IgfsDeleteWorker extends IgfsThread {
                 }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Failed to delete entry from the trash directory: " + entry.getKey(), e);
-
-                    sendDeleteMessage(new IgfsDeleteMessage(fileId, e));
                 }
             }
         }
@@ -346,25 +325,4 @@ public class IgfsDeleteWorker extends IgfsThread {
                 return true; // Directory entry was deleted concurrently.
         }
     }
-
-    /**
-     * Send delete message to all meta cache nodes in the grid.
-     *
-     * @param msg Message to send.
-     */
-    private void sendDeleteMessage(IgfsDeleteMessage msg) {
-        assert msg != null;
-
-        Collection<ClusterNode> nodes = meta.metaCacheNodes();
-
-        for (ClusterNode node : nodes) {
-            try {
-                igfsCtx.send(node, topic, msg, GridIoPolicy.IGFS_POOL);
-            }
-            catch (IgniteCheckedException e) {
-                U.warn(log, "Failed to send IGFS delete message to node [nodeId=" + node.id() +
-                    ", msg=" + msg + ", err=" + e.getMessage() + ']');
-            }
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
index fb67e20..4c64bc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
@@ -23,7 +23,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
@@ -100,14 +99,6 @@ public interface IgfsEx extends IgniteFileSystem {
     public long groupBlockSize();
 
     /**
-     * Asynchronously await for all entries existing in trash to be removed.
-     *
-     * @return Future which will be completed when all entries existed in trash by the time of invocation are removed.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException;
-
-    /**
      * Gets client file system log directory.
      *
      * @return Client file system log directory or {@code null} in case no client connections have been created yet.

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/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 9087ff0..262dfef 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
@@ -32,10 +32,9 @@ import org.apache.ignite.compute.ComputeJobResultPolicy;
 import org.apache.ignite.compute.ComputeTaskSplitAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
-import org.apache.ignite.events.DiscoveryEvent;
-import org.apache.ignite.events.Event;
 import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.igfs.IgfsBlockLocation;
+import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsInvalidPathException;
 import org.apache.ignite.igfs.IgfsMetrics;
@@ -51,9 +50,7 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
-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;
@@ -69,8 +66,6 @@ import org.apache.ignite.internal.processors.igfs.client.IgfsClientSummaryCallab
 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;
-import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.F;
@@ -100,11 +95,11 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
@@ -114,14 +109,10 @@ import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
 import static org.apache.ignite.events.EventType.EVT_IGFS_META_UPDATED;
-import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
-import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC;
 import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
 import static org.apache.ignite.igfs.IgfsMode.PROXY;
-import static org.apache.ignite.internal.GridTopic.TOPIC_IGFS;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
 
 /**
  * Cache-based IGFS implementation.
@@ -130,6 +121,9 @@ public final class IgfsImpl implements IgfsEx {
     /** Default permissions for file system entry. */
     private static final String PERMISSION_DFLT_VAL = "0777";
 
+    /** Index generator for async format threads. */
+    private static final AtomicInteger FORMAT_THREAD_IDX_GEN = new AtomicInteger();
+
     /** Default directory metadata. */
     static final Map<String, String> DFLT_DIR_META = F.asMap(IgfsUtils.PROP_PERMISSION, PERMISSION_DFLT_VAL);
 
@@ -169,24 +163,12 @@ public final class IgfsImpl implements IgfsEx {
     /** Writers map. */
     private final ConcurrentHashMap8<IgfsPath, IgfsFileWorkerBatch> workerMap = new ConcurrentHashMap8<>();
 
-    /** Delete futures. */
-    private final ConcurrentHashMap8<IgniteUuid, GridFutureAdapter<Object>> delFuts = new ConcurrentHashMap8<>();
-
-    /** Delete message listener. */
-    private final GridMessageListener delMsgLsnr = new FormatMessageListener();
-
-    /** Format discovery listener. */
-    private final GridLocalEventListener delDiscoLsnr = new FormatDiscoveryListener();
-
     /** Local metrics holder. */
     private final IgfsLocalMetrics metrics = new IgfsLocalMetrics();
 
     /** Client log directory. */
     private volatile String logDir;
 
-    /** Message topic. */
-    private Object topic;
-
     /** Eviction policy (if set). */
     private IgfsPerBlockLruEvictionPolicy evictPlc;
 
@@ -292,11 +274,6 @@ public final class IgfsImpl implements IgfsEx {
             }
         }
 
-        topic = F.isEmpty(name()) ? TOPIC_IGFS : TOPIC_IGFS.topic(name());
-
-        igfsCtx.kernalContext().io().addMessageListener(topic, delMsgLsnr);
-        igfsCtx.kernalContext().event().addLocalEventListener(delDiscoLsnr, EVT_NODE_LEFT, EVT_NODE_FAILED);
-
         dualPool = secondaryFs != null ? new IgniteThreadPoolExecutor(4, Integer.MAX_VALUE, 5000L,
             new LinkedBlockingQueue<Runnable>(), new IgfsThreadFactory(cfg.getName()), null) : null;
     }
@@ -332,9 +309,6 @@ public final class IgfsImpl implements IgfsEx {
             }
         }
 
-        igfsCtx.kernalContext().io().removeMessageListener(topic, delMsgLsnr);
-        igfsCtx.kernalContext().event().removeLocalEventListener(delDiscoLsnr);
-
         // Restore interrupted flag.
         if (interrupted)
             Thread.currentThread().interrupt();
@@ -1381,7 +1355,25 @@ public final class IgfsImpl implements IgfsEx {
     /** {@inheritDoc} */
     @Override public void format() {
         try {
-            formatAsync().get();
+            IgniteUuid id = meta.format();
+
+            // If ID is null, then file system is already empty.
+            if (id == null)
+                return;
+
+            while (true) {
+                if (enterBusy()) {
+                    try {
+                        if (!meta.exists(id))
+                            return;
+                    }
+                    finally {
+                        busyLock.leaveBusy();
+                    }
+                }
+
+                U.sleep(10);
+            }
         }
         catch (Exception e) {
             throw IgfsUtils.toIgfsException(e);
@@ -1394,69 +1386,16 @@ public final class IgfsImpl implements IgfsEx {
      * @return Future.
      */
     IgniteInternalFuture<?> formatAsync() {
-        try {
-            IgniteUuid id = meta.format();
-
-            if (id == null)
-                return new GridFinishedFuture<Object>();
-            else {
-                GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
-
-                GridFutureAdapter<Object> oldFut = delFuts.putIfAbsent(id, fut);
-
-                if (oldFut != null)
-                    return oldFut;
-                else {
-                    if (!meta.exists(id)) {
-                        // Safety in case response message was received before we put future into collection.
-                        fut.onDone();
-
-                        delFuts.remove(id, fut);
-                    }
+        GridFutureAdapter<?> fut = new GridFutureAdapter<>();
 
-                    return fut;
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<Object>(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
-        Collection<IgniteUuid> ids = meta.pendingDeletes();
+        Thread t = new Thread(new FormatRunnable(fut), "igfs-format-" + cfg.getName() + "-" +
+            FORMAT_THREAD_IDX_GEN.incrementAndGet());
 
-        if (!ids.isEmpty()) {
-            if (log.isDebugEnabled())
-                log.debug("Constructing delete future for trash entries: " + ids);
+        t.setDaemon(true);
 
-            GridCompoundFuture<Object, Object> resFut = new GridCompoundFuture<>();
+        t.start();
 
-            for (IgniteUuid id : ids) {
-                GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
-
-                IgniteInternalFuture<Object> oldFut = delFuts.putIfAbsent(id, fut);
-
-                if (oldFut != null)
-                    resFut.add(oldFut);
-                else {
-                    if (meta.exists(id))
-                        resFut.add(fut);
-                    else {
-                        fut.onDone();
-
-                        delFuts.remove(id, fut);
-                    }
-                }
-            }
-
-            resFut.markInitialized();
-
-            return resFut;
-        }
-        else
-            return new GridFinishedFuture<>();
+        return fut;
     }
 
     /**
@@ -1482,24 +1421,6 @@ public final class IgfsImpl implements IgfsEx {
         return new FileDescriptor(parentId, path.name(), fileInfo.id(), fileInfo.isFile());
     }
 
-    /**
-     * Check whether IGFS with the same name exists among provided attributes.
-     *
-     * @param attrs Attributes.
-     * @return {@code True} in case IGFS with the same name exists among provided attributes
-     */
-    private boolean sameIgfs(IgfsAttributes[] attrs) {
-        if (attrs != null) {
-            String igfsName = name();
-
-            for (IgfsAttributes attr : attrs) {
-                if (F.eq(igfsName, attr.igfsName()))
-                    return true;
-            }
-        }
-        return false;
-    }
-
     /** {@inheritDoc} */
     @Override public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
         Collection<IgfsPath> paths, @Nullable T arg) {
@@ -1905,81 +1826,6 @@ public final class IgfsImpl implements IgfsEx {
         }
     }
 
-    /**
-     * Format message listener required for format action completion.
-     */
-    private class FormatMessageListener implements GridMessageListener {
-        /** {@inheritDoc} */
-        @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-        @Override public void onMessage(UUID nodeId, Object msg) {
-            if (msg instanceof IgfsDeleteMessage) {
-                ClusterNode node = igfsCtx.kernalContext().discovery().node(nodeId);
-
-                if (node != null) {
-                    if (sameIgfs((IgfsAttributes[]) node.attribute(ATTR_IGFS))) {
-                        IgfsDeleteMessage msg0 = (IgfsDeleteMessage)msg;
-
-                        try {
-                            msg0.finishUnmarshal(igfsCtx.kernalContext().config().getMarshaller(), null);
-                        }
-                        catch (IgniteCheckedException e) {
-                            U.error(log, "Failed to unmarshal message (will ignore): " + msg0, e);
-
-                            return;
-                        }
-
-                        assert msg0.id() != null;
-
-                        GridFutureAdapter<?> fut = delFuts.remove(msg0.id());
-
-                        if (fut != null) {
-                            if (msg0.error() == null)
-                                fut.onDone();
-                            else
-                                fut.onDone(msg0.error());
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Discovery listener required for format actions completion.
-     */
-    private class FormatDiscoveryListener implements GridLocalEventListener {
-        /** {@inheritDoc} */
-        @Override public void onEvent(Event evt) {
-            assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
-
-            DiscoveryEvent evt0 = (DiscoveryEvent)evt;
-
-            if (evt0.eventNode() != null) {
-                if (sameIgfs((IgfsAttributes[]) evt0.eventNode().attribute(ATTR_IGFS))) {
-                    Collection<IgniteUuid> rmv = new HashSet<>();
-
-                    for (Map.Entry<IgniteUuid, GridFutureAdapter<Object>> fut : delFuts.entrySet()) {
-                        IgniteUuid id = fut.getKey();
-
-                        try {
-                            if (!meta.exists(id)) {
-                                fut.getValue().onDone();
-
-                                rmv.add(id);
-                            }
-                        }
-                        catch (IgniteCheckedException e) {
-                            U.error(log, "Failed to check file existence: " + id, e);
-                        }
-                    }
-
-                    for (IgniteUuid id : rmv)
-                        delFuts.remove(id);
-                }
-            }
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public IgniteUuid nextAffinityKey() {
         return safeOp(new Callable<IgniteUuid>() {
@@ -2079,4 +1925,39 @@ public final class IgfsImpl implements IgfsEx {
             return t;
         }
     }
+
+    /**
+     * Format runnable.
+     */
+    private class FormatRunnable implements Runnable {
+        /** Target future. */
+        private final GridFutureAdapter<?> fut;
+
+        /**
+         * Constructor.
+         *
+         * @param fut Future.
+         */
+        public FormatRunnable(GridFutureAdapter<?> fut) {
+            this.fut = fut;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            IgfsException err = null;
+
+            try {
+                format();
+            }
+            catch (Throwable err0) {
+                err = IgfsUtils.toIgfsException(err0);
+            }
+            finally {
+                if (err == null)
+                    fut.onDone();
+                else
+                    fut.onDone(err);
+            }
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/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 6fa9877..cfe549f 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
@@ -180,7 +180,7 @@ public class IgfsUtils {
      * @return Converted IGFS exception.
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public static IgfsException toIgfsException(Exception err) {
+    public static IgfsException toIgfsException(Throwable err) {
         IgfsException err0 = err instanceof IgfsException ? (IgfsException)err : null;
 
         IgfsException igfsErr = X.cause(err, IgfsException.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
index fd4ec17..4e0f12b 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
@@ -239,8 +239,6 @@ public class IgfsFragmentizerSelfTest extends IgfsFragmentizerAbstractSelfTest {
 
         igfs.format();
 
-        igfs.awaitDeletesAsync().get();
-
         GridTestUtils.retryAssert(log, 50, 100, new CA() {
             @Override public void apply() {
                 for (int i = 0; i < NODE_CNT; i++) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
index 3933e86..266945f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
@@ -41,27 +40,21 @@ 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 org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.transactions.Transaction;
 import org.jsr166.ThreadLocalRandom8;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicReference;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  * {@link IgfsAttributes} test case.
@@ -256,41 +249,6 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * Ensure that exception is not thrown in case PARTITIONED cache is oversized, but data is deleted concurrently.
-     *
-     * @throws Exception If failed.
-     */
-    public void testPartitionedOversizeDelay() throws Exception {
-        cacheMode = PARTITIONED;
-        nearEnabled = true;
-
-        checkOversizeDelay();
-    }
-
-    /**
-     * Ensure that exception is not thrown in case co-located cache is oversized, but data is deleted concurrently.
-     *
-     * @throws Exception If failed.
-     */
-    public void testColocatedOversizeDelay() throws Exception {
-        cacheMode = PARTITIONED;
-        nearEnabled = false;
-
-        checkOversizeDelay();
-    }
-
-    /**
-     * Ensure that exception is not thrown in case REPLICATED cache is oversized, but data is deleted concurrently.
-     *
-     * @throws Exception If failed.
-     */
-    public void testReplicatedOversizeDelay() throws Exception {
-        cacheMode = REPLICATED;
-
-        checkOversizeDelay();
-    }
-
-    /**
      * Ensure that IGFS size is correctly updated in case of preloading for PARTITIONED cache.
      *
      * @throws Exception If failed.
@@ -484,97 +442,6 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * Ensure that exception is not thrown or thrown with some delay when there is something in trash directory.
-     *
-     * @throws Exception If failed.
-     */
-    private void checkOversizeDelay() throws Exception {
-        final CountDownLatch latch = new CountDownLatch(1);
-
-        igfsMaxData = 256;
-        trashPurgeTimeout = 2000;
-
-        startUp();
-
-        IgfsImpl igfs = igfs(0);
-
-        final IgfsPath path = new IgfsPath("/file");
-        final IgfsPath otherPath = new IgfsPath("/fileOther");
-
-        // Fill cache with data up to it's limit.
-        IgfsOutputStream os = igfs.create(path, false);
-        os.write(chunk((int)igfsMaxData));
-        os.close();
-
-        final IgniteCache<IgniteUuid, IgfsEntryInfo> metaCache = igfs.context().kernalContext().cache().jcache(
-            igfs.configuration().getMetaCacheName());
-
-        // Start a transaction in a separate thread which will lock file ID.
-        final IgniteUuid id = igfs.context().meta().fileId(path);
-        final IgfsEntryInfo info = igfs.context().meta().info(id);
-
-        final AtomicReference<Throwable> err = new AtomicReference<>();
-
-        try {
-            new Thread(new Runnable() {
-                @Override public void run() {
-                    try {
-
-                        try (Transaction tx = metaCache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                            metaCache.get(id);
-
-                            latch.await();
-
-                            U.sleep(1000); // Sleep here so that data manager could "see" oversize.
-
-                            tx.commit();
-                        }
-                    }
-                    catch (Throwable e) {
-                        err.set(e);
-                    }
-                }
-            }).start();
-
-            // Now add file ID to trash listing so that delete worker could "see" it.
-            IgniteUuid trashId = IgfsUtils.randomTrashId();
-
-            try (Transaction tx = metaCache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                Map<String, IgfsListingEntry> listing = Collections.singletonMap(path.name(),
-                    new IgfsListingEntry(info));
-
-                // Clear root listing.
-                metaCache.put(IgfsUtils.ROOT_ID, IgfsUtils.createDirectory(IgfsUtils.ROOT_ID));
-
-                // Add file to trash listing.
-                IgfsEntryInfo trashInfo = metaCache.get(trashId);
-
-                if (trashInfo == null)
-                    metaCache.put(trashId, IgfsUtils.createDirectory(trashId).listing(listing));
-                else
-                    metaCache.put(trashId, trashInfo.listing(listing));
-
-                tx.commit();
-            }
-
-            assert metaCache.get(trashId) != null;
-
-            // Now the file is locked and is located in trash, try adding some more data.
-            os = igfs.create(otherPath, false);
-            os.write(new byte[1]);
-
-            latch.countDown();
-
-            os.close();
-
-            assert err.get() == null;
-        }
-        finally {
-            latch.countDown(); // Safety.
-        }
-    }
-
-    /**
      * Ensure that IGFS size is correctly updated in case of preloading.
      *
      * @throws Exception If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/065d2e70/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
index b38f3a2..ffa6f7d 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
@@ -41,7 +41,6 @@ import org.apache.ignite.igfs.mapreduce.IgfsTask;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
@@ -754,11 +753,6 @@ public class HadoopDefaultMapReducePlannerSelfTest extends HadoopAbstractSelfTes
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
-            return null;
-        }
-
-        /** {@inheritDoc} */
         @Nullable @Override public String clientLogDirectory() {
             return null;
         }


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

Posted by sb...@apache.org.
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/b96afb2f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b96afb2f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b96afb2f

Branch: refs/heads/ignite-3038
Commit: b96afb2fac5df060d340b7b92627186f3c82532b
Parents: dcd722a
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sun Jun 5 21:00:53 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sun Jun 5 21:00:53 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   |  49 ++++++-
 .../igfs/IgfsClientCacheSelfTest.java           | 139 -------------------
 .../igfs/IgfsDualAsyncClientSelfTest.java       |  28 ++++
 .../igfs/IgfsDualSyncClientSelfTest.java        |  28 ++++
 .../processors/igfs/IgfsOneClientNodeTest.java  |   5 +-
 .../igfs/IgfsPrimaryClientSelfTest.java         |  30 ++++
 .../igfs/IgfsPrimaryRelaxedClientSelfTest.java  |  28 ++++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  11 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   2 +-
 30 files changed, 1336 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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 2796839..b7b3fac 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/b96afb2f/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 35e77c4..3295249 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
@@ -247,7 +247,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/b96afb2f/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 e45e34e..6fa9877 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
@@ -684,6 +684,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;
+    }
+
+    /**
      * Check whether provided node contains IGFS with the given name.
      *
      * @param node Node.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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 3717d31..c898909 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;
@@ -4710,6 +4712,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/b96afb2f/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 2f6c7bf..3fb7b91 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;
@@ -55,6 +56,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 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 org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
@@ -189,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);
+
     static {
         PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration();
 
@@ -233,6 +241,13 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * @return Client flag.
+     */
+    protected boolean client() {
+        return false;
+    }
+
+    /**
      * @return Use optimzied marshaller flag.
      */
     protected boolean useOptimizedMarshaller() {
@@ -258,9 +273,29 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         igfsSecondaryFileSystem = createSecondaryFileSystemStack();
 
-        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG);
+        if (client()) {
+            // Start server.
+            startGridWithIgfs("ignite_srv", "igfs", mode, igfsSecondaryFileSystem, null, primaryIpFinder);
 
-        igfs = (IgfsImpl) ignite.fileSystem("igfs");
+            // Start client.
+            Ignition.setClientMode(true);
+
+            try {
+                Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem,
+                    PRIMARY_REST_CFG, primaryIpFinder);
+
+                igfs = (IgfsImpl) ignite.fileSystem("igfs");
+            }
+            finally {
+                Ignition.setClientMode(false);
+            }
+        }
+        else {
+            Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem,
+                PRIMARY_REST_CFG, primaryIpFinder);
+
+            igfs = (IgfsImpl) ignite.fileSystem("igfs");
+        }
     }
 
     /**
@@ -271,7 +306,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      */
     protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
         Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
-            SECONDARY_REST_CFG);
+            SECONDARY_REST_CFG, secondaryIpFinder);
 
         IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary");
 
@@ -300,12 +335,14 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param mode IGFS mode.
      * @param secondaryFs Secondary file system (optional).
      * @param restCfg Rest configuration string (optional).
+     * @param ipFinder IP finder.
      * @return Started grid instance.
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
     protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
-        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg) throws Exception {
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg,
+        TcpDiscoveryIpFinder ipFinder) throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
@@ -322,6 +359,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);
@@ -334,6 +372,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);
@@ -347,7 +386,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+        discoSpi.setIpFinder(ipFinder);
 
         prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/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/b96afb2f/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.java
new file mode 100644
index 0000000..64bce8e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedClientSelfTest.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 IgfsPrimaryRelaxedClientSelfTest extends IgfsPrimaryRelaxedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file


[21/24] ignite git commit: Merge remote-tracking branch 'upstream/gridgain-7.5.25' into gridgain-7.5.25

Posted by sb...@apache.org.
Merge remote-tracking branch 'upstream/gridgain-7.5.25' into gridgain-7.5.25


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

Branch: refs/heads/ignite-3038
Commit: ec887a329c45ea9134747c9f1a2ebe9f78a61b8e
Parents: ccbb059 e4337d2
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jun 8 11:50:41 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jun 8 11:50:41 2016 +0300

----------------------------------------------------------------------
 .../GridClientConnectionManagerAdapter.java     |  25 ++-
 .../connection/GridClientNioTcpConnection.java  |   3 +
 .../GridClientOptimizedMarshaller.java          |   4 +-
 .../GridClientZipOptimizedMarshaller.java       | 162 +++++++++++++++++++
 .../impl/GridTcpRouterNioListenerAdapter.java   |  11 +-
 .../message/GridClientHandshakeRequest.java     |   4 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |  19 ++-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  12 +-
 .../ignite/internal/util/nio/GridNioServer.java |  10 +-
 .../ignite/internal/visor/cache/VisorCache.java |  56 +------
 .../visor/cache/VisorCachePartition.java        |  89 ++++++++++
 .../visor/cache/VisorCachePartitions.java       |  88 ++++++++++
 .../visor/cache/VisorCachePartitionsTask.java   | 152 +++++++++++++++++
 .../internal/visor/cache/VisorCacheV3.java      |  68 +-------
 14 files changed, 564 insertions(+), 139 deletions(-)
----------------------------------------------------------------------



[03/24] ignite git commit: IGNITE-3232 - Inline transformers for IgniteRDD.savePairs and IgniteRDD.saveValues

Posted by sb...@apache.org.
IGNITE-3232 - Inline transformers for IgniteRDD.savePairs and IgniteRDD.saveValues


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

Branch: refs/heads/ignite-3038
Commit: 0e4ef3b2e842015ef27aa107a5ecdcc10a73a327
Parents: 50a8ec2
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Jun 2 16:03:12 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Jun 3 19:47:00 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spark/IgniteRDD.scala     | 75 +++++++++++++++++++
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |  9 +++
 .../org/apache/ignite/spark/IgniteRDDSpec.scala | 77 +++++++++++++++++++-
 3 files changed, 160 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e4ef3b2/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 0d8e730..036dfe6 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -180,6 +180,39 @@ class IgniteRDD[K, V] (
     }
 
     /**
+     * Saves values from given RDD into Ignite. A unique key will be generated for each value of the given RDD.
+     *
+     * @param rdd RDD instance to save values from.
+     * @param f Transformation function.
+     */
+    def saveValues[T](rdd: RDD[T], f: (T, IgniteContext[K, V]) \u21d2 V) = {
+        rdd.foreachPartition(it \u21d2 {
+            val ig = ic.ignite()
+
+            ensureCache()
+
+            val locNode = ig.cluster().localNode()
+
+            val node: Option[ClusterNode] = ig.cluster().forHost(locNode).nodes().find(!_.eq(locNode))
+
+            val streamer = ig.dataStreamer[Object, V](cacheName)
+
+            try {
+                it.foreach(t \u21d2 {
+                    val value = f(t, ic)
+
+                    val key = affinityKeyFunc(value, node.orNull)
+
+                    streamer.addData(key, value)
+                })
+            }
+            finally {
+                streamer.close()
+            }
+        })
+    }
+
+    /**
      * Saves values from the given key-value RDD into Ignite.
      *
      * @param rdd RDD instance to save values from.
@@ -209,6 +242,48 @@ class IgniteRDD[K, V] (
     }
 
     /**
+     * Saves values from the given RDD into Ignite.
+     *
+     * @param rdd RDD instance to save values from.
+     * @param f Transformation function.
+     * @param overwrite Boolean flag indicating whether the call on this method should overwrite existing
+     *      values in Ignite cache.
+     */
+    def savePairs[T](rdd: RDD[T], f: (T, IgniteContext[K, V]) \u21d2 (K, V), overwrite: Boolean) = {
+        rdd.foreachPartition(it \u21d2 {
+            val ig = ic.ignite()
+
+            // Make sure to deploy the cache
+            ensureCache()
+
+            val streamer = ig.dataStreamer[K, V](cacheName)
+
+            try {
+                streamer.allowOverwrite(overwrite)
+
+                it.foreach(t \u21d2 {
+                    val tup = f(t, ic)
+
+                    streamer.addData(tup._1, tup._2)
+                })
+            }
+            finally {
+                streamer.close()
+            }
+        })
+    }
+
+    /**
+     * Saves values from the given RDD into Ignite.
+     *
+     * @param rdd RDD instance to save values from.
+     * @param f Transformation function.
+     */
+    def savePairs[T](rdd: RDD[T], f: (T, IgniteContext[K, V]) \u21d2 (K, V)): Unit = {
+        savePairs(rdd, f, overwrite = false)
+    }
+
+    /**
      * Removes all values from the underlying Ignite cache.
      */
     def clear(): Unit = {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e4ef3b2/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
index 2e8702e..cac0e15 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
@@ -80,12 +80,21 @@ class JavaIgniteRDD[K, V](override val rdd: IgniteRDD[K, V])
 
     def saveValues(jrdd: JavaRDD[V]) = rdd.saveValues(JavaRDD.toRDD(jrdd))
 
+    def saveValues[T](jrdd: JavaRDD[T], f: (T, IgniteContext[K, V]) \u21d2 V) = rdd.saveValues(JavaRDD.toRDD(jrdd), f)
+
     def savePairs(jrdd: JavaPairRDD[K, V]) = {
         val rrdd: RDD[(K, V)] = JavaPairRDD.toRDD(jrdd)
 
         rdd.savePairs(rrdd)
     }
 
+    def savePairs[T](jrdd: JavaRDD[T], f: (T, IgniteContext[K, V]) \u21d2 (K, V), overwrite: Boolean = false) = {
+        rdd.savePairs(JavaRDD.toRDD(jrdd), f, overwrite)
+    }
+
+    def savePairs[T](jrdd: JavaRDD[T], f: (T, IgniteContext[K, V]) \u21d2 (K, V)): Unit =
+        savePairs(jrdd, f, overwrite = false)
+
     def clear(): Unit = rdd.clear()
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e4ef3b2/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
index 4e90bc8..61040d9 100644
--- a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
+++ b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
@@ -26,6 +26,7 @@ import org.apache.spark.SparkContext
 import org.junit.runner.RunWith
 import org.scalatest._
 import org.scalatest.junit.JUnitRunner
+import scala.collection.JavaConversions._
 
 import IgniteRDDSpec._
 
@@ -34,7 +35,7 @@ import scala.annotation.meta.field
 @RunWith(classOf[JUnitRunner])
 class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll with BeforeAndAfterEach {
     describe("IgniteRDD") {
-        it("should successfully store data to ignite") {
+        it("should successfully store data to ignite using savePairs") {
             val sc = new SparkContext("local[*]", "test")
 
             try {
@@ -59,6 +60,80 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
             }
         }
 
+        it("should successfully store data to ignite using savePairs with inline transformation") {
+            val sc = new SparkContext("local[*]", "test")
+
+            try {
+                val ic = new IgniteContext[String, String](sc,
+                    () \u21d2 configuration("client", client = true))
+
+                // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
+                ic.fromCache(PARTITIONED_CACHE_NAME).savePairs(
+                    sc.parallelize(0 to 10000, 2), (i: Int, ic) \u21d2 (String.valueOf(i), "val" + i))
+
+                // Check cache contents.
+                val ignite = Ignition.ignite("grid-0")
+
+                for (i \u2190 0 to 10000) {
+                    val res = ignite.cache[String, String](PARTITIONED_CACHE_NAME).get(String.valueOf(i))
+
+                    assert(res != null, "Value was not put to cache for key: " + i)
+                    assert("val" + i == res, "Invalid value stored for key: " + i)
+                }
+            }
+            finally {
+                sc.stop()
+            }
+        }
+
+        it("should successfully store data to ignite using saveValues") {
+            val sc = new SparkContext("local[*]", "test")
+
+            try {
+                val ic = new IgniteContext[String, String](sc,
+                    () \u21d2 configuration("client", client = true))
+
+                // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
+                ic.fromCache(PARTITIONED_CACHE_NAME).saveValues(
+                    sc.parallelize(0 to 10000, 2).map(i \u21d2 "val" + i))
+
+                // Check cache contents.
+                val ignite = Ignition.ignite("grid-0")
+
+                val values = ignite.cache[String, String](PARTITIONED_CACHE_NAME).toList.map(e \u21d2 e.getValue)
+
+                for (i \u2190 0 to 10000)
+                    assert(values.contains("val" + i), "Value not found for index: " + i)
+            }
+            finally {
+                sc.stop()
+            }
+        }
+
+        it("should successfully store data to ignite using saveValues with inline transformation") {
+            val sc = new SparkContext("local[*]", "test")
+
+            try {
+                val ic = new IgniteContext[String, String](sc,
+                    () \u21d2 configuration("client", client = true))
+
+                // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
+                ic.fromCache(PARTITIONED_CACHE_NAME).saveValues(
+                    sc.parallelize(0 to 10000, 2), (i: Int, ic) \u21d2 "val" + i)
+
+                // Check cache contents.
+                val ignite = Ignition.ignite("grid-0")
+
+                val values = ignite.cache[String, String](PARTITIONED_CACHE_NAME).toList.map(e \u21d2 e.getValue)
+
+                for (i \u2190 0 to 10000)
+                    assert(values.contains("val" + i), "Value not found for index: " + i)
+            }
+            finally {
+                sc.stop()
+            }
+        }
+
         it("should successfully read data from ignite") {
             val sc = new SparkContext("local[*]", "test")
 


[06/24] ignite git commit: IGFS-3246: IGFS: Infrastructure for client optimizations.

Posted by sb...@apache.org.
IGFS-3246: IGFS: Infrastructure for client optimizations.


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

Branch: refs/heads/ignite-3038
Commit: dcd722a3dc55d656e3517764c3b162d63f0d1ba0
Parents: 4506c21
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sun Jun 5 20:57:15 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sun Jun 5 20:57:15 2016 +0300

----------------------------------------------------------------------
 .../internal/cluster/ClusterGroupAdapter.java   |   6 +
 .../ignite/internal/cluster/ClusterGroupEx.java |  14 ++-
 .../internal/processors/igfs/IgfsContext.java   |  14 +--
 .../internal/processors/igfs/IgfsImpl.java      |   2 -
 .../processors/igfs/IgfsMetaManager.java        |  62 +++++++++-
 .../processors/igfs/IgfsNodePredicate.java      |  80 +++++++++++++
 .../internal/processors/igfs/IgfsProcessor.java |  37 ++++--
 .../internal/processors/igfs/IgfsUtils.java     |  21 ++++
 .../igfs/client/IgfsClientAbstractCallable.java | 112 +++++++++++++++++++
 .../multijvm/IgniteClusterProcessProxy.java     |   5 +
 10 files changed, 325 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
index 75168a1..c664f1e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
@@ -51,6 +51,7 @@ import org.apache.ignite.internal.IgniteServicesImpl;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.executor.GridExecutorService;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.processors.igfs.IgfsNodePredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -602,6 +603,11 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public ClusterGroup forIgfsMetadataDataNodes(@Nullable String igfsName, @Nullable String metaCacheName) {
+        return forPredicate(new IgfsNodePredicate(igfsName)).forDataNodes(metaCacheName);
+    }
+
+    /** {@inheritDoc} */
     @Override public final ClusterGroup forHost(ClusterNode node) {
         A.notNull(node, "node");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
index 59da7cf..21533a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupEx.java
@@ -29,7 +29,7 @@ public interface ClusterGroupEx extends ClusterGroup {
      * Creates projection for specified subject ID.
      *
      * @param subjId Subject ID.
-     * @return Internal projection.
+     * @return Cluster group.
      */
     public ClusterGroupEx forSubjectId(UUID subjId);
 
@@ -40,5 +40,15 @@ public interface ClusterGroupEx extends ClusterGroup {
      * @param clientNodes Flag to include client nodes.
      * @return Cluster group.
      */
-    public ClusterGroup forCacheNodes(@Nullable String cacheName, boolean affNodes, boolean nearNodes, boolean clientNodes);
+    public ClusterGroup forCacheNodes(@Nullable String cacheName, boolean affNodes, boolean nearNodes,
+        boolean clientNodes);
+
+    /**
+     * Create projection for IGFS server nodes.
+     *
+     * @param igfsName IGFS name.
+     * @param metaCacheName Metadata cache name.
+     * @return Cluster group.
+     */
+    public ClusterGroup forIgfsMetadataDataNodes(@Nullable String igfsName, @Nullable String metaCacheName);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
index 2b08f28..a638bf3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
@@ -24,11 +24,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.util.typedef.F;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
-
 /**
  * IGFS context holding all required components for IGFS instance.
  */
@@ -178,16 +175,7 @@ public class IgfsContext {
      * @return {@code True} if node has IGFS with this name, {@code false} otherwise.
      */
     public boolean igfsNode(ClusterNode node) {
-        assert node != null;
-
-        IgfsAttributes[] igfs = node.attribute(ATTR_IGFS);
-
-        if (igfs != null)
-            for (IgfsAttributes attrs : igfs)
-                if (F.eq(cfg.getName(), attrs.igfsName()))
-                    return true;
-
-        return false;
+        return IgfsUtils.isIgfsNode(node, cfg.getName());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/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 967d962..2796839 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
@@ -96,13 +96,11 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_RENAMED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_READ;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_WRITE;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_RENAMED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_META_UPDATED;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/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 a4212ba..35e77c4 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
@@ -18,10 +18,13 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.events.IgfsEvent;
@@ -37,6 +40,7 @@ import org.apache.ignite.igfs.IgfsPathIsNotDirectoryException;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
@@ -44,6 +48,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileLockProcessor;
@@ -139,13 +144,21 @@ public class IgfsMetaManager extends IgfsManager {
     /** Relaxed flag. */
     private final boolean relaxed;
 
+    /** Client flag. */
+    private final boolean client;
+
+    /** Compute facade for client tasks. */
+    private IgniteCompute cliCompute;
+
     /**
      * Constructor.
      *
      * @param relaxed Relaxed mode flag.
+     * @param client Client flag.
      */
-    public IgfsMetaManager(boolean relaxed) {
+    public IgfsMetaManager(boolean relaxed, boolean client) {
         this.relaxed = relaxed;
+        this.client = client;
     }
 
     /**
@@ -217,6 +230,53 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * @return Client flag.
+     */
+    boolean isClient() {
+        return client;
+    }
+
+    /**
+     * Run client task.
+     *
+     * @param task Task.
+     * @return Result.
+     */
+    <T> T runClientTask(IgfsClientAbstractCallable<T> task) {
+        try {
+            return clientCompute().call(task);
+        }
+        catch (ClusterTopologyException e) {
+            throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes left." , e);
+        }
+    }
+
+    /**
+     * Get compute facade for client tasks.
+     *
+     * @return Compute facade.
+     */
+    private IgniteCompute clientCompute() {
+        assert client;
+
+        IgniteCompute cliCompute0 = cliCompute;
+
+        if (cliCompute0 == null) {
+            IgniteEx ignite = igfsCtx.kernalContext().grid();
+
+            ClusterGroup cluster = ignite.cluster().forIgfsMetadataDataNodes(cfg.getName(), cfg.getMetaCacheName());
+
+            cliCompute0 = ignite.compute(cluster);
+
+            cliCompute = cliCompute0;
+        }
+
+        assert cliCompute0 != null;
+
+        return cliCompute0;
+    }
+
+    /**
      * Return nodes where meta cache is defined.
      *
      * @return Nodes where meta cache is defined.

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.java
new file mode 100644
index 0000000..e3a658a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNodePredicate.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;
+
+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.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * IGFS node predicate.
+ */
+public class IgfsNodePredicate implements IgnitePredicate<ClusterNode>, Binarylizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** IGFS name. */
+    private String igfsName;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsNodePredicate() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     */
+    public IgfsNodePredicate(@Nullable String igfsName) {
+        this.igfsName = igfsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(ClusterNode node) {
+        return IgfsUtils.isIgfsNode(node, igfsName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeString(igfsName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        igfsName = rawReader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsNodePredicate.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
index 778de99..92b43cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
@@ -95,10 +95,12 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
+        IgniteConfiguration igniteCfg = ctx.config();
+
+        if (igniteCfg.isDaemon())
             return;
 
-        FileSystemConfiguration[] cfgs = ctx.config().getFileSystemConfiguration();
+        FileSystemConfiguration[] cfgs = igniteCfg.getFileSystemConfiguration();
 
         assert cfgs != null && cfgs.length > 0;
 
@@ -108,10 +110,27 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
         for (FileSystemConfiguration cfg : cfgs) {
             FileSystemConfiguration cfg0 = new FileSystemConfiguration(cfg);
 
+            boolean metaClient = true;
+
+            CacheConfiguration[] cacheCfgs = igniteCfg.getCacheConfiguration();
+
+            if (cacheCfgs != null) {
+                for (CacheConfiguration cacheCfg : cacheCfgs) {
+                    if (F.eq(cacheCfg.getName(), cfg.getMetaCacheName())) {
+                        metaClient = false;
+
+                        break;
+                    }
+                }
+            }
+
+            if (igniteCfg.isClientMode() != null && igniteCfg.isClientMode())
+                metaClient = true;
+
             IgfsContext igfsCtx = new IgfsContext(
                 ctx,
                 cfg0,
-                new IgfsMetaManager(cfg0.isRelaxedConsistency()),
+                new IgfsMetaManager(cfg0.isRelaxedConsistency(), metaClient),
                 new IgfsDataManager(),
                 new IgfsServerManager(),
                 new IgfsFragmentizerManager());
@@ -126,19 +145,17 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("IGFS processor started.");
 
-        IgniteConfiguration gridCfg = ctx.config();
-
         // Node doesn't have IGFS if it:
         // is daemon;
         // doesn't have configured IGFS;
         // doesn't have configured caches.
-        if (gridCfg.isDaemon() || F.isEmpty(gridCfg.getFileSystemConfiguration()) ||
-            F.isEmpty(gridCfg.getCacheConfiguration()))
+        if (igniteCfg.isDaemon() || F.isEmpty(igniteCfg.getFileSystemConfiguration()) ||
+            F.isEmpty(igniteCfg.getCacheConfiguration()))
             return;
 
         final Map<String, CacheConfiguration> cacheCfgs = new HashMap<>();
 
-        F.forEach(gridCfg.getCacheConfiguration(), new CI1<CacheConfiguration>() {
+        F.forEach(igniteCfg.getCacheConfiguration(), new CI1<CacheConfiguration>() {
             @Override public void apply(CacheConfiguration c) {
                 cacheCfgs.put(c.getName(), c);
             }
@@ -146,9 +163,9 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
         Collection<IgfsAttributes> attrVals = new ArrayList<>();
 
-        assert gridCfg.getFileSystemConfiguration() != null;
+        assert igniteCfg.getFileSystemConfiguration() != null;
 
-        for (FileSystemConfiguration igfsCfg : gridCfg.getFileSystemConfiguration()) {
+        for (FileSystemConfiguration igfsCfg : igniteCfg.getFileSystemConfiguration()) {
             CacheConfiguration cacheCfg = cacheCfgs.get(igfsCfg.getDataCacheName());
 
             if (cacheCfg == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/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 1b97565..e45e34e 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
@@ -56,6 +56,7 @@ import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
@@ -681,4 +682,24 @@ public class IgfsUtils {
         else
             return null;
     }
+
+    /**
+     * Check whether provided node contains IGFS with the given name.
+     *
+     * @param node Node.
+     * @param igfsName IGFS name.
+     * @return {@code True} if it contains IGFS.
+     */
+    public static boolean isIgfsNode(ClusterNode node, String igfsName) {
+        assert node != null;
+
+        IgfsAttributes[] igfs = node.attribute(ATTR_IGFS);
+
+        if (igfs != null)
+            for (IgfsAttributes attrs : igfs)
+                if (F.eq(igfsName, attrs.igfsName()))
+                    return true;
+
+        return false;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/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
new file mode 100644
index 0000000..b83ed13
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
@@ -0,0 +1,112 @@
+/*
+ * 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.Ignite;
+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.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsEx;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract callable for IGFS tasks initiated on client node and passed to data node.
+ */
+public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>, Binarylizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** IGFS name. */
+    protected String igfsName;
+
+    /** Injected instance. */
+    @IgniteInstanceResource
+    private transient Ignite ignite;
+
+    /**
+     * Default constructor.
+     */
+    protected IgfsClientAbstractCallable() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     */
+    protected IgfsClientAbstractCallable(@Nullable String igfsName) {
+        this.igfsName = igfsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final T call() throws Exception {
+        assert ignite != null;
+
+        IgfsEx igfs = (IgfsEx)ignite.fileSystem(igfsName);
+
+        return call0(igfs.context());
+    }
+
+    /**
+     * Execute task.
+     *
+     * @param ctx IGFS ocntext.
+     * @return Result.
+     * @throws Exception If failed.
+     */
+    protected abstract T call0(IgfsContext ctx) throws Exception;
+
+    /** {@inheritDoc} */
+    @Override public final void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeString(igfsName);
+
+        writeBinary0(rawWriter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public final void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader rawReader = reader.rawReader();
+
+        igfsName = rawReader.readString();
+
+        readBinary0(rawReader);
+    }
+
+    /**
+     * Write binary.
+     *
+     * @param rawWriter Raw writer.
+     */
+    protected abstract void writeBinary0(BinaryRawWriter rawWriter);
+
+    /**
+     * Read binary.
+     *
+     * @param rawReader Raw reader.
+     */
+    protected abstract void readBinary0(BinaryRawReader rawReader);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dcd722a3/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
index 633e9d0..76a88d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
@@ -69,6 +69,11 @@ public class IgniteClusterProcessProxy implements IgniteClusterEx {
     }
 
     /** {@inheritDoc} */
+    @Override public ClusterGroup forIgfsMetadataDataNodes(@Nullable String igfsName, @Nullable String metaCacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public ClusterNode localNode() {
         return compute.call(new LocalNodeTask());
     }


[20/24] ignite git commit: IGNITE-3274: Hadoop: Fixed NPE in BasicUserNameMapper.

Posted by sb...@apache.org.
IGNITE-3274: Hadoop: Fixed NPE in BasicUserNameMapper.


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

Branch: refs/heads/ignite-3038
Commit: ccbb059a1d89ec631a4c1c85bd0d242ba21d2579
Parents: 065d2e7
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jun 8 11:50:22 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jun 8 11:50:22 2016 +0300

----------------------------------------------------------------------
 .../ignite/hadoop/util/BasicUserNameMapper.java  |  4 +---
 .../hadoop/util/BasicUserNameMapperSelfTest.java | 19 ++++++++++++++++++-
 2 files changed, 19 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ccbb059a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/BasicUserNameMapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/BasicUserNameMapper.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/BasicUserNameMapper.java
index aea7196..c34808a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/BasicUserNameMapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/BasicUserNameMapper.java
@@ -41,9 +41,7 @@ public class BasicUserNameMapper implements UserNameMapper {
 
     /** {@inheritDoc} */
     @Nullable @Override public String map(String name) {
-        assert mappings != null;
-
-        String res = mappings.get(name);
+        String res = mappings != null ? mappings.get(name) : null;
 
         return res != null ? res : useDfltUsrName ? dfltUsrName : name;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ccbb059a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java
index 54b03f9..fd8fdef 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java
@@ -28,13 +28,30 @@ import java.util.Map;
  */
 public class BasicUserNameMapperSelfTest extends GridCommonAbstractTest {
     /**
+     * Test null mappings.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNullMappings() throws Exception {
+        checkNullOrEmptyMappings(null);
+    }
+
+    /**
      * Test empty mappings.
      *
      * @throws Exception If failed.
      */
     public void testEmptyMappings() throws Exception {
-        Map<String, String> map = new HashMap<>();
+        checkNullOrEmptyMappings(new HashMap<String, String>());
+    }
 
+    /**
+     * Check null or empty mappings.
+     *
+     * @param map Mappings.
+     * @throws Exception If failed.
+     */
+    private void checkNullOrEmptyMappings(@Nullable Map<String, String> map) throws Exception {
         BasicUserNameMapper mapper = create(map, false, null);
 
         assertNull(mapper.map(null));


[11/24] ignite git commit: IGNITE-3240: fixed serialization of String and other primitive arrays by CacheObjectBinaryProcessorImpl

Posted by sb...@apache.org.
IGNITE-3240: fixed serialization of String and other primitive arrays by CacheObjectBinaryProcessorImpl


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

Branch: refs/heads/ignite-3038
Commit: 9e9252c2e4ed7380b3933909ac297f6618708980
Parents: e409b67
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Jun 6 10:20:13 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Jun 6 10:20:13 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryUtils.java     | 16 ++++
 .../processors/cache/CacheObjectContext.java    |  3 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java | 78 +++++++++++++++++++-
 3 files changed, 95 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9e9252c2/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index e431474..0250e3b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -655,6 +655,22 @@ public class BinaryUtils {
     }
 
     /**
+     * @param arr Array to check.
+     * @return {@code true} if this array is of a known type.
+     */
+    public static boolean knownArray(Object arr) {
+        if (arr == null)
+            return false;
+
+        Class<?> cls =  arr.getClass();
+
+        return cls == byte[].class || cls == short[].class || cls == int[].class || cls == long[].class ||
+            cls == float[].class || cls == double[].class || cls == char[].class || cls == boolean[].class ||
+            cls == String[].class || cls == UUID[].class || cls == Date[].class || cls == Timestamp[].class ||
+            cls == BigDecimal[].class;
+    }
+
+    /**
      * Attempts to create a new collection of the same known type. Will return null if collection type is unknown.
      *
      * @param col Collection.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e9252c2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
index d22bc75..d7fdb83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
@@ -199,6 +199,9 @@ import org.apache.ignite.internal.util.typedef.F;
      * @return Result.
      */
     public Object[] unwrapBinariesInArrayIfNeeded(Object[] arr, boolean keepBinary, boolean cpy) {
+        if (BinaryUtils.knownArray(arr))
+            return arr;
+
         Object[] res = new Object[arr.length];
 
         for (int i = 0; i < arr.length; i++)

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e9252c2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index f776146..56cca39 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -17,13 +17,17 @@
 
 package org.apache.ignite.internal.processors.cache.binary;
 
+import java.math.BigDecimal;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
+import java.util.UUID;
 import javax.cache.Cache;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
@@ -60,6 +64,7 @@ import org.apache.ignite.lang.IgniteBiInClosure;
 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 org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -422,6 +427,75 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     /**
      * @throws Exception If failed.
      */
+    public void testBasicArrays() throws Exception {
+        IgniteCache<Integer, Object> cache = jcache(0);
+
+        checkArrayClass(cache, new String[] {"abc"});
+
+        checkArrayClass(cache, new byte[] {1});
+
+        checkArrayClass(cache, new short[] {1});
+
+        checkArrayClass(cache, new int[] {1});
+
+        checkArrayClass(cache, new long[] {1});
+
+        checkArrayClass(cache, new float[] {1});
+
+        checkArrayClass(cache, new double[] {1});
+
+        checkArrayClass(cache, new char[] {'a'});
+
+        checkArrayClass(cache, new boolean[] {false});
+
+        checkArrayClass(cache, new UUID[] {UUID.randomUUID()});
+
+        checkArrayClass(cache, new Date[] {new Date()});
+
+        checkArrayClass(cache, new Timestamp[] {new Timestamp(System.currentTimeMillis())});
+
+        checkArrayClass(cache, new BigDecimal[] {new BigDecimal(100)});
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomArrays() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-3244");
+
+        IgniteCache<Integer, TestObject[]> cache = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            TestObject[] arr = new TestObject[] {new TestObject(i)};
+
+            cache.put(0, arr);
+        }
+
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            TestObject[] obj = cache.get(i);
+
+            assertEquals(1, obj.length);
+            assertEquals(i, obj[0].val);
+        }
+    }
+
+    /**
+     * @param cache Ignite cache.
+     * @param arr Array to check.
+     */
+    private void checkArrayClass(IgniteCache<Integer, Object> cache, Object arr) {
+        cache.put(0, arr);
+
+        Object res = cache.get(0);
+
+        assertEquals(arr.getClass(), res.getClass());
+        GridTestUtils.deepEquals(arr, res);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetTx1() throws Exception {
         checkGetTx(PESSIMISTIC, REPEATABLE_READ);
     }
@@ -651,7 +725,6 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
             for (int j = 0; j < 10; j++)
                 keys.add(i++);
 
-
             cacheBinaryAsync.getAll(keys);
 
             Map<Integer, BinaryObject> objs = cacheBinaryAsync.<Map<Integer, BinaryObject>>future().get();
@@ -1045,7 +1118,8 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
      * No-op entry processor.
      */
     private static class ObjectEntryProcessor implements EntryProcessor<Integer, TestObject, Boolean> {
-        @Override public Boolean process(MutableEntry<Integer, TestObject> entry, Object... args) throws EntryProcessorException {
+        @Override
+        public Boolean process(MutableEntry<Integer, TestObject> entry, Object... args) throws EntryProcessorException {
             TestObject obj = entry.getValue();
 
             entry.setValue(new TestObject(obj.val));


[22/24] ignite git commit: IGNITE-3276: IGFS: Reworked output stream.

Posted by sb...@apache.org.
IGNITE-3276: IGFS: Reworked output stream.


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

Branch: refs/heads/ignite-3038
Commit: ec5706fff899b71c8f3a15a1476744004f876c6e
Parents: ec887a3
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jun 8 14:41:49 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jun 8 14:41:49 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  |  34 ++
 .../processors/igfs/IgfsDataManager.java        |  28 +-
 .../internal/processors/igfs/IgfsImpl.java      |  45 +-
 .../processors/igfs/IgfsMetaManager.java        | 442 ++++++++-------
 .../igfs/IgfsOutputStreamAdapter.java           | 265 ---------
 .../processors/igfs/IgfsOutputStreamImpl.java   | 558 +++++++++++--------
 .../igfs/meta/IgfsMetaFileUnlockProcessor.java  |  69 ++-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  12 +-
 .../igfs/IgfsDataManagerSelfTest.java           |  12 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |  38 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |   2 +-
 11 files changed, 733 insertions(+), 772 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
index 625ba95..074636a 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
@@ -20,6 +20,7 @@ package org.apache.ignite.configuration;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
+import org.apache.ignite.igfs.IgfsOutputStream;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -91,6 +92,9 @@ public class FileSystemConfiguration {
     /** Default value of relaxed consistency flag. */
     public static final boolean DFLT_RELAXED_CONSISTENCY = true;
 
+    /** Default value of update file length on flush flag. */
+    public static final boolean DFLT_UPDATE_FILE_LEN_ON_FLUSH = false;
+
     /** IGFS instance name. */
     private String name;
 
@@ -178,6 +182,9 @@ public class FileSystemConfiguration {
     /** Relaxed consistency flag. */
     private boolean relaxedConsistency = DFLT_RELAXED_CONSISTENCY;
 
+    /** Update file length on flush flag. */
+    private boolean updateFileLenOnFlush = DFLT_UPDATE_FILE_LEN_ON_FLUSH;
+
     /**
      * Constructs default configuration.
      */
@@ -225,6 +232,7 @@ public class FileSystemConfiguration {
         relaxedConsistency = cfg.isRelaxedConsistency();
         seqReadsBeforePrefetch = cfg.getSequentialReadsBeforePrefetch();
         trashPurgeTimeout = cfg.getTrashPurgeTimeout();
+        updateFileLenOnFlush = cfg.isUpdateFileLengthOnFlush();
     }
 
     /**
@@ -922,6 +930,32 @@ public class FileSystemConfiguration {
         this.relaxedConsistency = relaxedConsistency;
     }
 
+    /**
+     * Get whether to update file length on flush.
+     * <p>
+     * Controls whether to update file length or not when {@link IgfsOutputStream#flush()} method is invoked. You
+     * may want to set this property to true in case you want to read from a file which is being written at the
+     * same time.
+     * <p>
+     * Defaults to {@link #DFLT_UPDATE_FILE_LEN_ON_FLUSH}.
+     *
+     * @return Whether to update file length on flush.
+     */
+    public boolean isUpdateFileLengthOnFlush() {
+        return updateFileLenOnFlush;
+    }
+
+    /**
+     * Set whether to update file length on flush.
+     * <p>
+     * Set {@link #isUpdateFileLengthOnFlush()} for more information.
+     *
+     * @param updateFileLenOnFlush Whether to update file length on flush.
+     */
+    public void setUpdateFileLengthOnFlush(boolean updateFileLenOnFlush) {
+        this.updateFileLenOnFlush = updateFileLenOnFlush;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(FileSystemConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index 57a8c6c..34d77f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -473,18 +473,18 @@ public class IgfsDataManager extends IgfsManager {
     /**
      * Registers write future in igfs data manager.
      *
-     * @param fileInfo File info of file opened to write.
+     * @param fileId File ID.
      * @return Future that will be completed when all ack messages are received or when write failed.
      */
-    public IgniteInternalFuture<Boolean> writeStart(IgfsEntryInfo fileInfo) {
-        WriteCompletionFuture fut = new WriteCompletionFuture(fileInfo.id());
+    public IgniteInternalFuture<Boolean> writeStart(IgniteUuid fileId) {
+        WriteCompletionFuture fut = new WriteCompletionFuture(fileId);
 
-        WriteCompletionFuture oldFut = pendingWrites.putIfAbsent(fileInfo.id(), fut);
+        WriteCompletionFuture oldFut = pendingWrites.putIfAbsent(fileId, fut);
 
-        assert oldFut == null : "Opened write that is being concurrently written: " + fileInfo;
+        assert oldFut == null : "Opened write that is being concurrently written: " + fileId;
 
         if (log.isDebugEnabled())
-            log.debug("Registered write completion future for file output stream [fileInfo=" + fileInfo +
+            log.debug("Registered write completion future for file output stream [fileId=" + fileId +
                 ", fut=" + fut + ']');
 
         return fut;
@@ -493,17 +493,23 @@ public class IgfsDataManager extends IgfsManager {
     /**
      * Notifies data manager that no further writes will be performed on stream.
      *
-     * @param fileInfo File info being written.
+     * @param fileId File ID.
+     * @param await Await completion.
+     * @throws IgniteCheckedException If failed.
      */
-    public void writeClose(IgfsEntryInfo fileInfo) {
-        WriteCompletionFuture fut = pendingWrites.get(fileInfo.id());
+    public void writeClose(IgniteUuid fileId, boolean await) throws IgniteCheckedException {
+        WriteCompletionFuture fut = pendingWrites.get(fileId);
 
-        if (fut != null)
+        if (fut != null) {
             fut.markWaitingLastAck();
+
+            if (await)
+                fut.get();
+        }
         else {
             if (log.isDebugEnabled())
                 log.debug("Failed to find write completion future for file in pending write map (most likely it was " +
-                    "failed): " + fileInfo);
+                    "failed): " + fileId);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/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 262dfef..bc26ace 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
@@ -1051,7 +1051,7 @@ public final class IgfsImpl implements IgfsEx {
 
                     batch = newBatch(path, desc.out());
 
-                    IgfsEventAwareOutputStream os = new IgfsEventAwareOutputStream(path, desc.info(),
+                    IgfsOutputStreamImpl os = new IgfsOutputStreamImpl(igfsCtx, path, desc.info(),
                         bufferSize(bufSize), mode, batch);
 
                     IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_WRITE);
@@ -1081,7 +1081,7 @@ public final class IgfsImpl implements IgfsEx {
 
                 assert res != null;
 
-                return new IgfsEventAwareOutputStream(path, res, bufferSize(bufSize), mode, null);
+                return new IgfsOutputStreamImpl(igfsCtx, path, res, bufferSize(bufSize), mode, null);
             }
         });
     }
@@ -1116,7 +1116,7 @@ public final class IgfsImpl implements IgfsEx {
 
                     batch = newBatch(path, desc.out());
 
-                    return new IgfsEventAwareOutputStream(path, desc.info(), bufferSize(bufSize), mode, batch);
+                    return new IgfsOutputStreamImpl(igfsCtx, path, desc.info(), bufferSize(bufSize), mode, batch);
                 }
 
                 final List<IgniteUuid> ids = meta.idsForPath(path);
@@ -1157,7 +1157,7 @@ public final class IgfsImpl implements IgfsEx {
 
                 assert res != null;
 
-                return new IgfsEventAwareOutputStream(path, res, bufferSize(bufSize), mode, null);
+                return new IgfsOutputStreamImpl(igfsCtx, path, res, bufferSize(bufSize), mode, null);
             }
         });
     }
@@ -1680,43 +1680,6 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /**
-     * IGFS output stream extension that fires events.
-     */
-    private class IgfsEventAwareOutputStream extends IgfsOutputStreamImpl {
-        /** Close guard. */
-        private final AtomicBoolean closeGuard = new AtomicBoolean(false);
-
-        /**
-         * Constructs file output stream.
-         *
-         * @param path Path to stored file.
-         * @param fileInfo File info.
-         * @param bufSize The size of the buffer to be used.
-         * @param mode IGFS mode.
-         * @param batch Optional secondary file system batch.
-         */
-        IgfsEventAwareOutputStream(IgfsPath path, IgfsEntryInfo fileInfo, int bufSize, IgfsMode mode,
-            @Nullable IgfsFileWorkerBatch batch) {
-            super(igfsCtx, path, fileInfo, bufSize, mode, batch, metrics);
-
-            metrics.incrementFilesOpenedForWrite();
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("NonSynchronizedMethodOverridesSynchronizedMethod")
-        @Override protected void onClose() throws IOException {
-            if (closeGuard.compareAndSet(false, true)) {
-                super.onClose();
-
-                metrics.decrementFilesOpenedForWrite();
-
-                if (evts.isRecordable(EVT_IGFS_FILE_CLOSED_WRITE))
-                    evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_CLOSED_WRITE, bytes()));
-            }
-        }
-    }
-
-    /**
      * IGFS input stream extension that fires events.
      */
     private class IgfsEventAwareInputStream extends IgfsInputStreamImpl {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/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 465116b..91bb334 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
@@ -44,7 +44,6 @@ import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
@@ -280,24 +279,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Return nodes where meta cache is defined.
-     *
-     * @return Nodes where meta cache is defined.
-     */
-    Collection<ClusterNode> metaCacheNodes() {
-        if (busyLock.enterBusy()) {
-            try {
-                return igfsCtx.kernalContext().discovery().cacheNodes(metaCache.name(), AffinityTopologyVersion.NONE);
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to get meta cache nodes because Grid is stopping.");
-    }
-
-    /**
      * Gets file ID for specified path.
      *
      * @param path Path.
@@ -630,21 +611,36 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Remove explicit lock on file held by the current thread.
+     * Remove explicit lock on file held by the current stream.
      *
-     * @param info File info to unlock.
+     * @param fileId File ID.
+     * @param lockId Lock ID.
      * @param modificationTime Modification time to write to file info.
      * @throws IgniteCheckedException If failed.
      */
-    public void unlock(final IgfsEntryInfo info, final long modificationTime) throws IgniteCheckedException {
-        validTxState(false);
+    public void unlock(final IgniteUuid fileId, final IgniteUuid lockId, final long modificationTime)
+        throws IgniteCheckedException {
+        unlock(fileId, lockId, modificationTime, false, 0, null);
+    }
 
-        assert info != null;
+    /**
+     * Remove explicit lock on file held by the current stream.
+     *
+     * @param fileId File ID.
+     * @param lockId Lock ID.
+     * @param modificationTime Modification time to write to file info.
+     * @param updateSpace Whether to update space.
+     * @param space Space.
+     * @param affRange Affinity range.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void unlock(final IgniteUuid fileId, final IgniteUuid lockId, final long modificationTime,
+        final boolean updateSpace, final long space, @Nullable final IgfsFileAffinityRange affRange)
+        throws IgniteCheckedException {
+        validTxState(false);
 
         if (busyLock.enterBusy()) {
             try {
-                final IgniteUuid lockId = info.lockId();
-
                 if (lockId == null)
                     return;
 
@@ -656,8 +652,6 @@ public class IgfsMetaManager extends IgfsManager {
                         @Override public Void applyx() throws IgniteCheckedException {
                             validTxState(true);
 
-                            IgniteUuid fileId = info.id();
-
                             // Lock file ID for this transaction.
                             IgfsEntryInfo oldInfo = info(fileId);
 
@@ -665,12 +659,13 @@ public class IgfsMetaManager extends IgfsManager {
                                 throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not " +
                                     "found): " + fileId));
 
-                            if (!F.eq(info.lockId(), oldInfo.lockId()))
+                            if (!F.eq(lockId, oldInfo.lockId()))
                                 throw new IgniteCheckedException("Failed to unlock file (inconsistent file lock ID) " +
-                                    "[fileId=" + fileId + ", lockId=" + info.lockId() + ", actualLockId=" +
+                                    "[fileId=" + fileId + ", lockId=" + lockId + ", actualLockId=" +
                                     oldInfo.lockId() + ']');
 
-                            id2InfoPrj.invoke(fileId, new IgfsMetaFileUnlockProcessor(modificationTime));
+                            id2InfoPrj.invoke(fileId,
+                                new IgfsMetaFileUnlockProcessor(modificationTime, updateSpace, space, affRange));
 
                             return null;
                         }
@@ -688,7 +683,7 @@ public class IgfsMetaManager extends IgfsManager {
             }
         }
         else
-            throw new IllegalStateException("Failed to unlock file system entry because Grid is stopping: " + info);
+            throw new IllegalStateException("Failed to unlock file system entry because Grid is stopping: " + fileId);
     }
 
     /**
@@ -1449,38 +1444,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Check whether there are any pending deletes and return collection of pending delete entry IDs.
-     *
-     * @return Collection of entry IDs to be deleted.
-     * @throws IgniteCheckedException If operation failed.
-     */
-    public Collection<IgniteUuid> pendingDeletes() throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                Collection<IgniteUuid> ids = new HashSet<>();
-
-                for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++) {
-                    IgniteUuid trashId = IgfsUtils.trashId(i);
-
-                    IgfsEntryInfo trashInfo = getInfo(trashId);
-
-                    if (trashInfo != null && trashInfo.hasChildren()) {
-                        for (IgfsListingEntry entry : trashInfo.listing().values())
-                            ids.add(entry.fileId());
-                    }
-                }
-
-                return ids;
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to get pending deletes because Grid is stopping.");
-    }
-
-    /**
      * Update file info (file properties) in cache in existing transaction.
      *
      * @param fileId File ID to update information for.
@@ -1545,27 +1508,26 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Reserve space for file.
      *
-     * @param path File path.
      * @param fileId File ID.
      * @param space Space.
      * @param affRange Affinity range.
      * @return New file info.
      */
-    public IgfsEntryInfo reserveSpace(IgfsPath path, IgniteUuid fileId, long space, IgfsFileAffinityRange affRange)
+    public IgfsEntryInfo reserveSpace(IgniteUuid fileId, long space, IgfsFileAffinityRange affRange)
         throws IgniteCheckedException {
         validTxState(false);
 
         if (busyLock.enterBusy()) {
             try {
                 if (log.isDebugEnabled())
-                    log.debug("Reserve file space [path=" + path + ", id=" + fileId + ']');
+                    log.debug("Reserve file space: " + fileId);
 
                 try (IgniteInternalTx tx = startTx()) {
                     // Lock file ID for this transaction.
                     IgfsEntryInfo oldInfo = info(fileId);
 
                     if (oldInfo == null)
-                        throw fsException("File has been deleted concurrently [path=" + path + ", id=" + fileId + ']');
+                        throw fsException("File has been deleted concurrently: " + fileId);
 
                     IgfsEntryInfo newInfo =
                         invokeAndGet(fileId, new IgfsMetaFileReserveSpaceProcessor(space, affRange));
@@ -1583,8 +1545,7 @@ public class IgfsMetaManager extends IgfsManager {
             }
         }
         else
-            throw new IllegalStateException("Failed to reseve file space because Grid is stopping [path=" + path +
-                ", id=" + fileId + ']');
+            throw new IllegalStateException("Failed to reserve file space because Grid is stopping:" + fileId);
     }
 
     /**
@@ -1882,121 +1843,8 @@ public class IgfsMetaManager extends IgfsManager {
                 // Events to fire (can be done outside of a transaction).
                 final Deque<IgfsEvent> pendingEvts = new LinkedList<>();
 
-                SynchronizationTask<IgfsSecondaryOutputStreamDescriptor> task =
-                    new SynchronizationTask<IgfsSecondaryOutputStreamDescriptor>() {
-                        /** Output stream to the secondary file system. */
-                        private OutputStream out;
-
-                        @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
-                            IgfsEntryInfo> infos) throws Exception {
-                            validTxState(true);
-
-                            assert !infos.isEmpty();
-
-                            // Determine the first existing parent.
-                            IgfsPath parentPath = null;
-
-                            for (IgfsPath curPath : infos.keySet()) {
-                                if (parentPath == null || curPath.isSubDirectoryOf(parentPath))
-                                    parentPath = curPath;
-                            }
-
-                            assert parentPath != null;
-
-                            IgfsEntryInfo parentInfo = infos.get(parentPath);
-
-                            // Delegate to the secondary file system.
-                            out = simpleCreate ? fs.create(path, overwrite) :
-                                fs.create(path, bufSize, overwrite, replication, blockSize, props);
-
-                            IgfsPath parent0 = path.parent();
-
-                            assert parent0 != null : "path.parent() is null (are we creating ROOT?): " + path;
-
-                            // If some of the parent directories were missing, synchronize again.
-                            if (!parentPath.equals(parent0)) {
-                                parentInfo = synchronize(fs, parentPath, parentInfo, parent0, true, null);
-
-                                // Fire notification about missing directories creation.
-                                if (evts.isRecordable(EventType.EVT_IGFS_DIR_CREATED)) {
-                                    IgfsPath evtPath = parent0;
-
-                                    while (!parentPath.equals(evtPath)) {
-                                        pendingEvts.addFirst(new IgfsEvent(evtPath, locNode,
-                                            EventType.EVT_IGFS_DIR_CREATED));
-
-                                        evtPath = evtPath.parent();
-
-                                        assert evtPath != null; // If this fails, then ROOT does not exist.
-                                    }
-                                }
-                            }
-
-                            // Get created file info.
-                            IgfsFile status = fs.info(path);
-
-                            if (status == null)
-                                throw fsException("Failed to open output stream to the file created in " +
-                                    "the secondary file system because it no longer exists: " + path);
-                            else if (status.isDirectory())
-                                throw fsException("Failed to open output stream to the file created in " +
-                                    "the secondary file system because the path points to a directory: " + path);
-
-                            IgfsEntryInfo newInfo = IgfsUtils.createFile(
-                                IgniteUuid.randomUuid(),
-                                status.blockSize(),
-                                status.length(),
-                                affKey,
-                                createFileLockId(false),
-                                igfsCtx.igfs().evictExclude(path, false),
-                                status.properties(),
-                                status.accessTime(),
-                                status.modificationTime()
-                            );
-
-                            // Add new file info to the listing optionally removing the previous one.
-                            assert parentInfo != null;
-
-                            IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), path.name(), newInfo);
-
-                            if (oldId != null) {
-                                IgfsEntryInfo oldInfo = info(oldId);
-
-                                assert oldInfo != null; // Otherwise cache is in inconsistent state.
-
-                                // The contact is that we cannot overwrite a file locked for writing:
-                                if (oldInfo.lockId() != null)
-                                    throw fsException("Failed to overwrite file (file is opened for writing) [path=" +
-                                        path + ", fileId=" + oldId + ", lockId=" + oldInfo.lockId() + ']');
-
-                                id2InfoPrj.remove(oldId); // Remove the old one.
-                                id2InfoPrj.invoke(parentInfo.id(), new IgfsMetaDirectoryListingRemoveProcessor(
-                                    path.name(), parentInfo.listing().get(path.name()).fileId()));
-
-                                createNewEntry(newInfo, parentInfo.id(), path.name()); // Put new one.
-
-                                igfsCtx.data().delete(oldInfo);
-                            }
-
-                            // Record CREATE event if needed.
-                            if (oldId == null && evts.isRecordable(EventType.EVT_IGFS_FILE_CREATED))
-                                pendingEvts.add(new IgfsEvent(path, locNode, EventType.EVT_IGFS_FILE_CREATED));
-
-                            return new IgfsSecondaryOutputStreamDescriptor(newInfo, out);
-                        }
-
-                        @Override public IgfsSecondaryOutputStreamDescriptor onFailure(Exception err)
-                            throws IgniteCheckedException {
-                            U.closeQuiet(out);
-
-                            U.error(log, "File create in DUAL mode failed [path=" + path + ", simpleCreate=" +
-                                simpleCreate + ", props=" + props + ", overwrite=" + overwrite + ", bufferSize=" +
-                                bufSize + ", replication=" + replication + ", blockSize=" + blockSize + ']', err);
-
-                            throw new IgniteCheckedException("Failed to create the file due to secondary file system " +
-                                "exception: " + path, err);
-                        }
-                    };
+                CreateFileSynchronizationTask task = new CreateFileSynchronizationTask(fs, path, simpleCreate, props,
+                    overwrite, bufSize, replication, blockSize, affKey, pendingEvts);
 
                 try {
                     return synchronizeAndExecute(task, fs, false, path.parent());
@@ -2956,29 +2804,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Synchronization task interface.
-     */
-    private static interface SynchronizationTask<T> {
-        /**
-         * Callback handler in case synchronization was successful.
-         *
-         * @param infos Map from paths to corresponding infos.
-         * @return Task result.
-         * @throws Exception If failed.
-         */
-        public T onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception;
-
-        /**
-         * Callback handler in case synchronization failed.
-         *
-         * @param err Optional exception.
-         * @return Task result.
-         * @throws IgniteCheckedException In case exception is to be thrown in that case.
-         */
-        public T onFailure(Exception err) throws IgniteCheckedException;
-    }
-
-    /**
      * Append routine.
      *
      * @param path Path.
@@ -3352,4 +3177,205 @@ public class IgfsMetaManager extends IgfsManager {
         if (delWorker0 != null)
             delWorker0.signal();
     }
+
+    /**
+     * Synchronization task interface.
+     */
+    private static interface SynchronizationTask<T> {
+        /**
+         * Callback handler in case synchronization was successful.
+         *
+         * @param infos Map from paths to corresponding infos.
+         * @return Task result.
+         * @throws Exception If failed.
+         */
+        public T onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception;
+
+        /**
+         * Callback handler in case synchronization failed.
+         *
+         * @param err Optional exception.
+         * @return Task result.
+         * @throws IgniteCheckedException In case exception is to be thrown in that case.
+         */
+        public T onFailure(Exception err) throws IgniteCheckedException;
+    }
+
+    /**
+     * Synchronization task to create a file.
+     */
+    private class CreateFileSynchronizationTask implements SynchronizationTask<IgfsSecondaryOutputStreamDescriptor> {
+        /** Secondary file system. */
+        private IgfsSecondaryFileSystem fs;
+
+        /** Path. */
+        private IgfsPath path;
+
+        /** Simple create flag. */
+        private boolean simpleCreate;
+
+        /** Properties. */
+        private Map<String, String> props;
+
+        /** Overwrite flag. */
+        private boolean overwrite;
+
+        /** Buffer size. */
+        private int bufSize;
+
+        /** Replication factor. */
+        private short replication;
+
+        /** Block size. */
+        private long blockSize;
+
+        /** Affinity key. */
+        private IgniteUuid affKey;
+
+        /** Pending events. */
+        private Deque<IgfsEvent> pendingEvts;
+
+        /** Output stream to the secondary file system. */
+        private OutputStream out;
+
+        /**
+         * Constructor.
+         *
+         * @param fs Secondary file system.
+         * @param path Path.
+         * @param simpleCreate Simple create flag.
+         * @param props Properties.
+         * @param overwrite Overwrite flag.
+         * @param bufSize Buffer size.
+         * @param replication Replication factor.
+         * @param blockSize Block size.
+         * @param affKey Affinity key.
+         * @param pendingEvts Pending events.
+         */
+        public CreateFileSynchronizationTask(IgfsSecondaryFileSystem fs, IgfsPath path, boolean simpleCreate,
+            @Nullable Map<String, String> props, boolean overwrite, int bufSize, short replication, long blockSize,
+            IgniteUuid affKey, Deque<IgfsEvent> pendingEvts) {
+            this.fs = fs;
+            this.path = path;
+            this.simpleCreate = simpleCreate;
+            this.props = props;
+            this.overwrite = overwrite;
+            this.bufSize = bufSize;
+            this.replication = replication;
+            this.blockSize = blockSize;
+            this.affKey = affKey;
+            this.pendingEvts = pendingEvts;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath, IgfsEntryInfo> infos)
+            throws Exception {
+            validTxState(true);
+
+            assert !infos.isEmpty();
+
+            // Determine the first existing parent.
+            IgfsPath parentPath = null;
+
+            for (IgfsPath curPath : infos.keySet()) {
+                if (parentPath == null || curPath.isSubDirectoryOf(parentPath))
+                    parentPath = curPath;
+            }
+
+            assert parentPath != null;
+
+            IgfsEntryInfo parentInfo = infos.get(parentPath);
+
+            // Delegate to the secondary file system.
+            out = simpleCreate ? fs.create(path, overwrite) :
+                fs.create(path, bufSize, overwrite, replication, blockSize, props);
+
+            IgfsPath parent0 = path.parent();
+
+            assert parent0 != null : "path.parent() is null (are we creating ROOT?): " + path;
+
+            // If some of the parent directories were missing, synchronize again.
+            if (!parentPath.equals(parent0)) {
+                parentInfo = synchronize(fs, parentPath, parentInfo, parent0, true, null);
+
+                // Fire notification about missing directories creation.
+                if (evts.isRecordable(EventType.EVT_IGFS_DIR_CREATED)) {
+                    IgfsPath evtPath = parent0;
+
+                    while (!parentPath.equals(evtPath)) {
+                        pendingEvts.addFirst(new IgfsEvent(evtPath, locNode,
+                            EventType.EVT_IGFS_DIR_CREATED));
+
+                        evtPath = evtPath.parent();
+
+                        assert evtPath != null; // If this fails, then ROOT does not exist.
+                    }
+                }
+            }
+
+            // Get created file info.
+            IgfsFile status = fs.info(path);
+
+            if (status == null)
+                throw fsException("Failed to open output stream to the file created in " +
+                    "the secondary file system because it no longer exists: " + path);
+            else if (status.isDirectory())
+                throw fsException("Failed to open output stream to the file created in " +
+                    "the secondary file system because the path points to a directory: " + path);
+
+            IgfsEntryInfo newInfo = IgfsUtils.createFile(
+                IgniteUuid.randomUuid(),
+                status.blockSize(),
+                status.length(),
+                affKey,
+                createFileLockId(false),
+                igfsCtx.igfs().evictExclude(path, false),
+                status.properties(),
+                status.accessTime(),
+                status.modificationTime()
+            );
+
+            // Add new file info to the listing optionally removing the previous one.
+            assert parentInfo != null;
+
+            IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), path.name(), newInfo);
+
+            if (oldId != null) {
+                IgfsEntryInfo oldInfo = info(oldId);
+
+                assert oldInfo != null; // Otherwise cache is in inconsistent state.
+
+                // The contact is that we cannot overwrite a file locked for writing:
+                if (oldInfo.lockId() != null)
+                    throw fsException("Failed to overwrite file (file is opened for writing) [path=" +
+                        path + ", fileId=" + oldId + ", lockId=" + oldInfo.lockId() + ']');
+
+                id2InfoPrj.remove(oldId); // Remove the old one.
+                id2InfoPrj.invoke(parentInfo.id(), new IgfsMetaDirectoryListingRemoveProcessor(
+                    path.name(), parentInfo.listing().get(path.name()).fileId()));
+
+                createNewEntry(newInfo, parentInfo.id(), path.name()); // Put new one.
+
+                igfsCtx.data().delete(oldInfo);
+            }
+
+            // Record CREATE event if needed.
+            if (oldId == null && evts.isRecordable(EventType.EVT_IGFS_FILE_CREATED))
+                pendingEvts.add(new IgfsEvent(path, locNode, EventType.EVT_IGFS_FILE_CREATED));
+
+            return new IgfsSecondaryOutputStreamDescriptor(newInfo, out);
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsSecondaryOutputStreamDescriptor onFailure(Exception err) throws IgniteCheckedException {
+            U.closeQuiet(out);
+
+            U.error(log, "File create in DUAL mode failed [path=" + path + ", simpleCreate=" +
+                simpleCreate + ", props=" + props + ", overwrite=" + overwrite + ", bufferSize=" +
+                bufSize + ", replication=" + replication + ", blockSize=" + blockSize + ']', err);
+
+            throw new IgniteCheckedException("Failed to create the file due to secondary file system " +
+                "exception: " + path, err);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamAdapter.java
deleted file mode 100644
index 43de61e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamAdapter.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.igfs;
-
-import java.io.DataInput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.igfs.IgfsOutputStream;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Output stream to store data into grid cache with separate blocks.
- */
-@SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-abstract class IgfsOutputStreamAdapter extends IgfsOutputStream {
-    /** Path to file. */
-    protected final IgfsPath path;
-
-    /** Buffer size. */
-    private final int bufSize;
-
-    /** Flag for this stream open/closed state. */
-    private boolean closed;
-
-    /** Local buffer to store stream data as consistent block. */
-    private ByteBuffer buf;
-
-    /** Bytes written. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    protected long bytes;
-
-    /** Time consumed by write operations. */
-    protected long time;
-
-    /**
-     * Constructs file output stream.
-     *
-     * @param path Path to stored file.
-     * @param bufSize The size of the buffer to be used.
-     */
-    IgfsOutputStreamAdapter(IgfsPath path, int bufSize) {
-        assert path != null;
-        assert bufSize > 0;
-
-        this.path = path;
-        this.bufSize = bufSize;
-    }
-
-    /**
-     * Gets number of written bytes.
-     *
-     * @return Written bytes.
-     */
-    public long bytes() {
-        return bytes;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void write(int b) throws IOException {
-        checkClosed(null, 0);
-
-        long startTime = System.nanoTime();
-
-        b &= 0xFF;
-
-        if (buf == null)
-            buf = ByteBuffer.allocate(bufSize);
-
-        buf.put((byte)b);
-
-        if (buf.position() >= bufSize)
-            sendData(true); // Send data to server.
-
-        time += System.nanoTime() - startTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void write(byte[] b, int off, int len) throws IOException {
-        A.notNull(b, "b");
-
-        if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length) || ((off + len) < 0)) {
-            throw new IndexOutOfBoundsException("Invalid bounds [data.length=" + b.length + ", offset=" + off +
-                ", length=" + len + ']');
-        }
-
-        checkClosed(null, 0);
-
-        if (len == 0)
-            return; // Done.
-
-        long startTime = System.nanoTime();
-
-        if (buf == null) {
-            // Do not allocate and copy byte buffer if will send data immediately.
-            if (len >= bufSize) {
-                buf = ByteBuffer.wrap(b, off, len);
-
-                sendData(false);
-
-                return;
-            }
-
-            buf = ByteBuffer.allocate(Math.max(bufSize, len));
-        }
-
-        if (buf.remaining() < len)
-            // Expand buffer capacity, if remaining size is less then data size.
-            buf = ByteBuffer.allocate(buf.position() + len).put((ByteBuffer)buf.flip());
-
-        assert len <= buf.remaining() : "Expects write data size less or equal then remaining buffer capacity " +
-            "[len=" + len + ", buf.remaining=" + buf.remaining() + ']';
-
-        buf.put(b, off, len);
-
-        if (buf.position() >= bufSize)
-            sendData(true); // Send data to server.
-
-        time += System.nanoTime() - startTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void transferFrom(DataInput in, int len) throws IOException {
-        checkClosed(in, len);
-
-        long startTime = System.nanoTime();
-
-        // Send all IPC data from the local buffer before streaming.
-        if (buf != null && buf.position() > 0)
-            sendData(true);
-
-        try {
-            storeDataBlocks(in, len);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IOException(e.getMessage(), e);
-        }
-
-        time += System.nanoTime() - startTime;
-    }
-
-    /**
-     * Flushes this output stream and forces any buffered output bytes to be written out.
-     *
-     * @exception IOException  if an I/O error occurs.
-     */
-    @Override public synchronized void flush() throws IOException {
-        checkClosed(null, 0);
-
-        // Send all IPC data from the local buffer.
-        if (buf != null && buf.position() > 0)
-            sendData(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public final synchronized void close() throws IOException {
-        // Do nothing if stream is already closed.
-        if (closed)
-            return;
-
-        try {
-            // Send all IPC data from the local buffer.
-            try {
-                flush();
-            }
-            finally {
-                onClose(); // "onClose()" routine must be invoked anyway!
-            }
-        }
-        finally {
-            // Mark this stream closed AFTER flush.
-            closed = true;
-        }
-    }
-
-    /**
-     * Store data blocks in file.<br/>
-     * Note! If file concurrently deleted we'll get lost blocks.
-     *
-     * @param data Data to store.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected abstract void storeDataBlock(ByteBuffer data) throws IgniteCheckedException, IOException;
-
-    /**
-     * Store data blocks in file reading appropriate number of bytes from given data input.
-     *
-     * @param in Data input to read from.
-     * @param len Data length to store.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected abstract void storeDataBlocks(DataInput in, int len) throws IgniteCheckedException, IOException;
-
-    /**
-     * Close callback. It will be called only once in synchronized section.
-     *
-     * @throws IOException If failed.
-     */
-    protected void onClose() throws IOException {
-        // No-op.
-    }
-
-    /**
-     * Validate this stream is open.
-     *
-     * @throws IOException If this stream is closed.
-     */
-    private void checkClosed(@Nullable DataInput in, int len) throws IOException {
-        assert Thread.holdsLock(this);
-
-        if (closed) {
-            // Must read data from stream before throwing exception.
-            if (in != null)
-                in.skipBytes(len);
-
-            throw new IOException("Stream has been closed: " + this);
-        }
-    }
-
-    /**
-     * Send all local-buffered data to server.
-     *
-     * @param flip Whether to flip buffer on sending data. We do not want to flip it if sending wrapped
-     *      byte array.
-     * @throws IOException In case of IO exception.
-     */
-    private void sendData(boolean flip) throws IOException {
-        assert Thread.holdsLock(this);
-
-        try {
-            if (flip)
-                buf.flip();
-
-            storeDataBlock(buf);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IOException("Failed to store data into file: " + path, e);
-        }
-
-        buf = null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgfsOutputStreamAdapter.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
index 21e5fb6..7741a25 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
@@ -18,12 +18,13 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsMode;
+import org.apache.ignite.igfs.IgfsOutputStream;
 import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.igfs.IgfsPathNotFoundException;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -32,8 +33,8 @@ import org.jetbrains.annotations.Nullable;
 import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.concurrent.atomic.AtomicBoolean;
 
+import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_WRITE;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC;
 import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
 import static org.apache.ignite.igfs.IgfsMode.PROXY;
@@ -41,25 +42,44 @@ import static org.apache.ignite.igfs.IgfsMode.PROXY;
 /**
  * Output stream to store data into grid cache with separate blocks.
  */
-class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
+class IgfsOutputStreamImpl extends IgfsOutputStream {
     /** Maximum number of blocks in buffer. */
     private static final int MAX_BLOCKS_CNT = 16;
 
     /** IGFS context. */
-    private IgfsContext igfsCtx;
+    private final IgfsContext igfsCtx;
 
-    /** Meta info manager. */
-    private final IgfsMetaManager meta;
+    /** Path to file. */
+    private final IgfsPath path;
 
-    /** Data manager. */
-    private final IgfsDataManager data;
+    /** Buffer size. */
+    private final int bufSize;
+
+    /** IGFS mode. */
+    private final IgfsMode mode;
+
+    /** File worker batch. */
+    private final IgfsFileWorkerBatch batch;
+
+    /** Mutex for synchronization. */
+    private final Object mux = new Object();
+
+    /** Flag for this stream open/closed state. */
+    private boolean closed;
+
+    /** Local buffer to store stream data as consistent block. */
+    private ByteBuffer buf;
+
+    /** Bytes written. */
+    private long bytes;
+
+    /** Time consumed by write operations. */
+    private long time;
 
     /** File descriptor. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
     private IgfsEntryInfo fileInfo;
 
     /** Space in file to write data. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
     private long space;
 
     /** Intermediate remainder to keep data. */
@@ -68,21 +88,6 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
     /** Data length in remainder. */
     private int remainderDataLen;
 
-    /** Write completion future. */
-    private final IgniteInternalFuture<Boolean> writeCompletionFut;
-
-    /** IGFS mode. */
-    private final IgfsMode mode;
-
-    /** File worker batch. */
-    private final IgfsFileWorkerBatch batch;
-
-    /** Ensures that onClose)_ routine is called no more than once. */
-    private final AtomicBoolean onCloseGuard = new AtomicBoolean();
-
-    /** Local IGFS metrics. */
-    private final IgfsLocalMetrics metrics;
-
     /** Affinity written by this output stream. */
     private IgfsFileAffinityRange streamRange;
 
@@ -95,259 +100,234 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
      * @param bufSize The size of the buffer to be used.
      * @param mode Grid IGFS mode.
      * @param batch Optional secondary file system batch.
-     * @param metrics Local IGFS metrics.
      */
     IgfsOutputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, int bufSize, IgfsMode mode,
-        @Nullable IgfsFileWorkerBatch batch, IgfsLocalMetrics metrics) {
-        super(path, optimizeBufferSize(bufSize, fileInfo));
-
-        assert fileInfo != null;
-        assert fileInfo.isFile() : "Unexpected file info: " + fileInfo;
-        assert mode != null && mode != PROXY;
-        assert mode == PRIMARY && batch == null || batch != null;
-        assert metrics != null;
+        @Nullable IgfsFileWorkerBatch batch) {
+        assert fileInfo != null && fileInfo.isFile() : "Unexpected file info: " + fileInfo;
+        assert mode != null && mode != PROXY && (mode == PRIMARY && batch == null || batch != null);
 
         // File hasn't been locked.
         if (fileInfo.lockId() == null)
             throw new IgfsException("Failed to acquire file lock (concurrently modified?): " + path);
 
-        assert !IgfsUtils.DELETE_LOCK_ID.equals(fileInfo.lockId());
+        synchronized (mux) {
+            this.path = path;
+            this.bufSize = optimizeBufferSize(bufSize, fileInfo);
+            this.igfsCtx = igfsCtx;
+            this.fileInfo = fileInfo;
+            this.mode = mode;
+            this.batch = batch;
 
-        this.igfsCtx = igfsCtx;
-        meta = igfsCtx.meta();
-        data = igfsCtx.data();
+            streamRange = initialStreamRange(fileInfo);
 
-        this.fileInfo = fileInfo;
-        this.mode = mode;
-        this.batch = batch;
-        this.metrics = metrics;
-
-        streamRange = initialStreamRange(fileInfo);
+            igfsCtx.data().writeStart(fileInfo.id());
+        }
 
-        writeCompletionFut = data.writeStart(fileInfo);
+        igfsCtx.igfs().localMetrics().incrementFilesOpenedForWrite();
     }
 
-    /**
-     * Optimize buffer size.
-     *
-     * @param bufSize Requested buffer size.
-     * @param fileInfo File info.
-     * @return Optimized buffer size.
-     */
-    @SuppressWarnings("IfMayBeConditional")
-    private static int optimizeBufferSize(int bufSize, IgfsEntryInfo fileInfo) {
-        assert bufSize > 0;
-
-        if (fileInfo == null)
-            return bufSize;
-
-        int blockSize = fileInfo.blockSize();
+    /** {@inheritDoc} */
+    @Override public void write(int b) throws IOException {
+        synchronized (mux) {
+            checkClosed(null, 0);
 
-        if (blockSize <= 0)
-            return bufSize;
+            b &= 0xFF;
 
-        if (bufSize <= blockSize)
-            // Optimize minimum buffer size to be equal file's block size.
-            return blockSize;
+            long startTime = System.nanoTime();
 
-        int maxBufSize = blockSize * MAX_BLOCKS_CNT;
+            if (buf == null)
+                buf = allocateNewBuffer();
 
-        if (bufSize > maxBufSize)
-            // There is no profit or optimization from larger buffers.
-            return maxBufSize;
+            buf.put((byte)b);
 
-        if (fileInfo.length() == 0)
-            // Make buffer size multiple of block size (optimized for new files).
-            return bufSize / blockSize * blockSize;
+            sendBufferIfFull();
 
-        return bufSize;
+            time += System.nanoTime() - startTime;
+        }
     }
 
     /** {@inheritDoc} */
-    @Override protected synchronized void storeDataBlock(ByteBuffer block) throws IgniteCheckedException, IOException {
-        int writeLen = block.remaining();
+    @SuppressWarnings("NullableProblems")
+    @Override public void write(byte[] b, int off, int len) throws IOException {
+        A.notNull(b, "b");
 
-        preStoreDataBlocks(null, writeLen);
+        if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length) || ((off + len) < 0)) {
+            throw new IndexOutOfBoundsException("Invalid bounds [data.length=" + b.length + ", offset=" + off +
+                ", length=" + len + ']');
+        }
 
-        int blockSize = fileInfo.blockSize();
+        synchronized (mux) {
+            checkClosed(null, 0);
+
+            // Check if there is anything to write.
+            if (len == 0)
+                return;
 
-        // If data length is not enough to fill full block, fill the remainder and return.
-        if (remainderDataLen + writeLen < blockSize) {
-            if (remainder == null)
-                remainder = new byte[blockSize];
-            else if (remainder.length != blockSize) {
-                assert remainderDataLen == remainder.length;
+            long startTime = System.nanoTime();
 
-                byte[] allocated = new byte[blockSize];
+            if (buf == null) {
+                if (len >= bufSize) {
+                    // Send data right away.
+                    ByteBuffer tmpBuf = ByteBuffer.wrap(b, off, len);
 
-                U.arrayCopy(remainder, 0, allocated, 0, remainder.length);
+                    send(tmpBuf, tmpBuf.remaining());
+                }
+                else {
+                    buf = allocateNewBuffer();
 
-                remainder = allocated;
+                    buf.put(b, off, len);
+                }
             }
+            else {
+                // Re-allocate buffer if needed.
+                if (buf.remaining() < len)
+                    buf = ByteBuffer.allocate(buf.position() + len).put((ByteBuffer)buf.flip());
 
-            block.get(remainder, remainderDataLen, writeLen);
+                buf.put(b, off, len);
 
-            remainderDataLen += writeLen;
-        }
-        else {
-            remainder = data.storeDataBlocks(fileInfo, fileInfo.length() + space, remainder, remainderDataLen, block,
-                false, streamRange, batch);
+                sendBufferIfFull();
+            }
 
-            remainderDataLen = remainder == null ? 0 : remainder.length;
+            time += System.nanoTime() - startTime;
         }
     }
 
     /** {@inheritDoc} */
-    @Override protected synchronized void storeDataBlocks(DataInput in, int len) throws IgniteCheckedException, IOException {
-        preStoreDataBlocks(in, len);
-
-        int blockSize = fileInfo.blockSize();
-
-        // If data length is not enough to fill full block, fill the remainder and return.
-        if (remainderDataLen + len < blockSize) {
-            if (remainder == null)
-                remainder = new byte[blockSize];
-            else if (remainder.length != blockSize) {
-                assert remainderDataLen == remainder.length;
+    @Override public void transferFrom(DataInput in, int len) throws IOException {
+        synchronized (mux) {
+            checkClosed(in, len);
 
-                byte[] allocated = new byte[blockSize];
+            long startTime = System.nanoTime();
 
-                U.arrayCopy(remainder, 0, allocated, 0, remainder.length);
+            // Clean-up local buffer before streaming.
+            sendBufferIfNotEmpty();
 
-                remainder = allocated;
-            }
-
-            in.readFully(remainder, remainderDataLen, len);
-
-            remainderDataLen += len;
-        }
-        else {
-            remainder = data.storeDataBlocks(fileInfo, fileInfo.length() + space, remainder, remainderDataLen, in, len,
-                false, streamRange, batch);
+            // Perform transfer.
+            send(in, len);
 
-            remainderDataLen = remainder == null ? 0 : remainder.length;
+            time += System.nanoTime() - startTime;
         }
     }
 
     /**
-     * Initializes data loader if it was not initialized yet and updates written space.
+     * Flushes this output stream and forces any buffered output bytes to be written out.
      *
-     * @param len Data length to be written.
+     * @exception IOException  if an I/O error occurs.
      */
-    private void preStoreDataBlocks(@Nullable DataInput in, int len) throws IgniteCheckedException, IOException {
-        // Check if any exception happened while writing data.
-        if (writeCompletionFut.isDone()) {
-            assert ((GridFutureAdapter)writeCompletionFut).isFailed();
+    @Override public void flush() throws IOException {
+        synchronized (mux) {
+            checkClosed(null, 0);
 
-            if (in != null)
-                in.skipBytes(len);
+            sendBufferIfNotEmpty();
 
-            writeCompletionFut.get();
-        }
+            flushRemainder();
+
+            awaitAcks();
+
+            // Update file length if needed.
+            if (igfsCtx.configuration().isUpdateFileLengthOnFlush() && space > 0) {
+                try {
+                    IgfsEntryInfo fileInfo0 = igfsCtx.meta().reserveSpace(fileInfo.id(), space, streamRange);
+
+                    if (fileInfo0 == null)
+                        throw new IOException("File was concurrently deleted: " + path);
+                    else
+                        fileInfo = fileInfo0;
+
+                    streamRange = initialStreamRange(fileInfo);
 
-        bytes += len;
-        space += len;
+                    space = 0;
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IOException("Failed to update file length data [path=" + path +
+                        ", space=" + space + ']', e);
+                }
+            }
+        }
     }
 
     /**
-     * Flushes this output stream and forces any buffered output bytes to be written out.
+     * Await acknowledgments.
      *
-     * @exception IOException  if an I/O error occurs.
+     * @throws IOException If failed.
      */
-    @Override public synchronized void flush() throws IOException {
-        boolean exists;
-
+    private void awaitAcks() throws IOException {
         try {
-            exists = meta.exists(fileInfo.id());
+            igfsCtx.data().awaitAllAcksReceived(fileInfo.id());
         }
         catch (IgniteCheckedException e) {
-            throw new IOException("File to read file metadata: " + path, e);
+            throw new IOException("Failed to wait for flush acknowledge: " + fileInfo.id, e);
         }
+    }
 
-        if (!exists) {
-            onClose(true);
-
-            throw new IOException("File was concurrently deleted: " + path);
-        }
-
-        super.flush();
-
+    /**
+     * Flush remainder.
+     *
+     * @throws IOException If failed.
+     */
+    private void flushRemainder() throws IOException {
         try {
             if (remainder != null) {
-                data.storeDataBlocks(fileInfo, fileInfo.length() + space, null, 0,
+                igfsCtx.data().storeDataBlocks(fileInfo, fileInfo.length() + space, null, 0,
                     ByteBuffer.wrap(remainder, 0, remainderDataLen), true, streamRange, batch);
 
                 remainder = null;
                 remainderDataLen = 0;
             }
-
-            if (space > 0) {
-                data.awaitAllAcksReceived(fileInfo.id());
-
-                IgfsEntryInfo fileInfo0 = meta.reserveSpace(path, fileInfo.id(), space, streamRange);
-
-                if (fileInfo0 == null)
-                    throw new IOException("File was concurrently deleted: " + path);
-                else
-                    fileInfo = fileInfo0;
-
-                streamRange = initialStreamRange(fileInfo);
-
-                space = 0;
-            }
         }
         catch (IgniteCheckedException e) {
-            throw new IOException("Failed to flush data [path=" + path + ", space=" + space + ']', e);
+            throw new IOException("Failed to flush data (remainder) [path=" + path + ", space=" + space + ']', e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override protected void onClose() throws IOException {
-        onClose(false);
-    }
+    @Override public final void close() throws IOException {
+        synchronized (mux) {
+            // Do nothing if stream is already closed.
+            if (closed)
+                return;
 
-    /**
-     * Close callback. It will be called only once in synchronized section.
-     *
-     * @param deleted Whether we already know that the file was deleted.
-     * @throws IOException If failed.
-     */
-    private void onClose(boolean deleted) throws IOException {
-        assert Thread.holdsLock(this);
-
-        if (onCloseGuard.compareAndSet(false, true)) {
-            // Notify backing secondary file system batch to finish.
-            if (mode != PRIMARY) {
-                assert batch != null;
+            // Set closed flag immediately.
+            closed = true;
 
-                batch.finish();
-            }
+            // Flush data.
+            IOException err = null;
 
-            // Ensure file existence.
-            boolean exists;
+            boolean flushSuccess = false;
 
             try {
-                exists = !deleted && meta.exists(fileInfo.id());
-            }
-            catch (IgniteCheckedException e) {
-                throw new IOException("File to read file metadata: " + path, e);
-            }
+                sendBufferIfNotEmpty();
 
-            if (exists) {
-                IOException err = null;
+                flushRemainder();
 
-                try {
-                    data.writeClose(fileInfo);
+                igfsCtx.data().writeClose(fileInfo.id(), true);
 
-                    writeCompletionFut.get();
-                }
-                catch (IgniteCheckedException e) {
-                    err = new IOException("Failed to close stream [path=" + path + ", fileInfo=" + fileInfo + ']', e);
-                }
+                flushSuccess = true;
+            }
+            catch (Exception e) {
+                err = new IOException("Failed to flush data during stream close [path=" + path +
+                    ", fileInfo=" + fileInfo + ']', e);
+            }
 
-                metrics.addWrittenBytesTime(bytes, time);
+            // Unlock the file after data is flushed.
+            try {
+                if (flushSuccess && space > 0)
+                    igfsCtx.meta().unlock(fileInfo.id(), fileInfo.lockId(), System.currentTimeMillis(), true,
+                        space, streamRange);
+                else
+                    igfsCtx.meta().unlock(fileInfo.id(), fileInfo.lockId(), System.currentTimeMillis());
+            }
+            catch (Exception e) {
+                if (err == null)
+                    err = new IOException("File to release file lock: " + path, e);
+                else
+                    err.addSuppressed(e);
+            }
+
+            // Finally, await secondary file system flush.
+            if (batch != null) {
+                batch.finish();
 
-                // Await secondary file system processing to finish.
                 if (mode == DUAL_SYNC) {
                     try {
                         batch.await();
@@ -356,40 +336,141 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
                         if (err == null)
                             err = new IOException("Failed to close secondary file system stream [path=" + path +
                                 ", fileInfo=" + fileInfo + ']', e);
+                        else
+                            err.addSuppressed(e);
                     }
                 }
+            }
 
-                long modificationTime = System.currentTimeMillis();
+            // Throw error, if any.
+            if (err != null)
+                throw err;
 
-                try {
-                    meta.unlock(fileInfo, modificationTime);
-                }
-                catch (IgfsPathNotFoundException ignore) {
-                    data.delete(fileInfo); // Safety to ensure that all data blocks are deleted.
+            igfsCtx.igfs().localMetrics().addWrittenBytesTime(bytes, time);
+            igfsCtx.igfs().localMetrics().decrementFilesOpenedForWrite();
 
-                    throw new IOException("File was concurrently deleted: " + path);
-                }
-                catch (IgniteCheckedException e) {
-                    throw new IOException("File to read file metadata: " + path, e);
+            GridEventStorageManager evts = igfsCtx.kernalContext().event();
+
+            if (evts.isRecordable(EVT_IGFS_FILE_CLOSED_WRITE))
+                evts.record(new IgfsEvent(path, igfsCtx.kernalContext().discovery().localNode(),
+                    EVT_IGFS_FILE_CLOSED_WRITE, bytes));
+        }
+    }
+
+    /**
+     * Validate this stream is open.
+     *
+     * @throws IOException If this stream is closed.
+     */
+    private void checkClosed(@Nullable DataInput in, int len) throws IOException {
+        assert Thread.holdsLock(mux);
+
+        if (closed) {
+            // Must read data from stream before throwing exception.
+            if (in != null)
+                in.skipBytes(len);
+
+            throw new IOException("Stream has been closed: " + this);
+        }
+    }
+
+    /**
+     * Send local buffer if it full.
+     *
+     * @throws IOException If failed.
+     */
+    private void sendBufferIfFull() throws IOException {
+        if (buf.position() >= bufSize)
+            sendBuffer();
+    }
+
+    /**
+     * Send local buffer if at least something is stored there.
+     *
+     * @throws IOException
+     */
+    private void sendBufferIfNotEmpty() throws IOException {
+        if (buf != null && buf.position() > 0)
+            sendBuffer();
+    }
+
+    /**
+     * Send all local-buffered data to server.
+     *
+     * @throws IOException In case of IO exception.
+     */
+    private void sendBuffer() throws IOException {
+        buf.flip();
+
+        send(buf, buf.remaining());
+
+        buf = null;
+    }
+
+    /**
+     * Store data block.
+     *
+     * @param data Block.
+     * @param writeLen Write length.
+     * @throws IOException If failed.
+     */
+    private void send(Object data, int writeLen) throws IOException {
+        assert Thread.holdsLock(mux);
+        assert data instanceof ByteBuffer || data instanceof DataInput;
+
+        try {
+            // Increment metrics.
+            bytes += writeLen;
+            space += writeLen;
+
+            int blockSize = fileInfo.blockSize();
+
+            // If data length is not enough to fill full block, fill the remainder and return.
+            if (remainderDataLen + writeLen < blockSize) {
+                if (remainder == null)
+                    remainder = new byte[blockSize];
+                else if (remainder.length != blockSize) {
+                    assert remainderDataLen == remainder.length;
+
+                    byte[] allocated = new byte[blockSize];
+
+                    U.arrayCopy(remainder, 0, allocated, 0, remainder.length);
+
+                    remainder = allocated;
                 }
 
-                if (err != null)
-                    throw err;
+                if (data instanceof ByteBuffer)
+                    ((ByteBuffer) data).get(remainder, remainderDataLen, writeLen);
+                else
+                    ((DataInput) data).readFully(remainder, remainderDataLen, writeLen);
+
+                remainderDataLen += writeLen;
             }
             else {
-                try {
-                    if (mode == DUAL_SYNC)
-                        batch.await();
+                if (data instanceof ByteBuffer) {
+                    remainder = igfsCtx.data().storeDataBlocks(fileInfo, fileInfo.length() + space, remainder,
+                        remainderDataLen, (ByteBuffer) data, false, streamRange, batch);
                 }
-                catch (IgniteCheckedException e) {
-                    throw new IOException("Failed to close secondary file system stream [path=" + path +
-                        ", fileInfo=" + fileInfo + ']', e);
-                }
-                finally {
-                    data.delete(fileInfo);
+                else {
+                    remainder = igfsCtx.data().storeDataBlocks(fileInfo, fileInfo.length() + space, remainder,
+                        remainderDataLen, (DataInput) data, writeLen, false, streamRange, batch);
                 }
+
+                remainderDataLen = remainder == null ? 0 : remainder.length;
             }
         }
+        catch (IgniteCheckedException e) {
+            throw new IOException("Failed to store data into file: " + path, e);
+        }
+    }
+
+    /**
+     * Allocate new buffer.
+     *
+     * @return New buffer.
+     */
+    private ByteBuffer allocateNewBuffer() {
+        return ByteBuffer.allocate(bufSize);
     }
 
     /**
@@ -421,11 +502,46 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
 
         IgniteUuid prevAffKey = map == null ? null : map.affinityKey(lastBlockOff, false);
 
-        IgniteUuid affKey = data.nextAffinityKey(prevAffKey);
+        IgniteUuid affKey = igfsCtx.data().nextAffinityKey(prevAffKey);
 
         return affKey == null ? null : new IgfsFileAffinityRange(off, off, affKey);
     }
 
+    /**
+     * Optimize buffer size.
+     *
+     * @param bufSize Requested buffer size.
+     * @param fileInfo File info.
+     * @return Optimized buffer size.
+     */
+    private static int optimizeBufferSize(int bufSize, IgfsEntryInfo fileInfo) {
+        assert bufSize > 0;
+
+        if (fileInfo == null)
+            return bufSize;
+
+        int blockSize = fileInfo.blockSize();
+
+        if (blockSize <= 0)
+            return bufSize;
+
+        if (bufSize <= blockSize)
+            // Optimize minimum buffer size to be equal file's block size.
+            return blockSize;
+
+        int maxBufSize = blockSize * MAX_BLOCKS_CNT;
+
+        if (bufSize > maxBufSize)
+            // There is no profit or optimization from larger buffers.
+            return maxBufSize;
+
+        if (fileInfo.length() == 0)
+            // Make buffer size multiple of block size (optimized for new files).
+            return bufSize / blockSize * blockSize;
+
+        return bufSize;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsOutputStreamImpl.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
index 6827e4a..8d23490 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
@@ -24,8 +24,11 @@ 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.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
+import org.apache.ignite.internal.processors.igfs.IgfsFileMap;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
 
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
@@ -46,6 +49,15 @@ public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, I
     /** Modification time. */
     private long modificationTime;
 
+    /** Whether to update space. */
+    private boolean updateSpace;
+
+    /** Space. */
+    private long space;
+
+    /** Affinity range. */
+    private IgfsFileAffinityRange affRange;
+
     /**
      * Default constructor.
      */
@@ -57,17 +69,36 @@ public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, I
      * Constructor.
      *
      * @param modificationTime Modification time.
+     * @param updateSpace Whether to update space.
+     * @param space Space.
+     * @param affRange Affinity range.
      */
-    public IgfsMetaFileUnlockProcessor(long modificationTime) {
+    public IgfsMetaFileUnlockProcessor(long modificationTime, boolean updateSpace, long space,
+        @Nullable IgfsFileAffinityRange affRange) {
         this.modificationTime = modificationTime;
+        this.updateSpace = updateSpace;
+        this.space = space;
+        this.affRange = affRange;
     }
 
     /** {@inheritDoc} */
     @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
         throws EntryProcessorException {
-        IgfsEntryInfo old = entry.getValue();
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        assert oldInfo != null;
+
+        IgfsEntryInfo newInfo = oldInfo.unlock(modificationTime);
+
+        if (updateSpace) {
+            IgfsFileMap newMap = new IgfsFileMap(newInfo.fileMap());
 
-        entry.setValue(old.unlock(modificationTime));
+            newMap.addRange(affRange);
+
+            newInfo = newInfo.length(newInfo.length() + space).fileMap(newMap);
+        }
+
+        entry.setValue(newInfo);
 
         return null;
     }
@@ -75,11 +106,27 @@ public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, I
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeLong(modificationTime);
+
+        if (updateSpace) {
+            out.writeBoolean(true);
+            out.writeLong(space);
+            out.writeObject(affRange);
+        }
+        else
+            out.writeBoolean(false);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         modificationTime = in.readLong();
+
+        if (in.readBoolean()) {
+            updateSpace = true;
+            space = in.readLong();
+            affRange = (IgfsFileAffinityRange)in.readObject();
+        }
+        else
+            updateSpace = false;
     }
 
     /** {@inheritDoc} */
@@ -87,6 +134,14 @@ public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, I
         BinaryRawWriter out = writer.rawWriter();
 
         out.writeLong(modificationTime);
+
+        if (updateSpace) {
+            out.writeBoolean(true);
+            out.writeLong(space);
+            out.writeObject(affRange);
+        }
+        else
+            out.writeBoolean(false);
     }
 
     /** {@inheritDoc} */
@@ -94,6 +149,14 @@ public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, I
         BinaryRawReader in = reader.rawReader();
 
         modificationTime = in.readLong();
+
+        if (in.readBoolean()) {
+            updateSpace = true;
+            space = in.readLong();
+            affRange = in.readObject();
+        }
+        else
+            updateSpace = false;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/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 76a038d..261a494 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
@@ -2987,10 +2987,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param chunks Expected data.
      * @throws Exception If failed.
      */
-    protected void checkFile(IgfsImpl igfs, UniversalFileSystemAdapter igfsSecondary, IgfsPath file,
+    protected void checkFile(@Nullable IgfsImpl igfs, UniversalFileSystemAdapter igfsSecondary, IgfsPath file,
         @Nullable byte[]... chunks) throws Exception {
-        checkExist(igfs, file);
-        checkFileContent(igfs, file, chunks);
+        if (igfs != null) {
+            checkExist(igfs, file);
+            checkFileContent(igfs, file, chunks);
+        }
 
         if (dual) {
             checkExist(igfsSecondary, file);
@@ -3016,16 +3018,18 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
                 is = igfs.open(file);
 
                 int chunkIdx = 0;
+                int pos = 0;
 
                 for (byte[] chunk : chunks) {
                     byte[] buf = new byte[chunk.length];
 
-                    is.readFully(0, buf);
+                    is.readFully(pos, buf);
 
                     assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + igfs.name() + ", chunkIdx=" + chunkIdx +
                         ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']';
 
                     chunkIdx++;
+                    pos += chunk.length;
                 }
 
                 is.close();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
index 013bb18..0d1a66f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
@@ -178,7 +178,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             rnd.nextBytes(data);
 
-            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
+            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info.id());
 
             expectsStoreFail(info, data, "Not enough space reserved to store data");
 
@@ -195,7 +195,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             assert remainder == null;
 
-            mgr.writeClose(info);
+            mgr.writeClose(info.id(), false);
 
             fut.get(3000);
 
@@ -269,7 +269,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             info = info.length(info.length() + data.length + remainder.length);
 
-            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
+            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info.id());
 
             IgfsFileAffinityRange range = new IgfsFileAffinityRange();
 
@@ -287,7 +287,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             assert left2 == null;
 
-            mgr.writeClose(info);
+            mgr.writeClose(info.id(), false);
 
             fut.get(3000);
 
@@ -358,7 +358,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             info = info.length(info.length() + data.length * writesCnt);
 
-            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
+            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info.id());
 
             for (int j = 0; j < 64; j++) {
                 Arrays.fill(data, (byte)(j / 4));
@@ -369,7 +369,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
                 assert left == null : "No remainder should be returned if flush is true: " + Arrays.toString(left);
             }
 
-            mgr.writeClose(info);
+            mgr.writeClose(info.id(), false);
 
             assertTrue(range.regionEqual(new IgfsFileAffinityRange(0, writesCnt * chunkSize - 1, null)));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
index 0ce1036..29bb2cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
@@ -445,27 +445,41 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testInvalidEndpointTcpPort() throws Exception {
+    public void testZeroEndpointTcpPort() throws Exception {
+        checkIvalidPort(0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeEndpointTcpPort() throws Exception {
+        checkIvalidPort(-1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTooBigEndpointTcpPort() throws Exception {
+        checkIvalidPort(65536);
+    }
+
+    /**
+     * Check invalid port handling.
+     *
+     * @param port Port.
+     * @throws Exception If failed.
+     */
+    private void checkIvalidPort(int port) throws Exception {
         final String failMsg = "IGFS endpoint TCP port is out of range";
         g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
 
         final String igfsCfgName = "igfs-cfg";
         final IgfsIpcEndpointConfiguration igfsEndpointCfg = new IgfsIpcEndpointConfiguration();
-        igfsEndpointCfg.setPort(0);
+        igfsEndpointCfg.setPort(port);
         g1IgfsCfg1.setName(igfsCfgName);
         g1IgfsCfg1.setIpcEndpointConfiguration(igfsEndpointCfg);
 
         checkGridStartFails(g1Cfg, failMsg, true);
-
-        igfsEndpointCfg.setPort(-1);
-        g1IgfsCfg1.setIpcEndpointConfiguration(igfsEndpointCfg);
-
-        checkGridStartFails(g1Cfg, failMsg, true);
-
-        igfsEndpointCfg.setPort(65536);
-        g1IgfsCfg1.setIpcEndpointConfiguration(igfsEndpointCfg);
-
-        checkGridStartFails(g1Cfg, failMsg, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec5706ff/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
index 57174ea..e5abfea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
@@ -137,7 +137,7 @@ public class IgfsTaskSelfTest extends IgfsCommonAbstractTest {
         metaCacheCfg.setName("metaCache");
         metaCacheCfg.setCacheMode(REPLICATED);
         metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
         IgniteConfiguration cfg = new IgniteConfiguration();
 


[10/24] ignite git commit: ignite-gg-11181 - scanCount with offheap index fix

Posted by sb...@apache.org.
ignite-gg-11181 - scanCount with offheap index fix


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

Branch: refs/heads/ignite-3038
Commit: a0c6ae542cdc1f885b038beef168a35b14e23ad5
Parents: e409b67
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Jun 6 00:14:05 2016 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Jun 6 00:14:05 2016 +0300

----------------------------------------------------------------------
 .../unsafe/GridOffheapSnapTreeSelfTest.java     |   2 +-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  23 +--
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |  17 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  | 195 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 5 files changed, 226 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a0c6ae54/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffheapSnapTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffheapSnapTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffheapSnapTreeSelfTest.java
index 463b6dc..92d9ec2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffheapSnapTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffheapSnapTreeSelfTest.java
@@ -313,7 +313,7 @@ public class GridOffheapSnapTreeSelfTest extends GridCommonAbstractTest {
         }
 
         @Override public int hashCode() {
-            return ptr;
+            throw new IllegalStateException();
         }
 
         @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a0c6ae54/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
index ca5442a..fe6851d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
@@ -351,41 +351,37 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
 
     /** {@inheritDoc} */
     @Override public void setKeyAndVersion(SearchRow old) {
-        assert false;
+        throw new IllegalStateException();
     }
 
     /** {@inheritDoc} */
     @Override public void setKey(long key) {
-        assert false;
+        throw new IllegalStateException();
     }
 
     /** {@inheritDoc} */
     @Override public Row getCopy() {
-        assert false;
-
-        return null;
+        throw new IllegalStateException();
     }
 
     /** {@inheritDoc} */
     @Override public void setDeleted(boolean deleted) {
-        assert false;
+        throw new IllegalStateException();
     }
 
     /** {@inheritDoc} */
     @Override public long getKey() {
-        assert false;
-
-        return 0;
+        throw new IllegalStateException();
     }
 
     /** {@inheritDoc} */
     @Override public void setSessionId(int sesId) {
-        assert false;
+        throw new IllegalStateException();
     }
 
     /** {@inheritDoc} */
     @Override public void setVersion(int ver) {
-        assert false;
+        throw new IllegalStateException();
     }
 
     /**
@@ -469,4 +465,9 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
             throw new IllegalStateException();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public final int hashCode() {
+        throw new IllegalStateException();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a0c6ae54/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOffheap.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOffheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOffheap.java
index 2dd9f25..ee68431 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOffheap.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOffheap.java
@@ -170,7 +170,7 @@ public class GridH2KeyValueRowOffheap extends GridH2AbstractKeyValueRow {
             }
         }
         else
-            assert false : col;
+            throw new IllegalStateException("Column: " + col);
 
         Data data = Data.create(null, bytes);
 
@@ -378,4 +378,19 @@ public class GridH2KeyValueRowOffheap extends GridH2AbstractKeyValueRow {
     @Override protected void addOffheapRowId(SB sb) {
         sb.a('-').a(ptr);
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+
+        if (obj instanceof GridH2KeyValueRowOffheap) {
+            GridH2KeyValueRowOffheap row = (GridH2KeyValueRowOffheap)obj;
+
+            if (pointer() == row.pointer())
+                return true;
+        }
+
+        return false;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a0c6ae54/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapIndexScanTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapIndexScanTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapIndexScanTest.java
new file mode 100644
index 0000000..dbc8a65
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapIndexScanTest.java
@@ -0,0 +1,195 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+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 org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+
+/**
+ * Based scanCount with offheap index issue.
+ */
+public class IgniteCacheOffheapIndexScanTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static IgniteCache<Integer, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        CacheConfiguration<?,?> cacheCfg = new CacheConfiguration<>();
+
+        cacheCfg.setCacheMode(LOCAL);
+        cacheCfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+        cacheCfg.setSqlOnheapRowCacheSize(256);
+        cacheCfg.setIndexedTypes(
+            Integer.class, Person.class
+        );
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(1, false);
+
+        cache = grid(0).cache(null);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryPlan() throws Exception {
+        for (int i = 0 ; i < 1000; i++)
+            cache.put(i, new Person(i, "firstName" + i, "lastName" + i, i % 100));
+
+        final AtomicBoolean end = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                while(!end.get())
+                    cache.query(new SqlFieldsQuery("select _val from Person")).getAll();
+
+                return null;
+            }
+        }, 5);
+
+        for (int i = 0; i < 150; i++) {
+            String plan = (String)cache.query(new SqlFieldsQuery(
+                "explain analyze select count(*) from Person where salary = 50")).getAll().get(0).get(0);
+
+            assertTrue(plan, plan.contains("scanCount: 11 "));
+
+            Thread.sleep(100);
+        }
+
+        end.set(true);
+
+        fut.get();
+    }
+
+    /**
+     * Person record used for query test.
+     */
+    public static class Person implements Serializable {
+        /** Person ID. */
+        @QuerySqlField(index = true)
+        private int id;
+
+        /** Organization ID. */
+        @QuerySqlField(index = true)
+        private int orgId;
+
+        /** First name (not-indexed). */
+        @QuerySqlField
+        private String firstName;
+
+        /** Last name (not indexed). */
+        @QuerySqlField
+        private String lastName;
+
+        /** Salary. */
+        @QuerySqlField(index = true)
+        private double salary;
+
+        /**
+         * Constructs empty person.
+         */
+        public Person() {
+            // No-op.
+        }
+
+        /**
+         * Constructs person record that is not linked to any organization.
+         *
+         * @param id Person ID.
+         * @param firstName First name.
+         * @param lastName Last name.
+         * @param salary Salary.
+         */
+        public Person(int id, String firstName, String lastName, double salary) {
+            this(id, 0, firstName, lastName, salary);
+        }
+
+        /**
+         * Constructs person record.
+         *
+         * @param id Person ID.
+         * @param orgId Organization ID.
+         * @param firstName First name.
+         * @param lastName Last name.
+         * @param salary Salary.
+         */
+        public Person(int id, int orgId, String firstName, String lastName, double salary) {
+            this.id = id;
+            this.orgId = orgId;
+            this.firstName = firstName;
+            this.lastName = lastName;
+            this.salary = salary;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            return this == o || (o instanceof Person) && id == ((Person)o).id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "Person [firstName=" + firstName +
+                ", id=" + id +
+                ", orgId=" + orgId +
+                ", lastName=" + lastName +
+                ", salary=" + salary +
+                ']';
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a0c6ae54/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 3c0e424..c82fd79 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySel
 import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapIndexScanTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
@@ -100,6 +101,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheOffheapIndexScanTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
         suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);


[04/24] ignite git commit: IGNITE-3232 - Fixed the number of iterations to find an affinity key.

Posted by sb...@apache.org.
IGNITE-3232 - Fixed the number of iterations to find an affinity key.


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

Branch: refs/heads/ignite-3038
Commit: c10097e0a89ae2c3e0c29283243e96d19281ee3b
Parents: 0e4ef3b
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Jun 2 22:52:39 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Jun 3 19:47:13 2016 +0300

----------------------------------------------------------------------
 .../spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c10097e0/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 036dfe6..9ef9846 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -334,7 +334,8 @@ class IgniteRDD[K, V] (
     private def affinityKeyFunc(value: V, node: ClusterNode): IgniteUuid = {
         val aff = ic.ignite().affinity[IgniteUuid](cacheName)
 
-        Stream.from(1, 1000).map(_ \u21d2 IgniteUuid.randomUuid()).find(node == null || aff.mapKeyToNode(_).eq(node))
+        Stream.from(1, Math.max(1000, aff.partitions() * 2))
+            .map(_ \u21d2 IgniteUuid.randomUuid()).find(node == null || aff.mapKeyToNode(_).eq(node))
             .getOrElse(IgniteUuid.randomUuid())
     }
 }


[23/24] ignite git commit: Merge remote-tracking branch 'remotes/community/gridgain-7.5.25' into ignite-3038

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/community/gridgain-7.5.25' into ignite-3038


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

Branch: refs/heads/ignite-3038
Commit: bcd401128413dadd49968c3a4953b3a62d980c97
Parents: c0e8c42 ec5706f
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 10 09:07:40 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 10 09:07:40 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  |  36 +-
 .../java/org/apache/ignite/igfs/IgfsPath.java   |  26 +-
 .../org/apache/ignite/igfs/IgfsPathSummary.java |  32 +-
 .../ignite/internal/binary/BinaryUtils.java     |  16 +
 .../GridClientConnectionManagerAdapter.java     |  25 +-
 .../connection/GridClientNioTcpConnection.java  |   3 +
 .../GridClientOptimizedMarshaller.java          |   4 +-
 .../GridClientZipOptimizedMarshaller.java       | 162 ++++++
 .../impl/GridTcpRouterNioListenerAdapter.java   |  11 +-
 .../internal/cluster/ClusterGroupAdapter.java   |   6 +
 .../ignite/internal/cluster/ClusterGroupEx.java |  14 +-
 .../processors/cache/CacheObjectContext.java    |   3 +
 .../internal/processors/igfs/IgfsAsyncImpl.java |   6 -
 .../processors/igfs/IgfsBlockLocationImpl.java  |  87 ++-
 .../internal/processors/igfs/IgfsContext.java   |  14 +-
 .../processors/igfs/IgfsDataManager.java        |  89 ++-
 .../processors/igfs/IgfsDeleteWorker.java       |  42 --
 .../ignite/internal/processors/igfs/IgfsEx.java |   9 -
 .../internal/processors/igfs/IgfsFileImpl.java  |  34 +-
 .../internal/processors/igfs/IgfsImpl.java      | 377 +++++-------
 .../processors/igfs/IgfsInputStreamImpl.java    |   6 +-
 ...zySecondaryFileSystemPositionedReadable.java |  77 +++
 .../processors/igfs/IgfsMetaManager.java        | 569 ++++++++++++-------
 .../processors/igfs/IgfsNodePredicate.java      |  80 +++
 .../igfs/IgfsOutputStreamAdapter.java           | 265 ---------
 .../processors/igfs/IgfsOutputStreamImpl.java   | 558 +++++++++++-------
 .../internal/processors/igfs/IgfsProcessor.java |  37 +-
 .../internal/processors/igfs/IgfsUtils.java     |  57 +-
 .../igfs/client/IgfsClientAbstractCallable.java | 125 ++++
 .../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 +++
 .../meta/IgfsClientMetaIdsForPathCallable.java  |  65 +++
 .../meta/IgfsClientMetaInfoForPathCallable.java |  63 ++
 .../igfs/meta/IgfsMetaFileUnlockProcessor.java  |  69 ++-
 .../message/GridClientHandshakeRequest.java     |   4 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |  19 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  12 +-
 .../ignite/internal/util/IgniteUtils.java       |  40 ++
 .../ignite/internal/util/nio/GridNioServer.java |  10 +-
 .../ignite/internal/visor/cache/VisorCache.java |  56 +-
 .../visor/cache/VisorCachePartition.java        |  89 +++
 .../visor/cache/VisorCachePartitions.java       |  88 +++
 .../visor/cache/VisorCachePartitionsTask.java   | 152 +++++
 .../internal/visor/cache/VisorCacheV3.java      |  68 +--
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |   2 -
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  78 ++-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  64 ++-
 .../igfs/IgfsClientCacheSelfTest.java           | 139 -----
 .../igfs/IgfsDataManagerSelfTest.java           |  12 +-
 .../igfs/IgfsDualAsyncClientSelfTest.java       |  28 +
 .../igfs/IgfsDualSyncClientSelfTest.java        |  28 +
 .../processors/igfs/IgfsModesSelfTest.java      |   1 +
 .../processors/igfs/IgfsOneClientNodeTest.java  |   7 +-
 .../igfs/IgfsPrimaryClientSelfTest.java         |  30 +
 .../igfs/IgfsPrimaryRelaxedClientSelfTest.java  |  28 +
 .../igfs/IgfsProcessorValidationSelfTest.java   |  38 +-
 .../processors/igfs/IgfsSizeSelfTest.java       | 133 -----
 .../processors/igfs/IgfsTaskSelfTest.java       |   2 +-
 .../unsafe/GridOffheapSnapTreeSelfTest.java     |   2 +-
 .../multijvm/IgniteClusterProcessProxy.java     |   5 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  11 +-
 .../ignite/hadoop/util/BasicUserNameMapper.java |   4 +-
 .../shuffle/collections/HadoopMultimapBase.java |  90 ++-
 .../util/BasicUserNameMapperSelfTest.java       |  19 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   2 +-
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   1 +
 .../HadoopDefaultMapReducePlannerSelfTest.java  |   6 -
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  23 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |  17 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  | 195 +++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 .../org/apache/ignite/spark/IgniteRDD.scala     |  81 ++-
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |   9 +
 .../spark/JavaStandaloneIgniteRDDSelfTest.java  |   9 +-
 .../ignite/spark/EntityTestAllTypeFields.scala  |   7 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |  77 ++-
 86 files changed, 3907 insertions(+), 1606 deletions(-)
----------------------------------------------------------------------



[05/24] ignite git commit: IGNITE-3175 BigDecimal fields are not supported if query is executed from IgniteRDD: fix java.sql.Date mapping

Posted by sb...@apache.org.
IGNITE-3175 BigDecimal fields are not supported if query is executed from IgniteRDD: fix java.sql.Date mapping


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

Branch: refs/heads/ignite-3038
Commit: 4506c215b43ffb5e6319c318196e713c8c5aefbd
Parents: c10097e
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Jun 2 16:56:20 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Jun 3 19:49:59 2016 +0300

----------------------------------------------------------------------
 .../src/main/scala/org/apache/ignite/spark/IgniteRDD.scala  | 3 ++-
 .../ignite/spark/JavaStandaloneIgniteRDDSelfTest.java       | 9 ++++++++-
 .../org/apache/ignite/spark/EntityTestAllTypeFields.scala   | 7 +++++--
 3 files changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4506c215/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 9ef9846..fa96212 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -315,9 +315,10 @@ class IgniteRDD[K, V] (
         case "java.lang.Long" \u21d2 LongType
         case "java.lang.Float" \u21d2 FloatType
         case "java.lang.Double" \u21d2 DoubleType
-        case "java.math.BigDecimal" \u21d2 DecimalType.SYSTEM_DEFAULT
+        case "java.math.BigDecimal" \u21d2 DataTypes.createDecimalType()
         case "java.lang.String" \u21d2 StringType
         case "java.util.Date" \u21d2 DateType
+        case "java.sql.Date" \u21d2 DateType
         case "java.sql.Timestamp" \u21d2 TimestampType
         case "[B" \u21d2 BinaryType
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4506c215/modules/spark/src/test/java/org/apache/ignite/spark/JavaStandaloneIgniteRDDSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/java/org/apache/ignite/spark/JavaStandaloneIgniteRDDSelfTest.java b/modules/spark/src/test/java/org/apache/ignite/spark/JavaStandaloneIgniteRDDSelfTest.java
index e600c6c..343158b 100644
--- a/modules/spark/src/test/java/org/apache/ignite/spark/JavaStandaloneIgniteRDDSelfTest.java
+++ b/modules/spark/src/test/java/org/apache/ignite/spark/JavaStandaloneIgniteRDDSelfTest.java
@@ -39,6 +39,7 @@ import org.apache.spark.sql.DataFrame;
 import org.apache.spark.sql.Row;
 import scala.Tuple2;
 
+import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.util.List;
@@ -279,10 +280,16 @@ public class JavaStandaloneIgniteRDDSelfTest extends GridCommonAbstractTest {
                     assertTrue(String.format("+++ Fail on %s field", fieldName),
                         ((Comparable<BigDecimal>)val).compareTo((BigDecimal)res) == 0);
                 }
+                else if (val instanceof java.sql.Date)
+                    assertEquals(String.format("+++ Fail on %s field", fieldName),
+                        val.toString(), df.collect()[0].get(0).toString());
                 else if (val.getClass().isArray())
                     assertTrue(String.format("+++ Fail on %s field", fieldName), 1 <= df.count());
-                else
+                else {
+                    assertTrue(String.format("+++ Fail on %s field", fieldName), df.collect().length > 0);
+                    assertTrue(String.format("+++ Fail on %s field", fieldName), df.collect()[0].size() > 0);
                     assertEquals(String.format("+++ Fail on %s field", fieldName), val, df.collect()[0].get(0));
+                }
 
                 info(String.format("+++ Query on the filed: %s : %s passed", fieldName, f.getType().getSimpleName()));
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4506c215/modules/spark/src/test/scala/org/apache/ignite/spark/EntityTestAllTypeFields.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/scala/org/apache/ignite/spark/EntityTestAllTypeFields.scala b/modules/spark/src/test/scala/org/apache/ignite/spark/EntityTestAllTypeFields.scala
index a936091..36d8274 100644
--- a/modules/spark/src/test/scala/org/apache/ignite/spark/EntityTestAllTypeFields.scala
+++ b/modules/spark/src/test/scala/org/apache/ignite/spark/EntityTestAllTypeFields.scala
@@ -35,7 +35,9 @@ class EntityTestAllTypeFields(
     @ScalarCacheQuerySqlField(index = true) val dateVal: Date,
     @ScalarCacheQuerySqlField(index = true) val timestampVal: Timestamp,
     @ScalarCacheQuerySqlField(index = true) val byteArrVal: Array[Byte],
-    @ScalarCacheQuerySqlField(index = true) val bigDecVal: java.math.BigDecimal
+    @ScalarCacheQuerySqlField(index = true) val bigDecVal: java.math.BigDecimal,
+    @ScalarCacheQuerySqlField(index = true) val javaSqlDate: java.sql.Date
+
 ) extends Serializable {
     def this(
         i: Int
@@ -52,6 +54,7 @@ class EntityTestAllTypeFields(
             new Date(i),
             new Timestamp(i),
             Array(i.toByte, i.toByte),
-            new java.math.BigDecimal(i.toString))
+            new java.math.BigDecimal(i.toString),
+            new java.sql.Date(i))
     }
 }


[09/24] ignite git commit: IGNITE-3248: Optimized fileIds() usage.

Posted by sb...@apache.org.
IGNITE-3248: Optimized fileIds() usage.


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

Branch: refs/heads/ignite-3038
Commit: e409b67a14ed67025a4bdd5824d4cc3a02f4e920
Parents: b96afb2
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Sun Jun 5 21:04:03 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Sun Jun 5 21:04:03 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsImpl.java      | 18 +++---
 .../processors/igfs/IgfsInputStreamImpl.java    |  2 +-
 .../processors/igfs/IgfsMetaManager.java        | 34 ++++++++--
 .../meta/IgfsClientMetaIdsForPathCallable.java  | 65 ++++++++++++++++++++
 .../meta/IgfsClientMetaInfoForPathCallable.java | 63 +++++++++++++++++++
 .../processors/igfs/IgfsOneClientNodeTest.java  |  2 +-
 6 files changed, 167 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e409b67a/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 b7b3fac..9087ff0 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
@@ -660,7 +660,7 @@ public final class IgfsImpl implements IgfsEx {
                     return new IgfsFileImpl(path, info, data.groupBlockSize());
                 }
 
-                List<IgniteUuid> fileIds = meta.fileIds(path);
+                List<IgniteUuid> fileIds = meta.idsForPath(path);
 
                 IgniteUuid fileId = fileIds.get(fileIds.size() - 1);
 
@@ -938,7 +938,8 @@ public final class IgfsImpl implements IgfsEx {
                             }
                         }
                     }
-                } else if (mode == PRIMARY) {
+                }
+                else if (mode == PRIMARY) {
                     checkConflictWithPrimary(path);
 
                     throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
@@ -993,7 +994,7 @@ public final class IgfsImpl implements IgfsEx {
                     return os;
                 }
 
-                IgfsEntryInfo info = meta.info(meta.fileId(path));
+                IgfsEntryInfo info = meta.infoForPath(path);
 
                 if (info == null) {
                     checkConflictWithPrimary(path);
@@ -1144,7 +1145,7 @@ public final class IgfsImpl implements IgfsEx {
                     return new IgfsEventAwareOutputStream(path, desc.info(), bufferSize(bufSize), mode, batch);
                 }
 
-                final List<IgniteUuid> ids = meta.fileIds(path);
+                final List<IgniteUuid> ids = meta.idsForPath(path);
 
                 final IgniteUuid id = ids.get(ids.size() - 1);
 
@@ -1259,8 +1260,7 @@ public final class IgfsImpl implements IgfsEx {
                 IgfsMode mode = resolveMode(path);
 
                 // Check memory first.
-                IgniteUuid fileId = meta.fileId(path);
-                IgfsEntryInfo info = meta.info(fileId);
+                IgfsEntryInfo info = meta.infoForPath(path);
 
                 if (info == null && mode != PRIMARY) {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
@@ -1469,7 +1469,7 @@ public final class IgfsImpl implements IgfsEx {
     @Nullable private FileDescriptor getFileDescriptor(IgfsPath path) throws IgniteCheckedException {
         assert path != null;
 
-        List<IgniteUuid> ids = meta.fileIds(path);
+        List<IgniteUuid> ids = meta.idsForPath(path);
 
         IgfsEntryInfo fileInfo = meta.info(ids.get(ids.size() - 1));
 
@@ -1645,13 +1645,13 @@ public final class IgfsImpl implements IgfsEx {
 
         switch (mode) {
             case PRIMARY:
-                info = meta.info(meta.fileId(path));
+                info = meta.infoForPath(path);
 
                 break;
 
             case DUAL_SYNC:
             case DUAL_ASYNC:
-                info = meta.info(meta.fileId(path));
+                info = meta.infoForPath(path);
 
                 if (info == null) {
                     try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e409b67a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
index 447be93..de7071a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
@@ -319,7 +319,7 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
                 }
 
                 // Safety to ensure no orphaned data blocks exist in case file was concurrently deleted.
-               if (!meta.exists(fileInfo.id()))
+                if (!meta.exists(fileInfo.id()))
                     data.delete(fileInfo);
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e409b67a/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 3295249..fa748f8 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
@@ -49,6 +49,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable;
+import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable;
+import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileLockProcessor;
@@ -2097,6 +2099,28 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Get info for the given path.
+     *
+     * @param path Path.
+     * @return Info.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable public IgfsEntryInfo infoForPath(IgfsPath path) throws IgniteCheckedException {
+        return client ? runClientTask(new IgfsClientMetaInfoForPathCallable(cfg.getName(), path)) : info(fileId(path));
+    }
+
+    /**
+     * Get IDs for the given path.
+     *
+     * @param path Path.
+     * @return IDs.
+     * @throws IgniteCheckedException If failed.
+     */
+    public List<IgniteUuid> idsForPath(IgfsPath path) throws IgniteCheckedException {
+        return client ? runClientTask(new IgfsClientMetaIdsForPathCallable(cfg.getName(), path)) : fileIds(path);
+    }
+
+    /**
      * Open file in DUAL mode.
      *
      * @param fs Secondary file system.
@@ -2106,15 +2130,14 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If input stream open has failed.
      */
     public IgfsSecondaryInputStreamDescriptor openDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
-        final int bufSize)
-        throws IgniteCheckedException {
+        final int bufSize) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 assert fs != null;
                 assert path != null;
 
                 // First, try getting file info without any transactions and synchronization.
-                IgfsEntryInfo info = info(fileId(path));
+                IgfsEntryInfo info = infoForPath(path);
 
                 if (info != null) {
                     if (!info.isFile())
@@ -2176,7 +2199,7 @@ public class IgfsMetaManager extends IgfsManager {
         if (busyLock.enterBusy()) {
             try {
                 // First, try getting file info without any transactions and synchronization.
-                IgfsEntryInfo info = info(fileId(path));
+                IgfsEntryInfo info = infoForPath(path);
 
                 if (info != null)
                     return info;
@@ -2652,10 +2675,9 @@ public class IgfsMetaManager extends IgfsManager {
             List<List<IgniteUuid>> pathIds = new ArrayList<>(paths.length);
 
             for (IgfsPath path : paths)
-                pathIds.add(fileIds(path));
+                pathIds.add(idsForPath(path));
 
             // Start pessimistic.
-
             try (IgniteInternalTx tx = startTx()) {
                 // Lock the very first existing parents and possibly the leaf as well.
                 Map<IgfsPath, IgfsPath> pathToParent = new HashMap<>();

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/e409b67a/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 3bfd372..c5f85bc 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
@@ -128,6 +128,6 @@ 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.");
     }
 }
\ No newline at end of file