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/16 20:42:37 UTC

[5/6] ignite git commit: Main part of refactoring - applied new scheme.

Main part of refactoring - applied new scheme.


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

Branch: refs/heads/ignite-2813
Commit: 9480b0a308b9f8a607491e643cb0a4853e8f3d75
Parents: 12be73e
Author: thatcoach <pp...@list.ru>
Authored: Wed Mar 16 22:35:43 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Wed Mar 16 22:35:43 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDataManager.java        | 110 +++----
 .../processors/igfs/IgfsDeleteWorker.java       |  31 +-
 .../processors/igfs/IgfsDirectoryInfo.java      | 271 +++-------------
 .../internal/processors/igfs/IgfsEntryInfo.java | 158 +++++++++
 .../internal/processors/igfs/IgfsFileImpl.java  |  15 +-
 .../internal/processors/igfs/IgfsFileInfo.java  | 220 +++----------
 .../igfs/IgfsFragmentizerManager.java           |  26 +-
 .../internal/processors/igfs/IgfsImpl.java      |  28 +-
 .../processors/igfs/IgfsInputStreamAdapter.java |   5 +-
 .../processors/igfs/IgfsInputStreamImpl.java    |  32 +-
 .../processors/igfs/IgfsListingEntry.java       |  12 +-
 .../processors/igfs/IgfsMetaManager.java        | 318 +++++++++----------
 .../processors/igfs/IgfsOutputStreamImpl.java   |  23 +-
 .../IgfsSecondaryInputStreamDescriptor.java     |   6 +-
 .../IgfsSecondaryOutputStreamDescriptor.java    |   9 +-
 .../internal/processors/igfs/IgfsUtils.java     |  28 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |   9 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  59 ++--
 .../igfs/IgfsDataManagerSelfTest.java           |  40 +--
 .../igfs/IgfsDualAbstractSelfTest.java          |  22 +-
 .../processors/igfs/IgfsFileInfoSelfTest.java   |  11 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |  57 ++--
 .../processors/igfs/IgfsProcessorSelfTest.java  |  38 +--
 .../processors/igfs/IgfsSizeSelfTest.java       |  40 +--
 .../processors/igfs/IgfsStreamsSelfTest.java    |  33 +-
 .../testframework/junits/GridAbstractTest.java  |  67 ++--
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   8 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |  26 +-
 28 files changed, 757 insertions(+), 945 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9480b0a3/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 b77b926..9368f35 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,12 +58,41 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-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;
-import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+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.*;
+import static org.apache.ignite.internal.GridTopic.*;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
 
 /**
  * Cache based file's data container.
@@ -368,7 +368,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 +476,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 +495,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 +524,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 +557,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 +579,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 +595,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 +616,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 +646,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 +721,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 +736,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 +845,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 +1327,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 +1373,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 +1632,10 @@ public class IgfsDataManager extends IgfsManager {
      */
     private class AsyncDeleteWorker extends GridWorker {
         /** File info for stop request. */
-        private final IgfsFileInfo stopInfo;
+        private final IgfsEntryInfo stopInfo;
 
         /** Delete requests queue. */
-        private BlockingQueue<IgniteBiTuple<GridFutureAdapter<Object>, IgfsFileInfo>> delReqs =
+        private BlockingQueue<IgniteBiTuple<GridFutureAdapter<Object>, IgfsEntryInfo>> delReqs =
             new LinkedBlockingQueue<>();
 
         /**
@@ -1662,7 +1662,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));
@@ -1674,10 +1674,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) {
@@ -1738,7 +1738,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/9480b0a3/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..5c5e8cc 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,8 +30,16 @@ 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 static org.apache.ignite.events.EventType.EVT_IGFS_FILE_PURGED;
-import static org.apache.ignite.internal.GridTopic.TOPIC_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 static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.GridTopic.*;
 
 /**
  * IGFS worker for removal from the trash directory.
@@ -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/9480b0a3/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 dc5478f..01c8ff9 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,11 +17,9 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import org.apache.ignite.igfs.IgfsPath;
 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;
 
@@ -39,28 +37,10 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** File length in bytes. */
-    private long len;
-
-    /** File block size, {@code zero} for directories. */
-    private int blockSize;
-
-    /** File lock ID. */
-    private IgniteUuid lockId;
-
-    /** Affinity key used for single-node file collocation. */
-    private IgniteUuid affKey;
-
-    /** File affinity map. */
-    private IgfsFileMap fileMap;
-
     /** Directory listing. */
     @GridToStringInclude
     private Map<String, IgfsListingEntry> listing;
 
-    /** Whether data blocks of this entry should never be excluded. */
-    private boolean evictExclude;
-
     /**
      * {@link Externalizable} support.
      */
@@ -74,49 +54,12 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
      * @param len New length.
      * @return Updated file info.
      */
-    public IgfsDirectoryInfo length(long len) {
-        IgfsDirectoryInfo res = copy();
-
-        res.len = len;
-
-        return res;
+    public IgfsEntryInfo length(long len) {
+        throw new UnsupportedOperationException("length");
     }
 
-    /**
-     * Update properties.
-     *
-     * @param props Properties.
-     * @return Updated file info.
-     */
-    public IgfsDirectoryInfo properties(@Nullable Map<String, String> props) {
-        IgfsDirectoryInfo res = copy();
-
-        res.props = props;
-
-        return res;
-    }
-
-    /**
-     * Update path.
-     *
-     * @param path Path.
-     * @return Updated file info.
-     */
-    public IgfsDirectoryInfo path(IgfsPath path) {
-        IgfsDirectoryInfo res = copy();
-
-        res.path = path;
-
-        return res;
-    }
-
-    /**
-     * Update listing.
-     *
-     * @param listing Listing.
-     * @return Updated file info.
-     */
-    public IgfsDirectoryInfo listing(@Nullable Map<String, IgfsListingEntry> listing) {
+    /** {@inheritDoc} */
+    @Override public IgfsDirectoryInfo listing(@Nullable Map<String, IgfsListingEntry> listing) {
         IgfsDirectoryInfo res = copy();
 
         res.listing = listing;
@@ -124,122 +67,40 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
         return res;
     }
 
-    /**
-     * Update access and modification time.
-     *
-     * @param accessTime Access time.
-     * @param modificationTime Modification time.
-     * @return Updated file info.
-     */
-    public IgfsDirectoryInfo accessModificationTime(long accessTime, long modificationTime) {
-        IgfsDirectoryInfo res = copy();
-
-        res.accessTime = accessTime;
-        res.modificationTime = modificationTime;
-
-        return res;
-    }
-
-    /**
-     * Lock file.
-     *
-     * @param lockId Lock ID.
-     * @return Update file info.
-     */
-    public IgfsDirectoryInfo lock(IgniteUuid lockId) {
-        assert lockId != null;
-        assert this.lockId == null;
-
-        IgfsDirectoryInfo res = copy();
-
-        res.lockId = lockId;
-
-        return res;
-    }
-
-    /**
-     * Unlock file.
-     *
-     * @param modificationTime Modification time.
-     * @return Updated file info.
-     */
-    public IgfsDirectoryInfo unlock(long modificationTime) {
-        IgfsDirectoryInfo res = copy();
-
-        res.lockId = null;
-        res.modificationTime = modificationTime;
-
-        return res;
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo lock(IgniteUuid lockId) {
+        throw new UnsupportedOperationException("lock");
     }
 
-    /**
-     * Update file map.
-     *
-     * @param fileMap File affinity map.
-     * @return Updated file info.
-     */
-    public IgfsDirectoryInfo fileMap(IgfsFileMap fileMap) {
-        IgfsDirectoryInfo res = copy();
-
-        res.fileMap = fileMap;
-
-        return res;
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo unlock(long modificationTime) {
+        throw new UnsupportedOperationException("unlock");
     }
 
-    /**
-     * Copy file info.
-     *
-     * @return Copy.
-     */
-    private IgfsDirectoryInfo copy() {
-        return new IgfsDirectoryInfo(isDirectory(), id, blockSize, len, affKey, listing, props,
-            fileMap(), lockId, accessTime, modificationTime, evictExclude);
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo fileMap(IgfsFileMap fileMap) {
+        throw new UnsupportedOperationException("fileMap");
     }
 
     /**
      * 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 accessTime Last access time.
      * @param modificationTime Last modification time.
-     * @param evictExclude Evict exclude flag.
      */
-    IgfsDirectoryInfo(boolean isDir, 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, long accessTime,
-                 long modificationTime, boolean evictExclude) {
+    IgfsDirectoryInfo(IgniteUuid id, @Nullable Map<String, IgfsListingEntry> listing,
+        @Nullable Map<String, String> props, long accessTime, long modificationTime) {
         super(id, props, accessTime, modificationTime);
 
-        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.len = isDir ? 0 : len;
-        this.blockSize = isDir ? 0 : blockSize;
-        this.affKey = affKey;
         this.listing = listing;
+    }
 
-        if (fileMap == null && !isDir)
-            fileMap = new IgfsFileMap();
-
-        this.fileMap = fileMap;
-        this.lockId = lockId;
-        this.evictExclude = evictExclude;
+    /** {@inheritDoc} */
+    protected IgfsDirectoryInfo copy() {
+        return new IgfsDirectoryInfo(id, listing, props, accessTime, modificationTime);
     }
 
     /** {@inheritDoc} */
@@ -247,64 +108,37 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
         return false;
     }
 
-    /**
-     * Get file size.
-     *
-     * @return File size.
-     */
+    /** {@inheritDoc} */
     public long length() {
-        assert isFile();
-
-        return len;
+        return 0;
     }
 
-    /**
-     * 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 0;
     }
 
-    /**
-     * @return Number of data blocks to store this file.
-     */
+    /** {@inheritDoc} */
     public long blocksCount() {
-        assert isFile();
-
-        return (len + blockSize() - 1) / blockSize();
+        return 0;
     }
 
-    /**
-     * @return Directory listing.
-     */
+    /** {@inheritDoc} */
     public Map<String, IgfsListingEntry> listing() {
         return listing != null ? listing : Collections.<String, IgfsListingEntry>emptyMap();
     }
 
-    /**
-     * @return {@code True} if at least one child exists.
-     */
+    /** {@inheritDoc} */
     public boolean hasChildren() {
         return !F.isEmpty(listing);
     }
 
-    /**
-     * @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);
     }
 
-    /**
-     * @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);
@@ -316,50 +150,31 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
         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;
+        return null;
     }
 
-    /**
-     * @return File affinity map.
-     */
+    /** {@inheritDoc} */
     public IgfsFileMap fileMap() {
-        return fileMap;
+        return null;
     }
 
-    /**
-     * 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;
+        return null;
     }
 
-    /**
-     * Get evict exclude flag.
-     *
-     * @return Evict exclude flag.
-     */
+    /** {@inheritDoc} */
     public boolean evictExclude() {
-        return evictExclude;
+        return true;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         super.writeExternal(out);
 
-        out.writeInt(blockSize);
-        out.writeLong(len);
-        U.writeGridUuid(out, lockId);
-        U.writeGridUuid(out, affKey);
         out.writeObject(listing);
-        out.writeObject(fileMap);
-        out.writeBoolean(evictExclude);
     }
 
     /** {@inheritDoc} */
@@ -367,19 +182,12 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         super.readExternal(in);
 
-        blockSize = in.readInt();
-        len = in.readLong();
-        lockId = U.readGridUuid(in);
-        affKey = U.readGridUuid(in);
         listing = (Map<String, IgfsListingEntry>)in.readObject();
-        fileMap = (IgfsFileMap)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());
+        return id.hashCode() ^ (props == null ? 0 : props.hashCode());
     }
 
     /** {@inheritDoc} */
@@ -392,8 +200,7 @@ public class IgfsDirectoryInfo extends IgfsEntryInfo {
 
         IgfsDirectoryInfo that = (IgfsDirectoryInfo)obj;
 
-        return id.equals(that.id) && blockSize == that.blockSize && len == that.len && F.eq(affKey, that.affKey) &&
-            F.eq(props, that.props) && F.eq(lockId, that.lockId);
+        return id.equals(that.id) && F.eq(props, that.props);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9480b0a3/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 8e26cf3..c5b1111 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
@@ -121,12 +121,170 @@ public abstract class IgfsEntryInfo implements Externalizable {
     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);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9480b0a3/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/9480b0a3/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 0589c1a..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,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import org.apache.ignite.igfs.IgfsPath;
-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;
@@ -54,10 +52,6 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
     /** File affinity map. */
     private IgfsFileMap fileMap;
 
-    /** Directory listing. */
-    @GridToStringInclude
-    private Map<String, IgfsListingEntry> listing;
-
     /** Whether data blocks of this entry should never be excluded. */
     private boolean evictExclude;
 
@@ -68,13 +62,8 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
         // No-op.
     }
 
-    /**
-     * Update length.
-     *
-     * @param len New length.
-     * @return Updated file info.
-     */
-    public IgfsFileInfo length(long len) {
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo length(long len) {
         IgfsFileInfo res = copy();
 
         res.len = len;
@@ -82,71 +71,13 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
         return res;
     }
 
-    /**
-     * Update properties.
-     *
-     * @param props Properties.
-     * @return Updated file info.
-     */
-    public IgfsFileInfo properties(@Nullable Map<String, String> props) {
-        IgfsFileInfo res = copy();
-
-        res.props = props;
-
-        return res;
-    }
-
-    /**
-     * Update path.
-     *
-     * @param path Path.
-     * @return Updated file info.
-     */
-    public IgfsFileInfo path(IgfsPath path) {
-        IgfsFileInfo res = copy();
-
-        res.path = path;
-
-        return res;
-    }
-
-    /**
-     * Update listing.
-     *
-     * @param listing Listing.
-     * @return Updated file info.
-     */
-    public IgfsFileInfo listing(@Nullable Map<String, IgfsListingEntry> listing) {
-        IgfsFileInfo res = copy();
-
-        res.listing = listing;
-
-        return res;
-    }
-
-    /**
-     * Update access and modification time.
-     *
-     * @param accessTime Access time.
-     * @param modificationTime Modification time.
-     * @return Updated file info.
-     */
-    public IgfsFileInfo accessModificationTime(long accessTime, long modificationTime) {
-        IgfsFileInfo res = copy();
-
-        res.accessTime = accessTime;
-        res.modificationTime = modificationTime;
-
-        return res;
+    /** {@inheritDoc} */
+    @Override public IgfsEntryInfo listing(@Nullable Map<String, IgfsListingEntry> listing) {
+        throw new UnsupportedOperationException("listing");
     }
 
-    /**
-     * Lock file.
-     *
-     * @param lockId Lock ID.
-     * @return Update file info.
-     */
-    public IgfsFileInfo lock(IgniteUuid lockId) {
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo lock(IgniteUuid lockId) {
         assert lockId != null;
         assert this.lockId == null;
 
@@ -157,13 +88,8 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
         return res;
     }
 
-    /**
-     * Unlock file.
-     *
-     * @param modificationTime Modification time.
-     * @return Updated file info.
-     */
-    public IgfsFileInfo unlock(long modificationTime) {
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo unlock(long modificationTime) {
         IgfsFileInfo res = copy();
 
         res.lockId = null;
@@ -172,13 +98,8 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
         return res;
     }
 
-    /**
-     * Update file map.
-     *
-     * @param fileMap File affinity map.
-     * @return Updated file info.
-     */
-    public IgfsFileInfo fileMap(IgfsFileMap fileMap) {
+    /** {@inheritDoc} */
+    @Override public IgfsFileInfo fileMap(IgfsFileMap fileMap) {
         IgfsFileInfo res = copy();
 
         res.fileMap = fileMap;
@@ -186,25 +107,19 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
         return res;
     }
 
-    /**
-     * Copy file info.
-     *
-     * @return Copy.
-     */
-    private IgfsFileInfo copy() {
-        return new IgfsFileInfo(isDirectory(), id, blockSize, len, affKey, listing, props,
-            fileMap(), lockId, 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.
@@ -212,29 +127,16 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
      * @param modificationTime Last modification time.
      * @param evictExclude Evict exclude flag.
      */
-    IgfsFileInfo(boolean isDir, 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, long accessTime,
-        long modificationTime, boolean evictExclude) {
+    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);
 
-        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.len = isDir ? 0 : len;
-        this.blockSize = isDir ? 0 : blockSize;
+        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;
@@ -244,108 +146,60 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
 
     /** {@inheritDoc} */
     public boolean isFile() {
-        // TODO: Cleanup.
-        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 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;
     }
 
-    /**
-     * @return File affinity map.
-     */
+    /** {@inheritDoc} */
     public IgfsFileMap fileMap() {
         return fileMap;
     }
 
-    /**
-     * 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;
     }
@@ -358,7 +212,6 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
         out.writeLong(len);
         U.writeGridUuid(out, lockId);
         U.writeGridUuid(out, affKey);
-        out.writeObject(listing);
         out.writeObject(fileMap);
         out.writeBoolean(evictExclude);
     }
@@ -372,7 +225,6 @@ public final class IgfsFileInfo extends IgfsEntryInfo {
         len = in.readLong();
         lockId = U.readGridUuid(in);
         affKey = U.readGridUuid(in);
-        listing = (Map<String, IgfsListingEntry>)in.readObject();
         fileMap = (IgfsFileMap)in.readObject();
         evictExclude = in.readBoolean();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9480b0a3/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 39202d4..0d1ce35 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
@@ -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,15 +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 = oldInfo.fileMap(newMap);
+            IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
 
             entry.setValue(newInfo);
 
@@ -411,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;
@@ -436,15 +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 = oldInfo.fileMap(newMap);
+            IgfsEntryInfo newInfo = oldInfo.fileMap(newMap);
 
             entry.setValue(newInfo);
 
@@ -503,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)
@@ -711,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();
@@ -785,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/9480b0a3/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..7fea118 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
@@ -621,7 +621,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;
@@ -636,7 +636,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))
@@ -690,7 +690,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;
 
@@ -857,7 +857,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) {
@@ -868,7 +868,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());
@@ -932,7 +932,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);
@@ -1033,7 +1033,7 @@ public final class IgfsImpl implements IgfsEx {
                 else
                     dirProps = fileProps = new HashMap<>(props);
 
-                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(path, false/*append*/, overwrite, dirProps,
+                IgniteBiTuple<IgfsEntryInfo, IgniteUuid> t2 = meta.create(path, false/*append*/, overwrite, dirProps,
                     cfg.getBlockSize(), affKey, evictExclude(path, true), fileProps);
 
                 assert t2 != null;
@@ -1104,7 +1104,7 @@ public final class IgfsImpl implements IgfsEx {
                 else
                     dirProps = fileProps = new HashMap<>(props);
 
-                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(path, true/*append*/, false/*overwrite*/,
+                IgniteBiTuple<IgfsEntryInfo, IgniteUuid> t2 = meta.create(path, true/*append*/, false/*overwrite*/,
                     dirProps, cfg.getBlockSize(), null/*affKey*/, evictExclude(path, true), fileProps);
 
                 assert t2 != null;
@@ -1179,7 +1179,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;
@@ -1276,7 +1276,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()) {
@@ -1386,7 +1386,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.
@@ -1556,7 +1556,7 @@ public final class IgfsImpl implements IgfsEx {
         assert path != null;
         assert mode != null;
 
-        IgfsFileInfo info = null;
+        IgfsEntryInfo info = null;
 
         switch (mode) {
             case PRIMARY:
@@ -1683,7 +1683,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);
 
@@ -1722,7 +1722,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/9480b0a3/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/9480b0a3/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/9480b0a3/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();
     }