You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dm...@apache.org on 2016/03/21 13:43:22 UTC

[01/17] ignite git commit: IGNITE-2853 - Fixed cancellation of the job that depends on a service

Repository: ignite
Updated Branches:
  refs/heads/ignite-2849 dd7f5d6e5 -> 4846b21f5


IGNITE-2853 - Fixed cancellation of the job that depends on a service


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

Branch: refs/heads/ignite-2849
Commit: d3420e6bc5e833a6eb1daaad25b11843f97328d5
Parents: 69d1f4b
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Mar 16 22:21:24 2016 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Mar 16 22:21:24 2016 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../ComputeJobCancelWithServiceSelfTest.java    | 154 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 3 files changed, 157 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d3420e6b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 5d8daf6..8df89f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -849,6 +849,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             startProcessor(createComponent(IgniteCacheObjectProcessor.class, ctx));
             startProcessor(new GridCacheProcessor(ctx));
             startProcessor(new GridQueryProcessor(ctx));
+            startProcessor(new GridServiceProcessor(ctx));
             startProcessor(new GridTaskSessionProcessor(ctx));
             startProcessor(new GridJobProcessor(ctx));
             startProcessor(new GridTaskProcessor(ctx));
@@ -860,7 +861,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             startProcessor((GridProcessor)(cfg.isPeerClassLoadingEnabled() ?
                 IgniteComponentType.HADOOP.create(ctx, true): // No-op when peer class loading is enabled.
                 IgniteComponentType.HADOOP.createIfInClassPath(ctx, cfg.getHadoopConfiguration() != null)));
-            startProcessor(new GridServiceProcessor(ctx));
             startProcessor(new DataStructuresProcessor(ctx));
             startProcessor(createComponent(PlatformProcessor.class, ctx));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d3420e6b/modules/core/src/test/java/org/apache/ignite/internal/ComputeJobCancelWithServiceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ComputeJobCancelWithServiceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ComputeJobCancelWithServiceSelfTest.java
new file mode 100644
index 0000000..2718ed9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/ComputeJobCancelWithServiceSelfTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskFuture;
+import org.apache.ignite.compute.ComputeTaskSplitAdapter;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+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;
+
+/**
+ * Test cancellation of a job that depends on service.
+ */
+public class ComputeJobCancelWithServiceSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJobCancel() throws Exception {
+        Ignite server = startGrid("server");
+
+        server.services().deployNodeSingleton("my-service", new MyService());
+
+        Ignition.setClientMode(true);
+
+        Ignite client = startGrid("client");
+
+        IgniteCompute compute = client.compute().withAsync();
+
+        compute.execute(new MyTask(), null);
+
+        ComputeTaskFuture<Integer> fut = compute.future();
+
+        Thread.sleep(3000);
+
+        server.close();
+
+        assertEquals(42, fut.get().intValue());
+    }
+
+    /** */
+    private static class MyService implements Service {
+        /** */
+        private volatile boolean cancelled;
+
+        /** {@inheritDoc} */
+        @Override public void init(ServiceContext ctx) throws Exception {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute(ServiceContext ctx) throws Exception {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel(ServiceContext ctx) {
+            cancelled = true;
+        }
+
+        /**
+         * @return Response.
+         */
+        public int hello() {
+            assertFalse("Service already cancelled!", cancelled);
+
+            return 42;
+        }
+    }
+
+    /** */
+    private static class MyTask extends ComputeTaskSplitAdapter<Object, Integer> {
+        /** {@inheritDoc} */
+        @Override protected Collection<? extends ComputeJob> split(int gridSize, Object arg) {
+            return Collections.singletonList(new ComputeJobAdapter() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override
+                public Object execute() throws IgniteException {
+                    MyService svc = ignite.services().service("my-service");
+
+                    while (!isCancelled()) {
+                        try {
+                            Thread.sleep(1000);
+
+                            svc.hello();
+                        }
+                        catch (InterruptedException e) {
+                            // No-op.
+                        }
+                    }
+
+                    assertTrue(isCancelled());
+
+                    return svc.hello();
+                }
+            });
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer reduce(List<ComputeJobResult> results) {
+            assertEquals(1, results.size());
+
+            return results.get(0).getData();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d3420e6b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index 6233bab..a8d6e5c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import java.util.Set;
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.ComputeJobCancelWithServiceSelfTest;
 import org.apache.ignite.internal.GridCommunicationSelfTest;
 import org.apache.ignite.internal.GridDiscoveryEventSelfTest;
 import org.apache.ignite.internal.GridDiscoverySelfTest;
@@ -114,6 +115,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteUpdateNotifierPerClusterSettingSelfTest.class);
         suite.addTestSuite(GridLocalEventListenerSelfTest.class);
         suite.addTestSuite(IgniteTopologyPrintFormatSelfTest.class);
+        suite.addTestSuite(ComputeJobCancelWithServiceSelfTest.class);
 
         // Managed Services.
         suite.addTestSuite(GridServiceProcessorSingleNodeSelfTest.class);


[16/17] ignite git commit: Merge remote-tracking branch 'remotes/apache-main/master' into ignite-2849

Posted by dm...@apache.org.
Merge remote-tracking branch 'remotes/apache-main/master' into ignite-2849


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

Branch: refs/heads/ignite-2849
Commit: 9d5dbd3159cb508296aaa4cf239aa305d43856f8
Parents: dd7f5d6 69f526a
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Mar 21 15:32:32 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Mar 21 15:32:32 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  |   47 +
 .../java/org/apache/ignite/igfs/IgfsPath.java   |   32 +-
 .../apache/ignite/internal/IgniteKernal.java    |    2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   10 +-
 .../ignite/internal/binary/BinaryContext.java   |   43 +
 .../ignite/internal/binary/BinaryUtils.java     |   37 +
 .../managers/communication/GridIoManager.java   |   10 +
 .../GridCachePartitionExchangeManager.java      |    3 +
 .../processors/cache/GridCacheUtils.java        |   14 +-
 .../IgfsColocatedMetadataAffinityKeyMapper.java |   47 +
 .../processors/igfs/IgfsDataManager.java        |  103 +-
 .../processors/igfs/IgfsDeleteWorker.java       |   27 +-
 .../processors/igfs/IgfsDirectoryInfo.java      |  235 ++
 .../internal/processors/igfs/IgfsEntryInfo.java |  340 +++
 .../processors/igfs/IgfsFileAffinityRange.java  |   32 +-
 .../internal/processors/igfs/IgfsFileImpl.java  |   15 +-
 .../internal/processors/igfs/IgfsFileInfo.java  |  535 +----
 .../internal/processors/igfs/IgfsFileMap.java   |   41 +-
 .../processors/igfs/IgfsFileWorkerBatch.java    |   75 +-
 .../IgfsFileWorkerBatchCancelledException.java  |   51 +
 .../igfs/IgfsFragmentizerManager.java           |  181 +-
 .../internal/processors/igfs/IgfsImpl.java      |   83 +-
 .../processors/igfs/IgfsInputStreamAdapter.java |    5 +-
 .../processors/igfs/IgfsInputStreamImpl.java    |   32 +-
 .../processors/igfs/IgfsIpcHandler.java         |   16 +-
 .../processors/igfs/IgfsListingEntry.java       |   37 +-
 .../processors/igfs/IgfsMetaManager.java        | 2182 ++++++------------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   19 +-
 .../internal/processors/igfs/IgfsPathIds.java   |  291 +++
 .../processors/igfs/IgfsPathsCreateResult.java  |   77 +
 .../IgfsSecondaryInputStreamDescriptor.java     |    6 +-
 .../IgfsSecondaryOutputStreamDescriptor.java    |    9 +-
 .../internal/processors/igfs/IgfsUtils.java     |  148 +-
 .../meta/IgfsMetaDirectoryCreateProcessor.java  |  170 ++
 .../IgfsMetaDirectoryListingAddProcessor.java   |  131 ++
 ...IgfsMetaDirectoryListingRemoveProcessor.java |  134 ++
 ...gfsMetaDirectoryListingReplaceProcessor.java |  130 ++
 .../igfs/meta/IgfsMetaFileCreateProcessor.java  |  164 ++
 .../igfs/meta/IgfsMetaFileLockProcessor.java    |  107 +
 .../meta/IgfsMetaFileRangeDeleteProcessor.java  |  111 +
 .../meta/IgfsMetaFileRangeUpdateProcessor.java  |  120 +
 .../meta/IgfsMetaFileReserveSpaceProcessor.java |  120 +
 .../igfs/meta/IgfsMetaFileUnlockProcessor.java  |  103 +
 .../igfs/meta/IgfsMetaUpdatePathProcessor.java  |  102 +
 .../meta/IgfsMetaUpdatePropertiesProcessor.java |  121 +
 .../igfs/meta/IgfsMetaUpdateTimesProcessor.java |  113 +
 .../internal/processors/igfs/package-info.java  |    2 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   21 +-
 .../ignite/internal/util/nio/GridNioServer.java |   52 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    8 +
 .../tcp/TcpCommunicationSpiMBean.java           |    8 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |    4 +-
 .../ComputeJobCancelWithServiceSelfTest.java    |  154 ++
 .../processors/igfs/IgfsAbstractSelfTest.java   |   72 +-
 .../igfs/IgfsDataManagerSelfTest.java           |   55 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   17 +-
 .../processors/igfs/IgfsFileInfoSelfTest.java   |   31 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   63 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   57 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   35 +-
 .../processors/igfs/IgfsStartCacheTest.java     |    9 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |   25 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    3 +-
 .../testsuites/IgniteKernalSelfTestSuite.java   |    2 +
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   32 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   11 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   10 +-
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |   21 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   26 +-
 69 files changed, 4539 insertions(+), 2590 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9d5dbd31/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------


[13/17] ignite git commit: IGNITE-2861: IGFS: Moved metadata processors into separate top-level classes to simplify code. Also cleaned up IgfsMetaManager from unused code.

Posted by dm...@apache.org.
IGNITE-2861: IGFS: Moved metadata processors into separate top-level classes to simplify code. Also cleaned up IgfsMetaManager from unused code.


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

Branch: refs/heads/ignite-2849
Commit: 865e376ad469bf9929b9f6f98ff0882e44c951a8
Parents: c506c44
Author: thatcoach <pp...@list.ru>
Authored: Sat Mar 19 21:13:35 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Sat Mar 19 21:13:35 2016 +0300

----------------------------------------------------------------------
 .../igfs/IgfsFragmentizerManager.java           |  141 +--
 .../processors/igfs/IgfsMetaManager.java        | 1142 ++----------------
 .../internal/processors/igfs/IgfsPathIds.java   |    2 +-
 .../meta/IgfsMetaDirectoryCreateProcessor.java  |  117 ++
 .../IgfsMetaDirectoryListingAddProcessor.java   |   92 ++
 ...IgfsMetaDirectoryListingRemoveProcessor.java |   89 ++
 ...gfsMetaDirectoryListingReplaceProcessor.java |   84 ++
 .../igfs/meta/IgfsMetaFileCreateProcessor.java  |  110 ++
 .../igfs/meta/IgfsMetaFileLockProcessor.java    |   63 +
 .../meta/IgfsMetaFileRangeDeleteProcessor.java  |   74 ++
 .../meta/IgfsMetaFileRangeUpdateProcessor.java  |   81 ++
 .../meta/IgfsMetaFileReserveSpaceProcessor.java |   75 ++
 .../igfs/meta/IgfsMetaFileUnlockProcessor.java  |   60 +
 .../igfs/meta/IgfsMetaUpdatePathProcessor.java  |   66 +
 .../meta/IgfsMetaUpdatePropertiesProcessor.java |   78 ++
 .../igfs/meta/IgfsMetaUpdateTimesProcessor.java |   68 ++
 16 files changed, 1155 insertions(+), 1187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
index 194a8ac..99e7cd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
@@ -26,13 +26,14 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileRangeDeleteProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileRangeUpdateProcessor;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridSpinReadWriteLock;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.LT;
-import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -40,13 +41,6 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -294,7 +288,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
                     case RANGE_STATUS_INITIAL: {
                         // Mark range as moving.
                         updated = igfsCtx.meta().updateInfo(
-                            fileId, new RangeUpdateProcessor(range, RANGE_STATUS_MOVING));
+                            fileId, new IgfsMetaFileRangeUpdateProcessor(range, RANGE_STATUS_MOVING));
 
                         if (updated == null) {
                             igfsCtx.data().cleanBlocks(fileInfo, range, true);
@@ -311,7 +305,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
 
                         // Mark range as moved.
                         updated = igfsCtx.meta().updateInfo(
-                            fileId, new RangeUpdateProcessor(range, RANGE_STATUS_MOVED));
+                            fileId, new IgfsMetaFileRangeUpdateProcessor(range, RANGE_STATUS_MOVED));
 
                         if (updated == null) {
                             igfsCtx.data().cleanBlocks(fileInfo, range, true);
@@ -327,7 +321,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
                         igfsCtx.data().cleanBlocks(fileInfo, range, false);
 
                         // Remove range from map.
-                        updated = igfsCtx.meta().updateInfo(fileId, new RangeDeleteProcessor(range));
+                        updated = igfsCtx.meta().updateInfo(fileId, new IgfsMetaFileRangeDeleteProcessor(range));
 
                         if (updated == null)
                             igfsCtx.data().cleanBlocks(fileInfo, range, true);
@@ -343,131 +337,6 @@ public class IgfsFragmentizerManager extends IgfsManager {
     }
 
     /**
-     * Update range processor.
-     */
-    private static class RangeUpdateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Range. */
-        private IgfsFileAffinityRange range;
-
-        /** Status. */
-        private int status;
-
-        /**
-         * Constructor.
-         */
-        public RangeUpdateProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param range Range.
-         * @param status Status.
-         */
-        public RangeUpdateProcessor(IgfsFileAffinityRange range, int status) {
-            this.range = range;
-            this.status = status;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo oldInfo = entry.getValue();
-
-            IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
-
-            newMap.updateRangeStatus(range, status);
-
-            IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
-
-            entry.setValue(newInfo);
-
-            return newInfo;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeObject(range);
-            out.writeInt(status);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            range = (IgfsFileAffinityRange)in.readObject();
-            status = in.readInt();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(RangeUpdateProcessor.class, this);
-        }
-    }
-
-    /**
-     * Delete range processor.
-     */
-    private static class RangeDeleteProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Range. */
-        private IgfsFileAffinityRange range;
-
-        /**
-         * Constructor.
-         */
-        public RangeDeleteProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param range Range.
-         */
-        public RangeDeleteProcessor(IgfsFileAffinityRange range) {
-            this.range = range;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo oldInfo = entry.getValue();
-
-            IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
-
-            newMap.deleteRange(range);
-
-            IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
-
-            entry.setValue(newInfo);
-
-            return newInfo;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeObject(range);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            range = (IgfsFileAffinityRange)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(RangeDeleteProcessor.class, this);
-        }
-    }
-
-    /**
      * Fragmentizer coordinator thread.
      */
     private class FragmentizerCoordinator extends GridWorker implements GridLocalEventListener, GridMessageListener {

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/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 d66d9be..1aa49ed 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,17 @@ 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.task.GridInternal;
+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;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileReserveSpaceProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileUnlockProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRemoveProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingReplaceProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdatePathProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdatePropertiesProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor;
 import org.apache.ignite.internal.util.GridLeanMap;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.lang.GridClosureException;
@@ -52,22 +62,15 @@ import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 
 import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
+import javax.cache.processor.EntryProcessorResult;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -89,7 +92,6 @@ import java.util.concurrent.CountDownLatch;
 /**
  * Cache based structure (meta data) manager.
  */
-@SuppressWarnings("all")
 public class IgfsMetaManager extends IgfsManager {
     /** Comparator for Id sorting. */
     private static final Comparator<IgniteUuid> PATH_ID_SORTING_COMPARATOR
@@ -161,6 +163,7 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("RedundantCast")
     @Override protected void onKernalStart0() throws IgniteCheckedException {
         metaCache = igfsCtx.kernalContext().cache().getOrStartCache(cfg.getMetaCacheName());
 
@@ -511,9 +514,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 assert fileId != null;
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     // Lock file ID for this transaction.
                     IgfsEntryInfo oldInfo = info(fileId);
 
@@ -532,9 +533,6 @@ public class IgfsMetaManager extends IgfsManager {
                 catch (GridClosureException e) {
                     throw U.cast(e);
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -593,12 +591,12 @@ public class IgfsMetaManager extends IgfsManager {
                                 throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not " +
                                     "found): " + fileId));
 
-                            if (!info.lockId().equals(oldInfo.lockId()))
+                            if (!F.eq(info.lockId(), oldInfo.lockId()))
                                 throw new IgniteCheckedException("Failed to unlock file (inconsistent file lock ID) " +
                                     "[fileId=" + fileId + ", lockId=" + info.lockId() + ", actualLockId=" +
                                     oldInfo.lockId() + ']');
 
-                            id2InfoPrj.invoke(fileId, new FileUnlockProcessor(modificationTime));
+                            id2InfoPrj.invoke(fileId, new IgfsMetaFileUnlockProcessor(modificationTime));
 
                             return null;
                         }
@@ -701,8 +699,6 @@ public class IgfsMetaManager extends IgfsManager {
         throws IgniteCheckedException {
         assert IgfsUtils.isRootOrTrashId(id);
 
-        long time = System.currentTimeMillis();
-
         IgfsEntryInfo info = IgfsUtils.createDirectory(id);
 
         IgfsEntryInfo oldInfo = id2InfoPrj.getAndPutIfAbsent(id, info);
@@ -898,9 +894,7 @@ public class IgfsMetaManager extends IgfsManager {
                 srcPathIds.addExistingIds(lockIds);
                 dstPathIds.addExistingIds(lockIds);
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     // Obtain the locks.
                     final Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
@@ -942,9 +936,6 @@ public class IgfsMetaManager extends IgfsManager {
                     // Set the new path to the info to simplify event creation:
                     return srcInfo.path(newPath);
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -956,41 +947,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Verify path integrity.
-     *
-     * @param path Path to verify.
-     * @param expIds Expected IDs for this path. Might contain additional elements, e.g. because they were created
-     *     on a child path.
-     * @param infos Locked infos.
-     * @return verification result.
-     */
-    private static boolean verifyPathIntegrity(IgfsPath path, List<IgniteUuid> expIds,
-        Map<IgniteUuid, IgfsEntryInfo> infos) {
-        List<String> pathParts = path.components();
-
-        assert pathParts.size() < expIds.size();
-
-        for (int i = 0; i < pathParts.size(); i++) {
-            IgniteUuid parentId = expIds.get(i);
-
-            // If parent ID is null, it doesn't exist.
-            if (parentId != null) {
-                IgfsEntryInfo parentInfo = infos.get(parentId);
-
-                // If parent info is null, it doesn't exist.
-                if (parentInfo != null) {
-                    if (parentInfo.hasChild(pathParts.get(i), expIds.get(i + 1)))
-                        continue;
-                }
-            }
-
-            return false;
-        }
-
-        return true;
-    }
-
-    /**
      * Move or rename file in existing transaction.
      *
      * @param fileId File ID to move or rename.
@@ -1069,6 +1025,7 @@ public class IgfsMetaManager extends IgfsManager {
      * elements moved to TRASH folder.
      * @throws IgniteCheckedException On error.
      */
+    @SuppressWarnings("RedundantCast")
     IgniteUuid format() throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
@@ -1076,9 +1033,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 IgniteUuid trashId = IgfsUtils.randomTrashId();
 
-                final IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     // NB: We may lock root because its id is less than any other id:
                     final IgfsEntryInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, trashId).get(IgfsUtils.ROOT_ID);
 
@@ -1097,7 +1052,7 @@ public class IgfsMetaManager extends IgfsManager {
                     IgfsEntryInfo newInfo = IgfsUtils.createDirectory(
                         IgniteUuid.randomUuid(),
                         transferListing,
-                        (Map<String,String>)null
+                        (Map<String, String>) null
                     );
 
                     createNewEntry(newInfo, trashId, newInfo.id().toString());
@@ -1112,9 +1067,6 @@ public class IgfsMetaManager extends IgfsManager {
 
                     return newInfo.id();
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1127,9 +1079,8 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Move path to the trash directory.
      *
-     * @param parentId Parent ID.
-     * @param pathName Path name.
-     * @param pathId Path ID.
+     * @param path Path.
+     * @param recursive Recursive flag.
      * @return ID of an entry located directly under the trash directory.
      * @throws IgniteCheckedException If failed.
      */
@@ -1159,9 +1110,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 allIds.add(trashId);
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     // Lock participants.
                     Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
 
@@ -1197,9 +1146,6 @@ public class IgfsMetaManager extends IgfsManager {
 
                     return victimId;
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1220,9 +1166,9 @@ public class IgfsMetaManager extends IgfsManager {
      * @return ID of an entry located directly under the trash directory.
      * @throws IgniteCheckedException If failed.
      */
+    @SuppressWarnings("RedundantCast")
     @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id,
-        IgniteUuid trashId)
-        throws IgniteCheckedException {
+        IgniteUuid trashId) throws IgniteCheckedException {
         validTxState(true);
 
         IgniteUuid resId;
@@ -1268,7 +1214,7 @@ public class IgfsMetaManager extends IgfsManager {
                 // Remove listing entries from root.
                 for (Map.Entry<String, IgfsListingEntry> entry : transferListing.entrySet())
                     id2InfoPrj.invoke(IgfsUtils.ROOT_ID,
-                        new ListingRemoveProcessor(entry.getKey(), entry.getValue().fileId()));
+                        new IgfsMetaDirectoryListingRemoveProcessor(entry.getKey(), entry.getValue().fileId()));
 
                 resId = newInfo.id();
             }
@@ -1304,9 +1250,7 @@ public class IgfsMetaManager extends IgfsManager {
                 assert listing != null;
                 validTxState(false);
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     Collection<IgniteUuid> res = new HashSet<>();
 
                     // Obtain all necessary locks in one hop.
@@ -1367,9 +1311,6 @@ public class IgfsMetaManager extends IgfsManager {
 
                     return res;
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1395,17 +1336,13 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 validTxState(false);
 
-                IgniteInternalTx tx = startTx();
-
-                try {
-                    boolean res = false;
-
+                try (IgniteInternalTx tx = startTx()) {
                     Map<IgniteUuid, IgfsEntryInfo> infos = lockIds(parentId, id);
 
                     IgfsEntryInfo victim = infos.get(id);
 
                     if (victim == null)
-                        return res;
+                        return false;
 
                     assert victim.isDirectory() || IgfsUtils.DELETE_LOCK_ID.equals(victim.lockId()) :
                         " isDir: " + victim.isDirectory() + ", lockId: " + victim.lockId();
@@ -1419,19 +1356,16 @@ public class IgfsMetaManager extends IgfsManager {
                         IgfsListingEntry childEntry = parentInfo.listing().get(name);
 
                         if (childEntry != null)
-                            id2InfoPrj.invoke(parentId, new ListingRemoveProcessor(name, id));
+                            id2InfoPrj.invoke(parentId, new IgfsMetaDirectoryListingRemoveProcessor(name, id));
 
                         id2InfoPrj.remove(id);
 
-                        res = true;
-                    }
+                        tx.commit();
 
-                    tx.commit();
+                        return true;
+                    }
 
-                    return res;
-                }
-                finally {
-                    tx.close();
+                    return false;
                 }
             }
             finally {
@@ -1499,7 +1433,7 @@ public class IgfsMetaManager extends IgfsManager {
             if (oldInfo == null)
                 return null;
 
-            return invokeAndGet(fileId, new UpdatePropertiesProcessor(props));
+            return invokeAndGet(fileId, new IgfsMetaUpdatePropertiesProcessor(props));
         }
         catch (GridClosureException e) {
             throw U.cast(e);
@@ -1520,18 +1454,13 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 validTxState(false);
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     IgfsEntryInfo info = updatePropertiesNonTx(fileId, props);
 
                     tx.commit();
 
                     return info;
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1560,16 +1489,15 @@ public class IgfsMetaManager extends IgfsManager {
                 if (log.isDebugEnabled())
                     log.debug("Reserve file space [path=" + path + ", id=" + fileId + ']');
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                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 + ']');
 
-                    IgfsEntryInfo newInfo = invokeAndGet(fileId, new FileReserveSpaceProcessor(space, affRange));
+                    IgfsEntryInfo newInfo =
+                        invokeAndGet(fileId, new IgfsMetaFileReserveSpaceProcessor(space, affRange));
 
                     tx.commit();
 
@@ -1578,9 +1506,6 @@ public class IgfsMetaManager extends IgfsManager {
                 catch (GridClosureException e) {
                     throw U.cast(e);
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1610,9 +1535,7 @@ public class IgfsMetaManager extends IgfsManager {
                 if (log.isDebugEnabled())
                     log.debug("Update file info [fileId=" + fileId + ", proc=" + proc + ']');
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     // Lock file ID for this transaction.
                     IgfsEntryInfo oldInfo = info(fileId);
 
@@ -1623,7 +1546,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     if (newInfo == null)
                         throw fsException("Failed to update file info with null value" +
-                            " [oldInfo=" + oldInfo + ", newInfo=" + newInfo + ", proc=" + proc + ']');
+                            " [oldInfo=" + oldInfo + ", newInfo=null, proc=" + proc + ']');
 
                     if (!oldInfo.id().equals(newInfo.id()))
                         throw fsException("Failed to update file info (file IDs differ)" +
@@ -1640,9 +1563,6 @@ public class IgfsMetaManager extends IgfsManager {
                 catch (GridClosureException e) {
                     throw U.cast(e);
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1679,9 +1599,7 @@ public class IgfsMetaManager extends IgfsManager {
                     assert lockIds.size() == pathIds.count();
 
                     // Start TX.
-                    IgniteInternalTx tx = startTx();
-
-                    try {
+                    try (IgniteInternalTx tx = startTx()) {
                         final Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                         if (!pathIds.verifyIntegrity(lockInfos))
@@ -1710,9 +1628,6 @@ public class IgfsMetaManager extends IgfsManager {
                         // We are done.
                         return true;
                     }
-                    finally {
-                        tx.close();
-                    }
                 }
                 finally {
                     busyLock.leaveBusy();
@@ -1735,18 +1650,13 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 validTxState(false);
 
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     Object prev = val != null ? metaCache.getAndPut(sampling, val) : metaCache.getAndRemove(sampling);
 
                     tx.commit();
 
                     return !F.eq(prev, val);
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1795,7 +1705,7 @@ public class IgfsMetaManager extends IgfsManager {
             throw fsException("Failed to create new metadata entry due to ID conflict: " + info.id());
 
         if (parentId != null)
-            id2InfoPrj.invoke(parentId, new ListingAddProcessor(name, new IgfsListingEntry(info)));
+            id2InfoPrj.invoke(parentId, new IgfsMetaDirectoryListingAddProcessor(name, new IgfsListingEntry(info)));
     }
 
     /**
@@ -1812,8 +1722,8 @@ public class IgfsMetaManager extends IgfsManager {
         IgniteUuid destId, String destName) throws IgniteCheckedException {
         validTxState(true);
 
-        id2InfoPrj.invoke(srcId, new ListingRemoveProcessor(srcName, entry.fileId()));
-        id2InfoPrj.invoke(destId, new ListingAddProcessor(destName, entry));
+        id2InfoPrj.invoke(srcId, new IgfsMetaDirectoryListingRemoveProcessor(srcName, entry.fileId()));
+        id2InfoPrj.invoke(destId, new IgfsMetaDirectoryListingAddProcessor(destName, entry));
     }
 
     /**
@@ -1825,7 +1735,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     private IgfsEntryInfo invokeLock(IgniteUuid id, boolean delete) throws IgniteCheckedException {
-        return invokeAndGet(id, new FileLockProcessor(createFileLockId(delete)));
+        return invokeAndGet(id, new IgfsMetaFileLockProcessor(createFileLockId(delete)));
     }
 
     /**
@@ -1838,7 +1748,7 @@ public class IgfsMetaManager extends IgfsManager {
     private void invokeUpdatePath(IgniteUuid id, IgfsPath path) throws IgniteCheckedException {
         validTxState(true);
 
-        id2InfoPrj.invoke(id, new UpdatePathProcessor(path));
+        id2InfoPrj.invoke(id, new IgfsMetaUpdatePathProcessor(path));
     }
 
     /**
@@ -1853,7 +1763,11 @@ public class IgfsMetaManager extends IgfsManager {
         throws IgniteCheckedException {
         validTxState(true);
 
-        return id2InfoPrj.invoke(id, proc).get();
+        EntryProcessorResult<IgfsEntryInfo> res = id2InfoPrj.invoke(id, proc);
+
+        assert res != null;
+
+        return res.get();
     }
 
     /**
@@ -1985,6 +1899,8 @@ public class IgfsMetaManager extends IgfsManager {
                             );
 
                             // 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) {
@@ -1998,12 +1914,12 @@ public class IgfsMetaManager extends IgfsManager {
                                         path + ", fileId=" + oldId + ", lockId=" + oldInfo.lockId() + ']');
 
                                 id2InfoPrj.remove(oldId); // Remove the old one.
-                                id2InfoPrj.invoke(parentInfo.id(),
-                                    new ListingRemoveProcessor(path.name(), parentInfo.listing().get(path.name()).fileId()));
+                                id2InfoPrj.invoke(parentInfo.id(), new IgfsMetaDirectoryListingRemoveProcessor(
+                                    path.name(), parentInfo.listing().get(path.name()).fileId()));
 
                                 createNewEntry(newInfo, parentInfo.id(), path.name()); // Put new one.
 
-                                IgniteInternalFuture<?> delFut = igfsCtx.data().delete(oldInfo);
+                                igfsCtx.data().delete(oldInfo);
                             }
 
                             // Record CREATE event if needed.
@@ -2084,13 +2000,13 @@ public class IgfsMetaManager extends IgfsManager {
                             if (remainder > 0) {
                                 int blockIdx = (int)(len / blockSize);
 
-                                IgfsSecondaryFileSystemPositionedReadable reader = fs.open(path, bufSize);
+                                try (IgfsSecondaryFileSystemPositionedReadable reader = fs.open(path, bufSize)) {
+                                    IgniteInternalFuture<byte[]> fut =
+                                        igfsCtx.data().dataBlock(info, path, blockIdx, reader);
 
-                                try {
-                                    igfsCtx.data().dataBlock(info, path, blockIdx, reader).get();
-                                }
-                                finally {
-                                    reader.close();
+                                    assert fut != null;
+
+                                    fut.get();
                                 }
                             }
 
@@ -2366,12 +2282,8 @@ public class IgfsMetaManager extends IgfsManager {
                         fs.rename(src, dest);
 
                         // Rename was successful, perform compensation in the local file system.
-                        if (destInfo == null) {
-                            // Move and rename.
-                            assert destParentInfo != null;
-
+                        if (destInfo == null)
                             moveNonTx(srcInfo.id(), src.name(), srcParentInfo.id(), dest.name(), destParentInfo.id());
-                        }
                         else {
                             // Move.
                             if (destInfo.isFile())
@@ -2622,6 +2534,8 @@ public class IgfsMetaManager extends IgfsManager {
                         status.modificationTime()
                     );
 
+                assert parentInfo != null;
+
                 IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), components.get(i), curInfo);
 
                 if (oldId != null)
@@ -2669,13 +2583,9 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Result of task execution.
      * @throws IgniteCheckedException If failed.
      */
-    private <T> T synchronizeAndExecute(SynchronizationTask<T> task,
-        IgfsSecondaryFileSystem fs,
-        boolean strict,
-        @Nullable Collection<IgniteUuid> extraLockIds,
-        IgfsPath... paths)
-        throws IgniteCheckedException
-    {
+    @SuppressWarnings({"Contract", "ConstantConditions"})
+    private <T> T synchronizeAndExecute(SynchronizationTask<T> task, IgfsSecondaryFileSystem fs, boolean strict,
+        @Nullable Collection<IgniteUuid> extraLockIds, IgfsPath... paths) throws IgniteCheckedException {
         assert task != null;
         assert fs != null;
         assert paths != null && paths.length > 0;
@@ -2696,9 +2606,8 @@ public class IgfsMetaManager extends IgfsManager {
                 pathIds.add(fileIds(path));
 
             // Start pessimistic.
-            IgniteInternalTx tx = startTx();
 
-            try {
+            try (IgniteInternalTx tx = startTx()) {
                 // Lock the very first existing parents and possibly the leaf as well.
                 Map<IgfsPath, IgfsPath> pathToParent = new HashMap<>();
 
@@ -2864,34 +2773,12 @@ public class IgfsMetaManager extends IgfsManager {
                 else
                     throw e;
             }
-            finally {
-                tx.close();
-            }
         }
 
         return res;
     }
 
     /**
-     * Update cached value with closure.
-     *
-     * @param cache Cache projection to work with.
-     * @param key Key to retrieve/update the value for.
-     * @param c Closure to apply to cached value.
-     * @return {@code True} if value was stored in cache, {@code false} otherwise.
-     * @throws IgniteCheckedException If operation failed.
-     */
-    private <K, V> boolean putx(IgniteInternalCache<K, V> cache, K key, IgniteClosure<V, V> c)
-        throws IgniteCheckedException {
-        validTxState(true);
-
-        V oldVal = cache.get(key);
-        V newVal = c.apply(oldVal);
-
-        return newVal == null ? cache.remove(key) : cache.put(key, newVal);
-    }
-
-    /**
      * Check transaction is (not) started.
      *
      * @param inTx Expected transaction state.
@@ -2927,16 +2814,14 @@ public class IgfsMetaManager extends IgfsManager {
                 validTxState(false);
 
                 // Start pessimistic transaction.
-                IgniteInternalTx tx = startTx();
-
-                try {
+                try (IgniteInternalTx tx = startTx()) {
                     Map<IgniteUuid, IgfsEntryInfo> infoMap = lockIds(fileId, parentId);
 
                     IgfsEntryInfo fileInfo = infoMap.get(fileId);
 
                     if (fileInfo == null)
                         throw fsException(new IgfsPathNotFoundException("Failed to update times " +
-                                "(path was not found): " + fileName));
+                            "(path was not found): " + fileName));
 
                     IgfsEntryInfo parentInfo = infoMap.get(parentId);
 
@@ -2947,20 +2832,17 @@ public class IgfsMetaManager extends IgfsManager {
                     // Validate listing.
                     if (!parentInfo.hasChild(fileName, fileId))
                         throw fsException(new IgfsConcurrentModificationException("Failed to update times " +
-                                "(file concurrently modified): " + fileName));
+                            "(file concurrently modified): " + fileName));
 
                     assert parentInfo.isDirectory();
 
-                    id2InfoPrj.invoke(fileId, new UpdateTimesProcessor(
+                    id2InfoPrj.invoke(fileId, new IgfsMetaUpdateTimesProcessor(
                         accessTime == -1 ? fileInfo.accessTime() : accessTime,
                         modificationTime == -1 ? fileInfo.modificationTime() : modificationTime)
                     );
 
                     tx.commit();
                 }
-                finally {
-                    tx.close();
-                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -2980,7 +2862,7 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * @param msg Error message.
+     * @param err Unchecked exception.
      * @return Checked exception.
      */
     private static IgniteCheckedException fsException(IgfsException err) {
@@ -3011,365 +2893,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Path descriptor.
-     */
-    private static class PathDescriptor {
-        /** Path. */
-        private final IgfsPath path;
-
-        /** Resolved IDs. */
-        private final List<IgniteUuid> ids;
-
-        /** Parent path. */
-        private IgfsPath parentPath;
-
-        /** Parent path info. */
-        private IgfsEntryInfo parentInfo;
-
-        /**
-         * Constructor.
-         *
-         * @param path Path.
-         * @param ids Resolved path IDs.
-         * @param parentPath Parent path.
-         * @param parentInfo Parent info.
-         */
-        PathDescriptor(IgfsPath path, List<IgniteUuid> ids, IgfsPath parentPath, IgfsEntryInfo parentInfo) {
-            assert path != null;
-            assert ids != null && !ids.isEmpty();
-            assert parentPath == null && parentInfo == null || parentPath != null && parentInfo != null;
-            assert parentPath == null || parentPath != null && path.isSubDirectoryOf(parentPath);
-
-            this.path = path;
-            this.ids = ids;
-            this.parentPath = parentPath;
-            this.parentInfo = parentInfo;
-        }
-
-        /**
-         * Get resolved path ids.
-         *
-         * @return Path ids.
-         */
-        private Collection<IgniteUuid> ids() {
-            return ids;
-        }
-
-        /**
-         * Get path ID from the end. E.g. endId(1) will return the last element.
-         * @param i Element index from the end.
-         *
-         * @return Path ID from the end.
-         */
-        private IgniteUuid endId(int i) {
-            return ids.get(ids.size() - i);
-        }
-
-        /**
-         * Update ID with the given index.
-         *
-         * @param newParentPath New parent path.
-         * @param newParentInfo New parent info.
-         */
-        private void updateParent(IgfsPath newParentPath, IgfsEntryInfo newParentInfo) {
-            assert newParentPath != null;
-            assert newParentInfo != null;
-            assert path.isSubDirectoryOf(newParentPath);
-
-            parentPath = newParentPath;
-            parentInfo = newParentInfo;
-
-            ids.set(newParentPath.components().size(), newParentInfo.id());
-        }
-
-        /**
-         * Get parent path.
-         *
-         * @return Parent path.
-         */
-        private IgfsPath parentPath() {
-            return parentPath;
-        }
-
-        /**
-         * Get parent path info.
-         *
-         * @return Parent path info.
-         */
-        private IgfsEntryInfo parentInfo() {
-            return parentInfo;
-        }
-    }
-
-    /**
-     * Remove entry from directory listing.
-     */
-    @GridInternal
-    private static final class ListingRemoveProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** File name. */
-        private String fileName;
-
-        /** Expected ID. */
-        private IgniteUuid fileId;
-
-        /**
-         * Default constructor.
-         */
-        public ListingRemoveProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param fileName File name.
-         * @param fileId File ID.
-         */
-        public ListingRemoveProcessor(String fileName, IgniteUuid fileId) {
-            this.fileName = fileName;
-            this.fileId = fileId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo fileInfo = e.getValue();
-
-            assert fileInfo != null;
-            assert fileInfo.isDirectory();
-
-            Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
-
-            listing.putAll(fileInfo.listing());
-
-            IgfsListingEntry oldEntry = listing.get(fileName);
-
-            if (oldEntry == null || !oldEntry.fileId().equals(fileId))
-                throw new IgniteException("Directory listing doesn't contain expected file" +
-                    " [listing=" + listing + ", fileName=" + fileName + "]");
-
-            // Modify listing in-place.
-            listing.remove(fileName);
-
-            e.setValue(fileInfo.listing(listing));
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, fileName);
-            U.writeGridUuid(out, fileId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            fileName = U.readString(in);
-            fileId = U.readGridUuid(in);
-        }
-    }
-
-    /**
-     * Update directory listing closure.
-     */
-    @GridInternal
-    private static final class ListingAddProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** File name to add into parent listing. */
-        private String fileName;
-
-        /** File ID.*/
-        private IgfsListingEntry entry;
-
-        /**
-         * Empty constructor required for {@link Externalizable}.
-         *
-         */
-        public ListingAddProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructs update directory listing closure.
-         *
-         * @param fileName File name to add into parent listing.
-         * @param entry Listing entry to add or remove.
-         */
-        private ListingAddProcessor(String fileName, IgfsListingEntry entry) {
-            assert fileName != null;
-            assert entry != null;
-
-            this.fileName = fileName;
-            this.entry = entry;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args) {
-            IgfsEntryInfo fileInfo = e.getValue();
-
-            assert fileInfo.isDirectory();
-
-            Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
-
-            // Modify listing in-place.
-            IgfsListingEntry oldEntry = listing.put(fileName, entry);
-
-            if (oldEntry != null && !oldEntry.fileId().equals(entry.fileId()))
-                throw new IgniteException("Directory listing contains unexpected file" +
-                    " [listing=" + listing + ", fileName=" + fileName + ", entry=" + entry +
-                    ", oldEntry=" + oldEntry + ']');
-
-            e.setValue(fileInfo.listing(listing));
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, fileName);
-            out.writeObject(entry);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            fileName = U.readString(in);
-            entry = (IgfsListingEntry)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ListingAddProcessor.class, this);
-        }
-    }
-
-    /**
-     * Listing replace processor.
-     */
-    private static final class ListingReplaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Name. */
-        private String name;
-
-        /** New ID. */
-        private IgniteUuid id;
-
-        /**
-         * Constructor.
-         */
-        public ListingReplaceProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param name Name.
-         * @param id ID.
-         */
-        public ListingReplaceProcessor(String name, IgniteUuid id) {
-            this.name = name;
-            this.id = id;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo fileInfo = e.getValue();
-
-            assert fileInfo.isDirectory();
-
-            Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
-
-            // Modify listing in-place.
-            IgfsListingEntry oldEntry = listing.get(name);
-
-            if (oldEntry == null)
-                throw new IgniteException("Directory listing doesn't contain expected entry: " + name);
-
-            listing.put(name, new IgfsListingEntry(id, oldEntry.isDirectory()));
-
-            e.setValue(fileInfo.listing(listing));
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, name);
-            out.writeObject(id);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            name = U.readString(in);
-            id = (IgniteUuid)in.readObject();
-        }
-    }
-
-    /**
-     * Update path closure.
-     */
-    @GridInternal
-    private static final class UpdatePathProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** New path. */
-        private IgfsPath path;
-
-        /**
-         * @param path Path.
-         */
-        private UpdatePathProcessor(IgfsPath path) {
-            this.path = path;
-        }
-
-        /**
-         * Default constructor (required by Externalizable).
-         */
-        public UpdatePathProcessor() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args) {
-            IgfsEntryInfo info = e.getValue();
-
-            IgfsEntryInfo newInfo = info.path(path);
-
-            e.setValue(newInfo);
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeObject(path);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            path = (IgfsPath)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(UpdatePathProcessor.class, this);
-        }
-    }
-
-    /**
      * Append routine.
      *
      * @param path Path.
@@ -3409,9 +2932,7 @@ public class IgfsMetaManager extends IgfsManager {
                     pathIds.addSurrogateIds(lockIds);
 
                     // Start TX.
-                    IgniteInternalTx tx = startTx();
-
-                    try {
+                    try (IgniteInternalTx tx = startTx()) {
                         Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                         if (!pathIds.verifyIntegrity(lockInfos))
@@ -3458,9 +2979,6 @@ public class IgfsMetaManager extends IgfsManager {
                             return new T2<>(res.info(), res.parentId());
                         }
                     }
-                    finally {
-                        tx.close();
-                    }
                 }
                 finally {
                     busyLock.leaveBusy();
@@ -3472,16 +2990,17 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Create a new file.
+     * Create a file.
      *
      * @param path Path.
-     * @param bufSize Buffer size.
+     * @param dirProps Directory properties.
      * @param overwrite Overwrite flag.
+     * @param blockSize Block size.
      * @param affKey Affinity key.
-     * @param replication Replication factor.
-     * @param props Properties.
-     * @param simpleCreate Whether new file should be created in secondary FS using create(Path, boolean) method.
-     * @return Tuple containing the created file info and its parent id.
+     * @param evictExclude Evict exclude flag.
+     * @param fileProps File properties.
+     * @return @return Tuple containing the created file info and its parent id.
+     * @throws IgniteCheckedException If failed.
      */
     IgniteBiTuple<IgfsEntryInfo, IgniteUuid> create(
         final IgfsPath path,
@@ -3518,9 +3037,7 @@ public class IgfsMetaManager extends IgfsManager {
                     }
 
                     // Start TX.
-                    IgniteInternalTx tx = startTx();
-
-                    try {
+                    try (IgniteInternalTx tx = startTx()) {
                         Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                         if (!pathIds.verifyIntegrity(lockInfos))
@@ -3548,19 +3065,19 @@ public class IgfsMetaManager extends IgfsManager {
                             // First step: add existing to trash listing.
                             IgniteUuid oldId = pathIds.lastId();
 
-                            id2InfoPrj.invoke(trashId, new ListingAddProcessor(oldId.toString(),
+                            id2InfoPrj.invoke(trashId, new IgfsMetaDirectoryListingAddProcessor(oldId.toString(),
                                 new IgfsListingEntry(oldInfo)));
 
                             // Second step: replace ID in parent directory.
                             String name = pathIds.lastPart();
                             IgniteUuid parentId = pathIds.lastParentId();
 
-                            id2InfoPrj.invoke(parentId, new ListingReplaceProcessor(name, overwriteId));
+                            id2InfoPrj.invoke(parentId, new IgfsMetaDirectoryListingReplaceProcessor(name, overwriteId));
 
                             // Third step: create the file.
                             long createTime = System.currentTimeMillis();
 
-                            IgfsEntryInfo newInfo = invokeAndGet(overwriteId, new FileCreateProcessor(createTime,
+                            IgfsEntryInfo newInfo = invokeAndGet(overwriteId, new IgfsMetaFileCreateProcessor(createTime,
                                 fileProps, blockSize, affKey, createFileLockId(false), evictExclude));
 
                             // Fourth step: update path of remove file.
@@ -3592,9 +3109,6 @@ public class IgfsMetaManager extends IgfsManager {
                             return new T2<>(res.info(), res.parentId());
                         }
                     }
-                    finally {
-                        tx.close();
-                    }
                 }
                 finally {
                     busyLock.leaveBusy();
@@ -3682,7 +3196,7 @@ public class IgfsMetaManager extends IgfsManager {
             return null;
 
         // First step: add new entry to the last existing element.
-        id2InfoPrj.invoke(lastExistingInfo.id(), new ListingAddProcessor(curPart,
+        id2InfoPrj.invoke(lastExistingInfo.id(), new IgfsMetaDirectoryListingAddProcessor(curPart,
             new IgfsListingEntry(curId, dir || !pathIds.isLastIndex(curIdx))));
 
         // Events support.
@@ -3699,7 +3213,7 @@ public class IgfsMetaManager extends IgfsManager {
             String nextPart = pathIds.part(nextIdx);
             IgniteUuid nextId = pathIds.surrogateId(nextIdx);
 
-            id2InfoPrj.invoke(curId, new DirectoryCreateProcessor(createTime, dirProps,
+            id2InfoPrj.invoke(curId, new IgfsMetaDirectoryCreateProcessor(createTime, dirProps,
                 nextPart, new IgfsListingEntry(nextId, dir || !pathIds.isLastIndex(nextIdx))));
 
             // Save event.
@@ -3720,9 +3234,9 @@ public class IgfsMetaManager extends IgfsManager {
         IgfsEntryInfo info;
 
         if (dir)
-            info = invokeAndGet(curId, new DirectoryCreateProcessor(createTime, dirProps));
+            info = invokeAndGet(curId, new IgfsMetaDirectoryCreateProcessor(createTime, dirProps));
         else
-            info = invokeAndGet(curId, new FileCreateProcessor(createTime, fileProps,
+            info = invokeAndGet(curId, new IgfsMetaFileCreateProcessor(createTime, fileProps,
                 blockSize, affKey, createFileLockId(false), evictExclude));
 
         createdPaths.add(pathIds.path());
@@ -3752,476 +3266,4 @@ public class IgfsMetaManager extends IgfsManager {
         else
             IgfsUtils.sendEvents(igfsCtx.kernalContext(), leafPath, EventType.EVT_IGFS_DIR_CREATED);
     }
-
-    /**
-     * File create processor.
-     */
-    private static class FileCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Create time. */
-        private long createTime;
-
-        /** Properties. */
-        private Map<String, String> props;
-
-        /** Block size. */
-        private int blockSize;
-
-        /** Affintiy key. */
-        private IgniteUuid affKey;
-
-        /** Lcok ID. */
-        private IgniteUuid lockId;
-
-        /** Evict exclude flag. */
-        private boolean evictExclude;
-
-        /**
-         * Constructor.
-         */
-        public FileCreateProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param createTime Create time.
-         * @param props Properties.
-         * @param blockSize Block size.
-         * @param affKey Affinity key.
-         * @param lockId Lock ID.
-         * @param evictExclude Evict exclude flag.
-         */
-        public FileCreateProcessor(long createTime, Map<String, String> props, int blockSize,
-            @Nullable IgniteUuid affKey, IgniteUuid lockId, boolean evictExclude) {
-            this.createTime = createTime;
-            this.props = props;
-            this.blockSize = blockSize;
-            this.affKey = affKey;
-            this.lockId = lockId;
-            this.evictExclude = evictExclude;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo info = IgfsUtils.createFile(
-                entry.getKey(),
-                blockSize,
-                0L,
-                affKey,
-                lockId,
-                evictExclude,
-                props,
-                createTime,
-                createTime
-            );
-
-            entry.setValue(info);
-
-            return info;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeLong(createTime);
-            U.writeStringMap(out, props);
-            out.writeInt(blockSize);
-            out.writeObject(affKey);
-            out.writeObject(lockId);
-            out.writeBoolean(evictExclude);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            createTime = in.readLong();
-            props = U.readStringMap(in);
-            blockSize = in.readInt();
-            affKey = (IgniteUuid)in.readObject();
-            lockId = (IgniteUuid)in.readObject();
-            evictExclude = in.readBoolean();
-        }
-    }
-
-    /**
-     * Directory create processor.
-     */
-    private static class DirectoryCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Create time. */
-        private long createTime;
-
-        /** Properties. */
-        private Map<String, String> props;
-
-        /** Child name (optional). */
-        private String childName;
-
-        /** Child entry (optional. */
-        private IgfsListingEntry childEntry;
-
-        /**
-         * Constructor.
-         */
-        public DirectoryCreateProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param createTime Create time.
-         * @param props Properties.
-         */
-        public DirectoryCreateProcessor(long createTime, Map<String, String> props) {
-            this(createTime, props, null, null);
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param createTime Create time.
-         * @param props Properties.
-         * @param childName Child name.
-         * @param childEntry Child entry.
-         */
-        public DirectoryCreateProcessor(long createTime, Map<String, String> props, String childName,
-            IgfsListingEntry childEntry) {
-            this.createTime = createTime;
-            this.props = props;
-            this.childName = childName;
-            this.childEntry = childEntry;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-
-            IgfsEntryInfo info = IgfsUtils.createDirectory(
-                entry.getKey(),
-                null,
-                props,
-                createTime,
-                createTime
-            );
-
-            if (childName != null)
-                info = info.listing(Collections.singletonMap(childName, childEntry));
-
-            entry.setValue(info);
-
-            return info;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeLong(createTime);
-            U.writeStringMap(out, props);
-
-            if (childName != null) {
-                out.writeBoolean(true);
-
-                U.writeString(out, childName);
-                out.writeObject(childEntry);
-            }
-            else
-                out.writeBoolean(false);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            createTime = in.readLong();
-            props = U.readStringMap(in);
-
-            if (in.readBoolean()) {
-                childName = U.readString(in);
-                childEntry = (IgfsListingEntry)in.readObject();
-            }
-        }
-    }
-
-    /**
-     * File lock entry processor.
-     */
-    private static class FileLockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Lock Id. */
-        private IgniteUuid lockId;
-
-        /**
-         * Default constructor.
-         */
-        public FileLockProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param lockId Lock ID.
-         */
-        public FileLockProcessor(IgniteUuid lockId) {
-            this.lockId = lockId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo oldInfo = entry.getValue();
-
-            IgfsEntryInfo newInfo = oldInfo.lock(lockId);
-
-            entry.setValue(newInfo);
-
-            return newInfo;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeGridUuid(out, lockId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            lockId = U.readGridUuid(in);
-        }
-    }
-
-    /**
-     * File unlock entry processor.
-     */
-    private static class FileUnlockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Modification time. */
-        private long modificationTime;
-
-        /**
-         * Default constructor.
-         */
-        public FileUnlockProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param modificationTime Modification time.
-         */
-        public FileUnlockProcessor(long modificationTime) {
-            this.modificationTime = modificationTime;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo old = entry.getValue();
-
-            entry.setValue(old.unlock(modificationTime));
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeLong(modificationTime);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            modificationTime = in.readLong();
-        }
-    }
-
-    /**
-     * File reserve space entry processor.
-     */
-    private static class FileReserveSpaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Space. */
-        private long space;
-
-        /** Affinity range. */
-        private IgfsFileAffinityRange affRange;
-
-        /**
-         * Default constructor.
-         */
-        public FileReserveSpaceProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param space Space.
-         * @param affRange
-         */
-        public FileReserveSpaceProcessor(long space, IgfsFileAffinityRange affRange) {
-            this.space = space;
-            this.affRange = affRange;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo oldInfo = entry.getValue();
-
-            IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
-
-            newMap.addRange(affRange);
-
-            IgfsEntryInfo newInfo = oldInfo.length(oldInfo.length() + space).fileMap(newMap);
-
-            entry.setValue(newInfo);
-
-            return newInfo;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeLong(space);
-            out.writeObject(affRange);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            space = in.readLong();
-            affRange = (IgfsFileAffinityRange)in.readObject();
-        }
-    }
-
-    /**
-     * Update properties processor.
-     */
-    private static class UpdatePropertiesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Properties to be updated. */
-        private Map<String, String> props;
-
-        /**
-         * Constructor.
-         */
-        public UpdatePropertiesProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param props Properties.
-         */
-        public UpdatePropertiesProcessor(Map<String, String> props) {
-            this.props = props;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-            IgfsEntryInfo oldInfo = entry.getValue();
-
-            Map<String, String> tmp = oldInfo.properties();
-
-            tmp = tmp == null ? new GridLeanMap<String, String>(props.size()) : new GridLeanMap<>(tmp);
-
-            for (Map.Entry<String, String> e : props.entrySet()) {
-                if (e.getValue() == null)
-                    // Remove properties with 'null' values.
-                    tmp.remove(e.getKey());
-                else
-                    // Add/overwrite property.
-                    tmp.put(e.getKey(), e.getValue());
-            }
-
-            IgfsEntryInfo newInfo = oldInfo.properties(tmp);
-
-            entry.setValue(newInfo);
-
-            return newInfo;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeStringMap(out, props);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            props = U.readStringMap(in);
-        }
-    }
-
-    /**
-     * Update times entry processor.
-     */
-    private static class UpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Access time. */
-        private long accessTime;
-
-        /** Modification time. */
-        private long modificationTime;
-
-        /**
-         * Default constructor.
-         */
-        public UpdateTimesProcessor() {
-            // No-op.
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param accessTime Access time.
-         * @param modificationTime Modification time.
-         */
-        public UpdateTimesProcessor(long accessTime, long modificationTime) {
-            this.accessTime = accessTime;
-            this.modificationTime = modificationTime;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
-            throws EntryProcessorException {
-
-            IgfsEntryInfo oldInfo = entry.getValue();
-
-            entry.setValue(oldInfo.accessModificationTime(accessTime, modificationTime));
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeLong(accessTime);
-            out.writeLong(modificationTime);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            accessTime = in.readLong();
-            modificationTime = in.readLong();
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
index 2903239..e2fe58d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
@@ -134,7 +134,7 @@ public class IgfsPathIds {
      *
      * @return Last ID.
      */
-    @Nullable public IgniteUuid lastId() {
+    public IgniteUuid lastId() {
         return ids[ids.length - 1];
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
new file mode 100644
index 0000000..ffba042
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
@@ -0,0 +1,117 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Directory create processor.
+ */
+public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Create time. */
+    private long createTime;
+
+    /** Properties. */
+    private Map<String, String> props;
+
+    /** Child name (optional). */
+    private String childName;
+
+    /** Child entry (optional. */
+    private IgfsListingEntry childEntry;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaDirectoryCreateProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param createTime Create time.
+     * @param props Properties.
+     */
+    public IgfsMetaDirectoryCreateProcessor(long createTime, Map<String, String> props) {
+        this(createTime, props, null, null);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param createTime Create time.
+     * @param props Properties.
+     * @param childName Child name.
+     * @param childEntry Child entry.
+     */
+    public IgfsMetaDirectoryCreateProcessor(long createTime, Map<String, String> props, String childName,
+        IgfsListingEntry childEntry) {
+        this.createTime = createTime;
+        this.props = props;
+        this.childName = childName;
+        this.childEntry = childEntry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+
+        IgfsEntryInfo info = IgfsUtils.createDirectory(
+            entry.getKey(),
+            null,
+            props,
+            createTime,
+            createTime
+        );
+
+        if (childName != null)
+            info = info.listing(Collections.singletonMap(childName, childEntry));
+
+        entry.setValue(info);
+
+        return info;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(createTime);
+        U.writeStringMap(out, props);
+
+        if (childName != null) {
+            out.writeBoolean(true);
+
+            U.writeString(out, childName);
+            out.writeObject(childEntry);
+        }
+        else
+            out.writeBoolean(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        createTime = in.readLong();
+        props = U.readStringMap(in);
+
+        if (in.readBoolean()) {
+            childName = U.readString(in);
+            childEntry = (IgfsListingEntry)in.readObject();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
new file mode 100644
index 0000000..ab5cd5d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
@@ -0,0 +1,92 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Update directory listing closure.
+ */
+public final class IgfsMetaDirectoryListingAddProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** File name to add into parent listing. */
+    private String fileName;
+
+    /** File ID.*/
+    private IgfsListingEntry entry;
+
+    /**
+     * Empty constructor required for {@link Externalizable}.
+     *
+     */
+    public IgfsMetaDirectoryListingAddProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructs update directory listing closure.
+     *
+     * @param fileName File name to add into parent listing.
+     * @param entry Listing entry to add or remove.
+     */
+    public IgfsMetaDirectoryListingAddProcessor(String fileName, IgfsListingEntry entry) {
+        assert fileName != null;
+        assert entry != null;
+
+        this.fileName = fileName;
+        this.entry = entry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args) {
+        IgfsEntryInfo fileInfo = e.getValue();
+
+        assert fileInfo.isDirectory();
+
+        Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
+
+        // Modify listing in-place.
+        IgfsListingEntry oldEntry = listing.put(fileName, entry);
+
+        if (oldEntry != null && !oldEntry.fileId().equals(entry.fileId()))
+            throw new IgniteException("Directory listing contains unexpected file" +
+                " [listing=" + listing + ", fileName=" + fileName + ", entry=" + entry +
+                ", oldEntry=" + oldEntry + ']');
+
+        e.setValue(fileInfo.listing(listing));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, fileName);
+        out.writeObject(entry);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        fileName = U.readString(in);
+        entry = (IgfsListingEntry)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaDirectoryListingAddProcessor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
new file mode 100644
index 0000000..181a73e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
@@ -0,0 +1,89 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Remove entry from directory listing.
+ */
+public class IgfsMetaDirectoryListingRemoveProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** File name. */
+    private String fileName;
+
+    /** Expected ID. */
+    private IgniteUuid fileId;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaDirectoryListingRemoveProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param fileName File name.
+     * @param fileId File ID.
+     */
+    public IgfsMetaDirectoryListingRemoveProcessor(String fileName, IgniteUuid fileId) {
+        this.fileName = fileName;
+        this.fileId = fileId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo fileInfo = e.getValue();
+
+        assert fileInfo != null;
+        assert fileInfo.isDirectory();
+
+        Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
+
+        listing.putAll(fileInfo.listing());
+
+        IgfsListingEntry oldEntry = listing.get(fileName);
+
+        if (oldEntry == null || !oldEntry.fileId().equals(fileId))
+            throw new IgniteException("Directory listing doesn't contain expected file" +
+                " [listing=" + listing + ", fileName=" + fileName + "]");
+
+        // Modify listing in-place.
+        listing.remove(fileName);
+
+        e.setValue(fileInfo.listing(listing));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, fileName);
+        U.writeGridUuid(out, fileId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        fileName = U.readString(in);
+        fileId = U.readGridUuid(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
new file mode 100644
index 0000000..4c4888c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
@@ -0,0 +1,84 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Listing replace processor.
+ */
+public final class IgfsMetaDirectoryListingReplaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Name. */
+    private String name;
+
+    /** New ID. */
+    private IgniteUuid id;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaDirectoryListingReplaceProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param name Name.
+     * @param id ID.
+     */
+    public IgfsMetaDirectoryListingReplaceProcessor(String name, IgniteUuid id) {
+        this.name = name;
+        this.id = id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo fileInfo = e.getValue();
+
+        assert fileInfo.isDirectory();
+
+        Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
+
+        // Modify listing in-place.
+        IgfsListingEntry oldEntry = listing.get(name);
+
+        if (oldEntry == null)
+            throw new IgniteException("Directory listing doesn't contain expected entry: " + name);
+
+        listing.put(name, new IgfsListingEntry(id, oldEntry.isDirectory()));
+
+        e.setValue(fileInfo.listing(listing));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, name);
+        out.writeObject(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        name = U.readString(in);
+        id = (IgniteUuid)in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
new file mode 100644
index 0000000..a07d764
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
@@ -0,0 +1,110 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Map;
+
+/**
+ * File create processor.
+ */
+public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Create time. */
+    private long createTime;
+
+    /** Properties. */
+    private Map<String, String> props;
+
+    /** Block size. */
+    private int blockSize;
+
+    /** Affintiy key. */
+    private IgniteUuid affKey;
+
+    /** Lcok ID. */
+    private IgniteUuid lockId;
+
+    /** Evict exclude flag. */
+    private boolean evictExclude;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaFileCreateProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param createTime Create time.
+     * @param props Properties.
+     * @param blockSize Block size.
+     * @param affKey Affinity key.
+     * @param lockId Lock ID.
+     * @param evictExclude Evict exclude flag.
+     */
+    public IgfsMetaFileCreateProcessor(long createTime, Map<String, String> props, int blockSize,
+        @Nullable IgniteUuid affKey, IgniteUuid lockId, boolean evictExclude) {
+        this.createTime = createTime;
+        this.props = props;
+        this.blockSize = blockSize;
+        this.affKey = affKey;
+        this.lockId = lockId;
+        this.evictExclude = evictExclude;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo info = IgfsUtils.createFile(
+            entry.getKey(),
+            blockSize,
+            0L,
+            affKey,
+            lockId,
+            evictExclude,
+            props,
+            createTime,
+            createTime
+        );
+
+        entry.setValue(info);
+
+        return info;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(createTime);
+        U.writeStringMap(out, props);
+        out.writeInt(blockSize);
+        out.writeObject(affKey);
+        out.writeObject(lockId);
+        out.writeBoolean(evictExclude);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        createTime = in.readLong();
+        props = U.readStringMap(in);
+        blockSize = in.readInt();
+        affKey = (IgniteUuid)in.readObject();
+        lockId = (IgniteUuid)in.readObject();
+        evictExclude = in.readBoolean();
+    }
+}


[07/17] ignite git commit: IGNITE-2834: IGFS: Implemented optional metadata co-location.

Posted by dm...@apache.org.
IGNITE-2834: IGFS: Implemented optional metadata co-location.


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

Branch: refs/heads/ignite-2849
Commit: 4af5316bfe87b9cba179ce9307beb5eadd3b6a7f
Parents: 88ffed1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Mar 18 17:07:58 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 18 17:07:58 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  | 47 ++++++++++++++++
 .../org/apache/ignite/internal/IgnitionEx.java  | 10 ++--
 .../processors/cache/GridCacheUtils.java        | 14 +----
 .../IgfsColocatedMetadataAffinityKeyMapper.java | 47 ++++++++++++++++
 .../internal/processors/igfs/IgfsUtils.java     | 57 ++++++++++++++++++++
 5 files changed, 157 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4af5316b/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 1a9c0fe..99d364e 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
@@ -19,6 +19,7 @@ package org.apache.ignite.configuration;
 
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
@@ -83,6 +84,9 @@ public class FileSystemConfiguration {
     /** Default value of whether to initialize default path modes. */
     public static final boolean DFLT_INIT_DFLT_PATH_MODES = true;
 
+    /** Default value of metadata co-location flag. */
+    public static boolean DFLT_COLOCATE_META = true;
+
     /** IGFS instance name. */
     private String name;
 
@@ -164,6 +168,9 @@ public class FileSystemConfiguration {
     /** Whether to initialize default path modes. */
     private boolean initDfltPathModes = DFLT_INIT_DFLT_PATH_MODES;
 
+    /** Metadata co-location flag. */
+    private boolean colocateMeta = DFLT_COLOCATE_META;
+
     /**
      * Constructs default configuration.
      */
@@ -184,6 +191,7 @@ public class FileSystemConfiguration {
          */
         blockSize = cfg.getBlockSize();
         bufSize = cfg.getStreamBufferSize();
+        colocateMeta = cfg.isColocateMetadata();
         dataCacheName = cfg.getDataCacheName();
         dfltMode = cfg.getDefaultMode();
         dualModeMaxPendingPutsSize = cfg.getDualModeMaxPendingPutsSize();
@@ -830,6 +838,45 @@ public class FileSystemConfiguration {
         this.initDfltPathModes = initDfltPathModes;
     }
 
+    /**
+     * Get whether to co-locate metadata on a single node.
+     * <p>
+     * Normally Ignite spread ownership of particular keys among all cache nodes. Transaction with keys owned by
+     * different nodes will produce more network traffic and will require more time to complete comparing to
+     * transaction with keys owned only by a single node.
+     * <p>
+     * IGFS stores information about file system structure (metadata) inside a transactional cache configured through
+     * {@link #getMetaCacheName()} property. Metadata updates caused by operations on IGFS usually require several
+     * intearnal keys to be updated. As IGFS metadata cache usually operates in {@link CacheMode#REPLICATED} mode,
+     * meaning that all nodes have all metadata locally, it makes sense to give a hint to Ignite to co-locate
+     * ownership of all metadata keys on a single node. This will decrease amount of network trips required to update
+     * metadata and hence could improve performance.
+     * <p>
+     * This property should be disabled if you see excessive CPU and network load on a single node, which
+     * degrades performance and cannot be explained by business logic of your application.
+     * <p>
+     * This settings is only used if metadata cache is configured in {@code CacheMode#REPLICATED} mode. Otherwise it
+     * is ignored.
+     * <p>
+     * Defaults to {@link #DFLT_COLOCATE_META}.
+     *
+     * @return {@code True} if metadata co-location is enabled.
+     */
+    public boolean isColocateMetadata() {
+        return colocateMeta;
+    }
+
+    /**
+     * Set metadata co-location flag.
+     * <p>
+     * See {@link #isColocateMetadata()} for more information.
+     *
+     * @param colocateMeta Whether metadata co-location is enabled.
+     */
+    public void setColocateMetadata(boolean colocateMeta) {
+        this.colocateMeta = colocateMeta;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(FileSystemConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af5316b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 4796581..533b6d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -62,6 +62,7 @@ import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -1950,14 +1951,11 @@ public class IgnitionEx {
 
             cfg.setCacheConfiguration(cacheCfgs.toArray(new CacheConfiguration[cacheCfgs.size()]));
 
-            // Iterate over IGFS caches and set "copyOnRead" flag to "false". Note that we do this after cloning
-            // to leave user object unchanged.
+            // Iterate over IGFS caches and prepare their configurations if needed.
             assert cfg.getCacheConfiguration() != null;
 
-            for (CacheConfiguration ccfg : cfg.getCacheConfiguration()) {
-                if (CU.isIgfsCache(cfg, ccfg.getName()))
-                    ccfg.setCopyOnRead(false);
-            }
+            for (CacheConfiguration ccfg : cfg.getCacheConfiguration())
+                IgfsUtils.prepareCacheConfiguration(cfg, ccfg);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af5316b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 94307ca..1cdd303 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -73,6 +73,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.lang.IgniteInClosureX;
 import org.apache.ignite.internal.util.typedef.C1;
@@ -1382,18 +1383,7 @@ public class GridCacheUtils {
      * @return {@code True} in this is IGFS data or meta cache.
      */
     public static boolean isIgfsCache(IgniteConfiguration cfg, @Nullable String cacheName) {
-        FileSystemConfiguration[] igfsCfgs = cfg.getFileSystemConfiguration();
-
-        if (igfsCfgs != null) {
-            for (FileSystemConfiguration igfsCfg : igfsCfgs) {
-                // IGFS config probably has not been validated yet => possible NPE, so we check for null.
-                if (igfsCfg != null &&
-                    (F.eq(cacheName, igfsCfg.getDataCacheName()) || F.eq(cacheName, igfsCfg.getMetaCacheName())))
-                    return true;
-            }
-        }
-
-        return false;
+        return IgfsUtils.isIgfsCache(cfg, cacheName);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af5316b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsColocatedMetadataAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsColocatedMetadataAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsColocatedMetadataAffinityKeyMapper.java
new file mode 100644
index 0000000..d8c10b0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsColocatedMetadataAffinityKeyMapper.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cache.affinity.AffinityKeyMapper;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Special implementation of affinity mapper which maps all metadata to the same primary.
+ */
+public class IgfsColocatedMetadataAffinityKeyMapper implements AffinityKeyMapper {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Affinity. */
+    private static final Integer AFF = 1;
+
+    /** {@inheritDoc} */
+    @Override public Object affinityKey(Object key) {
+        return AFF;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsColocatedMetadataAffinityKeyMapper.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af5316b/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 b176e21..ef7d5c7 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
@@ -20,9 +20,12 @@ package org.apache.ignite.internal.processors.igfs;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsPath;
@@ -264,4 +267,58 @@ public class IgfsUtils {
         if (evts.isRecordable(type))
             evts.record(new IgfsEvent(path, locNode, type));
     }
+
+    /**
+     * @param cfg Grid configuration.
+     * @param cacheName Cache name.
+     * @return {@code True} in this is IGFS data or meta cache.
+     */
+    public static boolean isIgfsCache(IgniteConfiguration cfg, @Nullable String cacheName) {
+        FileSystemConfiguration[] igfsCfgs = cfg.getFileSystemConfiguration();
+
+        if (igfsCfgs != null) {
+            for (FileSystemConfiguration igfsCfg : igfsCfgs) {
+                // IGFS config probably has not been validated yet => possible NPE, so we check for null.
+                if (igfsCfg != null) {
+                    if (F.eq(cacheName, igfsCfg.getDataCacheName()) || F.eq(cacheName, igfsCfg.getMetaCacheName()))
+                        return true;
+                }
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Prepare cache configuration if this is IGFS meta or data cache.
+     *
+     * @param cfg Configuration.
+     * @param ccfg Cache configuration.
+     */
+    public static void prepareCacheConfiguration(IgniteConfiguration cfg, CacheConfiguration ccfg) {
+        FileSystemConfiguration[] igfsCfgs = cfg.getFileSystemConfiguration();
+
+        if (igfsCfgs != null) {
+            for (FileSystemConfiguration igfsCfg : igfsCfgs) {
+                if (igfsCfg != null) {
+                    if (F.eq(ccfg.getName(), igfsCfg.getMetaCacheName())) {
+                        ccfg.setCopyOnRead(false);
+
+                        // Set co-located affinity mapper if needed.
+                        if (igfsCfg.isColocateMetadata() && ccfg.getCacheMode() == CacheMode.REPLICATED &&
+                            ccfg.getAffinityMapper() == null)
+                            ccfg.setAffinityMapper(new IgfsColocatedMetadataAffinityKeyMapper());
+
+                        return;
+                    }
+
+                    if (F.eq(ccfg.getName(), igfsCfg.getDataCacheName())) {
+                        ccfg.setCopyOnRead(false);
+
+                        return;
+                    }
+                }
+            }
+        }
+    }
 }
\ No newline at end of file


[05/17] ignite git commit: IGNITE-2860: IGFS: Reworked base meta operations.

Posted by dm...@apache.org.
IGNITE-2860: IGFS: Reworked base meta operations.


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

Branch: refs/heads/ignite-2849
Commit: ebf40752854cb12c5c6202ecb8546a0090482ad6
Parents: 3e53f17
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Mar 18 16:38:45 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 18 16:38:45 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/igfs/IgfsPath.java   |    9 +
 .../internal/processors/igfs/IgfsImpl.java      |   59 +-
 .../processors/igfs/IgfsMetaManager.java        | 1058 +++++++++---------
 .../internal/processors/igfs/IgfsPathIds.java   |  291 +++++
 .../processors/igfs/IgfsPathsCreateResult.java  |   77 ++
 .../internal/processors/igfs/IgfsUtils.java     |   23 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |   68 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   31 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   26 +-
 .../processors/igfs/IgfsStartCacheTest.java     |    9 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   10 +-
 11 files changed, 1039 insertions(+), 622 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/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 fb0621c..bbb4efb 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
@@ -159,6 +159,15 @@ public final class IgfsPath implements Comparable<IgfsPath>, Externalizable {
     }
 
     /**
+     * Get components in array form.
+     *
+     * @return Components array.
+     */
+    public String[] componentsArray() {
+        return path.length() == 1 ? new String[0] : path.substring(1).split(SLASH);
+    }
+
+    /**
      * Returns the parent of a path or {@code null} if at root.
      *
      * @return The parent of a path or {@code null} if at root.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/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 3065427..9ec583c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -17,24 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-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.AtomicLong;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -94,6 +76,25 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+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.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;
@@ -1033,8 +1034,15 @@ public final class IgfsImpl implements IgfsEx {
                 else
                     dirProps = fileProps = new HashMap<>(props);
 
-                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(path, false/*append*/, overwrite, dirProps,
-                    cfg.getBlockSize(), affKey, evictExclude(path, true), fileProps);
+                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(
+                    path,
+                    dirProps,
+                    overwrite,
+                    cfg.getBlockSize(),
+                    affKey,
+                    evictExclude(path, true),
+                    fileProps
+                );
 
                 assert t2 != null;
 
@@ -1104,8 +1112,15 @@ public final class IgfsImpl implements IgfsEx {
                 else
                     dirProps = fileProps = new HashMap<>(props);
 
-                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(path, true/*append*/, false/*overwrite*/,
-                    dirProps, cfg.getBlockSize(), null/*affKey*/, evictExclude(path, true), fileProps);
+                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.append(
+                    path,
+                    dirProps,
+                    create,
+                    cfg.getBlockSize(),
+                    null/*affKey*/,
+                    evictExclude(path, true),
+                    fileProps
+                );
 
                 assert t2 != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/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 b4774f2..d91b0bc 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
@@ -17,31 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.CountDownLatch;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
@@ -82,16 +57,36 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_CREATED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_RENAMED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CREATED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_RENAMED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.builder;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  * Cache based structure (meta data) manager.
@@ -334,6 +329,51 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Gets all file IDs for components of specified path. Result cannot be empty - there is at least root element.
+     * But each element (except the first) can be {@code null} if such files don't exist.
+     *
+     * @param path Path.
+     * @return Collection of file IDs for components of specified path.
+     * @throws IgniteCheckedException If failed.
+     */
+    public IgfsPathIds pathIds(IgfsPath path) throws IgniteCheckedException {
+        if (busyLock.enterBusy()) {
+            try {
+                validTxState(false);
+
+                // Prepare parts.
+                String[] components = path.componentsArray();
+
+                String[] parts = new String[components.length + 1];
+
+                System.arraycopy(components, 0, parts, 1, components.length);
+
+                // Prepare IDs.
+                IgniteUuid[] ids = new IgniteUuid[parts.length];
+
+                ids[0] = IgfsUtils.ROOT_ID;
+
+                for (int i = 1; i < ids.length; i++) {
+                    IgniteUuid id = fileId(ids[i - 1], parts[i], false);
+
+                    if (id != null)
+                        ids[i] = id;
+                    else
+                        break;
+                }
+
+                // Return.
+                return new IgfsPathIds(path, parts, ids);
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to get file IDS because Grid is stopping: " + path);
+    }
+
+    /**
      * Gets all file IDs for components of specified path possibly skipping existing transaction. Result cannot
      * be empty - there is at least root element. But each element (except the first) can be {@code null} if such
      * files don't exist.
@@ -823,107 +863,84 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 validTxState(false);
 
-                // 1. First get source and destination path IDs.
-                List<IgniteUuid> srcPathIds = fileIds(srcPath);
-                List<IgniteUuid> dstPathIds = fileIds(dstPath);
+                // Prepare path IDs.
+                IgfsPathIds srcPathIds = pathIds(srcPath);
+                IgfsPathIds dstPathIds = pathIds(dstPath);
 
-                final Set<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
+                // Source path must exists.
+                if (!srcPathIds.allExists())
+                    throw new IgfsPathNotFoundException("Failed to perform move because source path is not " +
+                        "found: " + srcPath);
 
-                allIds.addAll(srcPathIds);
+                // At this point we need to understand name of resulting entry. It will be either destination leaf
+                // or source leaf depending on existence.
+                String dstName;
 
-                final IgniteUuid dstLeafId = dstPathIds.get(dstPathIds.size() - 1);
+                if (dstPathIds.lastExists())
+                    //  Full destination path exists -> use source name.
+                    dstName = srcPathIds.lastPart();
+                else {
+                    if (dstPathIds.lastParentExists()) {
+                        // Destination path doesn't exists -> use destination name.
+                        dstName = dstPathIds.lastPart();
 
-                if (dstLeafId == null) {
-                    // Delete null entry for the unexisting destination element:
-                    dstPathIds.remove(dstPathIds.size() - 1);
+                        dstPathIds = dstPathIds.parent();
+                    }
+                    else
+                        // Destination parent is not found either -> exception.
+                        throw new IgfsPathNotFoundException("Failed to perform move because destination path is not " +
+                            "found: " + dstPath.parent());
                 }
 
-                allIds.addAll(dstPathIds);
+                // Lock participating IDs.
+                final Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                if (allIds.remove(null)) {
-                    throw new IgfsPathNotFoundException("Failed to perform move because some path component was " +
-                            "not found. [src=" + srcPath + ", dst=" + dstPath + ']');
-                }
+                srcPathIds.addExistingIds(lockIds);
+                dstPathIds.addExistingIds(lockIds);
 
-                // 2. Start transaction.
                 IgniteInternalTx tx = startTx();
 
                 try {
-                    // 3. Obtain the locks.
-                    final Map<IgniteUuid, IgfsFileInfo> allInfos = lockIds(allIds);
+                    // Obtain the locks.
+                    final Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
 
-                    // 4. Verify integrity of source directory.
-                    if (!verifyPathIntegrity(srcPath, srcPathIds, allInfos)) {
+                    // Verify integrity of source and destination paths.
+                    if (!srcPathIds.verifyIntegrity(lockInfos))
                         throw new IgfsPathNotFoundException("Failed to perform move because source directory " +
                             "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']');
-                    }
-
-                    // 5. Verify integrity of destination directory.
-                    final IgfsPath dstDirPath = dstLeafId != null ? dstPath : dstPath.parent();
 
-                    if (!verifyPathIntegrity(dstDirPath, dstPathIds, allInfos)) {
+                    if (!dstPathIds.verifyIntegrity(lockInfos))
                         throw new IgfsPathNotFoundException("Failed to perform move because destination directory " +
                             "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']');
-                    }
-
-                    // 6. Calculate source and destination targets which will be changed.
-                    IgniteUuid srcTargetId = srcPathIds.get(srcPathIds.size() - 2);
-                    IgfsFileInfo srcTargetInfo = allInfos.get(srcTargetId);
-                    String srcName = srcPath.name();
-
-                    IgniteUuid dstTargetId;
-                    IgfsFileInfo dstTargetInfo;
-                    String dstName;
 
-                    if (dstLeafId != null) {
-                        // Destination leaf exists. Check if it is an empty directory.
-                        IgfsFileInfo dstLeafInfo = allInfos.get(dstLeafId);
-
-                        assert dstLeafInfo != null;
-
-                        if (dstLeafInfo.isDirectory()) {
-                            // Destination is a directory.
-                            dstTargetId = dstLeafId;
-                            dstTargetInfo = dstLeafInfo;
-                            dstName = srcPath.name();
-                        }
-                        else {
-                            // Error, destination is existing file.
-                            throw new IgfsPathAlreadyExistsException("Failed to perform move " +
-                                "because destination points to " +
-                                "existing file [src=" + srcPath + ", dst=" + dstPath + ']');
-                        }
-                    }
-                    else {
-                        // Destination leaf doesn't exist, so we operate on parent.
-                        dstTargetId = dstPathIds.get(dstPathIds.size() - 1);
-                        dstTargetInfo = allInfos.get(dstTargetId);
-                        dstName = dstPath.name();
-                    }
+                    // Addiional check: is destination directory?
+                    IgfsFileInfo dstParentInfo = lockInfos.get(dstPathIds.lastId());
 
-                    assert dstTargetInfo != null;
-                    assert dstTargetInfo.isDirectory();
+                    if (dstParentInfo.isFile())
+                        throw new IgfsPathAlreadyExistsException("Failed to perform move because destination points " +
+                            "to existing file [src=" + srcPath + ", dst=" + dstPath + ']');
 
-                    // 7. Last check: does destination target already have listing entry with the same name?
-                    if (dstTargetInfo.hasChild(dstName)) {
+                    // Additional check: does destination already has child with the same name?
+                    if (dstParentInfo.hasChild(dstName))
                         throw new IgfsPathAlreadyExistsException("Failed to perform move because destination already " +
                             "contains entry with the same name existing file [src=" + srcPath +
                             ", dst=" + dstPath + ']');
-                    }
 
-                    // 8. Actual move: remove from source parent and add to destination target.
-                    IgfsListingEntry entry = srcTargetInfo.listing().get(srcName);
+                    // Actual move: remove from source parent and add to destination target.
+                    IgfsFileInfo srcParentInfo = lockInfos.get(srcPathIds.lastParentId());
 
-                    transferEntry(entry, srcTargetId, srcName, dstTargetId, dstName);
+                    IgfsFileInfo srcInfo = lockInfos.get(srcPathIds.lastId());
+                    String srcName = srcPathIds.lastPart();
+                    IgfsListingEntry srcEntry = srcParentInfo.listing().get(srcName);
 
-                    tx.commit();
+                    transferEntry(srcEntry, srcParentInfo.id(), srcName, dstParentInfo.id(), dstName);
 
-                    IgfsPath realNewPath = new IgfsPath(dstDirPath, dstName);
+                    tx.commit();
 
-                    IgfsFileInfo moved = allInfos.get(srcPathIds.get(srcPathIds.size() - 1));
+                    IgfsPath newPath = new IgfsPath(dstPathIds.path(), dstName);
 
                     // Set the new path to the info to simplify event creation:
-                    return IgfsFileInfo.builder(moved).path(realNewPath).build();
+                    return IgfsFileInfo.builder(srcInfo).path(newPath).build();
                 }
                 finally {
                     tx.close();
@@ -1117,72 +1134,57 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 validTxState(false);
 
-                final SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
+                IgfsPathIds pathIds = pathIds(path);
 
-                List<IgniteUuid> pathIdList = fileIds(path);
-
-                assert pathIdList.size() > 1;
+                // Continue only if the whole path present.
+                if (!pathIds.allExists())
+                    return null; // A fragment of the path no longer exists.
 
-                final IgniteUuid victimId = pathIdList.get(pathIdList.size() - 1);
+                IgniteUuid victimId = pathIds.lastId();
+                String victimName = pathIds.lastPart();
 
-                assert !IgfsUtils.isRootOrTrashId(victimId) : "Cannot delete root or trash directories.";
+                if (IgfsUtils.isRootId(victimId))
+                    throw new IgfsException("Cannot remove root directory");
 
-                allIds.addAll(pathIdList);
+                // Prepare IDs to lock.
+                SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                if (allIds.remove(null))
-                    return null; // A fragment of the path no longer exists.
+                pathIds.addExistingIds(allIds);
 
                 IgniteUuid trashId = IgfsUtils.randomTrashId();
 
-                boolean added = allIds.add(trashId);
-                assert added;
+                allIds.add(trashId);
 
-                final IgniteInternalTx tx = startTx();
+                IgniteInternalTx tx = startTx();
 
                 try {
-                    final Map<IgniteUuid, IgfsFileInfo> infoMap = lockIds(allIds);
+                    // Lock participants.
+                    Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(allIds);
 
-                    // Directory starure was changed concurrently, so the original path no longer exists:
-                    if (!verifyPathIntegrity(path, pathIdList, infoMap))
+                    // Ensure that all participants are still in place.
+                    if (!pathIds.verifyIntegrity(lockInfos))
                         return null;
 
-                    final IgfsFileInfo victimInfo = infoMap.get(victimId);
+                    IgfsFileInfo victimInfo = lockInfos.get(victimId);
 
+                    // Cannot delete non-empty directory if recursive flag is not set.
                     if (!recursive && victimInfo.hasChildren())
-                        // Throw exception if not empty and not recursive.
                         throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
                             "empty and recursive flag is not set).");
 
-                    IgfsFileInfo destInfo = infoMap.get(trashId);
-
-                    assert destInfo != null;
+                    // Prepare trash data.
+                    IgfsFileInfo trashInfo = lockInfos.get(trashId);
+                    final String trashName = victimId.toString();
 
-                    final String srcFileName = path.name();
+                    assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " +
+                        "destination directory (file already exists) [destName=" + trashName + ']';
 
-                    final String destFileName = victimId.toString();
+                    IgniteUuid parentId = pathIds.lastParentId();
+                    IgfsFileInfo parentInfo = lockInfos.get(parentId);
 
-                    assert !destInfo.hasChild(destFileName) : "Failed to add file name into the " +
-                        "destination directory (file already exists) [destName=" + destFileName + ']';
-
-                    IgfsFileInfo srcParentInfo = infoMap.get(pathIdList.get(pathIdList.size() - 2));
-
-                    assert srcParentInfo != null;
-
-                    IgniteUuid srcParentId = srcParentInfo.id();
-                    assert srcParentId.equals(pathIdList.get(pathIdList.size() - 2));
-
-                    IgfsListingEntry srcEntry = srcParentInfo.listing().get(srcFileName);
-
-                    assert srcEntry != null : "Deletion victim not found in parent listing [path=" + path +
-                        ", name=" + srcFileName + ", listing=" + srcParentInfo.listing() + ']';
-
-                    assert victimId.equals(srcEntry.fileId());
-
-                    transferEntry(srcEntry, srcParentId, srcFileName, trashId, destFileName);
+                    transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName);
 
                     if (victimInfo.isFile())
-                        // Update a file info of the removed file with a file path,
-                        // which will be used by delete worker for event notifications.
                         invokeUpdatePath(victimId, path);
 
                     tx.commit();
@@ -1647,74 +1649,57 @@ public class IgfsMetaManager extends IgfsManager {
      *
      * @param path The path to create.
      * @param props The properties to use for created directories.
-     * @return True iff a directory was created during the operation.
+     * @return True if a directory was created during the operation.
      * @throws IgniteCheckedException If a non-directory file exists on the requested path, and in case of other errors.
      */
     boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
-        assert props != null;
         validTxState(false);
 
-        DirectoryChainBuilder b = null;
-
         while (true) {
             if (busyLock.enterBusy()) {
                 try {
-                    b = new DirectoryChainBuilder(path, props);
+                    // Prepare path IDs.
+                    IgfsPathIds pathIds = pathIds(path);
+
+                    // Prepare lock IDs. Essentially, they consist of two parts: existing IDs and potential new IDs.
+                    Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
+
+                    pathIds.addExistingIds(lockIds);
+                    pathIds.addSurrogateIds(lockIds);
+
+                    assert lockIds.size() == pathIds.count();
 
                     // Start TX.
                     IgniteInternalTx tx = startTx();
 
                     try {
-                        final Map<IgniteUuid, IgfsFileInfo> lockedInfos = lockIds(b.idSet);
+                        final Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
 
-                        // If the path was changed, we close the current Tx and repeat the procedure again
-                        // starting from taking the path ids.
-                        if (verifyPathIntegrity(b.existingPath, b.idList, lockedInfos)) {
-                            // Locked path okay, trying to proceed with the remainder creation.
-                            IgfsFileInfo lowermostExistingInfo = lockedInfos.get(b.lowermostExistingId);
+                        if (!pathIds.verifyIntegrity(lockInfos))
+                            // Directory structure changed concurrently. So we simply re-try.
+                            continue;
 
-                            // Check only the lowermost directory in the existing directory chain
-                            // because others are already checked in #verifyPathIntegrity() above.
-                            if (!lowermostExistingInfo.isDirectory())
+                        // Check if the whole structure is already in place.
+                        if (pathIds.allExists()) {
+                            if (lockInfos.get(pathIds.lastExistingId()).isDirectory())
+                                return false;
+                            else
                                 throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
                                     "element is not a directory)");
+                        }
 
-                            if (b.existingIdCnt == b.components.size() + 1) {
-                                assert b.existingPath.equals(path);
-                                assert lockedInfos.size() == b.existingIdCnt;
-
-                                // The target directory already exists, nothing to do.
-                                // (The fact that all the path consisns of directories is already checked above).
-                                // Note that properties are not updated in this case.
-                                return false;
-                            }
-
-                            Map<String, IgfsListingEntry> parentListing = lowermostExistingInfo.listing();
-
-                            String shortName = b.components.get(b.existingIdCnt - 1);
-
-                            IgfsListingEntry entry = parentListing.get(shortName);
+                        IgfsPathsCreateResult res = createDirectory(pathIds, lockInfos, props);
 
-                            if (entry == null) {
-                                b.doBuild();
+                        if (res == null)
+                            continue;
 
-                                tx.commit();
+                        // Commit TX.
+                        tx.commit();
 
-                                break;
-                            }
-                            else {
-                                // Another thread created file or directory with the same name.
-                                if (!entry.isDirectory()) {
-                                    // Entry exists, and it is not a directory:
-                                    throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
-                                        "element is not a directory)");
-                                }
+                        generateCreateEvents(res.createdPaths(), false);
 
-                                // If this is a directory, we continue the repeat loop,
-                                // because we cannot lock this directory without
-                                // lock ordering rule violation.
-                            }
-                        }
+                        // We are done.
+                        return true;
                     }
                     finally {
                         tx.close();
@@ -1727,12 +1712,6 @@ public class IgfsMetaManager extends IgfsManager {
             else
                 throw new IllegalStateException("Failed to mkdir because Grid is stopping. [path=" + path + ']');
         }
-
-        assert b != null;
-
-        b.sendEvents();
-
-        return true;
     }
 
     /**
@@ -1960,11 +1939,12 @@ public class IgfsMetaManager extends IgfsManager {
                                 parentInfo = synchronize(fs, parentPath, parentInfo, parent0, true, null);
 
                                 // Fire notification about missing directories creation.
-                                if (evts.isRecordable(EVT_IGFS_DIR_CREATED)) {
+                                if (evts.isRecordable(EventType.EVT_IGFS_DIR_CREATED)) {
                                     IgfsPath evtPath = parent0;
 
                                     while (!parentPath.equals(evtPath)) {
-                                        pendingEvts.addFirst(new IgfsEvent(evtPath, locNode, EVT_IGFS_DIR_CREATED));
+                                        pendingEvts.addFirst(new IgfsEvent(evtPath, locNode,
+                                            EventType.EVT_IGFS_DIR_CREATED));
 
                                         evtPath = evtPath.parent();
 
@@ -2010,8 +1990,8 @@ public class IgfsMetaManager extends IgfsManager {
                             }
 
                             // Record CREATE event if needed.
-                            if (evts.isRecordable(EVT_IGFS_FILE_CREATED))
-                                pendingEvts.add(new IgfsEvent(path, locNode, EVT_IGFS_FILE_CREATED));
+                            if (evts.isRecordable(EventType.EVT_IGFS_FILE_CREATED))
+                                pendingEvts.add(new IgfsEvent(path, locNode, EventType.EVT_IGFS_FILE_CREATED));
 
                             return new IgfsSecondaryOutputStreamDescriptor(parentInfo.id(), newInfo, out);
                         }
@@ -2285,11 +2265,11 @@ public class IgfsMetaManager extends IgfsManager {
 
                         synchronize(fs, parentPath, parentPathInfo, path, true, null);
 
-                        if (evts.isRecordable(EVT_IGFS_DIR_CREATED)) {
+                        if (evts.isRecordable(EventType.EVT_IGFS_DIR_CREATED)) {
                             IgfsPath evtPath = path;
 
                             while (!parentPath.equals(evtPath)) {
-                                pendingEvts.addFirst(new IgfsEvent(evtPath, locNode, EVT_IGFS_DIR_CREATED));
+                                pendingEvts.addFirst(new IgfsEvent(evtPath, locNode, EventType.EVT_IGFS_DIR_CREATED));
 
                                 evtPath = evtPath.parent();
 
@@ -2386,15 +2366,15 @@ public class IgfsMetaManager extends IgfsManager {
 
                         // Record event if needed.
                         if (srcInfo.isFile()) {
-                            if (evts.isRecordable(EVT_IGFS_FILE_RENAMED))
+                            if (evts.isRecordable(EventType.EVT_IGFS_FILE_RENAMED))
                                 pendingEvts.add(new IgfsEvent(
                                     src,
                                     destInfo == null ? dest : new IgfsPath(dest, src.name()),
                                     locNode,
-                                    EVT_IGFS_FILE_RENAMED));
+                                    EventType.EVT_IGFS_FILE_RENAMED));
                         }
-                        else if (evts.isRecordable(EVT_IGFS_DIR_RENAMED))
-                            pendingEvts.add(new IgfsEvent(src, dest, locNode, EVT_IGFS_DIR_RENAMED));
+                        else if (evts.isRecordable(EventType.EVT_IGFS_DIR_RENAMED))
+                            pendingEvts.add(new IgfsEvent(src, dest, locNode, EventType.EVT_IGFS_DIR_RENAMED));
 
                         return true;
                     }
@@ -2896,7 +2876,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Transaction.
      */
     private IgniteInternalTx startTx() {
-        return metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+        return metaCache.txStartEx(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ);
     }
 
     /**
@@ -3178,6 +3158,14 @@ public class IgfsMetaManager extends IgfsManager {
         private IgfsListingEntry entry;
 
         /**
+         * Empty constructor required for {@link Externalizable}.
+         *
+         */
+        public ListingAddProcessor() {
+            // No-op.
+        }
+
+        /**
          * Constructs update directory listing closure.
          *
          * @param fileName File name to add into parent listing.
@@ -3191,19 +3179,10 @@ public class IgfsMetaManager extends IgfsManager {
             this.entry = entry;
         }
 
-        /**
-         * Empty constructor required for {@link Externalizable}.
-         *
-         */
-        public ListingAddProcessor() {
-            // No-op.
-        }
-
         /** {@inheritDoc} */
         @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args) {
             IgfsFileInfo fileInfo = e.getValue();
 
-            assert fileInfo != null : "File info not found for the child: " + entry.fileId();
             assert fileInfo.isDirectory();
 
             Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
@@ -3240,6 +3219,73 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Listing replace processor.
+     */
+    private static final class ListingReplaceProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+        Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Name. */
+        private String name;
+
+        /** New ID. */
+        private IgniteUuid id;
+
+        /**
+         * Constructor.
+         */
+        public ListingReplaceProcessor() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param name Name.
+         * @param id ID.
+         */
+        public ListingReplaceProcessor(String name, IgniteUuid id) {
+            this.name = name;
+            this.id = id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args)
+            throws EntryProcessorException {
+            IgfsFileInfo fileInfo = e.getValue();
+
+            assert fileInfo.isDirectory();
+
+            Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
+
+            // Modify listing in-place.
+            IgfsListingEntry oldEntry = listing.get(name);
+
+            if (oldEntry == null)
+                throw new IgniteException("Directory listing doesn't contain expected entry: " + name);
+
+            listing.put(name, new IgfsListingEntry(id, oldEntry.isDirectory()));
+
+            e.setValue(new IgfsFileInfo(listing, fileInfo));
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeString(out, name);
+            out.writeObject(id);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            name = U.readString(in);
+            id = (IgniteUuid)in.readObject();
+        }
+    }
+
+    /**
      * Update path closure.
      */
     @GridInternal
@@ -3291,170 +3337,92 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Create a new file.
+     * Append routine.
      *
      * @param path Path.
-     * @param bufSize Buffer size.
-     * @param overwrite Overwrite flag.
+     * @param dirProps Directory properties.
+     * @param create Create flag.
+     * @param blockSize Block size.
      * @param affKey Affinity key.
-     * @param replication Replication factor.
-     * @param props Properties.
-     * @param simpleCreate Whether new file should be created in secondary FS using create(Path, boolean) method.
-     * @return Tuple containing the created file info and its parent id.
+     * @param evictExclude Evict exclude flag.
+     * @param fileProps File properties.
+     * @return Tuple containing the file info and its parent id.
+     * @throws IgniteCheckedException If failed.
      */
-    IgniteBiTuple<IgfsFileInfo, IgniteUuid> create(
+    IgniteBiTuple<IgfsFileInfo, IgniteUuid> append(
         final IgfsPath path,
-        final boolean append,
-        final boolean overwrite,
         Map<String, String> dirProps,
+        final boolean create,
         final int blockSize,
         final @Nullable IgniteUuid affKey,
         final boolean evictExclude,
         @Nullable Map<String, String> fileProps) throws IgniteCheckedException {
         validTxState(false);
 
-        assert path != null;
-
-        final String name = path.name();
-
-        DirectoryChainBuilder b = null;
-
-        IgniteUuid trashId = IgfsUtils.randomTrashId();
-
         while (true) {
             if (busyLock.enterBusy()) {
                 try {
-                    b = new DirectoryChainBuilder(path, dirProps, fileProps, blockSize, affKey, evictExclude);
-
-                    // Start Tx:
-                    IgniteInternalTx tx = startTx();
-
-                    try {
-                        if (overwrite)
-                            // Lock also the TRASH directory because in case of overwrite we
-                            // may need to delete the old file:
-                            b.idSet.add(trashId);
-
-                        final Map<IgniteUuid, IgfsFileInfo> lockedInfos = lockIds(b.idSet);
-
-                        assert !overwrite || lockedInfos.get(trashId) != null; // TRASH must exist at this point.
-
-                        // If the path was changed, we close the current Tx and repeat the procedure again
-                        // starting from taking the path ids.
-                        if (verifyPathIntegrity(b.existingPath, b.idList, lockedInfos)) {
-                            // Locked path okay, trying to proceed with the remainder creation.
-                            final IgfsFileInfo lowermostExistingInfo = lockedInfos.get(b.lowermostExistingId);
+                    // Prepare path IDs.
+                    IgfsPathIds pathIds = pathIds(path);
 
-                            if (b.existingIdCnt == b.components.size() + 1) {
-                                // Full requestd path exists.
+                    // Fail-fast: create flag is not specified and some paths are missing.
+                    if (!pathIds.allExists() && !create)
+                        throw new IgfsPathNotFoundException("Failed to append because file is not found: " + path);
 
-                                assert b.existingPath.equals(path);
-                                assert lockedInfos.size() ==
-                                        (overwrite ? b.existingIdCnt + 1/*TRASH*/ : b.existingIdCnt);
-
-                                if (lowermostExistingInfo.isDirectory()) {
-                                    throw new IgfsPathAlreadyExistsException("Failed to "
-                                            + (append ? "open" : "create") + " file (path points to an " +
-                                        "existing directory): " + path);
-                                }
-                                else {
-                                    // This is a file.
-                                    assert lowermostExistingInfo.isFile();
-
-                                    final IgniteUuid parentId = b.idList.get(b.idList.size() - 2);
-
-                                    final IgniteUuid lockId = lowermostExistingInfo.lockId();
-
-                                    if (append) {
-                                        if (lockId != null)
-                                            throw fsException("Failed to open file (file is opened for writing) "
-                                                + "[fileName=" + name + ", fileId=" + lowermostExistingInfo.id()
-                                                + ", lockId=" + lockId + ']');
-
-                                        IgfsFileInfo lockedInfo = invokeLock(lowermostExistingInfo.id(), false);
-
-                                        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(lockedInfo, parentId);
-
-                                        tx.commit();
-
-                                        IgfsUtils.sendEvents(igfsCtx.kernalContext(), path,
-                                                EventType.EVT_IGFS_FILE_OPENED_WRITE);
-
-                                        return t2;
-                                    }
-                                    else if (overwrite) {
-                                        // Delete existing file, but fail if it is locked:
-                                        if (lockId != null)
-                                            throw fsException("Failed to overwrite file (file is opened for writing) " +
-                                                    "[fileName=" + name + ", fileId=" + lowermostExistingInfo.id()
-                                                    + ", lockId=" + lockId + ']');
+                    // Prepare lock IDs.
+                    Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                                        final IgfsListingEntry deletedEntry = lockedInfos.get(parentId).listing()
-                                                .get(name);
+                    pathIds.addExistingIds(lockIds);
+                    pathIds.addSurrogateIds(lockIds);
 
-                                        assert deletedEntry != null;
-
-                                        transferEntry(deletedEntry, parentId, name, trashId,
-                                            lowermostExistingInfo.id().toString());
-
-                                        // Update a file info of the removed file with a file path,
-                                        // which will be used by delete worker for event notifications.
-                                        invokeUpdatePath(lowermostExistingInfo.id(), path);
-
-                                        // Make a new locked info:
-                                        long t = System.currentTimeMillis();
-
-                                        final IgfsFileInfo newFileInfo = new IgfsFileInfo(cfg.getBlockSize(), 0L,
-                                            affKey, createFileLockId(false), evictExclude, fileProps, t, t);
-
-                                        assert newFileInfo.lockId() != null; // locked info should be created.
-
-                                        createNewEntry(newFileInfo, parentId, name);
-
-                                        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(newFileInfo, parentId);
+                    // Start TX.
+                    IgniteInternalTx tx = startTx();
 
-                                        tx.commit();
+                    try {
+                        Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
 
-                                        delWorker.signal();
+                        if (!pathIds.verifyIntegrity(lockInfos))
+                            // Directory structure changed concurrently. So we simply re-try.
+                            continue;
 
-                                        IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_WRITE);
+                        if (pathIds.allExists()) {
+                            // All participants are found. Simply open the stream.
+                            IgfsFileInfo info = lockInfos.get(pathIds.lastId());
 
-                                        return t2;
-                                    }
-                                    else {
-                                        throw new IgfsPathAlreadyExistsException("Failed to create file (file " +
-                                            "already exists and overwrite flag is false): " + path);
-                                    }
-                                }
-                            }
+                            // Check: is it a file?
+                            if (!info.isFile())
+                                throw new IgfsPathIsDirectoryException("Failed to open file for write." + path);
 
-                            // The full requested path does not exist.
+                            // Check if file already opened for write.
+                            if (info.lockId() != null)
+                                throw new IgfsException("File is already opened for write: " + path);
 
-                            // Check only the lowermost directory in the existing directory chain
-                            // because others are already checked in #verifyPathIntegrity() above.
-                            if (!lowermostExistingInfo.isDirectory())
-                                throw new IgfsParentNotDirectoryException("Failed to " + (append ? "open" : "create" )
-                                    + " file (parent element is not a directory)");
+                            // At this point we can open the stream safely.
+                            info = invokeLock(info.id(), false);
 
-                            final String uppermostFileToBeCreatedName = b.components.get(b.existingIdCnt - 1);
+                            IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(info, pathIds.lastParentId());
 
-                            if (!lowermostExistingInfo.hasChild(uppermostFileToBeCreatedName)) {
-                                b.doBuild();
+                            tx.commit();
 
-                                assert b.leafInfo != null;
-                                assert b.leafParentId != null;
+                            IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EventType.EVT_IGFS_FILE_OPENED_WRITE);
 
-                                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(b.leafInfo, b.leafParentId);
+                            return t2;
+                        }
+                        else {
+                            // Create file and parent folders.
+                            IgfsPathsCreateResult res =
+                                createFile(pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
 
-                                tx.commit();
+                            if (res == null)
+                                continue;
 
-                                b.sendEvents();
+                            // Commit.
+                            tx.commit();
 
-                                return t2;
-                            }
+                            // Generate events.
+                            generateCreateEvents(res.createdPaths(), true);
 
-                            // Another thread concurrently created file or directory in the path with
-                            // the name we need.
+                            return new T2<>(res.info(), res.parentId());
                         }
                     }
                     finally {
@@ -3464,224 +3432,292 @@ public class IgfsMetaManager extends IgfsManager {
                 finally {
                     busyLock.leaveBusy();
                 }
-            } else
-                throw new IllegalStateException("Failed to mkdir because Grid is stopping. [path=" + path + ']');
+            }
+            else
+                throw new IllegalStateException("Failed to append for file because Grid is stopping:" + path);
         }
     }
 
-    /** File chain builder. */
-    private class DirectoryChainBuilder {
-        /** The requested path to be created. */
-        private final IgfsPath path;
+    /**
+     * Create a new file.
+     *
+     * @param path Path.
+     * @param bufSize Buffer size.
+     * @param overwrite Overwrite flag.
+     * @param affKey Affinity key.
+     * @param replication Replication factor.
+     * @param props Properties.
+     * @param simpleCreate Whether new file should be created in secondary FS using create(Path, boolean) method.
+     * @return Tuple containing the created file info and its parent id.
+     */
+    IgniteBiTuple<IgfsFileInfo, IgniteUuid> create(
+        final IgfsPath path,
+        Map<String, String> dirProps,
+        final boolean overwrite,
+        final int blockSize,
+        final @Nullable IgniteUuid affKey,
+        final boolean evictExclude,
+        @Nullable Map<String, String> fileProps) throws IgniteCheckedException {
+        validTxState(false);
 
-        /** Full path components. */
-        private final List<String> components;
+        while (true) {
+            if (busyLock.enterBusy()) {
+                try {
+                    // Prepare path IDs.
+                    IgfsPathIds pathIds = pathIds(path);
 
-        /** The list of ids. */
-        private final List<IgniteUuid> idList;
+                    // Prepare lock IDs.
+                    Set<IgniteUuid> lockIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-        /** The set of ids. */
-        private final SortedSet<IgniteUuid> idSet = new TreeSet<IgniteUuid>(PATH_ID_SORTING_COMPARATOR);
+                    pathIds.addExistingIds(lockIds);
+                    pathIds.addSurrogateIds(lockIds);
 
-        /** The middle node properties. */
-        private final Map<String, String> props;
+                    // In overwrite mode we also lock ID of potential replacement as well as trash ID.
+                    IgniteUuid overwriteId = IgniteUuid.randomUuid();
+                    IgniteUuid trashId = IgfsUtils.randomTrashId();
 
-        /** The leaf node properties. */
-        private final Map<String, String> leafProps;
+                    if (overwrite) {
+                        lockIds.add(overwriteId);
 
-        /** The lowermost exsiting path id. */
-        private final IgniteUuid lowermostExistingId;
+                        // Trash ID is only added if we suspect conflict.
+                        if (pathIds.allExists())
+                            lockIds.add(trashId);
+                    }
 
-        /** The existing path. */
-        private final IgfsPath existingPath;
+                    // Start TX.
+                    IgniteInternalTx tx = startTx();
 
-        /** The created leaf info. */
-        private IgfsFileInfo leafInfo;
+                    try {
+                        Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
 
-        /** The leaf parent id. */
-        private IgniteUuid leafParentId;
+                        if (!pathIds.verifyIntegrity(lockInfos))
+                            // Directory structure changed concurrently. So we simply re-try.
+                            continue;
 
-        /** The number of existing ids. */
-        private final int existingIdCnt;
+                        if (pathIds.allExists()) {
+                            // All participants found.
+                            IgfsFileInfo oldInfo = lockInfos.get(pathIds.lastId());
 
-        /** Whether laef is directory. */
-        private final boolean leafDir;
+                            // Check: is it a file?
+                            if (!oldInfo.isFile())
+                                throw new IgfsPathIsDirectoryException("Failed to create a file: " + path);
 
-        /** Block size. */
-        private final int blockSize;
+                            // Check: can we overwrite it?
+                            if (!overwrite)
+                                throw new IgfsPathAlreadyExistsException("Failed to create a file: " + path);
 
-        /** Affinity key. */
-        private final IgniteUuid affKey;
+                            // Check if file already opened for write.
+                            if (oldInfo.lockId() != null)
+                                throw new IgfsException("File is already opened for write: " + path);
 
-        /** Evict exclude flag. */
-        private final boolean evictExclude;
+                            // At this point file can be re-created safely.
 
-        /**
-         * Constructor for directories.
-         *
-         * @param path Path.
-         * @param props Properties.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected DirectoryChainBuilder(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
-            this(path, props, props, true, 0, null, false);
-        }
+                            // First step: add existing to trash listing.
+                            IgniteUuid oldId = pathIds.lastId();
 
-        /**
-         * Constructor for files.
-         *
-         * @param path Path.
-         * @param dirProps Directory properties.
-         * @param fileProps File properties.
-         * @param blockSize Block size.
-         * @param affKey Affinity key (optional).
-         * @param evictExclude Evict exclude flag.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected DirectoryChainBuilder(IgfsPath path, Map<String, String> dirProps, Map<String, String> fileProps,
-            int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude)
-            throws IgniteCheckedException {
-            this(path, dirProps, fileProps, false, blockSize, affKey, evictExclude);
-        }
+                            id2InfoPrj.invoke(trashId, new ListingAddProcessor(oldId.toString(),
+                                new IgfsListingEntry(oldId, true)));
 
-        /**
-         * Constructor.
-         *
-         * @param path Path.
-         * @param props Middle properties.
-         * @param leafProps Leaf properties.
-         * @param leafDir Whether leaf is directory or file.
-         * @param blockSize Block size.
-         * @param affKey Affinity key (optional).
-         * @param evictExclude Evict exclude flag.
-         * @throws IgniteCheckedException If failed.
-         */
-        private DirectoryChainBuilder(IgfsPath path, Map<String,String> props, Map<String,String> leafProps,
-            boolean leafDir, int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude)
-            throws IgniteCheckedException {
-            this.path = path;
-            this.components = path.components();
-            this.idList = fileIds(path);
-            this.props = props;
-            this.leafProps = leafProps;
-            this.leafDir = leafDir;
-            this.blockSize = blockSize;
-            this.affKey = affKey;
-            this.evictExclude = evictExclude;
+                            // Second step: replace ID in parent directory.
+                            String name = pathIds.lastPart();
+                            IgniteUuid parentId = pathIds.lastParentId();
 
-            // Store all the non-null ids in the set & construct existing path in one loop:
-            IgfsPath existingPath = path.root();
+                            id2InfoPrj.invoke(parentId, new ListingReplaceProcessor(name, overwriteId));
 
-            assert idList.size() == components.size() + 1;
+                            // Third step: create the file.
+                            long createTime = System.currentTimeMillis();
 
-            // Find the lowermost existing id:
-            IgniteUuid lowermostExistingId = null;
+                            IgfsFileInfo newInfo = invokeAndGet(overwriteId, new FileCreateProcessor(createTime,
+                                fileProps, blockSize, affKey, createFileLockId(false), evictExclude));
 
-            int idIdx = 0;
+                            // Fourth step: update path of remove file.
+                            invokeUpdatePath(oldId, path);
 
-            for (IgniteUuid id : idList) {
-                if (id == null)
-                    break;
+                            // Prepare result and commit.
+                            IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(newInfo, parentId);
 
-                lowermostExistingId = id;
+                            tx.commit();
 
-                boolean added = idSet.add(id);
+                            IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EventType.EVT_IGFS_FILE_OPENED_WRITE);
 
-                assert added : "Not added id = " + id;
+                            return t2;
+                        }
+                        else {
+                            // Create file and parent folders.
+                            IgfsPathsCreateResult res =
+                                createFile(pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
 
-                if (idIdx >= 1) // skip root.
-                    existingPath = new IgfsPath(existingPath, components.get(idIdx - 1));
+                            if (res == null)
+                                continue;
 
-                idIdx++;
-            }
+                            // Commit.
+                            tx.commit();
 
-            assert idSet.contains(IgfsUtils.ROOT_ID);
+                            // Generate events.
+                            generateCreateEvents(res.createdPaths(), true);
 
-            this.lowermostExistingId = lowermostExistingId;
+                            return new T2<>(res.info(), res.parentId());
+                        }
+                    }
+                    finally {
+                        tx.close();
+                    }
+                }
+                finally {
+                    busyLock.leaveBusy();
+                }
+            }
+            else
+                throw new IllegalStateException("Failed to mkdir because Grid is stopping. [path=" + path + ']');
+        }
+    }
 
-            this.existingPath = existingPath;
+    /**
+     * Create directory and it's parents.
+     *
+     * @param pathIds Path IDs.
+     * @param lockInfos Lock infos.
+     * @param dirProps Directory properties.
+     * @return Result or {@code} if the first parent already contained child with the same name.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable IgfsPathsCreateResult createDirectory(IgfsPathIds pathIds, Map<IgniteUuid, IgfsFileInfo> lockInfos,
+        Map<String, String> dirProps) throws IgniteCheckedException {
+        // Check if entry we are going to write to is directory.
+        if (lockInfos.get(pathIds.lastExistingId()).isFile())
+            throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
+                "element is not a directory)");
+
+        return createFileOrDirectory(true, pathIds, lockInfos, dirProps, null, 0, null, false);
+    }
 
-            this.existingIdCnt = idSet.size();
-        }
+    /**
+     * Create file and all it's parents.
+     *
+     * @param pathIds Paths IDs.
+     * @param lockInfos Lock infos.
+     * @param dirProps Directory properties.
+     * @param fileProps File propertris.
+     * @param blockSize Block size.
+     * @param affKey Affinity key (optional)
+     * @param evictExclude Evict exclude flag.
+     * @return Result or {@code} if the first parent already contained child with the same name.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map<IgniteUuid, IgfsFileInfo> lockInfos,
+        Map<String, String> dirProps, Map<String, String> fileProps, int blockSize, @Nullable IgniteUuid affKey,
+        boolean evictExclude) throws IgniteCheckedException{
+        // Check if entry we are going to write to is directory.
+        if (lockInfos.get(pathIds.lastExistingId()).isFile())
+            throw new IgfsParentNotDirectoryException("Failed to open file for write " +
+                "(parent element is not a directory): " + pathIds.path());
+
+        return createFileOrDirectory(false, pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
+    }
 
-        /**
-         * Does the main portion of job building the renmaining path.
-         */
-        public final void doBuild() throws IgniteCheckedException {
-            // Fix current time. It will be used in all created entities.
-            long createTime = System.currentTimeMillis();
+    /**
+     * Ceate file or directory.
+     *
+     * @param dir Directory flag.
+     * @param pathIds Path IDs.
+     * @param lockInfos Lock infos.
+     * @param dirProps Directory properties.
+     * @param fileProps File properties.
+     * @param blockSize Block size.
+     * @param affKey Affinity key.
+     * @param evictExclude Evict exclude flag.
+     * @return Result.
+     * @throws IgniteCheckedException If failed.
+     */
+    private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pathIds,
+        Map<IgniteUuid, IgfsFileInfo> lockInfos, Map<String, String> dirProps, Map<String, String> fileProps,
+        int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude) throws IgniteCheckedException {
+        // This is our starting point.
+        int lastExistingIdx = pathIds.lastExistingIndex();
+        IgfsFileInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId());
+
+        // If current info already contains entry with the same name as it's child, then something
+        // has changed concurrently. We must re-try because we cannot get info of this unexpected
+        // element due to possible deadlocks.
+        int curIdx = lastExistingIdx + 1;
+
+        String curPart = pathIds.part(curIdx);
+        IgniteUuid curId = pathIds.surrogateId(curIdx);
+        IgniteUuid curParentId = lastExistingInfo.id();
+
+        if (lastExistingInfo.hasChild(curPart))
+            return null;
 
-            IgfsListingEntry childInfo = null;
-            String childName = null;
+        // First step: add new entry to the last existing element.
+        id2InfoPrj.invoke(lastExistingInfo.id(), new ListingAddProcessor(curPart,
+            new IgfsListingEntry(curId, dir || !pathIds.isLastIndex(curIdx))));
 
-            IgniteUuid parentId = null;
+        // Events support.
+        IgfsPath lastCreatedPath = pathIds.lastExistingPath();
 
-            // This loop creates the missing directory chain from the bottom to the top:
-            for (int i = components.size() - 1; i >= existingIdCnt - 1; i--) {
-                IgniteUuid childId = IgniteUuid.randomUuid();
-                boolean childDir;
+        List<IgfsPath> createdPaths = new ArrayList<>(pathIds.count() - curIdx);
 
-                if (childName == null) {
-                    assert childInfo == null;
+        // Second step: create middle directories.
+        long createTime = System.currentTimeMillis();
 
-                    if (leafDir) {
-                        childDir = true;
+        while (curIdx < pathIds.count() - 1) {
+            int nextIdx = curIdx + 1;
 
-                        leafInfo = invokeAndGet(childId, new DirectoryCreateProcessor(createTime, leafProps));
-                    }
-                    else {
-                        childDir = false;
+            String nextPart = pathIds.part(nextIdx);
+            IgniteUuid nextId = pathIds.surrogateId(nextIdx);
 
-                        leafInfo = invokeAndGet(childId, new FileCreateProcessor(createTime, leafProps, blockSize,
-                            affKey, createFileLockId(false), evictExclude));
-                    }
-                }
-                else {
-                    assert childInfo != null;
+            id2InfoPrj.invoke(curId, new DirectoryCreateProcessor(createTime, dirProps,
+                nextPart, new IgfsListingEntry(nextId, dir || !pathIds.isLastIndex(nextIdx))));
 
-                    childDir = true;
+            // Save event.
+            lastCreatedPath = new IgfsPath(lastCreatedPath, curPart);
 
-                    id2InfoPrj.invoke(childId, new DirectoryCreateProcessor(createTime, props, childName, childInfo));
+            createdPaths.add(lastCreatedPath);
 
-                    if (parentId == null)
-                        parentId = childId;
-                }
+            // Advance things further.
+            curIdx++;
 
-                childInfo = new IgfsListingEntry(childId, childDir);
+            curParentId = curId;
 
-                childName = components.get(i);
-            }
+            curPart = nextPart;
+            curId = nextId;
+        }
 
-            if (parentId == null)
-                parentId = lowermostExistingId;
+        // Third step: create leaf.
+        IgfsFileInfo info;
 
-            leafParentId = parentId;
+        if (dir)
+            info = invokeAndGet(curId, new DirectoryCreateProcessor(createTime, dirProps));
+        else
+            info = invokeAndGet(curId, new FileCreateProcessor(createTime, fileProps,
+                blockSize, affKey, createFileLockId(false), evictExclude));
 
-            // Now link the newly created directory chain to the lowermost existing parent:
-            id2InfoPrj.invoke(lowermostExistingId, new ListingAddProcessor(childName, childInfo));
-        }
+        createdPaths.add(pathIds.path());
 
-        /**
-         * Sends events.
-         */
-        public final void sendEvents() {
-            if (evts.isRecordable(EVT_IGFS_DIR_CREATED)) {
-                IgfsPath createdPath = existingPath;
+        return new IgfsPathsCreateResult(createdPaths, info, curParentId);
+    }
 
-                for (int i = existingPath.components().size(); i < components.size() - 1; i++) {
-                    createdPath = new IgfsPath(createdPath, components.get(i));
+    /**
+     * Generate events for created file or directory.
+     *
+     * @param createdPaths Created paths.
+     * @param file Whether file was created.
+     */
+    private void generateCreateEvents(List<IgfsPath> createdPaths, boolean file) {
+        if (evts.isRecordable(EventType.EVT_IGFS_DIR_CREATED)) {
+            for (int i = 0; i < createdPaths.size() - 1; i++)
+                IgfsUtils.sendEvents(igfsCtx.kernalContext(), createdPaths.get(i),
+                    EventType.EVT_IGFS_DIR_CREATED);
+        }
 
-                    IgfsUtils.sendEvents(igfsCtx.kernalContext(), createdPath, EVT_IGFS_DIR_CREATED);
-                }
-            }
+        IgfsPath leafPath = createdPaths.get(createdPaths.size() - 1);
 
-            if (leafDir)
-                IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_DIR_CREATED);
-            else {
-                IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_CREATED);
-                IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_WRITE);
-            }
+        if (file) {
+            IgfsUtils.sendEvents(igfsCtx.kernalContext(), leafPath, EventType.EVT_IGFS_FILE_CREATED);
+            IgfsUtils.sendEvents(igfsCtx.kernalContext(), leafPath, EventType.EVT_IGFS_FILE_OPENED_WRITE);
         }
+        else
+            IgfsUtils.sendEvents(igfsCtx.kernalContext(), leafPath, EventType.EVT_IGFS_DIR_CREATED);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
new file mode 100644
index 0000000..1f669b0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
@@ -0,0 +1,291 @@
+/*
+ * 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.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Path IDs abstraction. Contains path and corresponding IDs.
+ */
+public class IgfsPathIds {
+    /** Original path. */
+    private final IgfsPath path;
+
+    /** Path parts. */
+    private final String[] parts;
+
+    /** IDs. */
+    private final IgniteUuid[] ids;
+
+    /** Surrogate IDs for paths which doesn't exist yet. Initialized on demand. */
+    private IgniteUuid[] surrogateIds;
+
+    /** Index of last existing ID. */
+    private final int lastExistingIdx;
+
+    /**
+     * Constructor.
+     *
+     * @param path Path.
+     * @param parts Path parts.
+     * @param ids IDs.
+     */
+    public IgfsPathIds(IgfsPath path, String[] parts, IgniteUuid[] ids) {
+        assert path != null;
+        assert parts.length == ids.length;
+
+        this.path = path;
+        this.parts = parts;
+        this.ids = ids;
+
+        int lastExistingIdx0 = -1;
+
+        for (int i = parts.length - 1; i >= 0; i--) {
+            if (ids[i] != null) {
+                lastExistingIdx0 = i;
+
+                break;
+            }
+        }
+
+        assert lastExistingIdx0 >= 0;
+
+        lastExistingIdx = lastExistingIdx0;
+    }
+
+    /**
+     * Get parent entity.
+     *
+     * @return Parent entity.
+     */
+    public IgfsPathIds parent() {
+        assert ids.length > 1;
+
+        String[] parentParts = new String[parts.length - 1];
+        IgniteUuid[] parentIds = new IgniteUuid[ids.length - 1];
+
+        System.arraycopy(parts, 0, parentParts, 0, parentParts.length);
+        System.arraycopy(ids, 0, parentIds, 0, parentIds.length);
+
+        return new IgfsPathIds(path.parent(), parentParts, parentIds);
+    }
+
+    /**
+     * Get number of elements.
+     *
+     * @return ID count.
+     */
+    public int count() {
+        return ids.length;
+    }
+
+    /**
+     * Get original path.
+     *
+     * @return Path.
+     */
+    public IgfsPath path() {
+        return path;
+    }
+
+    /**
+     * Get path part at the given index.
+     *
+     * @param idx Index.
+     * @return Path part.
+     */
+    public String part(int idx) {
+        assert idx < parts.length;
+
+        return parts[idx];
+    }
+
+    /**
+     * Get last part of original path.
+     *
+     * @return Last part.
+     */
+    public String lastPart() {
+        return parts[parts.length - 1];
+    }
+
+    /**
+     * Get last ID.
+     *
+     * @return Last ID.
+     */
+    @Nullable public IgniteUuid lastId() {
+        return ids[ids.length - 1];
+    }
+
+    /**
+     * Get last parent ID.
+     *
+     * @return Last parent ID.
+     */
+    @Nullable public IgniteUuid lastParentId() {
+        return ids[ids.length - 2];
+    }
+
+    /**
+     * Whether provided index denotes last entry in the path.
+     *
+     * @param idx Index.
+     * @return {@code True} if last.
+     */
+    public boolean isLastIndex(int idx) {
+        return idx == parts.length - 1;
+    }
+
+    /**
+     * Get path of the last existing element.
+     *
+     * @return Path of the last existing element.
+     */
+    public IgfsPath lastExistingPath() {
+        IgfsPath path = new IgfsPath();
+
+        for (int i = 1; i <= lastExistingIdx; i++)
+            path = new IgfsPath(path, parts[i]);
+
+        return path;
+    }
+
+    /**
+     * Whether all parts exists.
+     *
+     * @return {@code True} if all parts were found.
+     */
+    public boolean allExists() {
+        return parts.length == lastExistingIdx + 1;
+    }
+
+    /**
+     * Whether last entry exists.
+     *
+     * @return {@code True} if exists.
+     */
+    public boolean lastExists() {
+        return lastExistingIdx == ids.length - 1;
+    }
+
+
+    /**
+     * Whether parent of the last entry exists.
+     *
+     * @return {@code True} if exists.
+     */
+    public boolean lastParentExists() {
+        return ids.length > 1 && lastExistingIdx == ids.length - 2;
+    }
+
+    /**
+     * Get ID of the last existing entry.
+     *
+     * @return ID of the last existing entry.
+     */
+    public IgniteUuid lastExistingId() {
+        return ids[lastExistingIdx];
+    }
+
+    /**
+     * Get index of the last existing entry.
+     *
+     * @return Index of the last existing entry.
+     */
+    public int lastExistingIndex() {
+        return lastExistingIdx;
+    }
+
+    /**
+     * Add existing IDs to provided collection.
+     *
+     * @param col Collection.
+     */
+    @SuppressWarnings("ManualArrayToCollectionCopy")
+    public void addExistingIds(Collection<IgniteUuid> col) {
+        for (int i = 0; i <= lastExistingIdx; i++)
+            col.add(ids[i]);
+    }
+
+    /**
+     * Add surrogate IDs to provided collection potentially creating them on demand.
+     *
+     * @param col Collection.
+     */
+    @SuppressWarnings("ManualArrayToCollectionCopy")
+    public void addSurrogateIds(Collection<IgniteUuid> col) {
+        if (surrogateIds == null) {
+            surrogateIds = new IgniteUuid[ids.length];
+
+            for (int i = lastExistingIdx + 1; i < surrogateIds.length; i++)
+                surrogateIds[i] = IgniteUuid.randomUuid();
+        }
+
+        for (int i = lastExistingIdx + 1; i < surrogateIds.length; i++)
+            col.add(surrogateIds[i]);
+    }
+
+    /**
+     * Get surrogate ID at the given index.
+     *
+     * @param idx Index.
+     * @return Surrogate ID.
+     */
+    public IgniteUuid surrogateId(int idx) {
+        assert surrogateIds != null;
+
+        assert idx > lastExistingIdx;
+        assert idx < surrogateIds.length;
+
+        return surrogateIds[idx];
+    }
+
+    /**
+     * Verify that observed paths are found in provided infos in the right order.
+     *
+     * @param infos Info.
+     * @return {@code True} if full integrity is preserved.
+     */
+    public boolean verifyIntegrity(Map<IgniteUuid, IgfsFileInfo> infos) {
+        for (int i = 0; i <= lastExistingIdx; i++) {
+            IgniteUuid curId = ids[i];
+            IgfsFileInfo curInfo = infos.get(curId);
+
+            // Check if required ID is there.
+            if (curInfo == null)
+                return false;
+
+            // For non-leaf entry we check if child exists.
+            if (i < lastExistingIdx) {
+                String childName = parts[i + 1];
+                IgniteUuid childId = ids[i + 1];
+
+                if (!curInfo.hasChild(childName, childId))
+                    return false;
+            }
+        }
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
new file mode 100644
index 0000000..3b620f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.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.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+import java.util.List;
+
+/**
+ * IGFS paths create result.
+ */
+public class IgfsPathsCreateResult {
+    /** Created paths. */
+    private final List<IgfsPath> paths;
+
+    /** Info of the last created file. */
+    private final IgfsFileInfo info;
+
+    /** Parent ID. */
+    private final IgniteUuid parentId;
+
+    /**
+     * Constructor.
+     *
+     * @param paths Created paths.
+     * @param info Info of the last created file.
+     * @param parentId Parent ID.
+     */
+    public IgfsPathsCreateResult(List<IgfsPath> paths, IgfsFileInfo info, IgniteUuid parentId) {
+        this.paths = paths;
+        this.info = info;
+        this.parentId = parentId;
+    }
+
+    /**
+     * @return Created paths.
+     */
+    public List<IgfsPath> createdPaths() {
+        return paths;
+    }
+
+    /**
+     * @return Info of the last created file.
+     */
+    public IgfsFileInfo info() {
+        return info;
+    }
+
+    /**
+     * @return Parent ID.
+     */
+    public IgniteUuid parentId() {
+        return parentId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsPathsCreateResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/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 edded2f..b176e21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.lang.reflect.Constructor;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
@@ -43,6 +39,10 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 
+import java.lang.reflect.Constructor;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -104,7 +104,17 @@ public class IgfsUtils {
      * @return {@code True} if this is root ID or trash ID.
      */
     public static boolean isRootOrTrashId(@Nullable IgniteUuid id) {
-        return id != null && (ROOT_ID.equals(id) || isTrashId(id));
+        return isRootId(id) || isTrashId(id);
+    }
+
+    /**
+     * Check whether provided ID is root ID.
+     *
+     * @param id ID.
+     * @return {@code True} if this is root ID.
+     */
+    public static boolean isRootId(@Nullable IgniteUuid id) {
+        return id != null && ROOT_ID.equals(id);
     }
 
     /**
@@ -114,7 +124,8 @@ public class IgfsUtils {
      * @return {@code True} if this is trash ID.
      */
     private static boolean isTrashId(IgniteUuid id) {
-        assert id != null;
+        if (id == null)
+            return false;
 
         UUID gid = id.globalId();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/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 52d8bd5..2acf59c 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
@@ -17,27 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.Field;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -79,6 +58,28 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Field;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -1449,19 +1450,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
                         createCtr.incrementAndGet();
                     }
                     catch (IgniteException e) {
-                        Throwable[] chain = X.getThrowables(e);
-
-                        Throwable cause = chain[chain.length - 1];
-
-                        if (!e.getMessage().startsWith("Failed to overwrite file (file is opened for writing)")
-                                && (cause == null
-                                    || !cause.getMessage().startsWith("Failed to overwrite file (file is opened for writing)"))) {
-
-                            System.out.println("Failed due to IgniteException exception. Cause:");
-                            cause.printStackTrace(System.out);
-
-                            err.compareAndSet(null, e);
-                        }
+                        // No-op.
                     }
                     catch (IOException e) {
                         err.compareAndSet(null, e);
@@ -1937,15 +1926,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
                         chunksCtr.incrementAndGet();
                     }
-                    catch (IgniteException e) {
-                        Throwable[] chain = X.getThrowables(e);
-
-                        Throwable cause = chain[chain.length - 1];
-
-                        if (!e.getMessage().startsWith("Failed to open file (file is opened for writing)")
-                                && (cause == null
-                                || !cause.getMessage().startsWith("Failed to open file (file is opened for writing)")))
-                            err.compareAndSet(null, e);
+                    catch (IgniteException ignore) {
+                        // No-op.
                     }
                     catch (IOException e) {
                         err.compareAndSet(null, e);


[06/17] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by dm...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-2849
Commit: 88ffed13527d43cbfa204ae90cc39722fa3a77c4
Parents: ebf4075 0b10e0c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Mar 18 16:38:58 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 18 16:38:58 2016 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 10 ++++
 .../GridCachePartitionExchangeManager.java      |  3 ++
 .../util/nio/GridNioRecoveryDescriptor.java     | 21 +++++++-
 .../ignite/internal/util/nio/GridNioServer.java | 52 +++++++++++++++++++-
 .../communication/tcp/TcpCommunicationSpi.java  |  8 +++
 .../tcp/TcpCommunicationSpiMBean.java           |  8 ++-
 6 files changed, 98 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[14/17] ignite git commit: IGNITE-2836: IGFS: Ensured that metadata can be serialized through BinaryMarshaller in the most efficient way.

Posted by dm...@apache.org.
IGNITE-2836: IGFS: Ensured that metadata can be serialized through BinaryMarshaller in the most efficient way.


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

Branch: refs/heads/ignite-2849
Commit: 19b4da3d7ddb637c6d6440ec797e3e83d24f56b7
Parents: 865e376
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 21 10:29:20 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 21 10:29:20 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/igfs/IgfsPath.java   | 23 ++++++++++-
 .../ignite/internal/binary/BinaryContext.java   | 43 ++++++++++++++++++++
 .../ignite/internal/binary/BinaryUtils.java     | 37 +++++++++++++++++
 .../processors/igfs/IgfsDirectoryInfo.java      | 27 +++++++++++-
 .../internal/processors/igfs/IgfsEntryInfo.java | 35 ++++++++++++++++
 .../processors/igfs/IgfsFileAffinityRange.java  | 32 +++++++++++++--
 .../internal/processors/igfs/IgfsFileInfo.java  | 37 ++++++++++++++++-
 .../internal/processors/igfs/IgfsFileMap.java   | 41 +++++++++++++++++--
 .../processors/igfs/IgfsListingEntry.java       | 25 +++++++++++-
 .../meta/IgfsMetaDirectoryCreateProcessor.java  | 38 ++++++++++++++++-
 .../IgfsMetaDirectoryListingAddProcessor.java   | 24 ++++++++++-
 ...IgfsMetaDirectoryListingRemoveProcessor.java | 32 ++++++++++++++-
 ...gfsMetaDirectoryListingReplaceProcessor.java | 31 +++++++++++++-
 .../igfs/meta/IgfsMetaFileCreateProcessor.java  | 39 +++++++++++++++++-
 .../igfs/meta/IgfsMetaFileLockProcessor.java    | 29 ++++++++++++-
 .../meta/IgfsMetaFileRangeDeleteProcessor.java  | 22 +++++++++-
 .../meta/IgfsMetaFileRangeUpdateProcessor.java  | 24 ++++++++++-
 .../meta/IgfsMetaFileReserveSpaceProcessor.java | 32 ++++++++++++++-
 .../igfs/meta/IgfsMetaFileUnlockProcessor.java  | 28 ++++++++++++-
 .../igfs/meta/IgfsMetaUpdatePathProcessor.java  | 23 ++++++++++-
 .../meta/IgfsMetaUpdatePropertiesProcessor.java | 28 ++++++++++++-
 .../igfs/meta/IgfsMetaUpdateTimesProcessor.java | 30 +++++++++++++-
 22 files changed, 653 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/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 bbb4efb..f1ceb2e 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
@@ -25,6 +25,13 @@ import java.net.URI;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+
+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.util.io.GridFilenameUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -42,7 +49,7 @@ import org.jetbrains.annotations.Nullable;
  *     IgfsFile file = igfs.info(filePath);
  * </pre>
  */
-public final class IgfsPath implements Comparable<IgfsPath>, Externalizable {
+public final class IgfsPath implements Comparable<IgfsPath>, Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -260,6 +267,20 @@ public final class IgfsPath implements Comparable<IgfsPath>, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeString(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        path = in.readString();
+    }
+
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return path.hashCode();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index b9b633f..c9ad1e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -60,10 +60,29 @@ import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey;
 import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey;
+import org.apache.ignite.internal.processors.igfs.IgfsDirectoryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
+import org.apache.ignite.internal.processors.igfs.IgfsFileInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsFileMap;
+import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRemoveProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingReplaceProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileCreateProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileLockProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileRangeDeleteProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileRangeUpdateProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileReserveSpaceProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileUnlockProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdatePathProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdatePropertiesProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.typedef.F;
@@ -97,6 +116,30 @@ public class BinaryContext {
     static {
         Set<String> sysClss = new HashSet<>();
 
+        // IGFS classes.
+        sysClss.add(IgfsPath.class.getName());
+
+        sysClss.add(IgfsDirectoryInfo.class.getName());
+        sysClss.add(IgfsFileAffinityRange.class.getName());
+        sysClss.add(IgfsFileInfo.class.getName());
+        sysClss.add(IgfsFileMap.class.getName());
+        sysClss.add(IgfsListingEntry.class.getName());
+
+        sysClss.add(IgfsMetaDirectoryCreateProcessor.class.getName());
+        sysClss.add(IgfsMetaDirectoryListingAddProcessor.class.getName());
+        sysClss.add(IgfsMetaDirectoryListingRemoveProcessor.class.getName());
+        sysClss.add(IgfsMetaDirectoryListingReplaceProcessor.class.getName());
+        sysClss.add(IgfsMetaFileCreateProcessor.class.getName());
+        sysClss.add(IgfsMetaFileLockProcessor.class.getName());
+        sysClss.add(IgfsMetaFileRangeDeleteProcessor.class.getName());
+        sysClss.add(IgfsMetaFileRangeUpdateProcessor.class.getName());
+        sysClss.add(IgfsMetaFileReserveSpaceProcessor.class.getName());
+        sysClss.add(IgfsMetaFileUnlockProcessor.class.getName());
+        sysClss.add(IgfsMetaUpdatePathProcessor.class.getName());
+        sysClss.add(IgfsMetaUpdatePropertiesProcessor.class.getName());
+        sysClss.add(IgfsMetaUpdateTimesProcessor.class.getName());
+
+        // Closure processor classes.
         sysClss.add(GridClosureProcessor.C1V2.class.getName());
         sysClss.add(GridClosureProcessor.C1MLAV2.class.getName());
         sysClss.add(GridClosureProcessor.C2V2.class.getName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/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 4a79f22..727501a 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
@@ -51,12 +51,15 @@ import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.binary.BinaryMapFactory;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.internal.binary.builder.BinaryLazyValue;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -1992,6 +1995,40 @@ public class BinaryUtils {
     }
 
     /**
+     * Write {@code IgniteUuid} instance.
+     *
+     * @param out Writer.
+     * @param val Value.
+     */
+    public static void writeIgniteUuid(BinaryRawWriter out, @Nullable IgniteUuid val) {
+        if (val != null) {
+            out.writeBoolean(true);
+
+            out.writeLong(val.globalId().getMostSignificantBits());
+            out.writeLong(val.globalId().getLeastSignificantBits());
+            out.writeLong(val.localId());
+        }
+        else
+            out.writeBoolean(false);
+    }
+
+    /**
+     * Read {@code IgniteUuid} instance.
+     *
+     * @param in Reader.
+     * @return Value.
+     */
+    @Nullable public static IgniteUuid readIgniteUuid(BinaryRawReader in) {
+        if (in.readBoolean()) {
+            UUID globalId = new UUID(in.readLong(), in.readLong());
+
+            return new IgniteUuid(globalId, in.readLong());
+        }
+        else
+            return null;
+    }
+
+    /**
      * Enum type.
      */
     private static class EnumType {

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java
index 01c8ff9..233c8ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java
@@ -17,6 +17,13 @@
 
 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.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -33,7 +40,7 @@ import java.util.Map;
 /**
  * IGFS directory info.
  */
-public class IgfsDirectoryInfo extends IgfsEntryInfo {
+public class IgfsDirectoryInfo extends IgfsEntryInfo implements Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -186,6 +193,24 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        writeBinary(out);
+
+        out.writeMap(listing);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        readBinary(in);
+
+        listing = in.readMap();
+    }
+
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return id.hashCode() ^ (props == null ? 0 : props.hashCode());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java
index c5b1111..d31ef72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java
@@ -17,7 +17,11 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
@@ -302,4 +306,35 @@ public abstract class IgfsEntryInfo implements Externalizable {
         modificationTime = in.readLong();
         path = (IgfsPath)in.readObject();
     }
+
+    /**
+     * Write binary content.
+     *
+     * @param out Writer.
+     */
+    protected void writeBinary(BinaryRawWriter out) {
+        BinaryUtils.writeIgniteUuid(out, id);
+        out.writeMap(props);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
+        out.writeObject(path);
+    }
+
+    /**
+     * Read binary content.
+     *
+     * @param in Reader.
+     */
+    protected void readBinary(BinaryRawReader in) {
+        id = BinaryUtils.readIgniteUuid(in);
+        props = in.readMap();
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
+        path = in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsEntryInfo.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
index 9ac914f..953461c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java
@@ -25,6 +25,14 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+
+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.binary.BinaryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -37,7 +45,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Affinity range.
  */
-public class IgfsFileAffinityRange implements Message, Externalizable {
+public class IgfsFileAffinityRange implements Message, Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -243,9 +251,7 @@ public class IgfsFileAffinityRange implements Message, Externalizable {
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeGridUuid(out, affKey);
-
         out.writeInt(status);
-
         out.writeLong(startOff);
         out.writeLong(endOff);
     }
@@ -253,9 +259,27 @@ public class IgfsFileAffinityRange implements Message, Externalizable {
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         affKey = U.readGridUuid(in);
-
         status = in.readInt();
+        startOff = in.readLong();
+        endOff = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
 
+        BinaryUtils.writeIgniteUuid(out, affKey);
+        out.writeInt(status);
+        out.writeLong(startOff);
+        out.writeLong(endOff);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        affKey = BinaryUtils.readIgniteUuid(in);
+        status = in.readInt();
         startOff = in.readLong();
         endOff = in.readLong();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index 30f505e..337f281 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -17,6 +17,13 @@
 
 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.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -33,7 +40,7 @@ import java.util.Map;
 /**
  * IGFS file info.
  */
-public final class IgfsFileInfo extends IgfsEntryInfo {
+public final class IgfsFileInfo extends IgfsEntryInfo implements Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -230,6 +237,34 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        writeBinary(out);
+
+        out.writeInt(blockSize);
+        out.writeLong(len);
+        BinaryUtils.writeIgniteUuid(out, lockId);
+        BinaryUtils.writeIgniteUuid(out, affKey);
+        out.writeObject(fileMap);
+        out.writeBoolean(evictExclude);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        readBinary(in);
+
+        blockSize = in.readInt();
+        len = in.readLong();
+        lockId = BinaryUtils.readIgniteUuid(in);
+        affKey = BinaryUtils.readIgniteUuid(in);
+        fileMap = in.readObject();
+        evictExclude = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return id.hashCode() ^ blockSize ^ (int)(len ^ (len >>> 32)) ^ (props == null ? 0 : props.hashCode()) ^
             (lockId == null ? 0 : lockId.hashCode());

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileMap.java
index 9ea69ea..2890e2a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileMap.java
@@ -24,7 +24,12 @@ import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-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.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -36,12 +41,12 @@ import static org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange.R
 /**
  * Auxiliary class that is responsible for managing file affinity keys allocation by ranges.
  */
-public class IgfsFileMap implements Externalizable {
+public class IgfsFileMap implements Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    @GridToStringInclude
     /** Sorted list of ranges in ascending order. */
+    @GridToStringInclude
     private List<IgfsFileAffinityRange> ranges;
 
     /**
@@ -358,6 +363,36 @@ public class IgfsFileMap implements Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        if (ranges == null)
+            out.writeInt(-1);
+        else {
+            assert !ranges.isEmpty();
+
+            out.writeInt(ranges.size());
+
+            for (IgfsFileAffinityRange range : ranges)
+                out.writeObject(range);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        int size = in.readInt();
+
+        if (size > 0) {
+            ranges = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++)
+                ranges.add((IgfsFileAffinityRange)in.readObject());
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsFileMap.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
index ea05ca3..2401656 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
@@ -17,6 +17,13 @@
 
 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.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -30,7 +37,7 @@ import java.io.ObjectOutput;
 /**
  * Directory listing entry.
  */
-public class IgfsListingEntry implements Externalizable {
+public class IgfsListingEntry implements Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -102,6 +109,22 @@ public class IgfsListingEntry implements Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        BinaryUtils.writeIgniteUuid(out, id);
+        out.writeBoolean(dir);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        id = BinaryUtils.readIgniteUuid(in);
+        dir = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean equals(Object other) {
         return this == other || other instanceof IgfsListingEntry && F.eq(id, ((IgfsListingEntry)other).id);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
index ffba042..ded66bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
@@ -1,8 +1,15 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
@@ -20,7 +27,7 @@ import java.util.Map;
  * Directory create processor.
  */
 public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -114,4 +121,33 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
             childEntry = (IgfsListingEntry)in.readObject();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeLong(createTime);
+        out.writeMap(props);
+        out.writeString(childName);
+
+        if (childName != null)
+            out.writeObject(childEntry);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        createTime = in.readLong();
+        props = in.readMap();
+        childName = in.readString();
+
+        if (childName != null)
+            childEntry = in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaDirectoryCreateProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
index ab5cd5d..d45dea1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
@@ -1,6 +1,12 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
 import org.apache.ignite.IgniteException;
+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.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -20,7 +26,7 @@ import java.util.Map;
  * Update directory listing closure.
  */
 public final class IgfsMetaDirectoryListingAddProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -86,6 +92,22 @@ public final class IgfsMetaDirectoryListingAddProcessor implements EntryProcesso
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeString(fileName);
+        out.writeObject(entry);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        fileName = in.readString();
+        entry = in.readObject();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsMetaDirectoryListingAddProcessor.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
index 181a73e..d5da117 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
@@ -1,9 +1,16 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
 import org.apache.ignite.IgniteException;
+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.binary.BinaryUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
-import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
@@ -21,7 +28,7 @@ import java.util.Map;
  * Remove entry from directory listing.
  */
 public class IgfsMetaDirectoryListingRemoveProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -86,4 +93,25 @@ public class IgfsMetaDirectoryListingRemoveProcessor implements EntryProcessor<I
         fileName = U.readString(in);
         fileId = U.readGridUuid(in);
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeString(fileName);
+        BinaryUtils.writeIgniteUuid(out, fileId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        fileName = in.readString();
+        fileId = BinaryUtils.readIgniteUuid(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaDirectoryListingRemoveProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
index 4c4888c..bf3c288 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
@@ -1,8 +1,16 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
 import org.apache.ignite.IgniteException;
+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.binary.BinaryUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
@@ -20,7 +28,7 @@ import java.util.Map;
  * Listing replace processor.
  */
 public final class IgfsMetaDirectoryListingReplaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -81,4 +89,25 @@ public final class IgfsMetaDirectoryListingReplaceProcessor implements EntryProc
         name = U.readString(in);
         id = (IgniteUuid)in.readObject();
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeString(name);
+        BinaryUtils.writeIgniteUuid(out, id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        name = in.readString();
+        id = BinaryUtils.readIgniteUuid(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaDirectoryListingReplaceProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
index a07d764..418227a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
@@ -1,7 +1,15 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.binary.BinaryUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
@@ -19,7 +27,7 @@ import java.util.Map;
  * File create processor.
  */
 public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -107,4 +115,33 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
         lockId = (IgniteUuid)in.readObject();
         evictExclude = in.readBoolean();
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeLong(createTime);
+        out.writeMap(props);
+        out.writeInt(blockSize);
+        BinaryUtils.writeIgniteUuid(out, affKey);
+        BinaryUtils.writeIgniteUuid(out, lockId);
+        out.writeBoolean(evictExclude);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        createTime = in.readLong();
+        props = in.readMap();
+        blockSize = in.readInt();
+        affKey = BinaryUtils.readIgniteUuid(in);
+        lockId = BinaryUtils.readIgniteUuid(in);
+        evictExclude = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileCreateProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
index 43a5d7d..ff71f13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
@@ -1,6 +1,14 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.binary.BinaryUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
@@ -16,7 +24,7 @@ import java.io.ObjectOutput;
  * File lock entry processor.
  */
 public class IgfsMetaFileLockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -60,4 +68,23 @@ public class IgfsMetaFileLockProcessor implements EntryProcessor<IgniteUuid, Igf
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         lockId = U.readGridUuid(in);
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        BinaryUtils.writeIgniteUuid(out, lockId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        lockId = BinaryUtils.readIgniteUuid(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileLockProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
index 5af09e5..cdf07d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
@@ -1,5 +1,11 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
 import org.apache.ignite.internal.processors.igfs.IgfsFileMap;
@@ -18,7 +24,7 @@ import java.io.ObjectOutput;
  * Delete range processor.
  */
 public class IgfsMetaFileRangeDeleteProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -68,6 +74,20 @@ public class IgfsMetaFileRangeDeleteProcessor implements EntryProcessor<IgniteUu
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeObject(range);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        range = in.readObject();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsMetaFileRangeDeleteProcessor.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
index ae886c8..e1224de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
@@ -1,5 +1,11 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
 import org.apache.ignite.internal.processors.igfs.IgfsFileMap;
@@ -18,7 +24,7 @@ import java.io.ObjectOutput;
  * Update range processor.
  */
 public class IgfsMetaFileRangeUpdateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -75,6 +81,22 @@ public class IgfsMetaFileRangeUpdateProcessor implements EntryProcessor<IgniteUu
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeObject(range);
+        out.writeInt(status);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        range = in.readObject();
+        status = in.readInt();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsMetaFileRangeUpdateProcessor.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
index 30f8e22..a54ddcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
@@ -1,8 +1,15 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.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 javax.cache.processor.EntryProcessor;
@@ -17,7 +24,7 @@ import java.io.ObjectOutput;
  * File reserve space entry processor.
  */
 public class IgfsMetaFileReserveSpaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -38,7 +45,7 @@ public class IgfsMetaFileReserveSpaceProcessor implements EntryProcessor<IgniteU
      * Constructor.
      *
      * @param space Space.
-     * @param affRange
+     * @param affRange Affinity range.
      */
     public IgfsMetaFileReserveSpaceProcessor(long space, IgfsFileAffinityRange affRange) {
         this.space = space;
@@ -72,4 +79,25 @@ public class IgfsMetaFileReserveSpaceProcessor implements EntryProcessor<IgniteU
         space = in.readLong();
         affRange = (IgfsFileAffinityRange)in.readObject();
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeLong(space);
+        out.writeObject(affRange);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        space = in.readLong();
+        affRange = in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileReserveSpaceProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/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 d535a1d..0d06c92 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
@@ -1,6 +1,13 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.IgfsEntryInfo;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 
 import javax.cache.processor.EntryProcessor;
@@ -15,7 +22,7 @@ import java.io.ObjectOutput;
  * File unlock entry processor.
  */
 public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -57,4 +64,23 @@ public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, I
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         modificationTime = in.readLong();
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        modificationTime = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileUnlockProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
index 32e5512..81c1806 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
@@ -1,8 +1,13 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
-import org.apache.ignite.internal.processors.igfs.IgfsMetaManager;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 
@@ -17,7 +22,7 @@ import java.io.ObjectOutput;
  * Update path closure.
  */
 public final class IgfsMetaUpdatePathProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -60,6 +65,20 @@ public final class IgfsMetaUpdatePathProcessor implements EntryProcessor<IgniteU
     }
 
     /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeObject(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        path = in.readObject();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsMetaUpdatePathProcessor.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
index a97c186..a9751c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
@@ -1,7 +1,14 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.IgfsEntryInfo;
 import org.apache.ignite.internal.util.GridLeanMap;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
@@ -18,7 +25,7 @@ import java.util.Map;
  * Update properties processor.
  */
 public class IgfsMetaUpdatePropertiesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -75,4 +82,23 @@ public class IgfsMetaUpdatePropertiesProcessor implements EntryProcessor<IgniteU
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         props = U.readStringMap(in);
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeMap(props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        props = in.readMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaUpdatePropertiesProcessor.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19b4da3d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
index 8acd289..b754f19 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
@@ -1,6 +1,13 @@
 package org.apache.ignite.internal.processors.igfs.meta;
 
+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.IgfsEntryInfo;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 
 import javax.cache.processor.EntryProcessor;
@@ -15,7 +22,7 @@ import java.io.ObjectOutput;
  * Update times entry processor.
  */
 public class IgfsMetaUpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
-    Externalizable {
+    Externalizable, Binarylizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -65,4 +72,25 @@ public class IgfsMetaUpdateTimesProcessor implements EntryProcessor<IgniteUuid,
         accessTime = in.readLong();
         modificationTime = in.readLong();
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaUpdateTimesProcessor.class, this);
+    }
 }


[02/17] ignite git commit: CacheEntryProcessorCopySelfTest should be added to suite conditionally

Posted by dm...@apache.org.
CacheEntryProcessorCopySelfTest should be added to suite conditionally


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

Branch: refs/heads/ignite-2849
Commit: 3e53f17224c2fccfcce50b347a0dcc6aca543413
Parents: d3420e6
Author: agura <ag...@gridgain.com>
Authored: Thu Mar 17 16:03:20 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Mar 17 16:14:01 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3e53f172/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 2613175..9892ff7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -180,7 +180,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheAtomicLocalWithStoreInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxInvokeTest.class);
         suite.addTestSuite(IgniteCacheEntryProcessorCallTest.class);
-        suite.addTestSuite(CacheEntryProcessorCopySelfTest.class);
+        GridTestUtils.addTestIfNeeded(suite, CacheEntryProcessorCopySelfTest.class, ignoredTests);
         suite.addTestSuite(IgniteCacheTxNearEnabledInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxLocalInvokeTest.class);
         suite.addTestSuite(IgniteCrossCacheTxStoreSelfTest.class);
@@ -295,6 +295,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(CachePutEventListenerErrorSelfTest.class);
 
         suite.addTestSuite(IgniteTxConfigCacheSelfTest.class);
+
         return suite;
     }
 }


[08/17] ignite git commit: IGNITE-2813: IGFS: Optimized metadata values splitting file and directory into separate classes.

Posted by dm...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 06ae40b..013bb18 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
@@ -17,12 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.nio.ByteBuffer;
-import java.security.SecureRandom;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.concurrent.Callable;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -44,6 +38,13 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
 
+import java.nio.ByteBuffer;
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.Callable;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -168,8 +169,8 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             long t = System.currentTimeMillis();
 
-            IgfsFileInfo info = new IgfsFileInfo(200, 0L, null, IgfsUtils.DELETE_LOCK_ID,
-                    false, null, t, t);
+            IgfsEntryInfo info = IgfsUtils.createFile(IgniteUuid.randomUuid(), 200, 0L, null,
+                IgfsUtils.DELETE_LOCK_ID, false, null, t, t);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
 
@@ -181,11 +182,11 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             expectsStoreFail(info, data, "Not enough space reserved to store data");
 
-            info = new IgfsFileInfo(info, info.length() + data.length - 3);
+            info = info.length(info.length() + data.length - 3);
 
             expectsStoreFail(info, data, "Not enough space reserved to store data");
 
-            info = new IgfsFileInfo(info, info.length() + 3);
+            info = info.length(info.length() + 3);
 
             IgfsFileAffinityRange range = new IgfsFileAffinityRange();
 
@@ -253,8 +254,8 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             long t = System.currentTimeMillis();
 
-            IgfsFileInfo info = new IgfsFileInfo(blockSize, 0L, null, IgfsUtils.DELETE_LOCK_ID,
-                false, null, t, t);
+            IgfsEntryInfo info = IgfsUtils.createFile(IgniteUuid.randomUuid(), blockSize, 0L, null,
+                IgfsUtils.DELETE_LOCK_ID, false, null, t, t);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
 
@@ -266,7 +267,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             rnd.nextBytes(remainder);
 
-            info = new IgfsFileInfo(info, info.length() + data.length + remainder.length);
+            info = info.length(info.length() + data.length + remainder.length);
 
             IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
 
@@ -279,7 +280,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             byte[] remainder2 = new byte[blockSize / 2];
 
-            info = new IgfsFileInfo(info, info.length() + remainder2.length);
+            info = info.length(info.length() + remainder2.length);
 
             byte[] left2 = mgr.storeDataBlocks(info, info.length(), left, left.length, ByteBuffer.wrap(remainder2),
                 false, range, null);
@@ -344,9 +345,8 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             long t = System.currentTimeMillis();
 
-            IgfsFileInfo info =
-                new IgfsFileInfo(blockSize, 0L, null, IgfsUtils.DELETE_LOCK_ID,
-                    false, null, t, t);
+            IgfsEntryInfo info = IgfsUtils.createFile(IgniteUuid.randomUuid(), blockSize, 0L, null,
+                IgfsUtils.DELETE_LOCK_ID, false, null, t, t);
 
             IgfsFileAffinityRange range = new IgfsFileAffinityRange();
 
@@ -356,7 +356,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
             byte[] data = new byte[chunkSize];
 
-            info = new IgfsFileInfo(info, info.length() + data.length * writesCnt);
+            info = info.length(info.length() + data.length * writesCnt);
 
             IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
 
@@ -416,7 +416,8 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
         long t = System.currentTimeMillis();
 
         //IgfsFileInfo info = new IgfsFileInfo(blockSize, 0);
-        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null, t, t);
+        IgfsEntryInfo info = IgfsUtils.createFile(IgniteUuid.randomUuid(), blockSize, 1024 * 1024, null, null, false,
+            null, t, t);
 
         for (int pos = 0; pos < 5 * grpSize; pos++) {
             assertEquals("Expects no affinity for zero length.", Collections.<IgfsBlockLocation>emptyList(),
@@ -466,7 +467,8 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
         long t = System.currentTimeMillis();
 
-        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null, t, t);
+        IgfsEntryInfo info = IgfsUtils.createFile(IgniteUuid.randomUuid(), blockSize, 1024 * 1024, null, null, false,
+            null, t, t);
 
         Collection<IgfsBlockLocation> affinity = mgr.affinity(info, 0, info.length());
 
@@ -499,7 +501,8 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
         long t = System.currentTimeMillis();
 
-        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null, t, t);
+        IgfsEntryInfo info = IgfsUtils.createFile(IgniteUuid.randomUuid(), blockSize, 1024 * 1024, null, null, false,
+            null, t, t);
 
         IgniteUuid affKey = IgniteUuid.randomUuid();
 
@@ -508,7 +511,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
         map.addRange(new IgfsFileAffinityRange(3 * BLOCK_SIZE, 5 * BLOCK_SIZE - 1, affKey));
         map.addRange(new IgfsFileAffinityRange(13 * BLOCK_SIZE, 17 * BLOCK_SIZE - 1, affKey));
 
-        info.fileMap(map);
+        info = info.fileMap(map);
 
         Collection<IgfsBlockLocation> affinity = mgr.affinity(info, 0, info.length());
 
@@ -542,7 +545,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
      * @param info File info.
      * @param affinity Affinity block locations to check.
      */
-    private void checkAffinity(int blockSize, IgfsFileInfo info, Iterable<IgfsBlockLocation> affinity) {
+    private void checkAffinity(int blockSize, IgfsEntryInfo info, Iterable<IgfsBlockLocation> affinity) {
         for (IgfsBlockLocation loc : affinity) {
             info("Going to check IGFS block location: " + loc);
 
@@ -574,7 +577,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
      * @param data Data to store.
      * @param msg Expected failure message.
      */
-    private void expectsStoreFail(final IgfsFileInfo reserved, final byte[] data, @Nullable String msg) {
+    private void expectsStoreFail(final IgfsEntryInfo reserved, final byte[] data, @Nullable String msg) {
         GridTestUtils.assertThrows(log, new Callable() {
             @Override public Object call() throws Exception {
                 IgfsFileAffinityRange range = new IgfsFileAffinityRange();
@@ -592,7 +595,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
      * @param fileInfo File to delete data for.
      * @param msg Expected failure message.
      */
-    private void expectsDeleteFail(final IgfsFileInfo fileInfo, @Nullable String msg) {
+    private void expectsDeleteFail(final IgfsEntryInfo fileInfo, @Nullable String msg) {
         GridTestUtils.assertThrows(log, new Callable() {
             @Override public Object call() throws Exception {
                 mgr.delete(fileInfo);
@@ -610,7 +613,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
      * @param len File part length to get affinity for.
      * @param msg Expected failure message.
      */
-    private void expectsAffinityFail(final IgfsFileInfo info, final long start, final long len,
+    private void expectsAffinityFail(final IgfsEntryInfo info, final long start, final long len,
         @Nullable String msg) {
         GridTestUtils.assertThrows(log, new Callable() {
             @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
index 0731436..6e13280 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
@@ -17,13 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CyclicBarrier;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.igfs.IgfsFile;
@@ -36,6 +29,14 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+
 import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC;
 import static org.apache.ignite.internal.processors.igfs.IgfsEx.PROP_PERMISSION;
@@ -1254,7 +1255,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         // Wait for a while for prefetch to finish.
         IgfsMetaManager meta = igfs.context().meta();
 
-        IgfsFileInfo info = meta.info(meta.fileId(FILE));
+        IgfsEntryInfo info = meta.info(meta.fileId(FILE));
 
         assert info != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
index feccdb5..486179c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
@@ -17,18 +17,19 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.Externalizable;
-import java.util.Random;
-import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.Externalizable;
+import java.util.Random;
+import java.util.concurrent.Callable;
+
 /**
- * {@link IgfsFileInfo} test case.
+ * {@link IgfsEntryInfo} test case.
  */
 public class IgfsFileInfoSelfTest extends IgfsCommonAbstractTest {
     /** Marshaller to test {@link Externalizable} interface. */
@@ -49,23 +50,7 @@ public class IgfsFileInfoSelfTest extends IgfsCommonAbstractTest {
 
             @SuppressWarnings("deprecation") // Suppress due to default constructor should never be used directly.
             @Nullable @Override public Object call() throws IgniteCheckedException {
-                for (int i = 0; i < 10000; i++) {
-                    testSerialization(new IgfsFileInfo());
-                    testSerialization(new IgfsFileInfo());
-
-                    long a = System.currentTimeMillis();
-                    long m = a + 1;
-
-                    testSerialization(new IgfsFileInfo(true, null, a, m));
-                    testSerialization(new IgfsFileInfo(false, null, a, m));
-
-                    IgfsFileInfo rndInfo =
-                        new IgfsFileInfo(rnd.nextInt(max), 0, false, null, a, m);
-
-                    testSerialization(rndInfo);
-                    testSerialization(new IgfsFileInfo(rndInfo, rnd.nextInt(max)));
-                    testSerialization(new IgfsFileInfo(rndInfo, F.asMap("desc", String.valueOf(rnd.nextLong()))));
-                }
+                testSerialization(IgfsUtils.createDirectory(IgniteUuid.randomUuid()));
 
                 return null;
             }
@@ -78,7 +63,7 @@ public class IgfsFileInfoSelfTest extends IgfsCommonAbstractTest {
      * @param info Node info to test serialization for.
      * @throws IgniteCheckedException If failed.
      */
-    public void testSerialization(IgfsFileInfo info) throws IgniteCheckedException {
+    public void testSerialization(IgfsEntryInfo info) throws IgniteCheckedException {
         assertEquals(info, mu(info));
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index 26424f0..3dc2791 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -147,12 +147,12 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         IgfsListingEntry dirEntry = mgr.directoryListing(ROOT_ID).get("dir");
         assertNotNull(dirEntry);
         assertTrue(dirEntry.isDirectory());
-        IgfsFileInfo dir = mgr.info(dirEntry.fileId());
+        IgfsEntryInfo dir = mgr.info(dirEntry.fileId());
 
         IgfsListingEntry fileEntry = mgr.directoryListing(ROOT_ID).get("file");
         assertNotNull(fileEntry);
         assertTrue(!fileEntry.isDirectory());
-        IgfsFileInfo file = mgr.info(fileEntry.fileId());
+        IgfsEntryInfo file = mgr.info(fileEntry.fileId());
 
         assertEquals(2, mgr.directoryListing(ROOT_ID).size());
 
@@ -166,7 +166,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
             String key1 = UUID.randomUUID().toString();
             String key2 = UUID.randomUUID().toString();
 
-            IgfsFileInfo info = mgr.info(fileId);
+            IgfsEntryInfo info = mgr.info(fileId);
 
             assertNull("Unexpected stored properties: " + info, info.properties().get(key1));
             assertNull("Unexpected stored properties: " + info, info.properties().get(key2));
@@ -197,24 +197,24 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertNull(mgr.updateProperties(dir.id(), F.asMap("p", "7")));
     }
 
-    private IgfsFileInfo mkdirsAndGetInfo(String path) throws IgniteCheckedException {
+    private IgfsEntryInfo mkdirsAndGetInfo(String path) throws IgniteCheckedException {
         IgfsPath p = path(path);
 
         mgr.mkdirs(p, IgfsImpl.DFLT_DIR_META);
 
         IgniteUuid id = mgr.fileId(p);
 
-        IgfsFileInfo info = mgr.info(id);
+        IgfsEntryInfo info = mgr.info(id);
 
         assert info.isDirectory();
 
         return info;
     }
 
-    private IgfsFileInfo createFileAndGetInfo(String path) throws IgniteCheckedException {
+    private IgfsEntryInfo createFileAndGetInfo(String path) throws IgniteCheckedException {
         IgfsPath p = path(path);
 
-        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = mgr.create(p, null, false, 400, null, false, null);
+        IgniteBiTuple<IgfsEntryInfo, IgniteUuid> t2 = mgr.create(p, null, false, 400, null, false, null);
 
         assert t2 != null;
         assert !t2.get1().isDirectory();
@@ -228,7 +228,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStructure() throws Exception {
-        IgfsFileInfo rootInfo = new IgfsFileInfo();
+        IgfsEntryInfo rootInfo = IgfsUtils.createDirectory(ROOT_ID);
 
         // Test empty structure.
         assertEmpty(mgr.directoryListing(ROOT_ID));
@@ -236,15 +236,15 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEquals(F.asMap(ROOT_ID, rootInfo), mgr.infos(Arrays.asList(ROOT_ID)));
 
         // Directories:
-        IgfsFileInfo a = mkdirsAndGetInfo("/a");
-        IgfsFileInfo b = mkdirsAndGetInfo("/a/b");
-        IgfsFileInfo k = mkdirsAndGetInfo("/a/b/k");
-        IgfsFileInfo z = mkdirsAndGetInfo("/a/k");
+        IgfsEntryInfo a = mkdirsAndGetInfo("/a");
+        IgfsEntryInfo b = mkdirsAndGetInfo("/a/b");
+        IgfsEntryInfo k = mkdirsAndGetInfo("/a/b/k");
+        IgfsEntryInfo z = mkdirsAndGetInfo("/a/k");
 
         // Files:
-        IgfsFileInfo f1 = createFileAndGetInfo("/f1");
-        IgfsFileInfo f2 = createFileAndGetInfo("/a/f2");
-        IgfsFileInfo f3 = createFileAndGetInfo("/a/b/f3");
+        IgfsEntryInfo f1 = createFileAndGetInfo("/f1");
+        IgfsEntryInfo f2 = createFileAndGetInfo("/a/f2");
+        IgfsEntryInfo f3 = createFileAndGetInfo("/a/b/f3");
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -256,11 +256,11 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
             "k", new IgfsListingEntry(k)), mgr.directoryListing(b.id()));
 
         // Validate empty files listings.
-        for (IgfsFileInfo info : Arrays.asList(f1, f2, f3))
+        for (IgfsEntryInfo info : Arrays.asList(f1, f2, f3))
             assertEmpty(mgr.directoryListing(info.id()));
 
         // Validate 'file info' operations.
-        for (IgfsFileInfo info : Arrays.asList(rootInfo, a, b, f1, f2, f3)) {
+        for (IgfsEntryInfo info : Arrays.asList(rootInfo, a, b, f1, f2, f3)) {
             assertEquals(info, mgr.info(info.id()));
             assertEquals(F.asMap(info.id(), info), mgr.infos(Arrays.asList(info.id())));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
index 4112846..bcc2314 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
@@ -17,17 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicInteger;
-import javax.cache.Cache;
 import org.apache.commons.io.IOUtils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -56,6 +45,18 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
 
+import javax.cache.Cache;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -227,8 +228,8 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
             IgfsFileImpl info = (IgfsFileImpl)igfs.info(path);
 
             for (int i = 0; i < nodesCount(); i++) {
-                IgfsFileInfo fileInfo =
-                    (IgfsFileInfo)grid(i).cachex(metaCacheName).localPeek(info.fileId(), ONHEAP_PEEK_MODES, null);
+                IgfsEntryInfo fileInfo =
+                    (IgfsEntryInfo)grid(i).cachex(metaCacheName).localPeek(info.fileId(), ONHEAP_PEEK_MODES, null);
 
                 assertNotNull(fileInfo);
                 assertNotNull(fileInfo.listing());
@@ -693,10 +694,10 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
 
         IgniteUuid fileId = U.field(igfs.info(path), "fileId");
 
-        GridCacheAdapter<IgniteUuid, IgfsFileInfo> metaCache = ((IgniteKernal)grid(0)).internalCache(META_CACHE_NAME);
+        GridCacheAdapter<IgniteUuid, IgfsEntryInfo> metaCache = ((IgniteKernal)grid(0)).internalCache(META_CACHE_NAME);
         GridCacheAdapter<IgfsBlockKey, byte[]> dataCache = ((IgniteKernal)grid(0)).internalCache(DATA_CACHE_NAME);
 
-        IgfsFileInfo info = metaCache.get(fileId);
+        IgfsEntryInfo info = metaCache.get(fileId);
 
         assertNotNull(info);
         assertTrue(info.isFile());

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 fcf4b3b..3933e86 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
@@ -17,17 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-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 org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
@@ -55,6 +44,18 @@ 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;
@@ -505,12 +506,12 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
         os.write(chunk((int)igfsMaxData));
         os.close();
 
-        final IgniteCache<IgniteUuid, IgfsFileInfo> metaCache = igfs.context().kernalContext().cache().jcache(
+        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 IgfsFileInfo info = igfs.context().meta().info(id);
+        final IgfsEntryInfo info = igfs.context().meta().info(id);
 
         final AtomicReference<Throwable> err = new AtomicReference<>();
 
@@ -543,15 +544,15 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
                     new IgfsListingEntry(info));
 
                 // Clear root listing.
-                metaCache.put(IgfsUtils.ROOT_ID, new IgfsFileInfo(IgfsUtils.ROOT_ID));
+                metaCache.put(IgfsUtils.ROOT_ID, IgfsUtils.createDirectory(IgfsUtils.ROOT_ID));
 
                 // Add file to trash listing.
-                IgfsFileInfo trashInfo = metaCache.get(trashId);
+                IgfsEntryInfo trashInfo = metaCache.get(trashId);
 
                 if (trashInfo == null)
-                    metaCache.put(trashId, new IgfsFileInfo(listing, new IgfsFileInfo(trashId)));
+                    metaCache.put(trashId, IgfsUtils.createDirectory(trashId).listing(listing));
                 else
-                    metaCache.put(trashId, new IgfsFileInfo(listing, trashInfo));
+                    metaCache.put(trashId, trashInfo.listing(listing));
 
                 tx.commit();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
index 8383a18..7b7078f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
@@ -17,17 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
@@ -53,6 +42,18 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -276,7 +277,7 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
 
             GridCacheAdapter<Object, Object> metaCache = ((IgniteKernal)grid(0)).internalCache(META_CACHE_NAME);
 
-            IgfsFileInfo fileInfo = (IgfsFileInfo)metaCache.get(fileImpl.fileId());
+            IgfsEntryInfo fileInfo = (IgfsEntryInfo)metaCache.get(fileImpl.fileId());
 
             IgfsFileMap map = fileInfo.fileMap();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
index 672b912..7a4648a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
@@ -41,11 +41,12 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadabl
 import org.apache.ignite.internal.processors.hadoop.HadoopPayloadAware;
 import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsProperties;
 import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsFileImpl;
-import org.apache.ignite.internal.processors.igfs.IgfsFileInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.jetbrains.annotations.Nullable;
 
@@ -350,13 +351,28 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
 
             Collection<IgfsFile> res = new ArrayList<>(statuses.length);
 
-            for (FileStatus status : statuses) {
-                IgfsFileInfo fsInfo = status.isDirectory() ?
-                    new IgfsFileInfo(true, properties(status), status.getAccessTime(), status.getModificationTime()) :
-                    new IgfsFileInfo((int)status.getBlockSize(), status.getLen(), null, null, false, properties(status),
-                        status.getAccessTime(), status.getModificationTime());
-
-                res.add(new IgfsFileImpl(new IgfsPath(path, status.getPath().getName()), fsInfo, 1));
+            for (FileStatus s : statuses) {
+                IgfsEntryInfo fsInfo = s.isDirectory() ?
+                    IgfsUtils.createDirectory(
+                        IgniteUuid.randomUuid(),
+                        null,
+                        properties(s),
+                        s.getAccessTime(),
+                        s.getModificationTime()
+                    ) :
+                    IgfsUtils.createFile(
+                        IgniteUuid.randomUuid(),
+                        (int)s.getBlockSize(),
+                        s.getLen(),
+                        null,
+                        null,
+                        false,
+                        properties(s),
+                        s.getAccessTime(),
+                        s.getModificationTime()
+                    );
+
+                res.add(new IgfsFileImpl(new IgfsPath(path, s.getPath().getName()), fsInfo, 1));
             }
 
             return res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
index 0ec8a78..40cf493 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.igfs;
 
-import java.io.IOException;
-import java.net.URI;
-import java.util.concurrent.Callable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -34,7 +31,7 @@ import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.processors.igfs.IgfsBlockKey;
 import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest;
-import org.apache.ignite.internal.processors.igfs.IgfsFileInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsImpl;
 import org.apache.ignite.internal.processors.igfs.IgfsMetaManager;
 import org.apache.ignite.internal.util.typedef.G;
@@ -44,6 +41,10 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.IOException;
+import java.net.URI;
+import java.util.concurrent.Callable;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -279,7 +280,7 @@ public abstract class HadoopIgfsDualAbstractSelfTest extends IgfsCommonAbstractT
         // Wait for a while for prefetch to finish (if any).
         IgfsMetaManager meta = igfs.context().meta();
 
-        IgfsFileInfo info = meta.info(meta.fileId(FILE));
+        IgfsEntryInfo info = meta.info(meta.fileId(FILE));
 
         IgfsBlockKey key = new IgfsBlockKey(info.id(), info.affinityKey(), info.evictExclude(), 2);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java
index e89d651..3013311 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.igfs;
 
+import org.apache.ignite.internal.igfs.common.IgfsLogger;
+import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -24,12 +29,6 @@ import java.io.FilenameFilter;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.ignite.internal.igfs.common.IgfsLogger;
-import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest;
-import org.apache.ignite.internal.processors.igfs.IgfsFileImpl;
-import org.apache.ignite.internal.processors.igfs.IgfsFileInfo;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
 import static org.apache.ignite.internal.igfs.common.IgfsLogger.DELIM_FIELD;
@@ -244,16 +243,6 @@ public class IgniteHadoopFileSystemLoggerSelfTest extends IgfsCommonAbstractTest
     }
 
     /**
-     * Create IGFS file with the given path.
-     *
-     * @param path File path.
-     * @return IGFS file instance.
-     */
-    private IgfsFile file(String path) {
-        return new IgfsFileImpl(new IgfsPath(path), new IgfsFileInfo(), 64 * 1024 * 1024);
-    }
-
-    /**
      * Ensure that log file has only the following lines.
      *
      * @param lines Expected lines.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index e03e4e1..ebc0c23 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -17,17 +17,6 @@
 
 package org.apache.ignite.testsuites;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.List;
 import junit.framework.TestSuite;
 import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
 import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
@@ -68,6 +57,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobTrackerSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceEmbeddedSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopSerializationWrapperSelfTest;
+import org.apache.ignite.internal.processors.hadoop.HadoopSnappyFullMapReduceTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopSnappyTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopSortingTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapperSelfTest;
@@ -76,7 +66,6 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTasksV1Test;
 import org.apache.ignite.internal.processors.hadoop.HadoopTasksV2Test;
 import org.apache.ignite.internal.processors.hadoop.HadoopV2JobSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopValidationSelfTest;
-import org.apache.ignite.internal.processors.hadoop.HadoopSnappyFullMapReduceTest;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimapSelftest;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopHashMapSelfTest;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipListSelfTest;
@@ -85,6 +74,18 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+
 import static org.apache.ignite.testframework.GridTestUtils.modeToPermissionSet;
 
 /**
@@ -184,6 +185,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSecondaryFileSystemConfigurationTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopTxConfigCacheTest.class.getName())));
+
         return suite;
     }
 


[11/17] ignite git commit: IGNITE-2860: IGFS: Fixed minor bug in append() operation.

Posted by dm...@apache.org.
IGNITE-2860: IGFS: Fixed minor bug in append() operation.


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

Branch: refs/heads/ignite-2849
Commit: c506c440900733580d208dbf32ba900da2f03fb9
Parents: 4794f87
Author: thatcoach <pp...@list.ru>
Authored: Fri Mar 18 21:15:04 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Fri Mar 18 21:15:04 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/igfs/IgfsMetaManager.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c506c440/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 84e4dae..d66d9be 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
@@ -3549,7 +3549,7 @@ public class IgfsMetaManager extends IgfsManager {
                             IgniteUuid oldId = pathIds.lastId();
 
                             id2InfoPrj.invoke(trashId, new ListingAddProcessor(oldId.toString(),
-                                new IgfsListingEntry(oldId, true)));
+                                new IgfsListingEntry(oldInfo)));
 
                             // Second step: replace ID in parent directory.
                             String name = pathIds.lastPart();


[17/17] ignite git commit: IGNITE-2849: disallowed to build a BinaryObject if old field's metadata is Object and the new one is different from it

Posted by dm...@apache.org.
IGNITE-2849: disallowed to build a BinaryObject if old field's metadata is Object and the new one is different from it


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

Branch: refs/heads/ignite-2849
Commit: 4846b21f5ebfa2b1cc8ea1cd02a811d168168ce8
Parents: 9d5dbd3
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Mar 21 15:43:16 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Mar 21 15:43:16 2016 +0300

----------------------------------------------------------------------
 .../binary/builder/BinaryObjectBuilderImpl.java |  9 +++--
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 36 +++++++++++++-------
 ...naryObjectBuilderDefaultMappersSelfTest.java |  2 +-
 3 files changed, 28 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4846b21f/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
index 3477dc1..16c51b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
@@ -383,18 +383,17 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
         else
             newFldTypeId = BinaryUtils.typeByClass(newVal.getClass());
 
-        String newFldTypeName = BinaryUtils.fieldTypeName(newFldTypeId);
-
         if (oldFldTypeName == null) {
             // It's a new field, we have to add it to metadata.
             if (fieldsMeta == null)
                 fieldsMeta = new HashMap<>();
 
-            fieldsMeta.put(name, BinaryUtils.fieldTypeId(newFldTypeName));
+            fieldsMeta.put(name, newFldTypeId);
         }
         else if (!nullFieldVal) {
-            if (!oldFldTypeName.equals(newFldTypeName) &&
-                BinaryUtils.fieldTypeId(oldFldTypeName) != GridBinaryMarshaller.OBJ) {
+            String newFldTypeName = BinaryUtils.fieldTypeName(newFldTypeId);
+
+            if (!F.eq(newFldTypeName, oldFldTypeName)) {
                 throw new BinaryObjectException(
                     "Wrong value has been set [" +
                         "typeName=" + (typeName == null ? meta.typeName() : typeName) +

http://git-wip-us.apache.org/repos/asf/ignite/blob/4846b21f/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
index e7988cf..e3e538b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
@@ -499,7 +499,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
 
-        mutObj.setField("foo", createdArr.clone());
+        mutObj.setField("foo", createdArr.clone(), Object.class);
 
         GridBinaryTestClasses.TestObjectContainer res = mutObj.build().deserialize();
 
@@ -556,7 +556,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         ArrayList<Object> list = Lists.newArrayList(mutObj, "a", Lists.newArrayList(1, 2));
 
-        mutObj.setField("foo", list);
+        mutObj.setField("foo", list, Object.class);
 
         GridBinaryTestClasses.TestObjectContainer res = mutObj.build().deserialize();
 
@@ -654,7 +654,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         List<Object> list = Lists.newLinkedList(Arrays.asList(mutObj, "a", Lists.newLinkedList(Arrays.asList(1, 2))));
 
-        mutObj.setField("foo", list);
+        mutObj.setField("foo", list, Object.class);
 
         GridBinaryTestClasses.TestObjectContainer res = mutObj.build().deserialize();
 
@@ -737,7 +737,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         Set<Object> c = Sets.newHashSet(mutObj, "a", Sets.newHashSet(1, 2));
 
-        mutObj.setField("foo", c);
+        mutObj.setField("foo", c, Object.class);
 
         GridBinaryTestClasses.TestObjectContainer res = mutObj.build().deserialize();
 
@@ -816,7 +816,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         Map<Object, Object> map = Maps.newHashMap(ImmutableMap.of(mutObj, "a", "b", mutObj));
 
-        mutObj.setField("foo", map);
+        mutObj.setField("foo", map, Object.class);
 
         GridBinaryTestClasses.TestObjectContainer res = mutObj.build().deserialize();
 
@@ -1005,11 +1005,16 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         builder.build();
 
-        builder = binaries().builder("SomeType");
+        try {
+            builder = binaries().builder("SomeType");
 
-        builder.setField("dateField", new Date());
+            builder.setField("dateField", new Date());
 
-        builder.build();
+            builder.build();
+        }
+        catch (BinaryObjectException ex) {
+            assertTrue(ex.getMessage().startsWith("Wrong value has been set"));
+        }
 
         builder = binaries().builder("SomeType");
 
@@ -1037,11 +1042,16 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         BinaryObject obj = builder.build();
 
-        builder = binaries().builder(obj);
+        try {
+            builder = binaries().builder(obj);
 
-        builder.setField("dateField", new Date());
+            builder.setField("dateField", new Date());
 
-        builder.build();
+            builder.build();
+        }
+        catch (BinaryObjectException ex) {
+            assertTrue(ex.getMessage().startsWith("Wrong value has been set"));
+        }
 
         builder = binaries().builder(obj);
 
@@ -1163,7 +1173,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         Date[] arr = {new Date()};
 
-        mutableObj.setField("foo", arr);
+        mutableObj.setField("foo", arr, Object.class);
 
         GridBinaryTestClasses.TestObjectContainer res = mutableObj.build().deserialize();
 
@@ -1182,7 +1192,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         Timestamp[] arr = {new Timestamp(100020003)};
 
-        mutableObj.setField("foo", arr);
+        mutableObj.setField("foo", arr, Object.class);
 
         GridBinaryTestClasses.TestObjectContainer res = mutableObj.build().deserialize();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4846b21f/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java
index 9659c39..4fc5c6c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderDefaultMappersSelfTest.java
@@ -146,7 +146,7 @@ public class BinaryObjectBuilderDefaultMappersSelfTest extends GridCommonAbstrac
 
         builder = builder(obj);
 
-        builder.setField("objField", "value");
+        builder.setField("objField", "value", Object.class);
         builder.setField("otherField", (Object)null);
 
         obj = builder.build();


[03/17] ignite git commit: Added ability to dump comm SPI stats to log. (cherry picked from commit fa356e3)

Posted by dm...@apache.org.
Added ability to dump comm SPI stats to log.
(cherry picked from commit fa356e3)


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

Branch: refs/heads/ignite-2849
Commit: 0b10e0c8766dc41aa723881a75a6a37f3406f1aa
Parents: 3e53f17
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Mar 18 16:09:35 2016 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Mar 18 16:17:18 2016 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 10 ++++
 .../GridCachePartitionExchangeManager.java      |  3 ++
 .../util/nio/GridNioRecoveryDescriptor.java     | 21 +++++++-
 .../ignite/internal/util/nio/GridNioServer.java | 52 +++++++++++++++++++-
 .../communication/tcp/TcpCommunicationSpi.java  |  8 +++
 .../tcp/TcpCommunicationSpiMBean.java           |  8 ++-
 6 files changed, 98 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b10e0c8/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 232ec2e..9ffbf4e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -1943,6 +1943,16 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         return getSpi().getOutboundMessagesQueueSize();
     }
 
+    /**
+     * Dumps SPI stats to logs in case TcpCommunicationSpi is used, no-op otherwise.
+     */
+    public void dumpStats() {
+        CommunicationSpi spi = getSpi();
+
+        if (spi instanceof TcpCommunicationSpi)
+            ((TcpCommunicationSpi)spi).dumpStats();
+    }
+
     /** {@inheritDoc} */
     @Override public void printMemoryStats() {
         X.println(">>>");

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b10e0c8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index a0f7f93..1681f2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -1130,6 +1130,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts())
             cacheCtx.preloader().dumpDebugInfo();
+
+        // Dump IO manager statistics.
+        cctx.gridIO().dumpStats();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b10e0c8/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 685d260..409bded 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -44,6 +44,9 @@ public class GridNioRecoveryDescriptor {
     /** Number of received messages. */
     private long rcvCnt;
 
+    /** Number of sent messages. */
+    private long sentCnt;
+
     /** Reserved flag. */
     private boolean reserved;
 
@@ -120,6 +123,13 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
+     * @return Number of sent messages.
+     */
+    public long sent() {
+        return sentCnt;
+    }
+
+    /**
      * @param lastAck Last acknowledged message.
      */
     public void lastAcknowledged(long lastAck) {
@@ -151,6 +161,8 @@ public class GridNioRecoveryDescriptor {
             if (resendCnt == 0) {
                 msgFuts.addLast(fut);
 
+                sentCnt++;
+
                 return msgFuts.size() < queueLimit;
             }
             else
@@ -187,6 +199,13 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
+     * @return Last acked message by remote node.
+     */
+    public long acked() {
+        return acked;
+    }
+
+    /**
      * Node left callback.
      *
      * @return {@code False} if descriptor is reserved.
@@ -379,4 +398,4 @@ public class GridNioRecoveryDescriptor {
     @Override public String toString() {
         return S.toString(GridNioRecoveryDescriptor.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b10e0c8/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 765b139..42c7ac7 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
@@ -551,6 +551,14 @@ public class GridNioServer<T> {
     }
 
     /**
+     *
+     */
+    public void dumpStats() {
+        for (int i = 0; i < clientWorkers.size(); i++)
+            clientWorkers.get(i).offer(new NioOperationFuture<Void>(null, NioOperation.DUMP_STATS));
+    }
+
+    /**
      * Establishes a session.
      *
      * @param ch Channel to register within the server and create session for.
@@ -1433,6 +1441,43 @@ public class GridNioServer<T> {
 
                                 break;
                             }
+
+                            case DUMP_STATS: {
+                                StringBuilder sb = new StringBuilder();
+
+                                Set<SelectionKey> keys = selector.keys();
+
+                                sb.append(U.nl())
+                                    .append(">> Selector info [idx=").append(idx)
+                                    .append(", keysCnt=").append(keys.size())
+                                    .append("]").append(U.nl());
+
+                                for (SelectionKey key : keys) {
+                                    GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+
+                                    sb.append("    Conn [")
+                                        .append("rmtAddr=").append(ses.remoteAddress())
+                                        .append(", locAddr=").append(ses.localAddress())
+                                        .append(", bytesRcvd=").append(ses.bytesReceived())
+                                        .append(", bytesSent=").append(ses.bytesSent());
+
+                                    if (ses.recoveryDescriptor() != null) {
+                                        sb.append(", msgsSent=").append(ses.recoveryDescriptor().sent())
+                                            .append(", msgsAckedByRmt=").append(ses.recoveryDescriptor().acked())
+                                            .append(", msgsRcvd=").append(ses.recoveryDescriptor().received());
+                                    }
+                                    else
+                                        sb.append(", recoveryDesc=null");
+
+                                    sb.append("]").append(U.nl());
+                                }
+
+                                if (log.isInfoEnabled())
+                                    log.info(sb.toString());
+
+                                // Complete the request just in case (none should wait on this future).
+                                req.onDone(true);
+                            }
                         }
                     }
 
@@ -1991,7 +2036,10 @@ public class GridNioServer<T> {
         PAUSE_READ,
 
         /** Resume read. */
-        RESUME_READ
+        RESUME_READ,
+
+        /** Dump statistics. */
+        DUMP_STATS
     }
 
     /**
@@ -2059,7 +2107,7 @@ public class GridNioServer<T> {
          * @param op Requested operation.
          */
         NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op) {
-            assert ses != null;
+            assert ses != null || op == NioOperation.DUMP_STATS : "Invalid params [ses=" + ses + ", op=" + op + ']';
             assert op != null;
             assert op != NioOperation.REGISTER;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b10e0c8/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 2a078ee..b283b82 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1380,6 +1380,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /** {@inheritDoc} */
+    @Override public void dumpStats() {
+        GridNioServer<Message> nioSrvr1 = nioSrvr;
+
+        if (nioSrvr1 != null)
+            nioSrvr1.dumpStats();
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
         initFailureDetectionTimeout();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b10e0c8/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index a785482..482e2ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -287,4 +287,10 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      */
     @MXBeanDescription("Slow client queue limit.")
     public int getSlowClientQueueLimit();
-}
\ No newline at end of file
+
+    /**
+     * Dumps SPI per-connection stats to logs.
+     */
+    @MXBeanDescription("Dumps SPI statistics to logs.")
+    public void dumpStats();
+}


[10/17] ignite git commit: IGNITE-2813: IGFS: Optimized metadata values splitting file and directory into separate classes.

Posted by dm...@apache.org.
IGNITE-2813: IGFS: Optimized metadata values splitting file and directory into separate classes.


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

Branch: refs/heads/ignite-2849
Commit: 4794f87b5ebe2865b6afa541ce601df42fb8f6e3
Parents: 4af5316
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Mar 18 17:45:48 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 18 17:45:48 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDataManager.java        | 103 ++--
 .../processors/igfs/IgfsDeleteWorker.java       |  27 +-
 .../processors/igfs/IgfsDirectoryInfo.java      | 210 ++++++++
 .../internal/processors/igfs/IgfsEntryInfo.java | 305 +++++++++++
 .../internal/processors/igfs/IgfsFileImpl.java  |  15 +-
 .../internal/processors/igfs/IgfsFileInfo.java  | 500 +++----------------
 .../processors/igfs/IgfsFileWorkerBatch.java    |  75 +--
 .../IgfsFileWorkerBatchCancelledException.java  |  51 ++
 .../igfs/IgfsFragmentizerManager.java           |  64 ++-
 .../internal/processors/igfs/IgfsImpl.java      |  28 +-
 .../processors/igfs/IgfsInputStreamAdapter.java |   5 +-
 .../processors/igfs/IgfsInputStreamImpl.java    |  32 +-
 .../processors/igfs/IgfsIpcHandler.java         |  16 +-
 .../processors/igfs/IgfsListingEntry.java       |  12 +-
 .../processors/igfs/IgfsMetaManager.java        | 432 ++++++++--------
 .../processors/igfs/IgfsOutputStreamImpl.java   |  19 +-
 .../internal/processors/igfs/IgfsPathIds.java   |   4 +-
 .../processors/igfs/IgfsPathsCreateResult.java  |   6 +-
 .../IgfsSecondaryInputStreamDescriptor.java     |   6 +-
 .../IgfsSecondaryOutputStreamDescriptor.java    |   9 +-
 .../internal/processors/igfs/IgfsUtils.java     |  68 +++
 .../internal/processors/igfs/package-info.java  |   2 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |   4 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |   4 +-
 .../igfs/IgfsDataManagerSelfTest.java           |  55 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |  17 +-
 .../processors/igfs/IgfsFileInfoSelfTest.java   |  31 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |  34 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |  31 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |  35 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |  25 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  32 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |  11 +-
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |  21 +-
 .../testsuites/IgniteHadoopTestSuite.java       |  26 +-
 35 files changed, 1321 insertions(+), 994 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 125d728..3825086 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
@@ -17,35 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.DataInput;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -87,6 +58,36 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import java.io.DataInput;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
 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.internal.GridTopic.TOPIC_IGFS;
@@ -368,7 +369,7 @@ public class IgfsDataManager extends IgfsManager {
      * @return Requested data block or {@code null} if nothing found.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteInternalFuture<byte[]> dataBlock(final IgfsFileInfo fileInfo, final IgfsPath path,
+    @Nullable public IgniteInternalFuture<byte[]> dataBlock(final IgfsEntryInfo fileInfo, final IgfsPath path,
         final long blockIdx, @Nullable final IgfsSecondaryFileSystemPositionedReadable secReader)
         throws IgniteCheckedException {
         //assert validTxState(any); // Allow this method call for any transaction state.
@@ -476,7 +477,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param fileInfo File info of file opened to write.
      * @return Future that will be completed when all ack messages are received or when write failed.
      */
-    public IgniteInternalFuture<Boolean> writeStart(IgfsFileInfo fileInfo) {
+    public IgniteInternalFuture<Boolean> writeStart(IgfsEntryInfo fileInfo) {
         WriteCompletionFuture fut = new WriteCompletionFuture(fileInfo.id());
 
         WriteCompletionFuture oldFut = pendingWrites.putIfAbsent(fileInfo.id(), fut);
@@ -495,7 +496,7 @@ public class IgfsDataManager extends IgfsManager {
      *
      * @param fileInfo File info being written.
      */
-    public void writeClose(IgfsFileInfo fileInfo) {
+    public void writeClose(IgfsEntryInfo fileInfo) {
         WriteCompletionFuture fut = pendingWrites.get(fileInfo.id());
 
         if (fut != null)
@@ -524,7 +525,7 @@ public class IgfsDataManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     @Nullable public byte[] storeDataBlocks(
-        IgfsFileInfo fileInfo,
+        IgfsEntryInfo fileInfo,
         long reservedLen,
         @Nullable byte[] remainder,
         int remainderLen,
@@ -557,7 +558,7 @@ public class IgfsDataManager extends IgfsManager {
      * @throws IOException If store failed.
      */
     @Nullable public byte[] storeDataBlocks(
-        IgfsFileInfo fileInfo,
+        IgfsEntryInfo fileInfo,
         long reservedLen,
         @Nullable byte[] remainder,
         int remainderLen,
@@ -579,7 +580,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param fileInfo File details to remove data for.
      * @return Delete future that will be completed when file is actually erased.
      */
-    public IgniteInternalFuture<Object> delete(IgfsFileInfo fileInfo) {
+    public IgniteInternalFuture<Object> delete(IgfsEntryInfo fileInfo) {
         if (!fileInfo.isFile()) {
             if (log.isDebugEnabled())
                 log.debug("Cannot delete content of not-data file: " + fileInfo);
@@ -595,7 +596,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param fileInfo File info.
      * @return Block key.
      */
-    public IgfsBlockKey blockKey(long blockIdx, IgfsFileInfo fileInfo) {
+    public IgfsBlockKey blockKey(long blockIdx, IgfsEntryInfo fileInfo) {
         if (fileInfo.affinityKey() != null)
             return new IgfsBlockKey(fileInfo.id(), fileInfo.affinityKey(), fileInfo.evictExclude(), blockIdx);
 
@@ -616,7 +617,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param range Range to clean up.
      * @param cleanNonColocated {@code True} if all blocks should be cleaned.
      */
-    public void cleanBlocks(IgfsFileInfo fileInfo, IgfsFileAffinityRange range, boolean cleanNonColocated) {
+    public void cleanBlocks(IgfsEntryInfo fileInfo, IgfsFileAffinityRange range, boolean cleanNonColocated) {
         long startIdx = range.startOffset() / fileInfo.blockSize();
 
         long endIdx = range.endOffset() / fileInfo.blockSize();
@@ -646,7 +647,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param fileInfo File info to move data for.
      * @param range Range to move.
      */
-    public void spreadBlocks(IgfsFileInfo fileInfo, IgfsFileAffinityRange range) {
+    public void spreadBlocks(IgfsEntryInfo fileInfo, IgfsFileAffinityRange range) {
         long startIdx = range.startOffset() / fileInfo.blockSize();
 
         long endIdx = range.endOffset() / fileInfo.blockSize();
@@ -721,7 +722,7 @@ public class IgfsDataManager extends IgfsManager {
      * @return Affinity blocks locations.
      * @throws IgniteCheckedException If failed.
      */
-    public Collection<IgfsBlockLocation> affinity(IgfsFileInfo info, long start, long len)
+    public Collection<IgfsBlockLocation> affinity(IgfsEntryInfo info, long start, long len)
         throws IgniteCheckedException {
         return affinity(info, start, len, 0);
     }
@@ -736,7 +737,7 @@ public class IgfsDataManager extends IgfsManager {
      * @return Affinity blocks locations.
      * @throws IgniteCheckedException If failed.
      */
-    public Collection<IgfsBlockLocation> affinity(IgfsFileInfo info, long start, long len, long maxLen)
+    public Collection<IgfsBlockLocation> affinity(IgfsEntryInfo info, long start, long len, long maxLen)
         throws IgniteCheckedException {
         assert validTxState(false);
         assert info.isFile() : "Failed to get affinity (not a file): " + info;
@@ -845,7 +846,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param maxLen Maximum allowed split length.
      * @param res Result collection to add regions to.
      */
-    private void affinity0(IgfsFileInfo info, long start, long len, long maxLen, Deque<IgfsBlockLocation> res) {
+    private void affinity0(IgfsEntryInfo info, long start, long len, long maxLen, Deque<IgfsBlockLocation> res) {
         long firstGrpIdx = start / grpBlockSize;
         long limitGrpIdx = (start + len + grpBlockSize - 1) / grpBlockSize;
 
@@ -1327,7 +1328,7 @@ public class IgfsDataManager extends IgfsManager {
      */
     private IgfsBlockKey createBlockKey(
         long block,
-        IgfsFileInfo fileInfo,
+        IgfsEntryInfo fileInfo,
         IgfsFileAffinityRange locRange
     ) {
         // If affinityKey is present, return block key as is.
@@ -1373,7 +1374,7 @@ public class IgfsDataManager extends IgfsManager {
          * @return Data remainder if {@code flush} flag is {@code false}.
          */
         @Nullable public byte[] storeDataBlocks(
-            IgfsFileInfo fileInfo,
+            IgfsEntryInfo fileInfo,
             long reservedLen,
             @Nullable byte[] remainder,
             final int remainderLen,
@@ -1632,10 +1633,10 @@ public class IgfsDataManager extends IgfsManager {
      */
     private class AsyncDeleteWorker extends GridWorker {
         /** File info for stop request. */
-        private final IgfsFileInfo stopInfo = new IgfsFileInfo();
+        private final IgfsEntryInfo stopInfo;
 
         /** Delete requests queue. */
-        private BlockingQueue<IgniteBiTuple<GridFutureAdapter<Object>, IgfsFileInfo>> delReqs =
+        private BlockingQueue<IgniteBiTuple<GridFutureAdapter<Object>, IgfsEntryInfo>> delReqs =
             new LinkedBlockingQueue<>();
 
         /**
@@ -1645,6 +1646,10 @@ public class IgfsDataManager extends IgfsManager {
          */
         protected AsyncDeleteWorker(@Nullable String gridName, String name, IgniteLogger log) {
             super(gridName, name, log);
+
+            long time = System.currentTimeMillis();
+
+            stopInfo = IgfsUtils.createDirectory(IgniteUuid.randomUuid());
         }
 
         /**
@@ -1658,7 +1663,7 @@ public class IgfsDataManager extends IgfsManager {
          * @param info File info to delete.
          * @return Future which completes when entry is actually removed.
          */
-        private IgniteInternalFuture<Object> deleteAsync(IgfsFileInfo info) {
+        private IgniteInternalFuture<Object> deleteAsync(IgfsEntryInfo info) {
             GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
 
             delReqs.offer(F.t(fut, info));
@@ -1670,10 +1675,10 @@ public class IgfsDataManager extends IgfsManager {
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             try {
                 while (!isCancelled()) {
-                    IgniteBiTuple<GridFutureAdapter<Object>, IgfsFileInfo> req = delReqs.take();
+                    IgniteBiTuple<GridFutureAdapter<Object>, IgfsEntryInfo> req = delReqs.take();
 
                     GridFutureAdapter<Object> fut = req.get1();
-                    IgfsFileInfo fileInfo = req.get2();
+                    IgfsEntryInfo fileInfo = req.get2();
 
                     // Identity check.
                     if (fileInfo == stopInfo) {
@@ -1734,7 +1739,7 @@ public class IgfsDataManager extends IgfsManager {
                 if (log.isDebugEnabled())
                     log.debug("Stopping asynchronous igfs file delete thread: " + name());
 
-                IgniteBiTuple<GridFutureAdapter<Object>, IgfsFileInfo> req = delReqs.poll();
+                IgniteBiTuple<GridFutureAdapter<Object>, IgfsEntryInfo> req = delReqs.poll();
 
                 while (req != null) {
                     req.get1().onCancelled();

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 ffddd3e..f6b26ab 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
@@ -17,13 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
@@ -37,6 +30,14 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_PURGED;
 import static org.apache.ignite.internal.GridTopic.TOPIC_IGFS;
 
@@ -162,7 +163,7 @@ public class IgfsDeleteWorker extends IgfsThread {
      * @param trashId Trash ID.
      */
     private void delete(IgniteUuid trashId) {
-        IgfsFileInfo info = null;
+        IgfsEntryInfo info = null;
 
         try {
             info = meta.info(trashId);
@@ -220,7 +221,7 @@ public class IgfsDeleteWorker extends IgfsThread {
         assert id != null;
 
         while (true) {
-            IgfsFileInfo info = meta.info(id);
+            IgfsEntryInfo info = meta.info(id);
 
             if (info != null) {
                 if (info.isDirectory()) {
@@ -234,7 +235,7 @@ public class IgfsDeleteWorker extends IgfsThread {
                     assert info.isFile();
 
                     // Lock the file with special lock Id to prevent concurrent writing:
-                    IgfsFileInfo lockedInfo = meta.lock(id, true);
+                    IgfsEntryInfo lockedInfo = meta.lock(id, true);
 
                     if (lockedInfo == null)
                         return false; // File is locked, we cannot delete it.
@@ -271,7 +272,7 @@ public class IgfsDeleteWorker extends IgfsThread {
         assert id != null;
 
         while (true) {
-            IgfsFileInfo info = meta.info(id);
+            IgfsEntryInfo info = meta.info(id);
 
             if (info != null) {
                 assert info.isDirectory();
@@ -298,12 +299,12 @@ public class IgfsDeleteWorker extends IgfsThread {
                             failedFiles++;
                     }
                     else {
-                        IgfsFileInfo fileInfo = meta.info(entry.getValue().fileId());
+                        IgfsEntryInfo fileInfo = meta.info(entry.getValue().fileId());
 
                         if (fileInfo != null) {
                             assert fileInfo.isFile();
 
-                            IgfsFileInfo lockedInfo = meta.lock(fileInfo.id(), true);
+                            IgfsEntryInfo lockedInfo = meta.lock(fileInfo.id(), true);
 
                             if (lockedInfo == null)
                                 // File is already locked:

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java
new file mode 100644
index 0000000..01c8ff9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDirectoryInfo.java
@@ -0,0 +1,210 @@
+/*
+ * 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.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * IGFS directory info.
+ */
+public class IgfsDirectoryInfo extends IgfsEntryInfo {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Directory listing. */
+    @GridToStringInclude
+    private Map<String, IgfsListingEntry> listing;
+
+    /**
+     * {@link Externalizable} support.
+     */
+    public IgfsDirectoryInfo() {
+        // No-op.
+    }
+
+    /**
+     * Update length.
+     *
+     * @param len New length.
+     * @return Updated file info.
+     */
+    public IgfsEntryInfo length(long len) {
+        throw new UnsupportedOperationException("length");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsDirectoryInfo listing(@Nullable Map<String, IgfsListingEntry> listing) {
+        IgfsDirectoryInfo res = copy();
+
+        res.listing = listing;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo lock(IgniteUuid lockId) {
+        throw new UnsupportedOperationException("lock");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo unlock(long modificationTime) {
+        throw new UnsupportedOperationException("unlock");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo fileMap(IgfsFileMap fileMap) {
+        throw new UnsupportedOperationException("fileMap");
+    }
+
+    /**
+     * Constructs file info.
+     *
+     * @param id ID or {@code null} to generate it automatically.
+     * @param listing Directory listing.
+     * @param props File properties.
+     * @param accessTime Last access time.
+     * @param modificationTime Last modification time.
+     */
+    IgfsDirectoryInfo(IgniteUuid id, @Nullable Map<String, IgfsListingEntry> listing,
+        @Nullable Map<String, String> props, long accessTime, long modificationTime) {
+        super(id, props, accessTime, modificationTime);
+
+        this.listing = listing;
+    }
+
+    /** {@inheritDoc} */
+    protected IgfsDirectoryInfo copy() {
+        return new IgfsDirectoryInfo(id, listing, props, accessTime, modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    public boolean isFile() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    public long length() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    public int blockSize() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    public long blocksCount() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    public Map<String, IgfsListingEntry> listing() {
+        return listing != null ? listing : Collections.<String, IgfsListingEntry>emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    public boolean hasChildren() {
+        return !F.isEmpty(listing);
+    }
+
+    /** {@inheritDoc} */
+    public boolean hasChild(String name) {
+        return listing != null && listing.containsKey(name);
+    }
+
+    /** {@inheritDoc} */
+    public boolean hasChild(String name, IgniteUuid expId) {
+        if (listing != null) {
+            IgfsListingEntry entry = listing.get(name);
+
+            if (entry != null)
+                return F.eq(expId, entry.fileId());
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable public IgniteUuid affinityKey() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    public IgfsFileMap fileMap() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable public IgniteUuid lockId() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    public boolean evictExclude() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        out.writeObject(listing);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        listing = (Map<String, IgfsListingEntry>)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return id.hashCode() ^ (props == null ? 0 : props.hashCode());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (obj == this)
+            return true;
+
+        if (obj == null || getClass() != obj.getClass())
+            return false;
+
+        IgfsDirectoryInfo that = (IgfsDirectoryInfo)obj;
+
+        return id.equals(that.id) && F.eq(props, that.props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsDirectoryInfo.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java
new file mode 100644
index 0000000..c5b1111
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEntryInfo.java
@@ -0,0 +1,305 @@
+/*
+ * 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.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Base IGFS entry.
+ */
+public abstract class IgfsEntryInfo implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** ID. */
+    protected IgniteUuid id;
+
+    /** Properties. */
+    protected Map<String, String> props;
+
+    /** Last access time. */
+    protected long accessTime;
+
+    /** Last modification time. */
+    protected long modificationTime;
+
+    /** Original file path. This is a helper field used only during real file delete. */
+    protected IgfsPath path;
+
+    /**
+     * Default constructor.
+     */
+    protected IgfsEntryInfo() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param id ID.
+     * @param props Properties.
+     * @param accessTime Access time.
+     * @param modificationTime Modification time.
+     */
+    protected IgfsEntryInfo(IgniteUuid id, @Nullable Map<String, String> props, long accessTime,
+        long modificationTime) {
+        assert id != null;
+
+        this.id = id;
+        this.props = props == null || props.isEmpty() ? null : props;
+        this.accessTime = accessTime;
+        this.modificationTime = modificationTime;
+    }
+
+    /**
+     * Gets this item ID.
+     *
+     * @return This item ID.
+     */
+    public IgniteUuid id() {
+        return id;
+    }
+
+    /**
+     * Get properties of the file.
+     *
+     * @return Properties of the file.
+     */
+    public Map<String, String> properties() {
+        return props == null ? Collections.<String, String>emptyMap() : Collections.unmodifiableMap(props);
+    }
+
+    /**
+     * @return Last access time.
+     */
+    public long accessTime() {
+        return accessTime;
+    }
+
+    /**
+     * @return Last modification time.
+     */
+    public long modificationTime() {
+        return modificationTime;
+    }
+
+    /**
+     * @return Original file path. This is a helper field used only in some operations like delete.
+     */
+    public IgfsPath path() {
+        return path;
+    }
+
+    /**
+     * @return {@code True} if this is a file.
+     */
+    public abstract boolean isFile();
+
+    /**
+     * Update length.
+     *
+     * @param len New length.
+     * @return Updated file info.
+     */
+    public abstract IgfsEntryInfo length(long len);
+
+    /**
+     * Update listing.
+     *
+     * @param listing Listing.
+     * @return Updated file info.
+     */
+    public abstract IgfsEntryInfo listing(@Nullable Map<String, IgfsListingEntry> listing);
+
+    /**
+     * Update properties.
+     *
+     * @param props Properties.
+     * @return Updated file info.
+     */
+    public IgfsEntryInfo properties(@Nullable Map<String, String> props) {
+        IgfsEntryInfo res = copy();
+
+        res.props = props;
+
+        return res;
+    }
+
+    /**
+     * Update path.
+     *
+     * @param path Path.
+     * @return Updated file info.
+     */
+    public IgfsEntryInfo path(IgfsPath path) {
+        IgfsEntryInfo res = copy();
+
+        res.path = path;
+
+        return res;
+    }
+
+    /**
+     * Update access and modification time.
+     *
+     * @param accessTime Access time.
+     * @param modificationTime Modification time.
+     * @return Updated file info.
+     */
+    public IgfsEntryInfo accessModificationTime(long accessTime, long modificationTime) {
+        IgfsEntryInfo res = copy();
+
+        res.accessTime = accessTime;
+        res.modificationTime = modificationTime;
+
+        return res;
+    }
+
+    /**
+     * Lock file.
+     *
+     * @param lockId Lock ID.
+     * @return Update file info.
+     */
+    public abstract IgfsEntryInfo lock(IgniteUuid lockId);
+
+    /**
+     * Unlock file.
+     *
+     * @param modificationTime Modification time.
+     * @return Updated file info.
+     */
+    public abstract IgfsEntryInfo unlock(long modificationTime);
+
+    /**
+     * Update file map.
+     *
+     * @param fileMap File affinity map.
+     * @return Updated file info.
+     */
+    public abstract IgfsEntryInfo fileMap(IgfsFileMap fileMap);
+
+    /**
+     * Copy file info.
+     *
+     * @return Copy.
+     */
+    protected abstract IgfsEntryInfo copy();
+
+    /**
+     * @return {@code True} if this is a directory.
+     */
+    public boolean isDirectory() {
+        return !isFile();
+    }
+
+    /**
+     * Get file size.
+     *
+     * @return File size.
+     */
+    public abstract long length();
+
+    /**
+     * Get single data block size to store this file.
+     *
+     * @return Single data block size to store this file.
+     */
+    public abstract int blockSize();
+
+    /**
+     * @return Number of data blocks to store this file.
+     */
+    public abstract long blocksCount();
+
+    /**
+     * @return Directory listing.
+     */
+    public abstract Map<String, IgfsListingEntry> listing();
+
+    /**
+     * @return {@code True} if at least one child exists.
+     */
+    public abstract boolean hasChildren();
+
+    /**
+     * @param name Child name.
+     * @return {@code True} if child with such name exists.
+     */
+    public abstract boolean hasChild(String name);
+
+    /**
+     * @param name Child name.
+     * @param expId Expected child ID.
+     * @return {@code True} if child with such name exists.
+     */
+    public abstract boolean hasChild(String name, IgniteUuid expId);
+
+    /**
+    * @return Affinity key used for single-node file collocation. If {@code null}, usual
+    *      mapper procedure is used for block affinity detection.
+    */
+    @Nullable public abstract IgniteUuid affinityKey();
+
+    /**
+     * @return File affinity map.
+     */
+    public abstract IgfsFileMap fileMap();
+
+    /**
+     * Get lock ID.
+     *
+     * @return Lock ID if file is locked or {@code null} if file is free of locks.
+     */
+    @Nullable public abstract IgniteUuid lockId();
+
+    /**
+     * Get evict exclude flag.
+     *
+     * @return Evict exclude flag.
+     */
+    public abstract boolean evictExclude();
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeGridUuid(out, id);
+        U.writeStringMap(out, props);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
+        out.writeObject(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        id = U.readGridUuid(in);
+        props = U.readStringMap(in);
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
+        path = (IgfsPath)in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 3576a06..be8d0fc 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,12 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Collections;
-import java.util.Map;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -31,6 +25,13 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.Map;
+
 /**
  * File or directory information.
  */
@@ -97,7 +98,7 @@ public final class IgfsFileImpl implements IgfsFile, Externalizable {
      *
      * @param path Path.
      */
-    public IgfsFileImpl(IgfsPath path, IgfsFileInfo info, long globalGrpBlockSize) {
+    public IgfsFileImpl(IgfsPath path, IgfsEntryInfo info, long globalGrpBlockSize) {
         A.notNull(path, "path");
         A.notNull(info, "info");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index 13c54ff..30f505e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -17,44 +17,32 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
 import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.Collections;
 import java.util.Map;
-import org.apache.ignite.configuration.FileSystemConfiguration;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.util.GridLeanMap;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteUuid;
-import org.jetbrains.annotations.Nullable;
 
 /**
- * Unmodifiable file information.
+ * IGFS file info.
  */
-public final class IgfsFileInfo implements Externalizable {
+public final class IgfsFileInfo extends IgfsEntryInfo {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Special access time value, indicating that the modification time value should be taken. */
-    private static final long ACCESS_TIME_TAKE_MODIFICATION_TIME = -1L;
-
-    /** Info ID. */
-    private IgniteUuid id;
-
     /** File length in bytes. */
     private long len;
 
     /** File block size, {@code zero} for directories. */
     private int blockSize;
 
-    /** File properties. */
-    private Map<String, String> props;
-
     /** File lock ID. */
     private IgniteUuid lockId;
 
@@ -64,491 +52,181 @@ public final class IgfsFileInfo implements Externalizable {
     /** File affinity map. */
     private IgfsFileMap fileMap;
 
-    /** Last access time. Modified on-demand. */
-    private long accessTime;
-
-    /** Last modification time. */
-    private long modificationTime;
-
-    /** Directory listing. */
-    @GridToStringInclude
-    private Map<String, IgfsListingEntry> listing;
-
     /** Whether data blocks of this entry should never be excluded. */
     private boolean evictExclude;
 
     /**
-     * Original file path. This is a helper field used only in some
-     * operations like delete.
-     */
-    private IgfsPath path;
-
-    /**
      * {@link Externalizable} support.
      */
     public IgfsFileInfo() {
-        this(IgfsUtils.ROOT_ID);
+        // No-op.
     }
 
-    /**
-     * Constructs directory file info with the given ID.
-     *
-     * @param id ID.
-     */
-    IgfsFileInfo(IgniteUuid id) {
-        this(true, id, 0, 0, null, null, null, null, false, ACCESS_TIME_TAKE_MODIFICATION_TIME,
-            System.currentTimeMillis(), false);
-    }
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo length(long len) {
+        IgfsFileInfo res = copy();
 
-    /**
-     * Constructs directory or file info with
-     * {@link org.apache.ignite.configuration.FileSystemConfiguration#DFLT_BLOCK_SIZE default} block size.
-     *
-     * @param isDir Constructs directory info if {@code true} or file info if {@code false}.
-     * @param props Meta properties to set.
-     * @param accessTime The access time.
-     * @param modificationTime The modification time.
-     */
-    public IgfsFileInfo(boolean isDir, @Nullable Map<String, String> props, long accessTime, long modificationTime) {
-        this(isDir, null, isDir ? 0 : FileSystemConfiguration.DFLT_BLOCK_SIZE, 0, null, null, props, null, false,
-            accessTime, modificationTime, false);
-    }
+        res.len = len;
 
-    /**
-     * Consturcts directory with random ID and provided listing.
-     *
-     * @param listing Listing.
-     */
-    IgfsFileInfo(Map<String, IgfsListingEntry> listing) {
-        this(listing, (Map<String,String>)null);
+        return res;
     }
 
-    /**
-     * Consturcts directory with random ID, provided listing and properties.
-     *
-     * @param listing Listing.
-     * @param props The properties to set for the new directory.
-     */
-    IgfsFileInfo(@Nullable Map<String, IgfsListingEntry> listing, @Nullable Map<String,String> props) {
-        this(true/*dir*/, null, 0, 0, null, listing, props, null, false, ACCESS_TIME_TAKE_MODIFICATION_TIME,
-            System.currentTimeMillis(), false);
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo listing(@Nullable Map<String, IgfsListingEntry> listing) {
+        throw new UnsupportedOperationException("listing");
     }
 
-    /**
-     * Constructs file info.
-     *
-     * @param blockSize Block size.
-     * @param len Length.
-     * @param affKey Affinity key.
-     * @param lockId Lock ID.
-     * @param props Properties.
-     * @param evictExclude Evict exclude flag.
-     * @param accessTime The access time.
-     * @param modificationTime The modification time.
-     */
-    public IgfsFileInfo(int blockSize, long len, @Nullable IgniteUuid affKey, @Nullable IgniteUuid lockId,
-        boolean evictExclude, @Nullable Map<String, String> props, long accessTime, long modificationTime) {
-        this(false, null, blockSize, len, affKey, null, props, lockId, true, accessTime, modificationTime,
-            evictExclude);
-    }
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo lock(IgniteUuid lockId) {
+        assert lockId != null;
+        assert this.lockId == null;
 
-    /**
-     * Constructs file information.
-     *
-     * @param info File information to copy data from.
-     * @param len Size of a file.
-     */
-    IgfsFileInfo(IgfsFileInfo info, long len) {
-        this(info.isDirectory(), info.id, info.blockSize, len, info.affKey, info.listing, info.props, info.fileMap(),
-            info.lockId, true, info.accessTime, info.modificationTime, info.evictExclude());
-    }
+        IgfsFileInfo res = copy();
 
-    /**
-     * Constructs file info.
-     *
-     * @param info File info.
-     * @param accessTime Last access time.
-     * @param modificationTime Last modification time.
-     */
-    IgfsFileInfo(IgfsFileInfo info, long accessTime, long modificationTime) {
-        this(info.isDirectory(), info.id, info.blockSize, info.len, info.affKey, info.listing, info.props,
-            info.fileMap(), info.lockId, false, accessTime, modificationTime, info.evictExclude());
-    }
+        res.lockId = lockId;
 
-    /**
-     * Constructs file information.
-     *
-     * @param info File information to copy data from.
-     * @param props File properties to set.
-     */
-    IgfsFileInfo(IgfsFileInfo info, @Nullable Map<String, String> props) {
-        this(info.isDirectory(), info.id, info.blockSize, info.len, info.affKey, info.listing, props,
-            info.fileMap(), info.lockId, true, info.accessTime, info.modificationTime, info.evictExclude());
+        return res;
     }
 
-    /**
-     * Constructs file info.
-     *
-     * @param blockSize Block size,
-     * @param len Size of a file.
-     * @param props File properties to set.
-     * @param evictExclude Evict exclude flag.
-     * @param accessTime The access time.
-     * @param modificationTime The modification time.
-     */
-    IgfsFileInfo(int blockSize, long len, boolean evictExclude, @Nullable Map<String, String> props,
-        long accessTime, long modificationTime) {
-        this(blockSize == 0, // NB The contract is: (blockSize == 0) <=> isDirectory()
-            null, blockSize, len, null, null, props, null, true, accessTime, modificationTime, evictExclude);
-    }
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo unlock(long modificationTime) {
+        IgfsFileInfo res = copy();
 
-    /**
-     * Constructs file information.
-     *
-     * @param info File information to copy data from.
-     * @param lockId Lock ID.
-     * @param modificationTime Last modification time.
-     */
-    IgfsFileInfo(IgfsFileInfo info, @Nullable IgniteUuid lockId, long modificationTime) {
-        this(info.isDirectory(), info.id, info.blockSize, info.len, info.affKey, info.listing, info.props,
-            info.fileMap(), lockId, true, info.accessTime, modificationTime, info.evictExclude());
+        res.lockId = null;
+        res.modificationTime = modificationTime;
+
+        return res;
     }
 
-    /**
-     * Constructs file info.
-     *
-     * @param listing New directory listing.
-     * @param old Old file info.
-     */
-    IgfsFileInfo(@Nullable Map<String, IgfsListingEntry> listing, IgfsFileInfo old) {
-        this(old.isDirectory(), old.id, old.blockSize, old.len, old.affKey, listing, old.props, old.fileMap(),
-            old.lockId, false, old.accessTime, old.modificationTime, old.evictExclude());
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo fileMap(IgfsFileMap fileMap) {
+        IgfsFileInfo res = copy();
+
+        res.fileMap = fileMap;
+
+        return res;
     }
 
-    /**
-     * Constructs file info.
-     *
-     * @param isDir Constructs directory info if {@code true} or file info if {@code false}.
-     * @param id ID or {@code null} to generate it automatically.
-     * @param blockSize Block size.
-     * @param len Size of a file.
-     * @param affKey Affinity key for data blocks.
-     * @param listing Directory listing.
-     * @param props File properties.
-     * @param lockId Lock ID.
-     * @param cpProps Flag to copy properties map.
-     * @param accessTime The access time.
-     * @param modificationTime Last modification time.
-     * @param evictExclude Evict exclude flag.
-     */
-    private IgfsFileInfo(boolean isDir, @Nullable IgniteUuid id, int blockSize, long len, @Nullable IgniteUuid affKey,
-        @Nullable Map<String, IgfsListingEntry> listing, @Nullable Map<String, String> props,
-        @Nullable IgniteUuid lockId, boolean cpProps, long accessTime, long modificationTime, boolean evictExclude) {
-        this(isDir, id, blockSize, len, affKey, listing, props, null, lockId, cpProps, accessTime,
-            modificationTime, evictExclude);
+    /** {@inheritDoc} */
+    @Override protected IgfsFileInfo copy() {
+        return new IgfsFileInfo(id, blockSize, len, affKey, props, fileMap, lockId, accessTime, modificationTime,
+            evictExclude);
     }
 
     /**
      * Constructs file info.
      *
-     * @param isDir Constructs directory info if {@code true} or file info if {@code false}.
      * @param id ID or {@code null} to generate it automatically.
      * @param blockSize Block size.
      * @param len Size of a file.
      * @param affKey Affinity key for data blocks.
-     * @param listing Directory listing.
      * @param props File properties.
      * @param fileMap File map.
      * @param lockId Lock ID.
-     * @param cpProps Flag to copy properties map.
      * @param accessTime Last access time.
      * @param modificationTime Last modification time.
      * @param evictExclude Evict exclude flag.
      */
-    private IgfsFileInfo(boolean isDir, @Nullable IgniteUuid id, int blockSize, long len, @Nullable IgniteUuid affKey,
-        @Nullable Map<String, IgfsListingEntry> listing, @Nullable Map<String, String> props,
-        @Nullable IgfsFileMap fileMap, @Nullable IgniteUuid lockId, boolean cpProps, long accessTime,
-        long modificationTime, boolean evictExclude) {
-        assert F.isEmpty(listing) || isDir;
-
-        if (isDir) {
-            assert len == 0 : "Directory length should be zero: " + len;
-            assert blockSize == 0 : "Directory block size should be zero: " + blockSize;
-        }
-        else {
-            assert len >= 0 : "File length cannot be negative: " + len;
-            assert blockSize > 0 : "File block size should be positive: " + blockSize;
-        }
-
-        this.id = id == null ? IgniteUuid.randomUuid() : id;
-        this.len = isDir ? 0 : len;
-        this.blockSize = isDir ? 0 : blockSize;
+    IgfsFileInfo(IgniteUuid id, int blockSize, long len, @Nullable IgniteUuid affKey,
+        @Nullable Map<String, String> props, @Nullable IgfsFileMap fileMap, @Nullable IgniteUuid lockId,
+        long accessTime, long modificationTime, boolean evictExclude) {
+        super(id, props, accessTime, modificationTime);
+
+        this.len = len;
+        this.blockSize = blockSize;
         this.affKey = affKey;
-        this.listing = listing;
 
-        if (fileMap == null && !isDir)
+        if (fileMap == null)
             fileMap = new IgfsFileMap();
 
         this.fileMap = fileMap;
-        this.accessTime = accessTime == ACCESS_TIME_TAKE_MODIFICATION_TIME ? modificationTime : accessTime;
-        this.modificationTime = modificationTime;
-
-        // Always make a copy of passed properties collection to escape concurrent modifications.
-        this.props = props == null || props.isEmpty() ? null :
-            cpProps ? new GridLeanMap<>(props) : props;
-
         this.lockId = lockId;
         this.evictExclude = evictExclude;
     }
 
-    /**
-     * A copy constructor, which takes all data from the specified
-     * object field-by-field.
-     *
-     * @param info An object to copy data info.
-     */
-    public IgfsFileInfo(IgfsFileInfo info) {
-        this(info.isDirectory(), info.id, info.blockSize, info.len, info.affKey, info.listing, info.props,
-            info.fileMap(), info.lockId, true, info.accessTime, info.modificationTime, info.evictExclude());
-    }
-
-    /**
-     * Creates a builder for the new instance of file info.
-     *
-     * @return A builder to construct a new unmodifiable instance
-     *         of this class.
-     */
-    public static Builder builder() {
-        return new Builder(new IgfsFileInfo());
-    }
-
-    /**
-     * Creates a builder for the new instance of file info,
-     * based on the specified origin.
-     *
-     * @param origin An origin for new instance, from which
-     *               the data will be copied.
-     * @return A builder to construct a new unmodifiable instance
-     *         of this class.
-     */
-    public static Builder builder(IgfsFileInfo origin) {
-        return new Builder(new IgfsFileInfo(origin));
-    }
-
-    /**
-     * Gets this item ID.
-     *
-     * @return This item ID.
-     */
-    public IgniteUuid id() {
-        return id;
-    }
-
-    /**
-     * Temporal hack to change ID before saving entry to cache. Currently we have too much constructors and adding
-     * more will make things even worse. Instead, we use this method until directories and files are split into
-     * separate entities.
-     *
-     * @param id ID.
-     * @deprecated Use only on not-yet-saved entries.
-     */
-    @Deprecated
-    public void id(IgniteUuid id) {
-        this.id = id;
-    }
-
-    /**
-     * @return {@code True} if this is a file.
-     */
+    /** {@inheritDoc} */
     public boolean isFile() {
-        return blockSize > 0;
-    }
-
-    /**
-     * @return {@code True} if this is a directory.
-     */
-    public boolean isDirectory() {
-        return blockSize == 0;
+        return true;
     }
 
-    /**
-     * Get file size.
-     *
-     * @return File size.
-     */
+    /** {@inheritDoc} */
     public long length() {
-        assert isFile();
-
         return len;
     }
 
-    /**
-     * Get single data block size to store this file.
-     *
-     * @return Single data block size to store this file.
-     */
+    /** {@inheritDoc} */
     public int blockSize() {
-        assert isFile();
-
         return blockSize;
     }
 
-    /**
-     * @return Number of data blocks to store this file.
-     */
+    /** {@inheritDoc} */
     public long blocksCount() {
-        assert isFile();
-
         return (len + blockSize() - 1) / blockSize();
     }
 
-    /**
-     * @return Last access time.
-     */
-    public long accessTime() {
-        return accessTime;
-    }
-
-    /**
-     * @return Last modification time.
-     */
-    public long modificationTime() {
-        return modificationTime;
-    }
-
-    /**
-     * @return Directory listing.
-     */
+    /** {@inheritDoc} */
     public Map<String, IgfsListingEntry> listing() {
-        return listing != null ? listing : Collections.<String, IgfsListingEntry>emptyMap();
+        return Collections.emptyMap();
     }
 
-    /**
-     * @return {@code True} if at least one child exists.
-     */
+    /** {@inheritDoc} */
     public boolean hasChildren() {
-        return !F.isEmpty(listing);
+        return false;
     }
 
-    /**
-     * @param name Child name.
-     * @return {@code True} if child with such name exists.
-     */
+    /** {@inheritDoc} */
     public boolean hasChild(String name) {
-        return listing != null && listing.containsKey(name);
+        return false;
     }
 
-    /**
-     * @param name Child name.
-     * @param expId Expected child ID.
-     * @return {@code True} if child with such name exists.
-     */
+    /** {@inheritDoc} */
     public boolean hasChild(String name, IgniteUuid expId) {
-        if (listing != null) {
-            IgfsListingEntry entry = listing.get(name);
-
-            if (entry != null)
-                return F.eq(expId, entry.fileId());
-        }
-
         return false;
     }
 
-    /**
-     * @return Affinity key used for single-node file collocation. If {@code null}, usual
-     *      mapper procedure is used for block affinity detection.
-     */
+    /** {@inheritDoc} */
     @Nullable public IgniteUuid affinityKey() {
         return affKey;
     }
 
-    /**
-     * @param affKey Affinity key used for single-node file collocation.
-     */
-    public void affinityKey(IgniteUuid affKey) {
-        this.affKey = affKey;
-    }
-
-    /**
-     * @return File affinity map.
-     */
+    /** {@inheritDoc} */
     public IgfsFileMap fileMap() {
         return fileMap;
     }
 
-    /**
-     * @param fileMap File affinity map.
-     */
-    public void fileMap(IgfsFileMap fileMap) {
-        this.fileMap = fileMap;
-    }
-
-    /**
-     * Get properties of the file.
-     *
-     * @return Properties of the file.
-     */
-    public Map<String, String> properties() {
-        return props == null || props.isEmpty() ? Collections.<String, String>emptyMap() :
-            Collections.unmodifiableMap(props);
-    }
-
-    /**
-     * Get lock ID.
-     *
-     * @return Lock ID if file is locked or {@code null} if file is free of locks.
-     */
+    /** {@inheritDoc} */
     @Nullable public IgniteUuid lockId() {
         return lockId;
     }
 
-    /**
-     * Get evict exclude flag.
-     *
-     * @return Evict exclude flag.
-     */
+    /** {@inheritDoc} */
     public boolean evictExclude() {
         return evictExclude;
     }
 
-    /**
-     * @return Original file path. This is a helper field used only in some operations like delete.
-     */
-    public IgfsPath path() {
-        return path;
-    }
-
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeGridUuid(out, id);
+        super.writeExternal(out);
+
         out.writeInt(blockSize);
         out.writeLong(len);
-        U.writeStringMap(out, props);
         U.writeGridUuid(out, lockId);
         U.writeGridUuid(out, affKey);
-        out.writeObject(listing);
         out.writeObject(fileMap);
-        out.writeLong(accessTime);
-        out.writeLong(modificationTime);
         out.writeBoolean(evictExclude);
-        out.writeObject(path);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        id = U.readGridUuid(in);
+        super.readExternal(in);
+
         blockSize = in.readInt();
         len = in.readLong();
-        props = U.readStringMap(in);
         lockId = U.readGridUuid(in);
         affKey = U.readGridUuid(in);
-        listing = (Map<String, IgfsListingEntry>)in.readObject();
         fileMap = (IgfsFileMap)in.readObject();
-        accessTime = in.readLong();
-        modificationTime = in.readLong();
         evictExclude = in.readBoolean();
-        path = (IgfsPath)in.readObject();
     }
 
     /** {@inheritDoc} */
@@ -575,42 +253,4 @@ public final class IgfsFileInfo implements Externalizable {
     @Override public String toString() {
         return S.toString(IgfsFileInfo.class, this);
     }
-
-    /**
-     * Builder for {@link IgfsFileInfo}.
-     */
-    @SuppressWarnings("PublicInnerClass")
-    public static class Builder {
-        /** Instance to build. */
-        private final IgfsFileInfo info;
-
-        /**
-         * Private constructor.
-         *
-         * @param info Instance to build.
-         */
-        private Builder(IgfsFileInfo info) {
-            this.info = info;
-        }
-
-        /**
-         * @param path A new path value.
-         * @return This builder instance (for chaining).
-         */
-        public Builder path(IgfsPath path) {
-            info.path = path;
-
-            return this;
-        }
-
-        /**
-         * Finishes instance construction and returns a resulting
-         * unmodifiable instance.
-         *
-         * @return A constructed instance.
-         */
-        public IgfsFileInfo build() {
-            return info;
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
index 130846b..17875a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatch.java
@@ -17,22 +17,23 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.concurrent.BlockingDeque;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.TimeUnit;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
  * Work batch is an abstraction of the logically grouped tasks.
  */
 public abstract class IgfsFileWorkerBatch implements Runnable {
     /** Stop marker. */
-    private static final byte[] STOP_MARKER = new byte[0];
+    private static final byte[] FINISH_MARKER = new byte[0];
 
     /** Cancel marker. */
     private static final byte[] CANCEL_MARKER = new byte[0];
@@ -49,9 +50,6 @@ public abstract class IgfsFileWorkerBatch implements Runnable {
     /** Output stream to the file. */
     private final OutputStream out;
 
-    /** Cancel flag. */
-    private volatile boolean cancelled;
-
     /** Finishing flag. */
     private volatile boolean finishing;
 
@@ -76,45 +74,55 @@ public abstract class IgfsFileWorkerBatch implements Runnable {
      * @return {@code True} in case write was enqueued.
      */
     synchronized boolean write(final byte[] data) {
-        if (!finishing) {
-            queue.add(data);
-
-            return true;
-        }
-        else
-            return false;
+        return offer(data, false, false);
     }
 
     /**
      * Add the last task to that batch which will release all the resources.
+     *
+     * @return {@code True} if finish was signalled.
      */
-    synchronized void finish() {
-        if (!finishing) {
-            finishing = true;
-
-            queue.add(STOP_MARKER);
-        }
+    synchronized boolean finish() {
+        return offer(FINISH_MARKER, false, true);
     }
 
     /**
      * Cancel batch processing.
+     *
+     * @return {@code True} if cancel was signalled.
      */
-    synchronized void cancel() {
-        queue.addFirst(CANCEL_MARKER);
+    synchronized boolean cancel() {
+        return offer(CANCEL_MARKER, true, true);
     }
 
     /**
-     * @return {@code True} if finish was called on this batch.
+     * Add request to queue.
+     *
+     * @param data Data.
+     * @param head Whether to add to head.
+     * @param finish Whether this is the last batch to be accepted.
+     * @return {@code True} if request was added to queue.
      */
-    boolean finishing() {
-        return finishing;
+    private synchronized boolean offer(byte[] data, boolean head, boolean finish) {
+        if (finishing)
+            return false;
+
+        if (head)
+            queue.addFirst(data);
+        else
+            queue.addLast(data);
+
+        if (finish)
+            finishing = true;
+
+        return true;
     }
 
     /**
-     * @return {@code True} if batch write was terminated abruptly due to explicit cancellation.
+     * @return {@code True} if finish was called on this batch.
      */
-    boolean cancelled() {
-        return cancelled;
+    boolean finishing() {
+        return finishing;
     }
 
     /**
@@ -129,16 +137,13 @@ public abstract class IgfsFileWorkerBatch implements Runnable {
                 try {
                     byte[] data = queue.poll(1000, TimeUnit.MILLISECONDS);
 
-                    if (data == STOP_MARKER) {
+                    if (data == FINISH_MARKER) {
                         assert queue.isEmpty();
 
                         break;
                     }
-                    else if (data == CANCEL_MARKER) {
-                        cancelled = true;
-
-                        throw new IgniteCheckedException("Write to file was cancelled due to node stop.");
-                    }
+                    else if (data == CANCEL_MARKER)
+                        throw new IgfsFileWorkerBatchCancelledException(path);
                     else if (data != null) {
                         try {
                             out.write(data);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatchCancelledException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatchCancelledException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatchCancelledException.java
new file mode 100644
index 0000000..4ae6964
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileWorkerBatchCancelledException.java
@@ -0,0 +1,51 @@
+/*
+ * 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.IgniteCheckedException;
+import org.apache.ignite.igfs.IgfsPath;
+
+/**
+ * Exception indicating that file batch processing was cancelled.
+ */
+public class IgfsFileWorkerBatchCancelledException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Path. */
+    private IgfsPath path;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsFileWorkerBatchCancelledException() {
+        // No-op.
+    }
+
+    public IgfsFileWorkerBatchCancelledException(IgfsPath path) {
+        this.path = path;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getMessage() {
+        if (path == null)
+            return "Asynchronous file processing was cancelled due to node stop.";
+        else
+            return "Asynchronous file processing was cancelled due to node stop: " + path;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
index 7cc5cb6..194a8ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
@@ -17,23 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
@@ -60,6 +43,23 @@ import org.jetbrains.annotations.Nullable;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
@@ -274,7 +274,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
         Collection<IgfsFileAffinityRange> ranges = req.fragmentRanges();
         IgniteUuid fileId = req.fileId();
 
-        IgfsFileInfo fileInfo = igfsCtx.meta().info(fileId);
+        IgfsEntryInfo fileInfo = igfsCtx.meta().info(fileId);
 
         if (fileInfo == null) {
             if (log.isDebugEnabled())
@@ -288,7 +288,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
 
         for (IgfsFileAffinityRange range : ranges) {
             try {
-                IgfsFileInfo updated;
+                IgfsEntryInfo updated;
 
                 switch (range.status()) {
                     case RANGE_STATUS_INITIAL: {
@@ -345,7 +345,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
     /**
      * Update range processor.
      */
-    private static class RangeUpdateProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
+    private static class RangeUpdateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -375,17 +375,15 @@ public class IgfsFragmentizerManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo oldInfo = entry.getValue();
+            IgfsEntryInfo oldInfo = entry.getValue();
 
             IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
 
             newMap.updateRangeStatus(range, status);
 
-            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, oldInfo.length());
-
-            newInfo.fileMap(newMap);
+            IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
 
             entry.setValue(newInfo);
 
@@ -413,7 +411,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
     /**
      * Delete range processor.
      */
-    private static class RangeDeleteProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
+    private static class RangeDeleteProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -438,17 +436,15 @@ public class IgfsFragmentizerManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo oldInfo = entry.getValue();
+            IgfsEntryInfo oldInfo = entry.getValue();
 
             IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
 
             newMap.deleteRange(range);
 
-            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, oldInfo.length());
-
-            newInfo.fileMap(newMap);
+            IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
 
             entry.setValue(newInfo);
 
@@ -507,7 +503,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
                 // If we have room for files, add them to fragmentizer.
                 try {
                     while (fragmentingFiles.size() < igfsCtx.configuration().getFragmentizerConcurrentFiles()) {
-                        IgfsFileInfo fileInfo = fileForFragmentizer(fragmentingFiles.keySet());
+                        IgfsEntryInfo fileInfo = fileForFragmentizer(fragmentingFiles.keySet());
 
                         // If no colocated files found, exit loop.
                         if (fileInfo == null)
@@ -715,7 +711,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
          *
          * @param fileInfo File info to process.
          */
-        private void requestFragmenting(IgfsFileInfo fileInfo) {
+        private void requestFragmenting(IgfsEntryInfo fileInfo) {
             IgfsFileMap map = fileInfo.fileMap();
 
             assert map != null && !map.ranges().isEmpty();
@@ -789,7 +785,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
      * @return File ID to process or {@code null} if there are no such files.
      * @throws IgniteCheckedException In case of error.
      */
-    @Nullable private IgfsFileInfo fileForFragmentizer(Collection<IgniteUuid> exclude) throws IgniteCheckedException {
+    @Nullable private IgfsEntryInfo fileForFragmentizer(Collection<IgniteUuid> exclude) throws IgniteCheckedException {
         return fragmentizerEnabled ? igfsCtx.meta().fileForFragmentizer(exclude) : null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 9ec583c..398428a 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
@@ -622,7 +622,7 @@ public final class IgfsImpl implements IgfsEx {
 
                     await(path);
 
-                    IgfsFileInfo info = meta.updateDual(secondaryFs, path, props);
+                    IgfsEntryInfo info = meta.updateDual(secondaryFs, path, props);
 
                     if (info == null)
                         return null;
@@ -637,7 +637,7 @@ public final class IgfsImpl implements IgfsEx {
                 if (fileId == null)
                     return null;
 
-                IgfsFileInfo info = meta.updateProperties(fileId, props);
+                IgfsEntryInfo info = meta.updateProperties(fileId, props);
 
                 if (info != null) {
                     if (evts.isRecordable(EVT_IGFS_META_UPDATED))
@@ -691,7 +691,7 @@ public final class IgfsImpl implements IgfsEx {
                     return null;
                 }
 
-                IgfsFileInfo info = meta.move(src, dest);
+                IgfsEntryInfo info = meta.move(src, dest);
 
                 int evtTyp = info.isFile() ? EVT_IGFS_FILE_RENAMED : EVT_IGFS_DIR_RENAMED;
 
@@ -858,7 +858,7 @@ public final class IgfsImpl implements IgfsEx {
                 IgniteUuid fileId = meta.fileId(path);
 
                 if (fileId != null) {
-                    IgfsFileInfo info = meta.info(fileId);
+                    IgfsEntryInfo info = meta.info(fileId);
 
                     // Handle concurrent deletion.
                     if (info != null) {
@@ -869,7 +869,7 @@ public final class IgfsImpl implements IgfsEx {
 
                         // Perform the listing.
                         for (Map.Entry<String, IgfsListingEntry> e : info.listing().entrySet()) {
-                            IgfsFileInfo childInfo = meta.info(e.getValue().fileId());
+                            IgfsEntryInfo childInfo = meta.info(e.getValue().fileId());
 
                             if (childInfo != null) {
                                 IgfsPath childPath = new IgfsPath(path, e.getKey());
@@ -933,7 +933,7 @@ public final class IgfsImpl implements IgfsEx {
                     return os;
                 }
 
-                IgfsFileInfo info = meta.info(meta.fileId(path));
+                IgfsEntryInfo info = meta.info(meta.fileId(path));
 
                 if (info == null) {
                     checkConflictWithPrimary(path);
@@ -1034,7 +1034,7 @@ public final class IgfsImpl implements IgfsEx {
                 else
                     dirProps = fileProps = new HashMap<>(props);
 
-                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(
+                IgniteBiTuple<IgfsEntryInfo, IgniteUuid> t2 = meta.create(
                     path,
                     dirProps,
                     overwrite,
@@ -1112,7 +1112,7 @@ public final class IgfsImpl implements IgfsEx {
                 else
                     dirProps = fileProps = new HashMap<>(props);
 
-                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.append(
+                IgniteBiTuple<IgfsEntryInfo, IgniteUuid> t2 = meta.append(
                     path,
                     dirProps,
                     create,
@@ -1194,7 +1194,7 @@ public final class IgfsImpl implements IgfsEx {
 
                 // Check memory first.
                 IgniteUuid fileId = meta.fileId(path);
-                IgfsFileInfo info = meta.info(fileId);
+                IgfsEntryInfo info = meta.info(fileId);
 
                 if (info == null && mode != PRIMARY) {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
@@ -1291,7 +1291,7 @@ public final class IgfsImpl implements IgfsEx {
     private void summary0(IgniteUuid fileId, IgfsPathSummary sum) throws IgniteCheckedException {
         assert sum != null;
 
-        IgfsFileInfo info = meta.info(fileId);
+        IgfsEntryInfo info = meta.info(fileId);
 
         if (info != null) {
             if (info.isDirectory()) {
@@ -1401,7 +1401,7 @@ public final class IgfsImpl implements IgfsEx {
 
         List<IgniteUuid> ids = meta.fileIds(path);
 
-        IgfsFileInfo fileInfo = meta.info(ids.get(ids.size() - 1));
+        IgfsEntryInfo fileInfo = meta.info(ids.get(ids.size() - 1));
 
         if (fileInfo == null)
             return null; // File does not exist.
@@ -1571,7 +1571,7 @@ public final class IgfsImpl implements IgfsEx {
         assert path != null;
         assert mode != null;
 
-        IgfsFileInfo info = null;
+        IgfsEntryInfo info = null;
 
         switch (mode) {
             case PRIMARY:
@@ -1698,7 +1698,7 @@ public final class IgfsImpl implements IgfsEx {
          * @param mode IGFS mode.
          * @param batch Optional secondary file system batch.
          */
-        IgfsEventAwareOutputStream(IgfsPath path, IgfsFileInfo fileInfo,
+        IgfsEventAwareOutputStream(IgfsPath path, IgfsEntryInfo fileInfo,
             IgniteUuid parentId, int bufSize, IgfsMode mode, @Nullable IgfsFileWorkerBatch batch) {
             super(igfsCtx, path, fileInfo, parentId, bufSize, mode, batch, metrics);
 
@@ -1737,7 +1737,7 @@ public final class IgfsImpl implements IgfsEx {
          * @param secReader Optional secondary file system reader.
          * @param metrics Metrics.
          */
-        IgfsEventAwareInputStream(IgfsContext igfsCtx, IgfsPath path, IgfsFileInfo fileInfo,
+        IgfsEventAwareInputStream(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo,
             int prefetchBlocks, int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader,
             IgfsLocalMetrics metrics) {
             super(igfsCtx, path, fileInfo, prefetchBlocks, seqReadsBeforePrefetch, secReader, metrics);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
index afcece7..07ab051 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
@@ -17,10 +17,11 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.IOException;
 import org.apache.ignite.igfs.IgfsInputStream;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
 
+import java.io.IOException;
+
 /**
  * Implementation adapter providing necessary methods.
  */
@@ -36,7 +37,7 @@ public abstract class IgfsInputStreamAdapter extends IgfsInputStream
      *
      * @return File info.
      */
-    public abstract IgfsFileInfo fileInfo();
+    public abstract IgfsEntryInfo fileInfo();
 
     /**
      * Reads bytes from given position.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 62f8034..5d41543 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
@@ -17,16 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.igfs.IgfsCorruptedFileException;
@@ -42,6 +32,17 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
 /**
  * Input stream to read data from grid cache with separate blocks.
  */
@@ -66,7 +67,7 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
     protected final IgfsPath path;
 
     /** File descriptor. */
-    private volatile IgfsFileInfo fileInfo;
+    private volatile IgfsEntryInfo fileInfo;
 
     /** The number of already read bytes. Important! Access to the property is guarded by this object lock. */
     private long pos;
@@ -121,7 +122,7 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
      * @param secReader Optional secondary file system reader.
      * @param metrics Local IGFS metrics.
      */
-    IgfsInputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsFileInfo fileInfo, int prefetchBlocks,
+    IgfsInputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, int prefetchBlocks,
         int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader, IgfsLocalMetrics metrics) {
         assert igfsCtx != null;
         assert path != null;
@@ -157,7 +158,7 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFileInfo fileInfo() {
+    @Override public IgfsEntryInfo fileInfo() {
         return fileInfo;
     }
 
@@ -404,7 +405,7 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
 
                 // This failure may be caused by file being fragmented.
                 if (fileInfo.fileMap() != null && !fileInfo.fileMap().ranges().isEmpty()) {
-                    IgfsFileInfo newInfo = meta.info(fileInfo.id());
+                    IgfsEntryInfo newInfo = meta.info(fileInfo.id());
 
                     // File was deleted.
                     if (newInfo == null)
@@ -534,7 +535,8 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
      * @return Requested data block or {@code null} if nothing found.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected IgniteInternalFuture<byte[]> dataBlock(IgfsFileInfo fileInfo, long blockIdx) throws IgniteCheckedException {
+    @Nullable protected IgniteInternalFuture<byte[]> dataBlock(IgfsEntryInfo fileInfo, long blockIdx)
+        throws IgniteCheckedException {
         return data.dataBlock(fileInfo, path, blockIdx, secReader);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
index 0ed7c0d..eadbdb2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
@@ -17,11 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.Closeable;
-import java.io.DataInput;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -46,6 +41,12 @@ import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicLong;
+
 /**
  * IGFS IPC handler.
  */
@@ -342,10 +343,7 @@ class IgfsIpcHandler implements IgfsServerHandler {
                                 log.debug("Opened IGFS input stream for file read [igfsName=" + igfs.name() + ", path=" +
                                     req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
 
-                            IgfsFileInfo info = new IgfsFileInfo(igfsIn.fileInfo(), null,
-                                igfsIn.fileInfo().modificationTime());
-
-                            res.response(new IgfsInputStreamDescriptor(streamId, info.length()));
+                            res.response(new IgfsInputStreamDescriptor(streamId, igfsIn.fileInfo().length()));
 
                             break;
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
index 4fe0dca..ea05ca3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
@@ -17,16 +17,16 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
 /**
  * Directory listing entry.
  */
@@ -52,7 +52,7 @@ public class IgfsListingEntry implements Externalizable {
      *
      * @param fileInfo File info to construct listing entry from.
      */
-    public IgfsListingEntry(IgfsFileInfo fileInfo) {
+    public IgfsListingEntry(IgfsEntryInfo fileInfo) {
         id = fileInfo.id();
         dir = fileInfo.isDirectory();
     }


[15/17] ignite git commit: IGNITE-2861: Added missing Apache headers.

Posted by dm...@apache.org.
IGNITE-2861: Added missing Apache headers.


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

Branch: refs/heads/ignite-2849
Commit: 69f526a481bf24e01f37c2b60412da9154e38083
Parents: 19b4da3
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 21 10:36:26 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 21 10:36:26 2016 +0300

----------------------------------------------------------------------
 .../meta/IgfsMetaDirectoryCreateProcessor.java     | 17 +++++++++++++++++
 .../meta/IgfsMetaDirectoryListingAddProcessor.java | 17 +++++++++++++++++
 .../IgfsMetaDirectoryListingRemoveProcessor.java   | 17 +++++++++++++++++
 .../IgfsMetaDirectoryListingReplaceProcessor.java  | 17 +++++++++++++++++
 .../igfs/meta/IgfsMetaFileCreateProcessor.java     | 17 +++++++++++++++++
 .../igfs/meta/IgfsMetaFileLockProcessor.java       | 17 +++++++++++++++++
 .../meta/IgfsMetaFileRangeDeleteProcessor.java     | 17 +++++++++++++++++
 .../meta/IgfsMetaFileRangeUpdateProcessor.java     | 17 +++++++++++++++++
 .../meta/IgfsMetaFileReserveSpaceProcessor.java    | 17 +++++++++++++++++
 .../igfs/meta/IgfsMetaFileUnlockProcessor.java     | 17 +++++++++++++++++
 .../igfs/meta/IgfsMetaUpdatePathProcessor.java     | 17 +++++++++++++++++
 .../meta/IgfsMetaUpdatePropertiesProcessor.java    | 17 +++++++++++++++++
 .../igfs/meta/IgfsMetaUpdateTimesProcessor.java    | 17 +++++++++++++++++
 13 files changed, 221 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
index ded66bf..dcca298 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
index d45dea1..f27bdd5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingAddProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.IgniteException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
index d5da117..15eb9c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRemoveProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.IgniteException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
index bf3c288..dfc9bfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingReplaceProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.IgniteException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
index 418227a..41745f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
index ff71f13..3258f10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
index cdf07d1..f3132a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
index e1224de..a6b756f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
index a54ddcc..a1ed55f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/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 0d06c92..6827e4a 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
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
index 81c1806..782b25b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
index a9751c6..fb4466b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/69f526a4/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
index b754f19..de03fac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * 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.meta;
 
 import org.apache.ignite.binary.BinaryObjectException;


[12/17] ignite git commit: IGNITE-2861: IGFS: Moved metadata processors into separate top-level classes to simplify code. Also cleaned up IgfsMetaManager from unused code.

Posted by dm...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
new file mode 100644
index 0000000..43a5d7d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileLockProcessor.java
@@ -0,0 +1,63 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * File lock entry processor.
+ */
+public class IgfsMetaFileLockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Lock Id. */
+    private IgniteUuid lockId;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaFileLockProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param lockId Lock ID.
+     */
+    public IgfsMetaFileLockProcessor(IgniteUuid lockId) {
+        this.lockId = lockId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsEntryInfo newInfo = oldInfo.lock(lockId);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeGridUuid(out, lockId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        lockId = U.readGridUuid(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
new file mode 100644
index 0000000..5af09e5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeDeleteProcessor.java
@@ -0,0 +1,74 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+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 javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Delete range processor.
+ */
+public class IgfsMetaFileRangeDeleteProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Range. */
+    private IgfsFileAffinityRange range;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaFileRangeDeleteProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param range Range.
+     */
+    public IgfsMetaFileRangeDeleteProcessor(IgfsFileAffinityRange range) {
+        this.range = range;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
+
+        newMap.deleteRange(range);
+
+        IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(range);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        range = (IgfsFileAffinityRange)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileRangeDeleteProcessor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
new file mode 100644
index 0000000..ae886c8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileRangeUpdateProcessor.java
@@ -0,0 +1,81 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+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 javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Update range processor.
+ */
+public class IgfsMetaFileRangeUpdateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Range. */
+    private IgfsFileAffinityRange range;
+
+    /** Status. */
+    private int status;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaFileRangeUpdateProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param range Range.
+     * @param status Status.
+     */
+    public IgfsMetaFileRangeUpdateProcessor(IgfsFileAffinityRange range, int status) {
+        this.range = range;
+        this.status = status;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
+
+        newMap.updateRangeStatus(range, status);
+
+        IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(range);
+        out.writeInt(status);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        range = (IgfsFileAffinityRange)in.readObject();
+        status = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaFileRangeUpdateProcessor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
new file mode 100644
index 0000000..30f8e22
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileReserveSpaceProcessor.java
@@ -0,0 +1,75 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+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.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * File reserve space entry processor.
+ */
+public class IgfsMetaFileReserveSpaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Space. */
+    private long space;
+
+    /** Affinity range. */
+    private IgfsFileAffinityRange affRange;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaFileReserveSpaceProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param space Space.
+     * @param affRange
+     */
+    public IgfsMetaFileReserveSpaceProcessor(long space, IgfsFileAffinityRange affRange) {
+        this.space = space;
+        this.affRange = affRange;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
+
+        newMap.addRange(affRange);
+
+        IgfsEntryInfo newInfo = oldInfo.length(oldInfo.length() + space).fileMap(newMap);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(space);
+        out.writeObject(affRange);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        space = in.readLong();
+        affRange = (IgfsFileAffinityRange)in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/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
new file mode 100644
index 0000000..d535a1d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileUnlockProcessor.java
@@ -0,0 +1,60 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * File unlock entry processor.
+ */
+public class IgfsMetaFileUnlockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Modification time. */
+    private long modificationTime;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaFileUnlockProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param modificationTime Modification time.
+     */
+    public IgfsMetaFileUnlockProcessor(long modificationTime) {
+        this.modificationTime = modificationTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo old = entry.getValue();
+
+        entry.setValue(old.unlock(modificationTime));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        modificationTime = in.readLong();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
new file mode 100644
index 0000000..32e5512
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePathProcessor.java
@@ -0,0 +1,66 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsMetaManager;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Update path closure.
+ */
+public final class IgfsMetaUpdatePathProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** New path. */
+    private IgfsPath path;
+
+    /**
+     * @param path Path.
+     */
+    public IgfsMetaUpdatePathProcessor(IgfsPath path) {
+        this.path = path;
+    }
+
+    /**
+     * Default constructor (required by Externalizable).
+     */
+    public IgfsMetaUpdatePathProcessor() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args) {
+        IgfsEntryInfo info = e.getValue();
+
+        IgfsEntryInfo newInfo = info.path(path);
+
+        e.setValue(newInfo);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        path = (IgfsPath)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaUpdatePathProcessor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
new file mode 100644
index 0000000..a97c186
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdatePropertiesProcessor.java
@@ -0,0 +1,78 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.util.GridLeanMap;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Map;
+
+/**
+ * Update properties processor.
+ */
+public class IgfsMetaUpdatePropertiesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Properties to be updated. */
+    private Map<String, String> props;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaUpdatePropertiesProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param props Properties.
+     */
+    public IgfsMetaUpdatePropertiesProcessor(Map<String, String> props) {
+        this.props = props;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        Map<String, String> tmp = oldInfo.properties();
+
+        tmp = tmp == null ? new GridLeanMap<String, String>(props.size()) : new GridLeanMap<>(tmp);
+
+        for (Map.Entry<String, String> e : props.entrySet()) {
+            if (e.getValue() == null)
+                // Remove properties with 'null' values.
+                tmp.remove(e.getKey());
+            else
+                // Add/overwrite property.
+                tmp.put(e.getKey(), e.getValue());
+        }
+
+        IgfsEntryInfo newInfo = oldInfo.properties(tmp);
+
+        entry.setValue(newInfo);
+
+        return newInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeStringMap(out, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        props = U.readStringMap(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/865e376a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
new file mode 100644
index 0000000..8acd289
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaUpdateTimesProcessor.java
@@ -0,0 +1,68 @@
+package org.apache.ignite.internal.processors.igfs.meta;
+
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Update times entry processor.
+ */
+public class IgfsMetaUpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Access time. */
+    private long accessTime;
+
+    /** Modification time. */
+    private long modificationTime;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsMetaUpdateTimesProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param accessTime Access time.
+     * @param modificationTime Modification time.
+     */
+    public IgfsMetaUpdateTimesProcessor(long accessTime, long modificationTime) {
+        this.accessTime = accessTime;
+        this.modificationTime = modificationTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
+        throws EntryProcessorException {
+
+        IgfsEntryInfo oldInfo = entry.getValue();
+
+        entry.setValue(oldInfo.accessModificationTime(accessTime, modificationTime));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
+    }
+}


[04/17] ignite git commit: IGNITE-2860: IGFS: Reworked base meta operations.

Posted by dm...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index 19a91ad..26424f0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -17,11 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
@@ -29,7 +24,6 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
 import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
@@ -38,6 +32,12 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.jetbrains.annotations.Nullable;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -142,7 +142,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEmpty(mgr.directoryListing(ROOT_ID));
 
         assertTrue(mgr.mkdirs(new IgfsPath("/dir"), IgfsImpl.DFLT_DIR_META));
-        assertNotNull(mgr.create(new IgfsPath("/file"), false, false, null, 400, null, false, null));
+        assertNotNull(mgr.create(new IgfsPath("/file"), null, false, 400, null, false, null));
 
         IgfsListingEntry dirEntry = mgr.directoryListing(ROOT_ID).get("dir");
         assertNotNull(dirEntry);
@@ -214,7 +214,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
     private IgfsFileInfo createFileAndGetInfo(String path) throws IgniteCheckedException {
         IgfsPath p = path(path);
 
-        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = mgr.create(p, false, false, null, 400, null, false, null);
+        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = mgr.create(p, null, false, 400, null, false, null);
 
         assert t2 != null;
         assert !t2.get1().isDirectory();
@@ -297,14 +297,13 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEquals(Arrays.asList(ROOT_ID, null, null, null, null), mgr.fileIds(new IgfsPath("/f7/a/b/f6")));
 
         // One of participated files does not exist in cache.
-        expectsRenameFail("/b8", "/b2", "Failed to perform move because some path component was not found.");
+        expectsRenameFail("/b8", "/b2");
 
-        expectsRenameFail("/a", "/b/b8", "Failed to perform move because some path component was not found.");
+        expectsRenameFail("/a", "/b/b8");
 
-        expectsRenameFail("/a/f2", "/a/b/f3", "Failed to perform move because destination points to existing file");
+        expectsRenameFail("/a/f2", "/a/b/f3");
 
-        expectsRenameFail("/a/k", "/a/b/", "Failed to perform move because destination already " +
-            "contains entry with the same name existing file");
+        expectsRenameFail("/a/k", "/a/b/");
 
         mgr.delete(a.id(), "k", z.id());
         mgr.delete(b.id(), "k", k.id());
@@ -414,17 +413,15 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
     /**
      * Test expected failures for 'move file' operation.
-     *
-     * @param msg Failure message if expected exception was not thrown.
      */
-    private void expectsRenameFail(final String src, final String dst, @Nullable String msg) {
+    private void expectsRenameFail(final String src, final String dst) {
         Throwable err = assertThrowsInherited(log, new Callable() {
             @Override public Object call() throws Exception {
                 mgr.move(new IgfsPath(src), new IgfsPath(dst));
 
                 return null;
             }
-        }, IgfsException.class, msg);
+        }, IgfsException.class, null);
 
         assertTrue("Unexpected cause: " + err, err instanceof IgfsException);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
index f567099..4112846 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
@@ -626,17 +626,17 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
     /** @throws Exception If failed. */
     public void testCreateOpenAppend() throws Exception {
         // Error - path points to root directory.
-        assertCreateFails("/", false, "Failed to create file (path points to an existing directory)");
+        assertCreateFails("/", false);
 
         // Create directories.
         igfs.mkdirs(path("/A/B1/C1"));
 
         // Error - path points to directory.
         for (String path : Arrays.asList("/A", "/A/B1", "/A/B1/C1")) {
-            assertCreateFails(path, false, "Failed to create file (path points to an existing directory)");
-            assertCreateFails(path, true, "Failed to create file (path points to an existing directory)");
-            assertAppendFails(path, false, "Failed to open file (path points to an existing directory)");
-            assertAppendFails(path, true, "Failed to open file (path points to an existing directory)");
+            assertCreateFails(path, false);
+            assertCreateFails(path, true);
+            assertAppendFails(path, false);
+            assertAppendFails(path, true);
             assertOpenFails(path, "Failed to open file (not a file)");
         }
 
@@ -647,13 +647,13 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
         for (String path : Arrays.asList("/A/a", "/A/B1/a", "/A/B1/C1/a")) {
             // Error - file doesn't exist.
             assertOpenFails(path, "File not found");
-            assertAppendFails(path, false, "File not found");
+            assertAppendFails(path, false);
 
             // Create new and write.
             assertEquals(text1, create(path, false, text1));
 
             // Error - file already exists.
-            assertCreateFails(path, false, "Failed to create file (file already exists and overwrite flag is false)");
+            assertCreateFails(path, false);
 
             // Overwrite existent.
             assertEquals(text2, create(path, true, text2));
@@ -669,7 +669,7 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
 
             // Error - file doesn't exist.
             assertOpenFails(path, "File not found");
-            assertAppendFails(path, false, "File not found");
+            assertAppendFails(path, false);
 
             // Create with append.
             assertEquals(text1, append(path, true, text1));
@@ -927,16 +927,15 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
      *
      * @param path File path to create.
      * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
-     * @param msg Failure message if expected exception was not thrown.
      */
-    private void assertCreateFails(final String path, final boolean overwrite, @Nullable String msg) {
+    private void assertCreateFails(final String path, final boolean overwrite) {
         GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
                 igfs.create(path(path), overwrite);
 
                 return false;
             }
-        }, IgfsException.class, msg);
+        }, IgfsException.class, null);
     }
 
     /**
@@ -944,16 +943,15 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
      *
      * @param path File path to append.
      * @param create Create file if it doesn't exist yet.
-     * @param msg Failure message if expected exception was not thrown.
      */
-    private void assertAppendFails(final String path, final boolean create, @Nullable String msg) {
+    private void assertAppendFails(final String path, final boolean create) {
         GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
                 igfs.append(path(path), create);
 
                 return false;
             }
-        }, IgfsException.class, msg);
+        }, IgfsException.class, null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStartCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStartCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStartCacheTest.java
index 56559cb..eff44f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStartCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStartCacheTest.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.BufferedWriter;
-import java.io.OutputStreamWriter;
-import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -35,6 +32,10 @@ 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 java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.util.concurrent.Callable;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -85,7 +86,7 @@ public class IgfsStartCacheTest extends IgfsCommonAbstractTest {
             metaCacheCfg.setName("metaCache");
             metaCacheCfg.setCacheMode(REPLICATED);
             metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-            dataCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+            metaCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 
             cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
             cfg.setFileSystemConfiguration(igfsCfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebf40752/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
index c013cae..1dd665a 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
@@ -46,11 +46,11 @@ public class IgfsEventsTestSuite extends TestSuite {
 
         TestSuite suite = new TestSuite("Ignite FS Events Test Suite");
 
-        suite.addTest(new TestSuite(ldr.loadClass(ShmemPrivate.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(ShmemPrimary.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(ShmemDualSync.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(ShmemDualAsync.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(LoopbackPrivate.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(LoopbackPrimary.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(LoopbackDualSync.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(LoopbackDualAsync.class.getName())));
 
@@ -66,7 +66,7 @@ public class IgfsEventsTestSuite extends TestSuite {
 
         TestSuite suite = new TestSuite("Ignite IGFS Events Test Suite Noarch Only");
 
-        suite.addTest(new TestSuite(ldr.loadClass(LoopbackPrivate.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(LoopbackPrimary.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(LoopbackDualSync.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(LoopbackDualAsync.class.getName())));
 
@@ -76,7 +76,7 @@ public class IgfsEventsTestSuite extends TestSuite {
     /**
      * Shared memory IPC in PRIVATE mode.
      */
-    public static class ShmemPrivate extends IgfsEventsAbstractSelfTest {
+    public static class ShmemPrimary extends IgfsEventsAbstractSelfTest {
         /** {@inheritDoc} */
         @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
             FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
@@ -95,7 +95,7 @@ public class IgfsEventsTestSuite extends TestSuite {
     /**
      * Loopback socket IPS in PRIVATE mode.
      */
-    public static class LoopbackPrivate extends IgfsEventsAbstractSelfTest {
+    public static class LoopbackPrimary extends IgfsEventsAbstractSelfTest {
         /** {@inheritDoc} */
         @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
             FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();


[09/17] ignite git commit: IGNITE-2813: IGFS: Optimized metadata values splitting file and directory into separate classes.

Posted by dm...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 d91b0bc..84e4dae 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
@@ -86,8 +86,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
 
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.builder;
-
 /**
  * Cache based structure (meta data) manager.
  */
@@ -117,7 +115,7 @@ public class IgfsMetaManager extends IgfsManager {
     private CountDownLatch metaCacheStartLatch;
 
     /** File ID to file info projection. */
-    private IgniteInternalCache<IgniteUuid, IgfsFileInfo> id2InfoPrj;
+    private IgniteInternalCache<IgniteUuid, IgfsEntryInfo> id2InfoPrj;
 
     /** Predefined key for sampling mode value. */
     private GridCacheInternal sampling;
@@ -175,7 +173,7 @@ public class IgfsMetaManager extends IgfsManager {
                 }
             });
 
-        id2InfoPrj = (IgniteInternalCache<IgniteUuid, IgfsFileInfo>)metaCache.<IgniteUuid, IgfsFileInfo>cache();
+        id2InfoPrj = (IgniteInternalCache<IgniteUuid, IgfsEntryInfo>)metaCache.<IgniteUuid, IgfsEntryInfo>cache();
 
         locNode = igfsCtx.kernalContext().discovery().localNode();
 
@@ -440,17 +438,17 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File info.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgfsFileInfo info(@Nullable IgniteUuid fileId) throws IgniteCheckedException {
+    @Nullable public IgfsEntryInfo info(@Nullable IgniteUuid fileId) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 if (fileId == null)
                     return null;
 
-                IgfsFileInfo info = getInfo(fileId);
+                IgfsEntryInfo info = getInfo(fileId);
 
                 // Force root ID always exist in cache.
                 if (info == null && IgfsUtils.ROOT_ID.equals(fileId))
-                    info = createSystemEntryIfAbsent(fileId);
+                    info = createSystemDirectoryIfAbsent(fileId);
 
                 return info;
             }
@@ -469,7 +467,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Files details.
      * @throws IgniteCheckedException If failed.
      */
-    public Map<IgniteUuid, IgfsFileInfo> infos(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
+    public Map<IgniteUuid, IgfsEntryInfo> infos(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 validTxState(false);
@@ -479,13 +477,13 @@ public class IgfsMetaManager extends IgfsManager {
                 if (F.isEmpty(fileIds))
                     return Collections.emptyMap();
 
-                Map<IgniteUuid, IgfsFileInfo> map = getInfos(fileIds);
+                Map<IgniteUuid, IgfsEntryInfo> map = getInfos(fileIds);
 
                 // Force root ID always exist in cache.
                 if (fileIds.contains(IgfsUtils.ROOT_ID) && !map.containsKey(IgfsUtils.ROOT_ID)) {
                     map = new GridLeanMap<>(map);
 
-                    map.put(IgfsUtils.ROOT_ID, createSystemEntryIfAbsent(IgfsUtils.ROOT_ID));
+                    map.put(IgfsUtils.ROOT_ID, createSystemDirectoryIfAbsent(IgfsUtils.ROOT_ID));
                 }
 
                 return map;
@@ -506,7 +504,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Locked file info or {@code null} if file cannot be locked or doesn't exist.
      * @throws IgniteCheckedException If the file with such id does not exist, or on another failure.
      */
-    public @Nullable IgfsFileInfo lock(IgniteUuid fileId, boolean delete) throws IgniteCheckedException {
+    public @Nullable IgfsEntryInfo lock(IgniteUuid fileId, boolean delete) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 validTxState(false);
@@ -517,7 +515,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 try {
                     // Lock file ID for this transaction.
-                    IgfsFileInfo oldInfo = info(fileId);
+                    IgfsEntryInfo oldInfo = info(fileId);
 
                     if (oldInfo == null)
                         return null;
@@ -525,7 +523,7 @@ public class IgfsMetaManager extends IgfsManager {
                     if (oldInfo.lockId() != null)
                         return null; // The file is already locked, we cannot lock it.
 
-                    IgfsFileInfo newInfo = invokeLock(fileId, delete);
+                    IgfsEntryInfo newInfo = invokeLock(fileId, delete);
 
                     tx.commit();
 
@@ -566,7 +564,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @param modificationTime Modification time to write to file info.
      * @throws IgniteCheckedException If failed.
      */
-    public void unlock(final IgfsFileInfo info, final long modificationTime) throws IgniteCheckedException {
+    public void unlock(final IgfsEntryInfo info, final long modificationTime) throws IgniteCheckedException {
         validTxState(false);
 
         assert info != null;
@@ -589,7 +587,7 @@ public class IgfsMetaManager extends IgfsManager {
                             IgniteUuid fileId = info.id();
 
                             // Lock file ID for this transaction.
-                            IgfsFileInfo oldInfo = info(fileId);
+                            IgfsEntryInfo oldInfo = info(fileId);
 
                             if (oldInfo == null)
                                 throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not " +
@@ -628,7 +626,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Locked file details. Resulting map doesn't contain details for not-existent files.
      * @throws IgniteCheckedException If failed.
      */
-    private Map<IgniteUuid, IgfsFileInfo> lockIds(IgniteUuid... fileIds) throws IgniteCheckedException {
+    private Map<IgniteUuid, IgfsEntryInfo> lockIds(IgniteUuid... fileIds) throws IgniteCheckedException {
         validTxState(true);
 
         assert fileIds != null && fileIds.length > 0;
@@ -668,7 +666,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Map with lock info.
      * @throws IgniteCheckedException If failed.
      */
-    private Map<IgniteUuid, IgfsFileInfo> lockIds(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
+    private Map<IgniteUuid, IgfsEntryInfo> lockIds(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
         assert isSorted(fileIds);
         validTxState(true);
 
@@ -676,7 +674,7 @@ public class IgfsMetaManager extends IgfsManager {
             log.debug("Locking file ids: " + fileIds);
 
         // Lock files and get their infos.
-        Map<IgniteUuid, IgfsFileInfo> map = getInfos(fileIds);
+        Map<IgniteUuid, IgfsEntryInfo> map = getInfos(fileIds);
 
         if (log.isDebugEnabled())
             log.debug("Locked file ids: " + fileIds);
@@ -684,7 +682,7 @@ public class IgfsMetaManager extends IgfsManager {
         for (IgniteUuid fileId : fileIds) {
             if (IgfsUtils.isRootOrTrashId(fileId)) {
                 if (!map.containsKey(fileId))
-                    map.put(fileId, createSystemEntryIfAbsent(fileId));
+                    map.put(fileId, createSystemDirectoryIfAbsent(fileId));
             }
         }
 
@@ -699,13 +697,15 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Value of created or existing system entry.
      * @throws IgniteCheckedException On error.
      */
-    private IgfsFileInfo createSystemEntryIfAbsent(IgniteUuid id)
+    private IgfsEntryInfo createSystemDirectoryIfAbsent(IgniteUuid id)
         throws IgniteCheckedException {
         assert IgfsUtils.isRootOrTrashId(id);
 
-        IgfsFileInfo info = new IgfsFileInfo(id);
+        long time = System.currentTimeMillis();
+
+        IgfsEntryInfo info = IgfsUtils.createDirectory(id);
 
-        IgfsFileInfo oldInfo = id2InfoPrj.getAndPutIfAbsent(id, info);
+        IgfsEntryInfo oldInfo = id2InfoPrj.getAndPutIfAbsent(id, info);
 
         if (oldInfo != null)
             info = oldInfo;
@@ -740,7 +740,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return First qualified file info.
      * @throws IgniteCheckedException If failed to get file for fragmentizer.
      */
-    public IgfsFileInfo fileForFragmentizer(Collection<IgniteUuid> exclude) throws IgniteCheckedException {
+    public IgfsEntryInfo fileForFragmentizer(Collection<IgniteUuid> exclude) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 return fileForFragmentizer0(IgfsUtils.ROOT_ID, exclude);
@@ -761,9 +761,9 @@ public class IgfsMetaManager extends IgfsManager {
      * @return First qualified file info.
      * @throws IgniteCheckedException If failed to get file for fragmentizer.
      */
-    private IgfsFileInfo fileForFragmentizer0(IgniteUuid parentId, Collection<IgniteUuid> exclude)
+    private IgfsEntryInfo fileForFragmentizer0(IgniteUuid parentId, Collection<IgniteUuid> exclude)
         throws IgniteCheckedException {
-        IgfsFileInfo info = info(parentId);
+        IgfsEntryInfo info = info(parentId);
 
         // Check if file was concurrently deleted.
         if (info == null)
@@ -775,7 +775,7 @@ public class IgfsMetaManager extends IgfsManager {
 
         for (IgfsListingEntry entry : listing.values()) {
             if (entry.isFile()) {
-                IgfsFileInfo fileInfo = info(entry.fileId());
+                IgfsEntryInfo fileInfo = info(entry.fileId());
 
                 if (fileInfo != null) {
                     if (!exclude.contains(fileInfo.id()) &&
@@ -785,7 +785,7 @@ public class IgfsMetaManager extends IgfsManager {
                 }
             }
             else {
-                IgfsFileInfo fileInfo = fileForFragmentizer0(entry.fileId(), exclude);
+                IgfsEntryInfo fileInfo = fileForFragmentizer0(entry.fileId(), exclude);
 
                 if (fileInfo != null)
                     return fileInfo;
@@ -807,7 +807,7 @@ public class IgfsMetaManager extends IgfsManager {
         throws IgniteCheckedException {
         assert fileId != null;
 
-        IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) :
+        IgfsEntryInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) :
             getInfo(fileId);
 
         return info == null ? Collections.<String, IgfsListingEntry>emptyMap() : info.listing();
@@ -822,7 +822,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File id already stored in meta cache or {@code null} if passed file info was stored.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteUuid putIfAbsentNonTx(IgniteUuid parentId, String fileName, IgfsFileInfo newFileInfo)
+    private IgniteUuid putIfAbsentNonTx(IgniteUuid parentId, String fileName, IgfsEntryInfo newFileInfo)
         throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Locking parent id [parentId=" + parentId + ", fileName=" + fileName + ", newFileInfo=" +
@@ -831,7 +831,7 @@ public class IgfsMetaManager extends IgfsManager {
         validTxState(true);
 
         // Lock only parent file ID.
-        IgfsFileInfo parentInfo = info(parentId);
+        IgfsEntryInfo parentInfo = info(parentId);
 
         if (parentInfo == null)
             throw fsException(new IgfsPathNotFoundException("Failed to lock parent directory (not found): " +
@@ -858,7 +858,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File info of renamed entry.
      * @throws IgniteCheckedException In case of exception.
      */
-    public IgfsFileInfo move(IgfsPath srcPath, IgfsPath dstPath) throws IgniteCheckedException {
+    public IgfsEntryInfo move(IgfsPath srcPath, IgfsPath dstPath) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 validTxState(false);
@@ -902,7 +902,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 try {
                     // Obtain the locks.
-                    final Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
+                    final Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                     // Verify integrity of source and destination paths.
                     if (!srcPathIds.verifyIntegrity(lockInfos))
@@ -914,7 +914,7 @@ public class IgfsMetaManager extends IgfsManager {
                             "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']');
 
                     // Addiional check: is destination directory?
-                    IgfsFileInfo dstParentInfo = lockInfos.get(dstPathIds.lastId());
+                    IgfsEntryInfo dstParentInfo = lockInfos.get(dstPathIds.lastId());
 
                     if (dstParentInfo.isFile())
                         throw new IgfsPathAlreadyExistsException("Failed to perform move because destination points " +
@@ -927,9 +927,9 @@ public class IgfsMetaManager extends IgfsManager {
                             ", dst=" + dstPath + ']');
 
                     // Actual move: remove from source parent and add to destination target.
-                    IgfsFileInfo srcParentInfo = lockInfos.get(srcPathIds.lastParentId());
+                    IgfsEntryInfo srcParentInfo = lockInfos.get(srcPathIds.lastParentId());
 
-                    IgfsFileInfo srcInfo = lockInfos.get(srcPathIds.lastId());
+                    IgfsEntryInfo srcInfo = lockInfos.get(srcPathIds.lastId());
                     String srcName = srcPathIds.lastPart();
                     IgfsListingEntry srcEntry = srcParentInfo.listing().get(srcName);
 
@@ -940,7 +940,7 @@ public class IgfsMetaManager extends IgfsManager {
                     IgfsPath newPath = new IgfsPath(dstPathIds.path(), dstName);
 
                     // Set the new path to the info to simplify event creation:
-                    return IgfsFileInfo.builder(srcInfo).path(newPath).build();
+                    return srcInfo.path(newPath);
                 }
                 finally {
                     tx.close();
@@ -965,7 +965,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return verification result.
      */
     private static boolean verifyPathIntegrity(IgfsPath path, List<IgniteUuid> expIds,
-        Map<IgniteUuid, IgfsFileInfo> infos) {
+        Map<IgniteUuid, IgfsEntryInfo> infos) {
         List<String> pathParts = path.components();
 
         assert pathParts.size() < expIds.size();
@@ -975,7 +975,7 @@ public class IgfsMetaManager extends IgfsManager {
 
             // If parent ID is null, it doesn't exist.
             if (parentId != null) {
-                IgfsFileInfo parentInfo = infos.get(parentId);
+                IgfsEntryInfo parentInfo = infos.get(parentId);
 
                 // If parent info is null, it doesn't exist.
                 if (parentInfo != null) {
@@ -1019,9 +1019,9 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         // Lock file ID and parent IDs for this transaction.
-        Map<IgniteUuid, IgfsFileInfo> infoMap = lockIds(srcParentId, fileId, destParentId);
+        Map<IgniteUuid, IgfsEntryInfo> infoMap = lockIds(srcParentId, fileId, destParentId);
 
-        IgfsFileInfo srcInfo = infoMap.get(srcParentId);
+        IgfsEntryInfo srcInfo = infoMap.get(srcParentId);
 
         if (srcInfo == null)
             throw fsException(new IgfsPathNotFoundException("Failed to lock source directory (not found?)" +
@@ -1030,7 +1030,7 @@ public class IgfsMetaManager extends IgfsManager {
         if (!srcInfo.isDirectory())
             throw fsException(new IgfsPathIsNotDirectoryException("Source is not a directory: " + srcInfo));
 
-        IgfsFileInfo destInfo = infoMap.get(destParentId);
+        IgfsEntryInfo destInfo = infoMap.get(destParentId);
 
         if (destInfo == null)
             throw fsException(new IgfsPathNotFoundException("Failed to lock destination directory (not found?)" +
@@ -1039,7 +1039,7 @@ public class IgfsMetaManager extends IgfsManager {
         if (!destInfo.isDirectory())
             throw fsException(new IgfsPathIsNotDirectoryException("Destination is not a directory: " + destInfo));
 
-        IgfsFileInfo fileInfo = infoMap.get(fileId);
+        IgfsEntryInfo fileInfo = infoMap.get(fileId);
 
         if (fileInfo == null)
             throw fsException(new IgfsPathNotFoundException("Failed to lock target file (not found?) [fileId=" +
@@ -1080,7 +1080,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 try {
                     // NB: We may lock root because its id is less than any other id:
-                    final IgfsFileInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, trashId).get(IgfsUtils.ROOT_ID);
+                    final IgfsEntryInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, trashId).get(IgfsUtils.ROOT_ID);
 
                     assert rootInfo != null;
 
@@ -1094,13 +1094,17 @@ public class IgfsMetaManager extends IgfsManager {
                     // Construct new info and move locked entries from root to it.
                     Map<String, IgfsListingEntry> transferListing = new HashMap<>(rootListingMap);
 
-                    IgfsFileInfo newInfo = new IgfsFileInfo(transferListing);
+                    IgfsEntryInfo newInfo = IgfsUtils.createDirectory(
+                        IgniteUuid.randomUuid(),
+                        transferListing,
+                        (Map<String,String>)null
+                    );
 
                     createNewEntry(newInfo, trashId, newInfo.id().toString());
 
                     // Remove listing entries from root.
                     // Note that root directory properties and other attributes are preserved:
-                    id2InfoPrj.put(IgfsUtils.ROOT_ID, new IgfsFileInfo(null/*listing*/, rootInfo));
+                    id2InfoPrj.put(IgfsUtils.ROOT_ID, rootInfo.listing(null));
 
                     tx.commit();
 
@@ -1159,13 +1163,13 @@ public class IgfsMetaManager extends IgfsManager {
 
                 try {
                     // Lock participants.
-                    Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(allIds);
+                    Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
 
                     // Ensure that all participants are still in place.
                     if (!pathIds.verifyIntegrity(lockInfos))
                         return null;
 
-                    IgfsFileInfo victimInfo = lockInfos.get(victimId);
+                    IgfsEntryInfo victimInfo = lockInfos.get(victimId);
 
                     // Cannot delete non-empty directory if recursive flag is not set.
                     if (!recursive && victimInfo.hasChildren())
@@ -1173,14 +1177,14 @@ public class IgfsMetaManager extends IgfsManager {
                             "empty and recursive flag is not set).");
 
                     // Prepare trash data.
-                    IgfsFileInfo trashInfo = lockInfos.get(trashId);
+                    IgfsEntryInfo trashInfo = lockInfos.get(trashId);
                     final String trashName = victimId.toString();
 
                     assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " +
                         "destination directory (file already exists) [destName=" + trashName + ']';
 
                     IgniteUuid parentId = pathIds.lastParentId();
-                    IgfsFileInfo parentInfo = lockInfos.get(parentId);
+                    IgfsEntryInfo parentInfo = lockInfos.get(parentId);
 
                     transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName);
 
@@ -1227,13 +1231,13 @@ public class IgfsMetaManager extends IgfsManager {
             // Handle special case when we deleting root directory.
             assert IgfsUtils.ROOT_ID.equals(id);
 
-            IgfsFileInfo rootInfo = getInfo(IgfsUtils.ROOT_ID);
+            IgfsEntryInfo rootInfo = getInfo(IgfsUtils.ROOT_ID);
 
             if (rootInfo == null)
                 return null; // Root was never created.
 
             // Ensure trash directory existence.
-            createSystemEntryIfAbsent(trashId);
+            createSystemDirectoryIfAbsent(trashId);
 
             Map<String, IgfsListingEntry> rootListing = rootInfo.listing();
 
@@ -1253,13 +1257,18 @@ public class IgfsMetaManager extends IgfsManager {
 
                 transferListing.putAll(rootListing);
 
-                IgfsFileInfo newInfo = new IgfsFileInfo(transferListing);
+                IgfsEntryInfo newInfo = IgfsUtils.createDirectory(
+                    IgniteUuid.randomUuid(),
+                    transferListing,
+                    (Map<String,String>)null
+                );
 
                 createNewEntry(newInfo, trashId, newInfo.id().toString());
 
                 // Remove listing entries from root.
                 for (Map.Entry<String, IgfsListingEntry> entry : transferListing.entrySet())
-                    id2InfoPrj.invoke(IgfsUtils.ROOT_ID, new ListingRemoveProcessor(entry.getKey(), entry.getValue().fileId()));
+                    id2InfoPrj.invoke(IgfsUtils.ROOT_ID,
+                        new ListingRemoveProcessor(entry.getKey(), entry.getValue().fileId()));
 
                 resId = newInfo.id();
             }
@@ -1268,7 +1277,7 @@ public class IgfsMetaManager extends IgfsManager {
         }
         else {
             // Ensure trash directory existence.
-            createSystemEntryIfAbsent(trashId);
+            createSystemDirectoryIfAbsent(trashId);
 
             moveNonTx(id, name, parentId, id.toString(), trashId);
 
@@ -1310,9 +1319,9 @@ public class IgfsMetaManager extends IgfsManager {
                     for (IgfsListingEntry childEntry : listing.values())
                         allIds[i++] = childEntry.fileId();
 
-                    Map<IgniteUuid, IgfsFileInfo> locks = lockIds(allIds);
+                    Map<IgniteUuid, IgfsEntryInfo> locks = lockIds(allIds);
 
-                    IgfsFileInfo parentInfo = locks.get(parentId);
+                    IgfsEntryInfo parentInfo = locks.get(parentId);
 
                     // Ensure parent is still in place.
                     if (parentInfo != null) {
@@ -1327,7 +1336,7 @@ public class IgfsMetaManager extends IgfsManager {
                             String childName = entry.getKey();
                             IgniteUuid childId = entry.getValue().fileId();
 
-                            IgfsFileInfo entryInfo = locks.get(childId);
+                            IgfsEntryInfo entryInfo = locks.get(childId);
 
                             if (entryInfo != null) {
                                 // File must be locked for deletion:
@@ -1351,7 +1360,7 @@ public class IgfsMetaManager extends IgfsManager {
                         }
 
                         // Update parent listing.
-                        id2InfoPrj.put(parentId, new IgfsFileInfo(newListing, parentInfo));
+                        id2InfoPrj.put(parentId, parentInfo.listing(newListing));
                     }
 
                     tx.commit();
@@ -1391,9 +1400,9 @@ public class IgfsMetaManager extends IgfsManager {
                 try {
                     boolean res = false;
 
-                    Map<IgniteUuid, IgfsFileInfo> infos = lockIds(parentId, id);
+                    Map<IgniteUuid, IgfsEntryInfo> infos = lockIds(parentId, id);
 
-                    IgfsFileInfo victim = infos.get(id);
+                    IgfsEntryInfo victim = infos.get(id);
 
                     if (victim == null)
                         return res;
@@ -1403,7 +1412,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     // Proceed only in case both parent and child exist.
                     if (infos.containsKey(parentId) && infos.containsKey(id)) {
-                        IgfsFileInfo parentInfo = infos.get(parentId);
+                        IgfsEntryInfo parentInfo = infos.get(parentId);
 
                         assert parentInfo != null;
 
@@ -1448,7 +1457,7 @@ public class IgfsMetaManager extends IgfsManager {
                 for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++) {
                     IgniteUuid trashId = IgfsUtils.trashId(i);
 
-                    IgfsFileInfo trashInfo = getInfo(trashId);
+                    IgfsEntryInfo trashInfo = getInfo(trashId);
 
                     if (trashInfo != null && trashInfo.hasChildren()) {
                         for (IgfsListingEntry entry : trashInfo.listing().values())
@@ -1474,7 +1483,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Updated file info or {@code null} if such file ID not found.
      * @throws IgniteCheckedException If operation failed.
      */
-    @Nullable private IgfsFileInfo updatePropertiesNonTx(final IgniteUuid fileId, Map<String, String> props)
+    @Nullable private IgfsEntryInfo updatePropertiesNonTx(final IgniteUuid fileId, Map<String, String> props)
         throws IgniteCheckedException {
         assert fileId != null;
         assert !F.isEmpty(props) : "Expects not-empty file's properties";
@@ -1485,7 +1494,7 @@ public class IgfsMetaManager extends IgfsManager {
             log.debug("Update file properties [fileId=" + fileId + ", props=" + props + ']');
 
         try {
-            final IgfsFileInfo oldInfo = info(fileId);
+            final IgfsEntryInfo oldInfo = info(fileId);
 
             if (oldInfo == null)
                 return null;
@@ -1505,7 +1514,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Updated file info or {@code null} if such file ID not found.
      * @throws IgniteCheckedException If operation failed.
      */
-    @Nullable public IgfsFileInfo updateProperties(IgniteUuid fileId, Map<String, String> props)
+    @Nullable public IgfsEntryInfo updateProperties(IgniteUuid fileId, Map<String, String> props)
         throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
@@ -1514,7 +1523,7 @@ public class IgfsMetaManager extends IgfsManager {
                 IgniteInternalTx tx = startTx();
 
                 try {
-                    IgfsFileInfo info = updatePropertiesNonTx(fileId, props);
+                    IgfsEntryInfo info = updatePropertiesNonTx(fileId, props);
 
                     tx.commit();
 
@@ -1542,7 +1551,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @param affRange Affinity range.
      * @return New file info.
      */
-    public IgfsFileInfo reserveSpace(IgfsPath path, IgniteUuid fileId, long space, IgfsFileAffinityRange affRange)
+    public IgfsEntryInfo reserveSpace(IgfsPath path, IgniteUuid fileId, long space, IgfsFileAffinityRange affRange)
         throws IgniteCheckedException {
         validTxState(false);
 
@@ -1555,12 +1564,12 @@ public class IgfsMetaManager extends IgfsManager {
 
                 try {
                     // Lock file ID for this transaction.
-                    IgfsFileInfo oldInfo = info(fileId);
+                    IgfsEntryInfo oldInfo = info(fileId);
 
                     if (oldInfo == null)
                         throw fsException("File has been deleted concurrently [path=" + path + ", id=" + fileId + ']');
 
-                    IgfsFileInfo newInfo = invokeAndGet(fileId, new FileReserveSpaceProcessor(space, affRange));
+                    IgfsEntryInfo newInfo = invokeAndGet(fileId, new FileReserveSpaceProcessor(space, affRange));
 
                     tx.commit();
 
@@ -1590,8 +1599,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Updated file info or {@code null} if such file ID not found.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgfsFileInfo updateInfo(IgniteUuid fileId,
-        EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo> proc) throws IgniteCheckedException {
+    @Nullable public IgfsEntryInfo updateInfo(IgniteUuid fileId,
+        EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo> proc) throws IgniteCheckedException {
         validTxState(false);
         assert fileId != null;
         assert proc != null;
@@ -1605,12 +1614,12 @@ public class IgfsMetaManager extends IgfsManager {
 
                 try {
                     // Lock file ID for this transaction.
-                    IgfsFileInfo oldInfo = info(fileId);
+                    IgfsEntryInfo oldInfo = info(fileId);
 
                     if (oldInfo == null)
                         return null; // File not found.
 
-                    IgfsFileInfo newInfo = invokeAndGet(fileId, proc);
+                    IgfsEntryInfo newInfo = invokeAndGet(fileId, proc);
 
                     if (newInfo == null)
                         throw fsException("Failed to update file info with null value" +
@@ -1673,7 +1682,7 @@ public class IgfsMetaManager extends IgfsManager {
                     IgniteInternalTx tx = startTx();
 
                     try {
-                        final Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
+                        final Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                         if (!pathIds.verifyIntegrity(lockInfos))
                             // Directory structure changed concurrently. So we simply re-try.
@@ -1779,7 +1788,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @param name Name in parent.
      * @throws IgniteCheckedException If failed.
      */
-    private void createNewEntry(IgfsFileInfo info, IgniteUuid parentId, String name) throws IgniteCheckedException {
+    private void createNewEntry(IgfsEntryInfo info, IgniteUuid parentId, String name) throws IgniteCheckedException {
         validTxState(true);
 
         if (!id2InfoPrj.putIfAbsent(info.id(), info))
@@ -1815,7 +1824,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Resulting file info.
      * @throws IgniteCheckedException If failed.
      */
-    private IgfsFileInfo invokeLock(IgniteUuid id, boolean delete) throws IgniteCheckedException {
+    private IgfsEntryInfo invokeLock(IgniteUuid id, boolean delete) throws IgniteCheckedException {
         return invokeAndGet(id, new FileLockProcessor(createFileLockId(delete)));
     }
 
@@ -1840,7 +1849,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return New file info.
      * @throws IgniteCheckedException If failed.
      */
-    private IgfsFileInfo invokeAndGet(IgniteUuid id, EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo> proc)
+    private IgfsEntryInfo invokeAndGet(IgniteUuid id, EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo> proc)
         throws IgniteCheckedException {
         validTxState(true);
 
@@ -1854,7 +1863,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Info.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private IgfsFileInfo getInfo(IgniteUuid id) throws IgniteCheckedException {
+    @Nullable private IgfsEntryInfo getInfo(IgniteUuid id) throws IgniteCheckedException {
         return id2InfoPrj.get(id);
     }
 
@@ -1865,7 +1874,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Infos map.
      * @throws IgniteCheckedException If failed.
      */
-    private Map<IgniteUuid, IgfsFileInfo> getInfos(Collection<IgniteUuid> ids) throws IgniteCheckedException {
+    private Map<IgniteUuid, IgfsEntryInfo> getInfos(Collection<IgniteUuid> ids) throws IgniteCheckedException {
         return id2InfoPrj.getAll(ids);
     }
 
@@ -1909,7 +1918,7 @@ public class IgfsMetaManager extends IgfsManager {
                         private OutputStream out;
 
                         @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
-                            IgfsFileInfo> infos) throws Exception {
+                            IgfsEntryInfo> infos) throws Exception {
                             validTxState(true);
 
                             assert !infos.isEmpty();
@@ -1924,7 +1933,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                             assert parentPath != null;
 
-                            IgfsFileInfo parentInfo = infos.get(parentPath);
+                            IgfsEntryInfo parentInfo = infos.get(parentPath);
 
                             // Delegate to the secondary file system.
                             out = simpleCreate ? fs.create(path, overwrite) :
@@ -1963,15 +1972,23 @@ public class IgfsMetaManager extends IgfsManager {
                                 throw fsException("Failed to open output stream to the file created in " +
                                     "the secondary file system because the path points to a directory: " + path);
 
-                            IgfsFileInfo newInfo = new IgfsFileInfo(status.blockSize(), status.length(), affKey,
-                                createFileLockId(false), igfsCtx.igfs().evictExclude(path, false), status.properties(),
-                                status.accessTime(), status.modificationTime());
+                            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.
                             IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), path.name(), newInfo);
 
                             if (oldId != null) {
-                                IgfsFileInfo oldInfo = info(oldId);
+                                IgfsEntryInfo oldInfo = info(oldId);
 
                                 assert oldInfo != null; // Otherwise cache is in inconsistent state.
 
@@ -2047,10 +2064,10 @@ public class IgfsMetaManager extends IgfsManager {
                         private OutputStream out;
 
                         @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
-                            IgfsFileInfo> infos) throws Exception {
+                            IgfsEntryInfo> infos) throws Exception {
                             validTxState(true);
 
-                            final IgfsFileInfo info = infos.get(path);
+                            final IgfsEntryInfo info = infos.get(path);
 
                             if (info.isDirectory())
                                 throw fsException("Failed to open output stream to the file in the " +
@@ -2083,7 +2100,7 @@ public class IgfsMetaManager extends IgfsManager {
                             }
 
                             // Set lock and return.
-                            IgfsFileInfo lockedInfo = invokeLock(info.id(), false);
+                            IgfsEntryInfo lockedInfo = invokeLock(info.id(), false);
 
                             return new IgfsSecondaryOutputStreamDescriptor(infos.get(path.parent()).id(),
                                 lockedInfo, out);
@@ -2129,7 +2146,7 @@ public class IgfsMetaManager extends IgfsManager {
                 assert path != null;
 
                 // First, try getting file info without any transactions and synchronization.
-                IgfsFileInfo info = info(fileId(path));
+                IgfsEntryInfo info = info(fileId(path));
 
                 if (info != null) {
                     if (!info.isFile())
@@ -2143,8 +2160,8 @@ public class IgfsMetaManager extends IgfsManager {
                 SynchronizationTask<IgfsSecondaryInputStreamDescriptor> task =
                     new SynchronizationTask<IgfsSecondaryInputStreamDescriptor>() {
                         @Override public IgfsSecondaryInputStreamDescriptor onSuccess(
-                            Map<IgfsPath, IgfsFileInfo> infos) throws Exception {
-                            IgfsFileInfo info = infos.get(path);
+                            Map<IgfsPath, IgfsEntryInfo> infos) throws Exception {
+                            IgfsEntryInfo info = infos.get(path);
 
                             if (info == null)
                                 throw fsException(new IgfsPathNotFoundException("File not found: " + path));
@@ -2183,7 +2200,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File info or {@code null} if file not found.
      * @throws IgniteCheckedException If sync task failed.
      */
-    @Nullable public IgfsFileInfo synchronizeFileDual(final IgfsSecondaryFileSystem fs, final IgfsPath path)
+    @Nullable public IgfsEntryInfo synchronizeFileDual(final IgfsSecondaryFileSystem fs, final IgfsPath path)
         throws IgniteCheckedException {
         assert fs != null;
         assert path != null;
@@ -2191,20 +2208,20 @@ public class IgfsMetaManager extends IgfsManager {
         if (busyLock.enterBusy()) {
             try {
                 // First, try getting file info without any transactions and synchronization.
-                IgfsFileInfo info = info(fileId(path));
+                IgfsEntryInfo info = info(fileId(path));
 
                 if (info != null)
                     return info;
 
                 // If failed, try synchronize.
-                SynchronizationTask<IgfsFileInfo> task =
-                    new SynchronizationTask<IgfsFileInfo>() {
-                        @Override public IgfsFileInfo onSuccess(Map<IgfsPath, IgfsFileInfo> infos)
+                SynchronizationTask<IgfsEntryInfo> task =
+                    new SynchronizationTask<IgfsEntryInfo>() {
+                        @Override public IgfsEntryInfo onSuccess(Map<IgfsPath, IgfsEntryInfo> infos)
                             throws Exception {
                             return infos.get(path);
                         }
 
-                        @Override public IgfsFileInfo onFailure(@Nullable Exception err) throws IgniteCheckedException {
+                        @Override public IgfsEntryInfo onFailure(@Nullable Exception err) throws IgniteCheckedException {
                             throw new IgniteCheckedException("Failed to synchronize path due to secondary file " +
                                 "system exception: " + path, err);
                         }
@@ -2246,7 +2263,7 @@ public class IgfsMetaManager extends IgfsManager {
                 final Deque<IgfsEvent> pendingEvts = new LinkedList<>();
 
                 SynchronizationTask<Boolean> task = new SynchronizationTask<Boolean>() {
-                    @Override public Boolean onSuccess(Map<IgfsPath, IgfsFileInfo> infos) throws Exception {
+                    @Override public Boolean onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception {
                         fs.mkdirs(path, props);
 
                         assert !infos.isEmpty();
@@ -2261,7 +2278,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                         assert parentPath != null;
 
-                        IgfsFileInfo parentPathInfo = infos.get(parentPath);
+                        IgfsEntryInfo parentPathInfo = infos.get(parentPath);
 
                         synchronize(fs, parentPath, parentPathInfo, path, true, null);
 
@@ -2330,11 +2347,11 @@ public class IgfsMetaManager extends IgfsManager {
                 final Collection<IgfsEvent> pendingEvts = new LinkedList<>();
 
                 SynchronizationTask<Boolean> task = new SynchronizationTask<Boolean>() {
-                    @Override public Boolean onSuccess(Map<IgfsPath, IgfsFileInfo> infos) throws Exception {
-                        IgfsFileInfo srcInfo = infos.get(src);
-                        IgfsFileInfo srcParentInfo = infos.get(src.parent());
-                        IgfsFileInfo destInfo = infos.get(dest);
-                        IgfsFileInfo destParentInfo = dest.parent() != null ? infos.get(dest.parent()) : null;
+                    @Override public Boolean onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception {
+                        IgfsEntryInfo srcInfo = infos.get(src);
+                        IgfsEntryInfo srcParentInfo = infos.get(src.parent());
+                        IgfsEntryInfo destInfo = infos.get(dest);
+                        IgfsEntryInfo destParentInfo = dest.parent() != null ? infos.get(dest.parent()) : null;
 
                         // Source path and destination (or destination parent) must exist.
                         if (srcInfo == null)
@@ -2424,8 +2441,8 @@ public class IgfsMetaManager extends IgfsManager {
                 final IgniteUuid trashId = IgfsUtils.randomTrashId();
 
                 SynchronizationTask<Boolean> task = new SynchronizationTask<Boolean>() {
-                    @Override public Boolean onSuccess(Map<IgfsPath, IgfsFileInfo> infos) throws Exception {
-                        IgfsFileInfo info = infos.get(path);
+                    @Override public Boolean onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception {
+                        IgfsEntryInfo info = infos.get(path);
 
                         if (info == null)
                             return false; // File doesn't exist in the secondary file system.
@@ -2482,7 +2499,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Update file info.
      * @throws IgniteCheckedException If update failed.
      */
-    public IgfsFileInfo updateDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
+    public IgfsEntryInfo updateDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
         final Map<String, String> props) throws IgniteCheckedException {
         assert fs != null;
         assert path != null;
@@ -2490,8 +2507,8 @@ public class IgfsMetaManager extends IgfsManager {
 
         if (busyLock.enterBusy()) {
             try {
-                SynchronizationTask<IgfsFileInfo> task = new SynchronizationTask<IgfsFileInfo>() {
-                    @Override public IgfsFileInfo onSuccess(Map<IgfsPath, IgfsFileInfo> infos) throws Exception {
+                SynchronizationTask<IgfsEntryInfo> task = new SynchronizationTask<IgfsEntryInfo>() {
+                    @Override public IgfsEntryInfo onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception {
                         if (infos.get(path) == null)
                             return null;
 
@@ -2500,7 +2517,7 @@ public class IgfsMetaManager extends IgfsManager {
                         return updatePropertiesNonTx(infos.get(path).id(), props);
                     }
 
-                    @Override public IgfsFileInfo onFailure(@Nullable Exception err) throws IgniteCheckedException {
+                    @Override public IgfsEntryInfo onFailure(@Nullable Exception err) throws IgniteCheckedException {
                         U.error(log, "Path update in DUAL mode failed [path=" + path + ", properties=" + props + ']',
                             err);
 
@@ -2531,12 +2548,12 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File info of the end path.
      * @throws IgniteCheckedException If failed.
      */
-    private IgfsFileInfo synchronize(IgfsSecondaryFileSystem fs,
+    private IgfsEntryInfo synchronize(IgfsSecondaryFileSystem fs,
         IgfsPath startPath,
-        IgfsFileInfo startPathInfo,
+        IgfsEntryInfo startPathInfo,
         IgfsPath endPath,
         boolean strict,
-        @Nullable Map<IgfsPath, IgfsFileInfo> created)
+        @Nullable Map<IgfsPath, IgfsEntryInfo> created)
         throws IgniteCheckedException
     {
         assert fs != null;
@@ -2544,7 +2561,7 @@ public class IgfsMetaManager extends IgfsManager {
 
         validTxState(true);
 
-        IgfsFileInfo parentInfo = startPathInfo;
+        IgfsEntryInfo parentInfo = startPathInfo;
 
         List<String> components = endPath.components();
 
@@ -2585,11 +2602,25 @@ public class IgfsMetaManager extends IgfsManager {
                 }
 
                 // Recreate the path locally.
-                IgfsFileInfo curInfo = status.isDirectory() ?
-                    new IgfsFileInfo(true, status.properties(), status.accessTime(), status.modificationTime()) :
-                    new IgfsFileInfo(igfsCtx.configuration().getBlockSize(), status.length(),
-                        igfsCtx.igfs().evictExclude(curPath, false), status.properties(),
-                        status.accessTime(), status.modificationTime());
+                IgfsEntryInfo curInfo = status.isDirectory() ?
+                    IgfsUtils.createDirectory(
+                        IgniteUuid.randomUuid(),
+                        null,
+                        status.properties(),
+                        status.accessTime(),
+                        status.modificationTime()
+                    ) :
+                    IgfsUtils.createFile(
+                        IgniteUuid.randomUuid(),
+                        igfsCtx.configuration().getBlockSize(),
+                        status.length(),
+                        null,
+                        null,
+                        igfsCtx.igfs().evictExclude(curPath, false),
+                        status.properties(),
+                        status.accessTime(),
+                        status.modificationTime()
+                    );
 
                 IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), components.get(i), curInfo);
 
@@ -2719,7 +2750,7 @@ public class IgfsMetaManager extends IgfsManager {
                         lockArr[idx++] = id;
                 }
 
-                Map<IgniteUuid, IgfsFileInfo> idToInfo = lockIds(lockArr);
+                Map<IgniteUuid, IgfsEntryInfo> idToInfo = lockIds(lockArr);
 
                 if (extraLockIds != null) {
                     for (IgniteUuid id : extraLockIds)
@@ -2762,9 +2793,9 @@ public class IgfsMetaManager extends IgfsManager {
                         continue; // Release all locks and try again.
                     else {
                         // Perform synchronization.
-                        Map<IgfsPath, IgfsFileInfo> infos = new HashMap<>();
+                        Map<IgfsPath, IgfsEntryInfo> infos = new HashMap<>();
 
-                        TreeMap<IgfsPath, IgfsFileInfo> created = new TreeMap<>();
+                        TreeMap<IgfsPath, IgfsEntryInfo> created = new TreeMap<>();
 
                         for (IgfsPath path : paths) {
                             IgfsPath parentPath = path.parent();
@@ -2781,7 +2812,7 @@ public class IgfsMetaManager extends IgfsManager {
                                 assert firstParentPath != null;
                                 assert pathToId.get(firstParentPath) != null;
 
-                                IgfsFileInfo info = synchronize(fs,
+                                IgfsEntryInfo info = synchronize(fs,
                                     firstParentPath,
                                     idToInfo.get(pathToId.get(firstParentPath)),
                                     path,
@@ -2899,15 +2930,15 @@ public class IgfsMetaManager extends IgfsManager {
                 IgniteInternalTx tx = startTx();
 
                 try {
-                    Map<IgniteUuid, IgfsFileInfo> infoMap = lockIds(fileId, parentId);
+                    Map<IgniteUuid, IgfsEntryInfo> infoMap = lockIds(fileId, parentId);
 
-                    IgfsFileInfo fileInfo = infoMap.get(fileId);
+                    IgfsEntryInfo fileInfo = infoMap.get(fileId);
 
                     if (fileInfo == null)
                         throw fsException(new IgfsPathNotFoundException("Failed to update times " +
                                 "(path was not found): " + fileName));
 
-                    IgfsFileInfo parentInfo = infoMap.get(parentId);
+                    IgfsEntryInfo parentInfo = infoMap.get(parentId);
 
                     if (parentInfo == null)
                         throw fsException(new IgfsPathNotFoundException("Failed to update times " +
@@ -2967,7 +2998,7 @@ public class IgfsMetaManager extends IgfsManager {
          * @return Task result.
          * @throws Exception If failed.
          */
-        public T onSuccess(Map<IgfsPath, IgfsFileInfo> infos) throws Exception;
+        public T onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception;
 
         /**
          * Callback handler in case synchronization failed.
@@ -2993,7 +3024,7 @@ public class IgfsMetaManager extends IgfsManager {
         private IgfsPath parentPath;
 
         /** Parent path info. */
-        private IgfsFileInfo parentInfo;
+        private IgfsEntryInfo parentInfo;
 
         /**
          * Constructor.
@@ -3003,7 +3034,7 @@ public class IgfsMetaManager extends IgfsManager {
          * @param parentPath Parent path.
          * @param parentInfo Parent info.
          */
-        PathDescriptor(IgfsPath path, List<IgniteUuid> ids, IgfsPath parentPath, IgfsFileInfo parentInfo) {
+        PathDescriptor(IgfsPath path, List<IgniteUuid> ids, IgfsPath parentPath, IgfsEntryInfo parentInfo) {
             assert path != null;
             assert ids != null && !ids.isEmpty();
             assert parentPath == null && parentInfo == null || parentPath != null && parentInfo != null;
@@ -3040,7 +3071,7 @@ public class IgfsMetaManager extends IgfsManager {
          * @param newParentPath New parent path.
          * @param newParentInfo New parent info.
          */
-        private void updateParent(IgfsPath newParentPath, IgfsFileInfo newParentInfo) {
+        private void updateParent(IgfsPath newParentPath, IgfsEntryInfo newParentInfo) {
             assert newParentPath != null;
             assert newParentInfo != null;
             assert path.isSubDirectoryOf(newParentPath);
@@ -3065,7 +3096,7 @@ public class IgfsMetaManager extends IgfsManager {
          *
          * @return Parent path info.
          */
-        private IgfsFileInfo parentInfo() {
+        private IgfsEntryInfo parentInfo() {
             return parentInfo;
         }
     }
@@ -3074,7 +3105,7 @@ public class IgfsMetaManager extends IgfsManager {
      * Remove entry from directory listing.
      */
     @GridInternal
-    private static final class ListingRemoveProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+    private static final class ListingRemoveProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3104,9 +3135,9 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args)
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo fileInfo = e.getValue();
+            IgfsEntryInfo fileInfo = e.getValue();
 
             assert fileInfo != null;
             assert fileInfo.isDirectory();
@@ -3124,7 +3155,7 @@ public class IgfsMetaManager extends IgfsManager {
             // Modify listing in-place.
             listing.remove(fileName);
 
-            e.setValue(new IgfsFileInfo(listing, fileInfo));
+            e.setValue(fileInfo.listing(listing));
 
             return null;
         }
@@ -3146,7 +3177,7 @@ public class IgfsMetaManager extends IgfsManager {
      * Update directory listing closure.
      */
     @GridInternal
-    private static final class ListingAddProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+    private static final class ListingAddProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3180,8 +3211,8 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args) {
-            IgfsFileInfo fileInfo = e.getValue();
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args) {
+            IgfsEntryInfo fileInfo = e.getValue();
 
             assert fileInfo.isDirectory();
 
@@ -3195,7 +3226,7 @@ public class IgfsMetaManager extends IgfsManager {
                     " [listing=" + listing + ", fileName=" + fileName + ", entry=" + entry +
                     ", oldEntry=" + oldEntry + ']');
 
-            e.setValue(new IgfsFileInfo(listing, fileInfo));
+            e.setValue(fileInfo.listing(listing));
 
             return null;
         }
@@ -3221,7 +3252,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Listing replace processor.
      */
-    private static final class ListingReplaceProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+    private static final class ListingReplaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3251,9 +3282,9 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args)
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo fileInfo = e.getValue();
+            IgfsEntryInfo fileInfo = e.getValue();
 
             assert fileInfo.isDirectory();
 
@@ -3267,7 +3298,7 @@ public class IgfsMetaManager extends IgfsManager {
 
             listing.put(name, new IgfsListingEntry(id, oldEntry.isDirectory()));
 
-            e.setValue(new IgfsFileInfo(listing, fileInfo));
+            e.setValue(fileInfo.listing(listing));
 
             return null;
         }
@@ -3289,7 +3320,7 @@ public class IgfsMetaManager extends IgfsManager {
      * Update path closure.
      */
     @GridInternal
-    private static final class UpdatePathProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+    private static final class UpdatePathProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3312,10 +3343,12 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args) {
-            IgfsFileInfo info = e.getValue();
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args) {
+            IgfsEntryInfo info = e.getValue();
 
-            e.setValue(builder(info).path(path).build());
+            IgfsEntryInfo newInfo = info.path(path);
+
+            e.setValue(newInfo);
 
             return null;
         }
@@ -3349,7 +3382,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Tuple containing the file info and its parent id.
      * @throws IgniteCheckedException If failed.
      */
-    IgniteBiTuple<IgfsFileInfo, IgniteUuid> append(
+    IgniteBiTuple<IgfsEntryInfo, IgniteUuid> append(
         final IgfsPath path,
         Map<String, String> dirProps,
         final boolean create,
@@ -3379,7 +3412,7 @@ public class IgfsMetaManager extends IgfsManager {
                     IgniteInternalTx tx = startTx();
 
                     try {
-                        Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
+                        Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                         if (!pathIds.verifyIntegrity(lockInfos))
                             // Directory structure changed concurrently. So we simply re-try.
@@ -3387,7 +3420,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                         if (pathIds.allExists()) {
                             // All participants are found. Simply open the stream.
-                            IgfsFileInfo info = lockInfos.get(pathIds.lastId());
+                            IgfsEntryInfo info = lockInfos.get(pathIds.lastId());
 
                             // Check: is it a file?
                             if (!info.isFile())
@@ -3400,7 +3433,7 @@ public class IgfsMetaManager extends IgfsManager {
                             // At this point we can open the stream safely.
                             info = invokeLock(info.id(), false);
 
-                            IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(info, pathIds.lastParentId());
+                            IgniteBiTuple<IgfsEntryInfo, IgniteUuid> t2 = new T2<>(info, pathIds.lastParentId());
 
                             tx.commit();
 
@@ -3450,7 +3483,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @param simpleCreate Whether new file should be created in secondary FS using create(Path, boolean) method.
      * @return Tuple containing the created file info and its parent id.
      */
-    IgniteBiTuple<IgfsFileInfo, IgniteUuid> create(
+    IgniteBiTuple<IgfsEntryInfo, IgniteUuid> create(
         final IgfsPath path,
         Map<String, String> dirProps,
         final boolean overwrite,
@@ -3488,7 +3521,7 @@ public class IgfsMetaManager extends IgfsManager {
                     IgniteInternalTx tx = startTx();
 
                     try {
-                        Map<IgniteUuid, IgfsFileInfo> lockInfos = lockIds(lockIds);
+                        Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
                         if (!pathIds.verifyIntegrity(lockInfos))
                             // Directory structure changed concurrently. So we simply re-try.
@@ -3496,7 +3529,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                         if (pathIds.allExists()) {
                             // All participants found.
-                            IgfsFileInfo oldInfo = lockInfos.get(pathIds.lastId());
+                            IgfsEntryInfo oldInfo = lockInfos.get(pathIds.lastId());
 
                             // Check: is it a file?
                             if (!oldInfo.isFile())
@@ -3527,14 +3560,14 @@ public class IgfsMetaManager extends IgfsManager {
                             // Third step: create the file.
                             long createTime = System.currentTimeMillis();
 
-                            IgfsFileInfo newInfo = invokeAndGet(overwriteId, new FileCreateProcessor(createTime,
+                            IgfsEntryInfo newInfo = invokeAndGet(overwriteId, new FileCreateProcessor(createTime,
                                 fileProps, blockSize, affKey, createFileLockId(false), evictExclude));
 
                             // Fourth step: update path of remove file.
                             invokeUpdatePath(oldId, path);
 
                             // Prepare result and commit.
-                            IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(newInfo, parentId);
+                            IgniteBiTuple<IgfsEntryInfo, IgniteUuid> t2 = new T2<>(newInfo, parentId);
 
                             tx.commit();
 
@@ -3581,7 +3614,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Result or {@code} if the first parent already contained child with the same name.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable IgfsPathsCreateResult createDirectory(IgfsPathIds pathIds, Map<IgniteUuid, IgfsFileInfo> lockInfos,
+    @Nullable IgfsPathsCreateResult createDirectory(IgfsPathIds pathIds, Map<IgniteUuid, IgfsEntryInfo> lockInfos,
         Map<String, String> dirProps) throws IgniteCheckedException {
         // Check if entry we are going to write to is directory.
         if (lockInfos.get(pathIds.lastExistingId()).isFile())
@@ -3604,7 +3637,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Result or {@code} if the first parent already contained child with the same name.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map<IgniteUuid, IgfsFileInfo> lockInfos,
+    @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map<IgniteUuid, IgfsEntryInfo> lockInfos,
         Map<String, String> dirProps, Map<String, String> fileProps, int blockSize, @Nullable IgniteUuid affKey,
         boolean evictExclude) throws IgniteCheckedException{
         // Check if entry we are going to write to is directory.
@@ -3630,11 +3663,11 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pathIds,
-        Map<IgniteUuid, IgfsFileInfo> lockInfos, Map<String, String> dirProps, Map<String, String> fileProps,
+        Map<IgniteUuid, IgfsEntryInfo> lockInfos, Map<String, String> dirProps, Map<String, String> fileProps,
         int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude) throws IgniteCheckedException {
         // This is our starting point.
         int lastExistingIdx = pathIds.lastExistingIndex();
-        IgfsFileInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId());
+        IgfsEntryInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId());
 
         // If current info already contains entry with the same name as it's child, then something
         // has changed concurrently. We must re-try because we cannot get info of this unexpected
@@ -3684,7 +3717,7 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         // Third step: create leaf.
-        IgfsFileInfo info;
+        IgfsEntryInfo info;
 
         if (dir)
             info = invokeAndGet(curId, new DirectoryCreateProcessor(createTime, dirProps));
@@ -3723,7 +3756,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * File create processor.
      */
-    private static class FileCreateProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
+    private static class FileCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3774,12 +3807,19 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo info = new IgfsFileInfo(blockSize, 0L, affKey, lockId, evictExclude, props,
-                createTime, createTime);
-
-            info.id(entry.getKey());
+            IgfsEntryInfo info = IgfsUtils.createFile(
+                entry.getKey(),
+                blockSize,
+                0L,
+                affKey,
+                lockId,
+                evictExclude,
+                props,
+                createTime,
+                createTime
+            );
 
             entry.setValue(info);
 
@@ -3810,7 +3850,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Directory create processor.
      */
-    private static class DirectoryCreateProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
+    private static class DirectoryCreateProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3861,15 +3901,19 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
 
-            IgfsFileInfo info = new IgfsFileInfo(true, props, createTime, createTime);
+            IgfsEntryInfo info = IgfsUtils.createDirectory(
+                entry.getKey(),
+                null,
+                props,
+                createTime,
+                createTime
+            );
 
             if (childName != null)
-                info = new IgfsFileInfo(Collections.singletonMap(childName, childEntry), info);
-
-            info.id(entry.getKey());
+                info = info.listing(Collections.singletonMap(childName, childEntry));
 
             entry.setValue(info);
 
@@ -3906,7 +3950,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * File lock entry processor.
      */
-    private static class FileLockProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
+    private static class FileLockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3931,11 +3975,11 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo oldInfo = entry.getValue();
+            IgfsEntryInfo oldInfo = entry.getValue();
 
-            IgfsFileInfo newInfo =  new IgfsFileInfo(oldInfo, lockId, oldInfo.modificationTime());
+            IgfsEntryInfo newInfo = oldInfo.lock(lockId);
 
             entry.setValue(newInfo);
 
@@ -3956,7 +4000,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * File unlock entry processor.
      */
-    private static class FileUnlockProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+    private static class FileUnlockProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -3981,11 +4025,11 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo old = entry.getValue();
+            IgfsEntryInfo old = entry.getValue();
 
-            entry.setValue(new IgfsFileInfo(old, null, modificationTime));
+            entry.setValue(old.unlock(modificationTime));
 
             return null;
         }
@@ -4004,7 +4048,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * File reserve space entry processor.
      */
-    private static class FileReserveSpaceProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
+    private static class FileReserveSpaceProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -4034,17 +4078,15 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo oldInfo = entry.getValue();
+            IgfsEntryInfo oldInfo = entry.getValue();
 
             IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap());
 
             newMap.addRange(affRange);
 
-            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, oldInfo.length() + space);
-
-            newInfo.fileMap(newMap);
+            IgfsEntryInfo newInfo = oldInfo.length(oldInfo.length() + space).fileMap(newMap);
 
             entry.setValue(newInfo);
 
@@ -4067,7 +4109,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Update properties processor.
      */
-    private static class UpdatePropertiesProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
+    private static class UpdatePropertiesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, IgfsEntryInfo>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -4092,9 +4134,9 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public IgfsEntryInfo process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
-            IgfsFileInfo oldInfo = entry.getValue();
+            IgfsEntryInfo oldInfo = entry.getValue();
 
             Map<String, String> tmp = oldInfo.properties();
 
@@ -4109,7 +4151,7 @@ public class IgfsMetaManager extends IgfsManager {
                     tmp.put(e.getKey(), e.getValue());
             }
 
-            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, tmp);
+            IgfsEntryInfo newInfo = oldInfo.properties(tmp);
 
             entry.setValue(newInfo);
 
@@ -4130,7 +4172,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Update times entry processor.
      */
-    private static class UpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+    private static class UpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
         Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
@@ -4160,12 +4202,12 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> entry, Object... args)
             throws EntryProcessorException {
 
-            IgfsFileInfo oldInfo = entry.getValue();
+            IgfsEntryInfo oldInfo = entry.getValue();
 
-            entry.setValue(new IgfsFileInfo(oldInfo, accessTime, modificationTime));
+            entry.setValue(oldInfo.accessModificationTime(accessTime, modificationTime));
 
             return null;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 3bf1011..ef2826b 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
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.DataInput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsMode;
@@ -33,6 +29,11 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 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.igfs.IgfsMode.DUAL_SYNC;
 import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
 import static org.apache.ignite.igfs.IgfsMode.PROXY;
@@ -55,7 +56,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
 
     /** File descriptor. */
     @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private IgfsFileInfo fileInfo;
+    private IgfsEntryInfo fileInfo;
 
     /** Parent ID. */
     private final IgniteUuid parentId;
@@ -102,7 +103,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
      * @param batch Optional secondary file system batch.
      * @param metrics Local IGFS metrics.
      */
-    IgfsOutputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsFileInfo fileInfo, IgniteUuid parentId,
+    IgfsOutputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, IgniteUuid parentId,
         int bufSize, IgfsMode mode, @Nullable IgfsFileWorkerBatch batch, IgfsLocalMetrics metrics) {
         super(path, optimizeBufferSize(bufSize, fileInfo));
 
@@ -143,7 +144,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
      * @return Optimized buffer size.
      */
     @SuppressWarnings("IfMayBeConditional")
-    private static int optimizeBufferSize(int bufSize, IgfsFileInfo fileInfo) {
+    private static int optimizeBufferSize(int bufSize, IgfsEntryInfo fileInfo) {
         assert bufSize > 0;
 
         if (fileInfo == null)
@@ -292,7 +293,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
             if (space > 0) {
                 data.awaitAllAcksReceived(fileInfo.id());
 
-                IgfsFileInfo fileInfo0 = meta.reserveSpace(path, fileInfo.id(), space, streamRange);
+                IgfsEntryInfo fileInfo0 = meta.reserveSpace(path, fileInfo.id(), space, streamRange);
 
                 if (fileInfo0 == null)
                     throw new IOException("File was concurrently deleted: " + path);
@@ -407,7 +408,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
      * @param fileInfo File info to build initial range for.
      * @return Affinity range.
      */
-    private IgfsFileAffinityRange initialStreamRange(IgfsFileInfo fileInfo) {
+    private IgfsFileAffinityRange initialStreamRange(IgfsEntryInfo fileInfo) {
         if (!igfsCtx.configuration().isFragmentizerEnabled())
             return null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
index 1f669b0..2903239 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
@@ -267,10 +267,10 @@ public class IgfsPathIds {
      * @param infos Info.
      * @return {@code True} if full integrity is preserved.
      */
-    public boolean verifyIntegrity(Map<IgniteUuid, IgfsFileInfo> infos) {
+    public boolean verifyIntegrity(Map<IgniteUuid, IgfsEntryInfo> infos) {
         for (int i = 0; i <= lastExistingIdx; i++) {
             IgniteUuid curId = ids[i];
-            IgfsFileInfo curInfo = infos.get(curId);
+            IgfsEntryInfo curInfo = infos.get(curId);
 
             // Check if required ID is there.
             if (curInfo == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
index 3b620f8..9462aa4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
@@ -31,7 +31,7 @@ public class IgfsPathsCreateResult {
     private final List<IgfsPath> paths;
 
     /** Info of the last created file. */
-    private final IgfsFileInfo info;
+    private final IgfsEntryInfo info;
 
     /** Parent ID. */
     private final IgniteUuid parentId;
@@ -43,7 +43,7 @@ public class IgfsPathsCreateResult {
      * @param info Info of the last created file.
      * @param parentId Parent ID.
      */
-    public IgfsPathsCreateResult(List<IgfsPath> paths, IgfsFileInfo info, IgniteUuid parentId) {
+    public IgfsPathsCreateResult(List<IgfsPath> paths, IgfsEntryInfo info, IgniteUuid parentId) {
         this.paths = paths;
         this.info = info;
         this.parentId = parentId;
@@ -59,7 +59,7 @@ public class IgfsPathsCreateResult {
     /**
      * @return Info of the last created file.
      */
-    public IgfsFileInfo info() {
+    public IgfsEntryInfo info() {
         return info;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
index 54c0525..76d6be8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
@@ -24,7 +24,7 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadabl
  */
 public class IgfsSecondaryInputStreamDescriptor {
     /** File info in the primary file system. */
-    private final IgfsFileInfo info;
+    private final IgfsEntryInfo info;
 
     /** Secondary file system input stream wrapper. */
     private final IgfsSecondaryFileSystemPositionedReadable secReader;
@@ -35,7 +35,7 @@ public class IgfsSecondaryInputStreamDescriptor {
      * @param info File info in the primary file system.
      * @param secReader Secondary file system reader.
      */
-    IgfsSecondaryInputStreamDescriptor(IgfsFileInfo info, IgfsSecondaryFileSystemPositionedReadable secReader) {
+    IgfsSecondaryInputStreamDescriptor(IgfsEntryInfo info, IgfsSecondaryFileSystemPositionedReadable secReader) {
         assert info != null;
         assert secReader != null;
 
@@ -46,7 +46,7 @@ public class IgfsSecondaryInputStreamDescriptor {
     /**
      * @return File info in the primary file system.
      */
-    IgfsFileInfo info() {
+    IgfsEntryInfo info() {
         return info;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java
index dd6372f..507ccfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java
@@ -17,9 +17,10 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import java.io.OutputStream;
 import org.apache.ignite.lang.IgniteUuid;
 
+import java.io.OutputStream;
+
 /**
  * Descriptor of an output stream opened to the secondary file system.
  */
@@ -28,7 +29,7 @@ public class IgfsSecondaryOutputStreamDescriptor {
     private final IgniteUuid parentId;
 
     /** File info in the primary file system. */
-    private final IgfsFileInfo info;
+    private final IgfsEntryInfo info;
 
     /** Output stream to the secondary file system. */
     private final OutputStream out;
@@ -40,7 +41,7 @@ public class IgfsSecondaryOutputStreamDescriptor {
      * @param info File info in the primary file system.
      * @param out Output stream to the secondary file system.
      */
-    IgfsSecondaryOutputStreamDescriptor(IgniteUuid parentId, IgfsFileInfo info, OutputStream out) {
+    IgfsSecondaryOutputStreamDescriptor(IgniteUuid parentId, IgfsEntryInfo info, OutputStream out) {
         assert parentId != null;
         assert info != null;
         assert out != null;
@@ -60,7 +61,7 @@ public class IgfsSecondaryOutputStreamDescriptor {
     /**
      * @return File info in the primary file system.
      */
-    IgfsFileInfo info() {
+    IgfsEntryInfo info() {
         return info;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 ef7d5c7..325f636 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
@@ -43,6 +43,7 @@ import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 
 import java.lang.reflect.Constructor;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 
@@ -321,4 +322,71 @@ public class IgfsUtils {
             }
         }
     }
+
+    /**
+     * Create empty directory with the given ID.
+     *
+     * @param id ID.
+     * @return File info.
+     */
+    public static IgfsDirectoryInfo createDirectory(IgniteUuid id) {
+        return createDirectory(id, null, null);
+    }
+
+    /**
+     * Create directory.
+     *
+     * @param id ID.
+     * @param listing Listing.
+     * @param props Properties.
+     * @return File info.
+     */
+    public static IgfsDirectoryInfo createDirectory(
+        IgniteUuid id,
+        @Nullable Map<String, IgfsListingEntry> listing,
+        @Nullable Map<String, String> props) {
+        long time = System.currentTimeMillis();
+
+        return createDirectory(id, listing, props, time, time);
+    }
+
+    /**
+     * Create directory.
+     *
+     * @param id ID.
+     * @param listing Listing.
+     * @param props Properties.
+     * @param createTime Create time.
+     * @param modificationTime Modification time.
+     * @return File info.
+     */
+    public static IgfsDirectoryInfo createDirectory(
+        IgniteUuid id,
+        @Nullable Map<String, IgfsListingEntry> listing,
+        @Nullable Map<String,String> props,
+        long createTime,
+        long modificationTime) {
+        return new IgfsDirectoryInfo(id, listing, props, createTime, modificationTime);
+    }
+
+    /**
+     * Create file.
+     *
+     * @param id File ID.
+     * @param blockSize Block size.
+     * @param len Length.
+     * @param affKey Affinity key.
+     * @param lockId Lock ID.
+     * @param evictExclude Evict exclude flag.
+     * @param props Properties.
+     * @param accessTime Access time.
+     * @param modificationTime Modification time.
+     * @return File info.
+     */
+    public static IgfsFileInfo createFile(IgniteUuid id, int blockSize, long len, @Nullable IgniteUuid affKey,
+        @Nullable IgniteUuid lockId, boolean evictExclude, @Nullable Map<String, String> props, long accessTime,
+        long modificationTime) {
+        return new IgfsFileInfo(id, blockSize, len, affKey, props, null, lockId, accessTime, modificationTime,
+            evictExclude);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java
index ea70f49..20051ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java
@@ -17,6 +17,6 @@
 
 /**
  * <!-- Package description. -->
- * Contains high performance file system processer.
+ * Contains high performance file system processor.
  */
 package org.apache.ignite.internal.processors.igfs;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
index 22f427a..d52a918 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
@@ -22,8 +22,8 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
-import org.apache.ignite.internal.processors.igfs.IgfsFileInfo;
 import org.apache.ignite.internal.processors.igfs.IgfsMetaManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -139,7 +139,7 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
         if (fileId == null)
             throw new IgfsPathNotFoundException("File not found: " + path);
 
-        IgfsFileInfo fileInfo = meta.info(fileId);
+        IgfsEntryInfo fileInfo = meta.info(fileId);
 
         do {
             if (fileInfo == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 2acf59c..edec572 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
@@ -3081,7 +3081,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param igfs The IGFS instance.
      * @return The data cache.
      */
-    protected static GridCacheAdapter<IgniteUuid, IgfsFileInfo> getMetaCache(IgniteFileSystem igfs) {
+    protected static GridCacheAdapter<IgniteUuid, IgfsEntryInfo> getMetaCache(IgniteFileSystem igfs) {
         String dataCacheName = igfs.configuration().getMetaCacheName();
 
         IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
@@ -3111,7 +3111,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
                 entry.getValue().await();
             }
             catch (IgniteCheckedException e) {
-                if (!entry.getValue().cancelled())
+                if (!(e instanceof IgfsFileWorkerBatchCancelledException))
                     throw e;
             }
         }