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

ignite git commit: IGNITE-2861: Done.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2861 [created] d10d60f47


IGNITE-2861: Done.


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

Branch: refs/heads/ignite-2861
Commit: d10d60f4744ebe66ff3a8eb1aa8e29b3950d570d
Parents: 4794f87
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Mar 18 17:57:12 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 18 17:57:12 2016 +0300

----------------------------------------------------------------------
 .../igfs/IgfsFragmentizerManager.java           | 141 +--
 .../processors/igfs/IgfsMetaManager.java        | 890 +------------------
 .../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 ++
 15 files changed, 1093 insertions(+), 995 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d10d60f4/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/d10d60f4/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..2ae0fdb 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,7 +62,6 @@ 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;
@@ -62,12 +71,6 @@ 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 java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -598,7 +601,7 @@ public class IgfsMetaManager extends IgfsManager {
                                     "[fileId=" + fileId + ", lockId=" + info.lockId() + ", actualLockId=" +
                                     oldInfo.lockId() + ']');
 
-                            id2InfoPrj.invoke(fileId, new FileUnlockProcessor(modificationTime));
+                            id2InfoPrj.invoke(fileId, new IgfsMetaFileUnlockProcessor(modificationTime));
 
                             return null;
                         }
@@ -1268,7 +1271,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();
             }
@@ -1419,7 +1422,7 @@ 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);
 
@@ -1499,7 +1502,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);
@@ -1569,7 +1572,7 @@ public class IgfsMetaManager extends IgfsManager {
                     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();
 
@@ -1795,7 +1798,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 +1815,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 +1828,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 +1841,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));
     }
 
     /**
@@ -1998,8 +2001,8 @@ 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.
 
@@ -2951,7 +2954,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     assert parentInfo.isDirectory();
 
-                    id2InfoPrj.invoke(fileId, new UpdateTimesProcessor(
+                    id2InfoPrj.invoke(fileId, new IgfsMetaUpdateTimesProcessor(
                         accessTime == -1 ? fileInfo.accessTime() : accessTime,
                         modificationTime == -1 ? fileInfo.modificationTime() : modificationTime)
                     );
@@ -3011,365 +3014,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.
@@ -3548,19 +3192,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(oldId, true)));
 
                             // 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.
@@ -3682,7 +3326,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 +3343,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 +3364,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 +3396,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/d10d60f4/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/d10d60f4/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/d10d60f4/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/d10d60f4/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/d10d60f4/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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10d60f4/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/d10d60f4/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/d10d60f4/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/d10d60f4/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/d10d60f4/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/d10d60f4/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/d10d60f4/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/d10d60f4/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();
+    }
+}