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

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

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

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

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

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

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

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

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

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

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

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