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

[40/50] [abbrv] ignite git commit: IGNITE-2817: IGFS: Optimized "updateProperties" and several other cache operations. Reafactored IgfsMetaManager a bit to simplify work with cache.

IGNITE-2817: IGFS: Optimized "updateProperties" and several other cache operations. Reafactored IgfsMetaManager a bit to simplify work with cache.


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

Branch: refs/heads/ignite-2407
Commit: dda1f0ed449111b7e1011a669b30bc3b4c5d2d92
Parents: 7b7233e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Mar 16 09:14:49 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Mar 16 09:14:49 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsImpl.java      |   4 +-
 .../processors/igfs/IgfsMetaManager.java        | 641 ++++++++++---------
 .../igfs/IgfsMetaManagerSelfTest.java           |  23 +-
 3 files changed, 336 insertions(+), 332 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dda1f0ed/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 f44eda8..3065427 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
@@ -636,9 +636,7 @@ public final class IgfsImpl implements IgfsEx {
                 if (fileId == null)
                     return null;
 
-                IgniteUuid parentId = fileIds.size() > 1 ? fileIds.get(fileIds.size() - 2) : null;
-
-                IgfsFileInfo info = meta.updateProperties(parentId, fileId, path.name(), props);
+                IgfsFileInfo info = meta.updateProperties(fileId, props);
 
                 if (info != null) {
                     if (evts.isRecordable(EVT_IGFS_META_UPDATED))

http://git-wip-us.apache.org/repos/asf/ignite/blob/dda1f0ed/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 463e7a8..df69d49 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
@@ -40,7 +40,6 @@ import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.EntryProcessorResult;
 import javax.cache.processor.MutableEntry;
 
 import org.apache.ignite.IgniteCheckedException;
@@ -238,7 +237,7 @@ public class IgfsMetaManager extends IgfsManager {
     @Nullable public IgniteUuid fileId(IgfsPath path) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 return fileId(path, false);
             }
@@ -322,7 +321,7 @@ public class IgfsMetaManager extends IgfsManager {
     public List<IgniteUuid> fileIds(IgfsPath path) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 return fileIds(path, false);
             }
@@ -407,7 +406,7 @@ public class IgfsMetaManager extends IgfsManager {
                 if (fileId == null)
                     return null;
 
-                IgfsFileInfo info = id2InfoPrj.get(fileId);
+                IgfsFileInfo info = getInfo(fileId);
 
                 // Force root ID always exist in cache.
                 if (info == null && IgfsUtils.ROOT_ID.equals(fileId))
@@ -433,13 +432,14 @@ public class IgfsMetaManager extends IgfsManager {
     public Map<IgniteUuid, IgfsFileInfo> infos(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
+
                 assert fileIds != null;
 
                 if (F.isEmpty(fileIds))
                     return Collections.emptyMap();
 
-                Map<IgniteUuid, IgfsFileInfo> map = id2InfoPrj.getAll(fileIds);
+                Map<IgniteUuid, IgfsFileInfo> map = getInfos(fileIds);
 
                 // Force root ID always exist in cache.
                 if (fileIds.contains(IgfsUtils.ROOT_ID) && !map.containsKey(IgfsUtils.ROOT_ID)) {
@@ -469,7 +469,8 @@ public class IgfsMetaManager extends IgfsManager {
     public @Nullable IgfsFileInfo lock(IgniteUuid fileId, boolean delete) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
+
                 assert fileId != null;
 
                 IgniteInternalTx tx = startTx();
@@ -484,11 +485,7 @@ public class IgfsMetaManager extends IgfsManager {
                     if (oldInfo.lockId() != null)
                         return null; // The file is already locked, we cannot lock it.
 
-                    IgniteUuid lockId = createFileLockId(delete);
-
-                    id2InfoPrj.invoke(fileId, new FileLockProcessor(lockId));
-
-                    IgfsFileInfo newInfo = id2InfoPrj.get(fileId);
+                    IgfsFileInfo newInfo = invokeLock(fileId, delete);
 
                     tx.commit();
 
@@ -530,7 +527,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     public void unlock(final IgfsFileInfo info, final long modificationTime) throws IgniteCheckedException {
-        assert validTxState(false);
+        validTxState(false);
+
         assert info != null;
 
         if (busyLock.enterBusy()) {
@@ -546,7 +544,7 @@ public class IgfsMetaManager extends IgfsManager {
                 try {
                     IgfsUtils.doInTransactionWithRetries(id2InfoPrj, new IgniteOutClosureX<Void>() {
                         @Override public Void applyx() throws IgniteCheckedException {
-                            assert validTxState(true);
+                            validTxState(true);
 
                             IgniteUuid fileId = info.id();
 
@@ -569,7 +567,7 @@ public class IgfsMetaManager extends IgfsManager {
                     });
                 }
                 finally {
-                    assert validTxState(false);
+                    validTxState(false);
 
                     if (interrupted)
                         Thread.currentThread().interrupt();
@@ -591,7 +589,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     private Map<IgniteUuid, IgfsFileInfo> lockIds(IgniteUuid... fileIds) throws IgniteCheckedException {
-        assert validTxState(true);
+        validTxState(true);
+
         assert fileIds != null && fileIds.length > 0;
 
         Arrays.sort(fileIds);
@@ -631,13 +630,13 @@ public class IgfsMetaManager extends IgfsManager {
      */
     private Map<IgniteUuid, IgfsFileInfo> lockIds(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
         assert isSorted(fileIds);
-        assert validTxState(true);
+        validTxState(true);
 
         if (log.isDebugEnabled())
             log.debug("Locking file ids: " + fileIds);
 
         // Lock files and get their infos.
-        Map<IgniteUuid, IgfsFileInfo> map = id2InfoPrj.getAll(fileIds);
+        Map<IgniteUuid, IgfsFileInfo> map = getInfos(fileIds);
 
         if (log.isDebugEnabled())
             log.debug("Locked file ids: " + fileIds);
@@ -769,7 +768,7 @@ public class IgfsMetaManager extends IgfsManager {
         assert fileId != null;
 
         IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) :
-            id2InfoPrj.get(fileId);
+            getInfo(fileId);
 
         return info == null ? Collections.<String, IgfsListingEntry>emptyMap() : info.listing();
     }
@@ -789,7 +788,7 @@ public class IgfsMetaManager extends IgfsManager {
             log.debug("Locking parent id [parentId=" + parentId + ", fileName=" + fileName + ", newFileInfo=" +
                 newFileInfo + ']');
 
-        assert validTxState(true);
+        validTxState(true);
 
         // Lock only parent file ID.
         IgfsFileInfo parentInfo = info(parentId);
@@ -806,10 +805,7 @@ public class IgfsMetaManager extends IgfsManager {
         if (childEntry != null)
             return childEntry.fileId();
 
-        if (!id2InfoPrj.putIfAbsent(newFileInfo.id(), newFileInfo))
-            throw fsException("Failed to add file details into cache: " + newFileInfo);
-
-        id2InfoPrj.invoke(parentId, new ListingAdd(fileName, new IgfsListingEntry(newFileInfo)));
+        createNewEntry(newFileInfo, parentId, fileName);
 
         return null;
     }
@@ -825,7 +821,7 @@ public class IgfsMetaManager extends IgfsManager {
     public IgfsFileInfo move(IgfsPath srcPath, IgfsPath dstPath) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 // 1. First get source and destination path IDs.
                 List<IgniteUuid> srcPathIds = fileIds(srcPath);
@@ -918,8 +914,7 @@ public class IgfsMetaManager extends IgfsManager {
                     // 8. Actual move: remove from source parent and add to destination target.
                     IgfsListingEntry entry = srcTargetInfo.listing().get(srcName);
 
-                    id2InfoPrj.invoke(srcTargetId, new ListingRemove(srcName, entry.fileId()));
-                    id2InfoPrj.invoke(dstTargetId, new ListingAdd(dstName, entry));
+                    transferEntry(entry, srcTargetId, srcName, dstTargetId, dstName);
 
                     tx.commit();
 
@@ -990,7 +985,7 @@ public class IgfsMetaManager extends IgfsManager {
      */
     private void moveNonTx(IgniteUuid fileId, @Nullable String srcFileName, IgniteUuid srcParentId, String destFileName,
         IgniteUuid destParentId) throws IgniteCheckedException {
-        assert validTxState(true);
+        validTxState(true);
 
         assert fileId != null;
         assert srcFileName != null;
@@ -1047,11 +1042,7 @@ public class IgfsMetaManager extends IgfsManager {
                 " directory (file already exists) [fileId=" + fileId + ", destFileName=" + destFileName +
                 ", destParentId=" + destParentId + ']'));
 
-        // Remove listing entry from the source parent listing.
-        id2InfoPrj.invoke(srcParentId, new ListingRemove(srcFileName, srcEntry.fileId()));
-
-        // Add listing entry into the destination parent listing.
-        id2InfoPrj.invoke(destParentId, new ListingAdd(destFileName, srcEntry));
+        transferEntry(srcEntry, srcParentId, srcFileName, destParentId, destFileName);
     }
 
     /**
@@ -1064,7 +1055,7 @@ public class IgfsMetaManager extends IgfsManager {
     IgniteUuid format() throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 IgniteUuid trashId = IgfsUtils.randomTrashId();
 
@@ -1088,11 +1079,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     IgfsFileInfo newInfo = new IgfsFileInfo(transferListing);
 
-                    id2InfoPrj.put(newInfo.id(), newInfo);
-
-                    // Add new info to trash listing.
-                    id2InfoPrj.invoke(trashId, new ListingAdd(newInfo.id().toString(),
-                        new IgfsListingEntry(newInfo)));
+                    createNewEntry(newInfo, trashId, newInfo.id().toString());
 
                     // Remove listing entries from root.
                     // Note that root directory properties and other attributes are preserved:
@@ -1128,7 +1115,7 @@ public class IgfsMetaManager extends IgfsManager {
     IgniteUuid softDelete(final IgfsPath path, final boolean recursive) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 final SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
@@ -1191,15 +1178,12 @@ public class IgfsMetaManager extends IgfsManager {
 
                     assert victimId.equals(srcEntry.fileId());
 
-                    id2InfoPrj.invoke(srcParentId, new ListingRemove(srcFileName, srcEntry.fileId()));
-
-                    // Add listing entry into the destination parent listing.
-                    id2InfoPrj.invoke(trashId, new ListingAdd(destFileName, srcEntry));
+                    transferEntry(srcEntry, srcParentId, srcFileName, trashId, destFileName);
 
                     if (victimInfo.isFile())
                         // Update a file info of the removed file with a file path,
                         // which will be used by delete worker for event notifications.
-                        id2InfoPrj.invoke(victimId, new UpdatePath(path));
+                        invokeUpdatePath(victimId, path);
 
                     tx.commit();
 
@@ -1233,7 +1217,7 @@ public class IgfsMetaManager extends IgfsManager {
     @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id,
         IgniteUuid trashId)
         throws IgniteCheckedException {
-        assert validTxState(true);
+        validTxState(true);
 
         IgniteUuid resId;
 
@@ -1241,7 +1225,7 @@ public class IgfsMetaManager extends IgfsManager {
             // Handle special case when we deleting root directory.
             assert IgfsUtils.ROOT_ID.equals(id);
 
-            IgfsFileInfo rootInfo = id2InfoPrj.get(IgfsUtils.ROOT_ID);
+            IgfsFileInfo rootInfo = getInfo(IgfsUtils.ROOT_ID);
 
             if (rootInfo == null)
                 return null; // Root was never created.
@@ -1269,11 +1253,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 IgfsFileInfo newInfo = new IgfsFileInfo(transferListing);
 
-                id2InfoPrj.getAndPut(newInfo.id(), newInfo);
-
-                // Add new info to trash listing.
-                id2InfoPrj.invoke(trashId, new ListingAdd(newInfo.id().toString(),
-                    new IgfsListingEntry(newInfo)));
+                createNewEntry(newInfo, trashId, newInfo.id().toString());
 
                 // Remove listing entries from root.
                 for (Map.Entry<String, IgfsListingEntry> entry : transferListing.entrySet())
@@ -1311,7 +1291,7 @@ public class IgfsMetaManager extends IgfsManager {
             try {
                 assert parentId != null;
                 assert listing != null;
-                assert validTxState(false);
+                validTxState(false);
 
                 IgniteInternalTx tx = startTx();
 
@@ -1353,7 +1333,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                                 // Delete only files or empty folders.
                                 if (!entryInfo.hasChildren()) {
-                                    id2InfoPrj.getAndRemove(childId);
+                                    id2InfoPrj.remove(childId);
 
                                     newListing.remove(childName);
 
@@ -1402,7 +1382,7 @@ public class IgfsMetaManager extends IgfsManager {
     boolean delete(IgniteUuid parentId, String name, IgniteUuid id) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 IgniteInternalTx tx = startTx();
 
@@ -1428,11 +1408,9 @@ public class IgfsMetaManager extends IgfsManager {
                         IgfsListingEntry childEntry = parentInfo.listing().get(name);
 
                         if (childEntry != null)
-                            id2InfoPrj.invoke(parentId, new ListingRemove(name, childEntry.fileId()));
+                            id2InfoPrj.invoke(parentId, new ListingRemove(name, id));
 
-                        IgfsFileInfo deleted = id2InfoPrj.getAndRemove(id);
-
-                        assert victim.id().equals(deleted.id());
+                        id2InfoPrj.remove(id);
 
                         res = true;
                     }
@@ -1468,7 +1446,7 @@ public class IgfsMetaManager extends IgfsManager {
                 for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++) {
                     IgniteUuid trashId = IgfsUtils.trashId(i);
 
-                    IgfsFileInfo trashInfo = id2InfoPrj.get(trashId);
+                    IgfsFileInfo trashInfo = getInfo(trashId);
 
                     if (trashInfo != null && trashInfo.hasChildren()) {
                         for (IgfsListingEntry entry : trashInfo.listing().values())
@@ -1489,75 +1467,28 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Update file info (file properties) in cache in existing transaction.
      *
-     * @param parentId Parent ID ({@code null} if file is root).
-     * @param fileName To update listing.
      * @param fileId File ID to update information for.
      * @param props Properties to set for the file.
      * @return Updated file info or {@code null} if such file ID not found.
      * @throws IgniteCheckedException If operation failed.
      */
-    @Nullable private IgfsFileInfo updatePropertiesNonTx(final @Nullable IgniteUuid parentId, final IgniteUuid fileId,
-        String fileName, Map<String, String> props) throws IgniteCheckedException {
+    @Nullable private IgfsFileInfo updatePropertiesNonTx(final IgniteUuid fileId, Map<String, String> props)
+        throws IgniteCheckedException {
         assert fileId != null;
         assert !F.isEmpty(props) : "Expects not-empty file's properties";
-        assert validTxState(true);
+
+        validTxState(true);
 
         if (log.isDebugEnabled())
             log.debug("Update file properties [fileId=" + fileId + ", props=" + props + ']');
 
         try {
-            final IgfsFileInfo oldInfo;
-            final IgfsFileInfo parentInfo;
-
-            // Lock file ID for this transaction.
-            if (parentId == null) {
-                oldInfo = info(fileId);
-                parentInfo = null;
-            }
-            else {
-                Map<IgniteUuid, IgfsFileInfo> locked = lockIds(parentId, fileId);
-
-                oldInfo = locked.get(fileId);
-                parentInfo = locked.get(parentId);
-
-                if (parentInfo == null)
-                    return null; // Parent not found.
-            }
+            final IgfsFileInfo oldInfo = info(fileId);
 
             if (oldInfo == null)
-                return null; // File not found.
+                return null;
 
-            if (parentInfo != null) {
-                if (!parentInfo.hasChild(fileName, fileId)) // File was removed or recreated.
-                    return null;
-            }
-
-            Map<String, String> tmp = oldInfo.properties();
-
-            tmp = tmp == null ? new GridLeanMap<String, String>(props.size()) : new GridLeanMap<>(tmp);
-
-            for (Map.Entry<String, String> e : props.entrySet()) {
-                if (e.getValue() == null)
-                    // Remove properties with 'null' values.
-                    tmp.remove(e.getKey());
-                else
-                    // Add/overwrite property.
-                    tmp.put(e.getKey(), e.getValue());
-            }
-
-            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, tmp);
-
-            id2InfoPrj.put(fileId, newInfo);
-
-            if (parentId != null) {
-                IgfsListingEntry entry = new IgfsListingEntry(newInfo);
-
-                assert id2InfoPrj.get(parentId) != null;
-
-                id2InfoPrj.invoke(parentId, new ListingAdd(fileName, entry));
-            }
-
-            return newInfo;
+            return invokeAndGet(fileId, new UpdatePropertiesProcessor(props));
         }
         catch (GridClosureException e) {
             throw U.cast(e);
@@ -1567,23 +1498,21 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Update file info (file properties) in cache.
      *
-     * @param parentId Parent ID ({@code null} if file is root).
-     * @param fileName To update listing.
      * @param fileId File ID to update information for.
      * @param props Properties to set for the file.
      * @return Updated file info or {@code null} if such file ID not found.
      * @throws IgniteCheckedException If operation failed.
      */
-    @Nullable public IgfsFileInfo updateProperties(@Nullable IgniteUuid parentId, IgniteUuid fileId, String fileName,
-        Map<String, String> props) throws IgniteCheckedException {
+    @Nullable public IgfsFileInfo updateProperties(IgniteUuid fileId, Map<String, String> props)
+        throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 IgniteInternalTx tx = startTx();
 
                 try {
-                    IgfsFileInfo info = updatePropertiesNonTx(parentId, fileId, fileName, props);
+                    IgfsFileInfo info = updatePropertiesNonTx(fileId, props);
 
                     tx.commit();
 
@@ -1598,8 +1527,8 @@ public class IgfsMetaManager extends IgfsManager {
             }
         }
         else
-            throw new IllegalStateException("Failed to update properties because Grid is stopping [parentId=" +
-                parentId + ", fileId=" + fileId + ", fileName=" + fileName + ", props=" + props + ']');
+            throw new IllegalStateException("Failed to update properties because Grid is stopping [fileId=" + fileId +
+                ", props=" + props + ']');
     }
 
     /**
@@ -1613,7 +1542,7 @@ public class IgfsMetaManager extends IgfsManager {
      */
     public IgfsFileInfo reserveSpace(IgfsPath path, IgniteUuid fileId, long space, IgfsFileAffinityRange affRange)
         throws IgniteCheckedException {
-        assert validTxState(false);
+        validTxState(false);
 
         if (busyLock.enterBusy()) {
             try {
@@ -1629,9 +1558,7 @@ public class IgfsMetaManager extends IgfsManager {
                     if (oldInfo == null)
                         throw fsException("File has been deleted concurrently [path=" + path + ", id=" + fileId + ']');
 
-                    id2InfoPrj.invoke(fileId, new FileReserveSpaceProcessor(space, affRange));
-
-                    IgfsFileInfo newInfo = id2InfoPrj.get(fileId);
+                    IgfsFileInfo newInfo = invokeAndGet(fileId, new FileReserveSpaceProcessor(space, affRange));
 
                     tx.commit();
 
@@ -1663,7 +1590,7 @@ public class IgfsMetaManager extends IgfsManager {
      */
     @Nullable public IgfsFileInfo updateInfo(IgniteUuid fileId, IgniteClosure<IgfsFileInfo, IgfsFileInfo> c)
         throws IgniteCheckedException {
-        assert validTxState(false);
+        validTxState(false);
         assert fileId != null;
         assert c != null;
 
@@ -1672,7 +1599,7 @@ public class IgfsMetaManager extends IgfsManager {
                 if (log.isDebugEnabled())
                     log.debug("Update file info [fileId=" + fileId + ", c=" + c + ']');
 
-                IgniteInternalTx tx = id2InfoPrj.isLockedByThread(fileId) ? null : startTx();
+                IgniteInternalTx tx = startTx();
 
                 try {
                     // Lock file ID for this transaction.
@@ -1732,14 +1659,14 @@ public class IgfsMetaManager extends IgfsManager {
      */
     boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
         assert props != null;
-        assert validTxState(false);
+        validTxState(false);
 
         DirectoryChainBuilder b = null;
 
         while (true) {
             if (busyLock.enterBusy()) {
                 try {
-                    b = new DirectoryChainBuilder(path, props, props);
+                    b = new DirectoryChainBuilder(path, props);
 
                     // Start TX.
                     IgniteInternalTx tx = startTx();
@@ -1873,6 +1800,106 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Put new entry to meta cache immediately linking it to parent.
+     *
+     * @param info Info to put.
+     * @param parentId Parent ID.
+     * @param name Name in parent.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void createNewEntry(IgfsFileInfo info, IgniteUuid parentId, String name) throws IgniteCheckedException {
+        validTxState(true);
+
+        if (!id2InfoPrj.putIfAbsent(info.id(), info))
+            throw fsException("Failed to create new metadata entry due to ID conflict: " + info.id());
+
+        if (parentId != null)
+            id2InfoPrj.invoke(parentId, new ListingAdd(name, new IgfsListingEntry(info)));
+    }
+
+    /**
+     * Transfer entry from one directory to another.
+     *
+     * @param entry Entry to be transfered.
+     * @param srcId Source ID.
+     * @param srcName Source name.
+     * @param destId Destination ID.
+     * @param destName Destination name.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void transferEntry(IgfsListingEntry entry, IgniteUuid srcId, String srcName,
+        IgniteUuid destId, String destName) throws IgniteCheckedException {
+        validTxState(true);
+
+        id2InfoPrj.invoke(srcId, new ListingRemove(srcName, entry.fileId()));
+        id2InfoPrj.invoke(destId, new ListingAdd(destName, entry));
+    }
+
+    /**
+     * Invoke lock processor.
+     *
+     * @param id File ID.
+     * @param delete Whether lock is taken for delete.
+     * @return Resulting file info.
+     * @throws IgniteCheckedException If failed.
+     */
+    private IgfsFileInfo invokeLock(IgniteUuid id, boolean delete) throws IgniteCheckedException {
+        return invokeAndGet(id, new FileLockProcessor(createFileLockId(delete)));
+    }
+
+    /**
+     * Invoke path update processor.
+     *
+     * @param id File ID.
+     * @param path Path to be updated.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void invokeUpdatePath(IgniteUuid id, IgfsPath path) throws IgniteCheckedException {
+        validTxState(true);
+
+        id2InfoPrj.invoke(id, new UpdatePath(path));
+    }
+
+    /**
+     * Invoke some processor and return new value.
+     *
+     * @param id ID.
+     * @param proc Processor.
+     * @return New file info.
+     * @throws IgniteCheckedException If failed.
+     */
+    private IgfsFileInfo invokeAndGet(IgniteUuid id, EntryProcessor<IgniteUuid, IgfsFileInfo, Void> proc)
+        throws IgniteCheckedException {
+        validTxState(true);
+
+        id2InfoPrj.invoke(id, proc);
+
+        return getInfo(id);
+    }
+
+    /**
+     * Get info.
+     *
+     * @param id ID.
+     * @return Info.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable private IgfsFileInfo getInfo(IgniteUuid id) throws IgniteCheckedException {
+        return id2InfoPrj.get(id);
+    }
+
+    /**
+     * Get several infos.
+     *
+     * @param ids IDs.
+     * @return Infos map.
+     * @throws IgniteCheckedException If failed.
+     */
+    private Map<IgniteUuid, IgfsFileInfo> getInfos(Collection<IgniteUuid> ids) throws IgniteCheckedException {
+        return id2InfoPrj.getAll(ids);
+    }
+
+    /**
      * Create the file in DUAL mode.
      *
      * @param fs File system.
@@ -1913,7 +1940,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                         @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
                             IgfsFileInfo> infos) throws Exception {
-                            assert validTxState(true);
+                            validTxState(true);
 
                             assert !infos.isEmpty();
 
@@ -1983,12 +2010,10 @@ public class IgfsMetaManager extends IgfsManager {
                                         path + ", fileId=" + oldId + ", lockId=" + oldInfo.lockId() + ']');
 
                                 id2InfoPrj.remove(oldId); // Remove the old one.
-                                id2InfoPrj.put(newInfo.id(), newInfo); // Put the new one.
-
                                 id2InfoPrj.invoke(parentInfo.id(),
                                     new ListingRemove(path.name(), parentInfo.listing().get(path.name()).fileId()));
-                                id2InfoPrj.invoke(parentInfo.id(),
-                                    new ListingAdd(path.name(), new IgfsListingEntry(newInfo)));
+
+                                createNewEntry(newInfo, parentInfo.id(), path.name()); // Put new one.
 
                                 IgniteInternalFuture<?> delFut = igfsCtx.data().delete(oldInfo);
                             }
@@ -2052,7 +2077,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                         @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
                             IgfsFileInfo> infos) throws Exception {
-                            assert validTxState(true);
+                            validTxState(true);
 
                             final IgfsFileInfo info = infos.get(path);
 
@@ -2087,11 +2112,7 @@ public class IgfsMetaManager extends IgfsManager {
                             }
 
                             // Set lock and return.
-                            IgniteUuid lockId = createFileLockId(false);
-
-                            id2InfoPrj.invoke(info.id(), new FileLockProcessor(lockId));
-
-                            IgfsFileInfo lockedInfo = id2InfoPrj.get(info.id());
+                            IgfsFileInfo lockedInfo = invokeLock(info.id(), false);
 
                             return new IgfsSecondaryOutputStreamDescriptor(infos.get(path.parent()).id(),
                                 lockedInfo, out);
@@ -2453,7 +2474,7 @@ public class IgfsMetaManager extends IgfsManager {
                         }
 
                         // Update the deleted file info with path information for delete worker.
-                        id2InfoPrj.invoke(info.id(), new UpdatePath(path));
+                        invokeUpdatePath(info.id(), path);
 
                         return true; // No additional handling is required.
                     }
@@ -2505,12 +2526,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                         fs.update(path, props);
 
-                        IgfsFileInfo parentInfo = infos.get(path.parent());
-
-                        assert path.parent() == null || parentInfo != null;
-
-                        return updatePropertiesNonTx(parentInfo == null ? null : parentInfo.id(),
-                            infos.get(path).id(), path.name(), props);
+                        return updatePropertiesNonTx(infos.get(path).id(), props);
                     }
 
                     @Override public IgfsFileInfo onFailure(@Nullable Exception err) throws IgniteCheckedException {
@@ -2865,7 +2881,7 @@ public class IgfsMetaManager extends IgfsManager {
      */
     private <K, V> boolean putx(IgniteInternalCache<K, V> cache, K key, IgniteClosure<V, V> c)
         throws IgniteCheckedException {
-        assert validTxState(true);
+        validTxState(true);
 
         V oldVal = cache.get(key);
         V newVal = c.apply(oldVal);
@@ -2877,16 +2893,10 @@ public class IgfsMetaManager extends IgfsManager {
      * Check transaction is (not) started.
      *
      * @param inTx Expected transaction state.
-     * @return Transaction state is correct.
      */
-    private boolean validTxState(boolean inTx) {
-        boolean txState = inTx == (id2InfoPrj.tx() != null);
-
-        assert txState : (inTx ? "Method cannot be called outside transaction " :
-            "Method cannot be called in transaction ") + "[tx=" + id2InfoPrj.tx() + ", threadId=" +
-            Thread.currentThread().getId() + ']';
-
-        return txState;
+    private void validTxState(boolean inTx) {
+        assert (inTx && id2InfoPrj.tx() != null) || (!inTx && id2InfoPrj.tx() == null) :
+            "Invalid TX state [expected=" + inTx + ", actual=" + (id2InfoPrj.tx() != null) + ']';
     }
 
     /**
@@ -2912,7 +2922,7 @@ public class IgfsMetaManager extends IgfsManager {
         long modificationTime) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
-                assert validTxState(false);
+                validTxState(false);
 
                 // Start pessimistic transaction.
                 IgniteInternalTx tx = startTx();
@@ -3310,7 +3320,8 @@ public class IgfsMetaManager extends IgfsManager {
         final @Nullable IgniteUuid affKey,
         final boolean evictExclude,
         @Nullable Map<String, String> fileProps) throws IgniteCheckedException {
-        assert validTxState(false);
+        validTxState(false);
+
         assert path != null;
 
         final String name = path.name();
@@ -3322,15 +3333,7 @@ public class IgfsMetaManager extends IgfsManager {
         while (true) {
             if (busyLock.enterBusy()) {
                 try {
-                    b = new DirectoryChainBuilder(path, dirProps, fileProps) {
-                        /** {@inheritDoc} */
-                        @Override protected IgfsFileInfo buildLeaf() {
-                            long t = System.currentTimeMillis();
-
-                            return new IgfsFileInfo(blockSize, 0L, affKey, createFileLockId(false),
-                                 evictExclude, leafProps, t, t);
-                        }
-                    };
+                    b = new DirectoryChainBuilder(path, dirProps, fileProps, blockSize, affKey, evictExclude);
 
                     // Start Tx:
                     IgniteInternalTx tx = startTx();
@@ -3377,18 +3380,7 @@ public class IgfsMetaManager extends IgfsManager {
                                                 + "[fileName=" + name + ", fileId=" + lowermostExistingInfo.id()
                                                 + ", lockId=" + lockId + ']');
 
-                                        IgniteUuid newLockId = createFileLockId(false);
-
-                                        EntryProcessorResult<IgfsFileInfo> result
-                                            = id2InfoPrj.invoke(lowermostExistingInfo.id(),
-                                                new LockFileProcessor(newLockId));
-
-                                        IgfsFileInfo lockedInfo = result.get();
-
-                                        assert lockedInfo != null; // we already checked lock above.
-                                        assert lockedInfo.lockId() != null;
-                                        assert lockedInfo.lockId().equals(newLockId);
-                                        assert lockedInfo.id().equals(lowermostExistingInfo.id());
+                                        IgfsFileInfo lockedInfo = invokeLock(lowermostExistingInfo.id(), false);
 
                                         IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(lockedInfo, parentId);
 
@@ -3411,15 +3403,12 @@ public class IgfsMetaManager extends IgfsManager {
 
                                         assert deletedEntry != null;
 
-                                        id2InfoPrj.invoke(parentId, new ListingRemove(name, deletedEntry.fileId()));
-
-                                        // Add listing entry into the destination parent listing.
-                                        id2InfoPrj.invoke(trashId,
-                                            new ListingAdd(lowermostExistingInfo.id().toString(), deletedEntry));
+                                        transferEntry(deletedEntry, parentId, name, trashId,
+                                            lowermostExistingInfo.id().toString());
 
                                         // Update a file info of the removed file with a file path,
                                         // which will be used by delete worker for event notifications.
-                                        id2InfoPrj.invoke(lowermostExistingInfo.id(), new UpdatePath(path));
+                                        invokeUpdatePath(lowermostExistingInfo.id(), path);
 
                                         // Make a new locked info:
                                         long t = System.currentTimeMillis();
@@ -3429,12 +3418,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                                         assert newFileInfo.lockId() != null; // locked info should be created.
 
-                                        boolean put = id2InfoPrj.putIfAbsent(newFileInfo.id(), newFileInfo);
-
-                                        assert put;
-
-                                        id2InfoPrj.invoke(parentId,
-                                                new ListingAdd(name, new IgfsListingEntry(newFileInfo)));
+                                        createNewEntry(newFileInfo, parentId, name);
 
                                         IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(newFileInfo, parentId);
 
@@ -3497,54 +3481,103 @@ public class IgfsMetaManager extends IgfsManager {
     /** File chain builder. */
     private class DirectoryChainBuilder {
         /** The requested path to be created. */
-        protected final IgfsPath path;
+        private final IgfsPath path;
 
         /** Full path components. */
-        protected final List<String> components;
+        private final List<String> components;
 
         /** The list of ids. */
-        protected final List<IgniteUuid> idList;
+        private final List<IgniteUuid> idList;
 
         /** The set of ids. */
-        protected final SortedSet<IgniteUuid> idSet;
+        private final SortedSet<IgniteUuid> idSet = new TreeSet<IgniteUuid>(PATH_ID_SORTING_COMPARATOR);
 
         /** The middle node properties. */
-        protected final Map<String, String> middleProps;
+        private final Map<String, String> middleProps;
 
         /** The leaf node properties. */
-        protected final Map<String, String> leafProps;
+        private final Map<String, String> leafProps;
 
         /** The lowermost exsiting path id. */
-        protected final IgniteUuid lowermostExistingId;
+        private final IgniteUuid lowermostExistingId;
 
         /** The existing path. */
-        protected final IgfsPath existingPath;
+        private final IgfsPath existingPath;
 
         /** The created leaf info. */
-        protected IgfsFileInfo leafInfo;
+        private IgfsFileInfo leafInfo;
 
         /** The leaf parent id. */
-        protected IgniteUuid leafParentId;
+        private IgniteUuid leafParentId;
 
         /** The number of existing ids. */
-        protected final int existingIdCnt;
+        private final int existingIdCnt;
+
+        /** Whether laef is directory. */
+        private final boolean leafDir;
+
+        /** Block size. */
+        private final int blockSize;
+
+        /** Affinity key. */
+        private final IgniteUuid affKey;
+
+        /** Evict exclude flag. */
+        private final boolean evictExclude;
 
         /**
-         * Creates the builder and performa all the initial calculations.
+         * Constructor for directories.
+         *
+         * @param path Path.
+         * @param props Properties.
+         * @throws IgniteCheckedException If failed.
          */
-        protected DirectoryChainBuilder(IgfsPath path,
-                 Map<String,String> middleProps, Map<String,String> leafProps) throws IgniteCheckedException {
-            this.path = path;
+        protected DirectoryChainBuilder(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+            this(path, props, props, true, 0, null, false);
+        }
 
-            this.components = path.components();
+        /**
+         * Constructor for files.
+         *
+         * @param path Path.
+         * @param dirProps Directory properties.
+         * @param fileProps File properties.
+         * @param blockSize Block size.
+         * @param affKey Affinity key (optional).
+         * @param evictExclude Evict exclude flag.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected DirectoryChainBuilder(IgfsPath path, Map<String, String> dirProps, Map<String, String> fileProps,
+            int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude)
+            throws IgniteCheckedException {
+            this(path, dirProps, fileProps, false, blockSize, affKey, evictExclude);
+        }
 
+        /**
+         * Constructor.
+         *
+         * @param path Path.
+         * @param middleProps Middle properties.
+         * @param leafProps Leaf properties.
+         * @param leafDir Whether leaf is directory or file.
+         * @param blockSize Block size.
+         * @param affKey Affinity key (optional).
+         * @param evictExclude Evict exclude flag.
+         * @throws IgniteCheckedException If failed.
+         */
+        private DirectoryChainBuilder(IgfsPath path, Map<String,String> middleProps, Map<String,String> leafProps,
+            boolean leafDir, int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude)
+            throws IgniteCheckedException {
+            this.path = path;
+            this.components = path.components();
             this.idList = fileIds(path);
-
-            this.idSet = new TreeSet<IgniteUuid>(PATH_ID_SORTING_COMPARATOR);
-
             this.middleProps = middleProps;
-
             this.leafProps = leafProps;
+            this.leafDir = leafDir;
+            this.blockSize = blockSize;
+            this.affKey = affKey;
+            this.evictExclude = evictExclude;
+
             // Store all the non-null ids in the set & construct existing path in one loop:
             IgfsPath existingPath = path.root();
 
@@ -3555,7 +3588,7 @@ public class IgfsMetaManager extends IgfsManager {
 
             int idIdx = 0;
 
-            for (IgniteUuid id: idList) {
+            for (IgniteUuid id : idList) {
                 if (id == null)
                     break;
 
@@ -3581,33 +3614,6 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         /**
-         * Builds middle nodes.
-         */
-        protected IgfsFileInfo buildMiddleNode(String childName, IgfsFileInfo childInfo) {
-            return new IgfsFileInfo(Collections.singletonMap(childName,
-                    new IgfsListingEntry(childInfo)), middleProps);
-        }
-
-        /**
-         * Builds leaf.
-         */
-        protected IgfsFileInfo buildLeaf()  {
-            long t = System.currentTimeMillis();
-
-            return new IgfsFileInfo(true, leafProps, t, t);
-        }
-
-        /**
-         * Links newly created chain to existing parent.
-         */
-        final void linkBuiltChainToExistingParent(String childName, IgfsFileInfo childInfo)
-                throws IgniteCheckedException {
-            assert childInfo != null;
-
-            id2InfoPrj.invoke(lowermostExistingId, new ListingAdd(childName, new IgfsListingEntry(childInfo)));
-        }
-
-        /**
          * Does the main portion of job building the renmaining path.
          */
         public final void doBuild() throws IgniteCheckedException {
@@ -3615,7 +3621,8 @@ public class IgfsMetaManager extends IgfsManager {
 
             String childName = null;
 
-            IgfsFileInfo newLeafInfo;
+            IgfsFileInfo newInfo;
+
             IgniteUuid parentId = null;
 
             // This loop creates the missing directory chain from the bottom to the top:
@@ -3625,28 +3632,29 @@ public class IgfsMetaManager extends IgfsManager {
                 if (childName == null) {
                     assert childInfo == null;
 
-                    newLeafInfo = buildLeaf();
+                    long t = System.currentTimeMillis();
 
-                    assert newLeafInfo != null;
+                    if (leafDir)
+                        newInfo = new IgfsFileInfo(true, leafProps, t, t);
+                    else
+                        newInfo = new IgfsFileInfo(blockSize, 0L, affKey, createFileLockId(false), evictExclude,
+                            leafProps, t, t);
 
-                    leafInfo = newLeafInfo;
+                    leafInfo = newInfo;
                 }
                 else {
                     assert childInfo != null;
 
-                    newLeafInfo = buildMiddleNode(childName, childInfo);
-
-                    assert newLeafInfo != null;
+                    newInfo = new IgfsFileInfo(Collections.singletonMap(childName,
+                        new IgfsListingEntry(childInfo)), middleProps);
 
                     if (parentId == null)
-                        parentId = newLeafInfo.id();
+                        parentId = newInfo.id();
                 }
 
-                boolean put = id2InfoPrj.putIfAbsent(newLeafInfo.id(), newLeafInfo);
+                id2InfoPrj.put(newInfo.id(), newInfo);
 
-                assert put; // Because we used a new id that should be unique.
-
-                childInfo = newLeafInfo;
+                childInfo = newInfo;
 
                 childName = components.get(i);
             }
@@ -3657,7 +3665,7 @@ public class IgfsMetaManager extends IgfsManager {
             leafParentId = parentId;
 
             // Now link the newly created directory chain to the lowermost existing parent:
-            linkBuiltChainToExistingParent(childName, childInfo);
+            id2InfoPrj.invoke(lowermostExistingId, new ListingAdd(childName, new IgfsListingEntry(childInfo)));
         }
 
         /**
@@ -3674,7 +3682,7 @@ public class IgfsMetaManager extends IgfsManager {
                 }
             }
 
-            if (leafInfo.isDirectory())
+            if (leafDir)
                 IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_DIR_CREATED);
             else {
                 IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_CREATED);
@@ -3684,66 +3692,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Processor closure to locks a file for writing.
-     */
-    private static class LockFileProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, IgfsFileInfo>,
-            Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** New lock id to lock the entry. */
-        private IgniteUuid newLockId;
-
-        /**
-         * Constructor.
-         */
-        public LockFileProcessor(IgniteUuid newLockId) {
-            assert newLockId != null;
-
-            this.newLockId = newLockId;
-        }
-
-        /**
-         * Empty constructor required for {@link Externalizable}.
-         */
-        public LockFileProcessor() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override @Nullable public IgfsFileInfo process(MutableEntry<IgniteUuid, IgfsFileInfo> entry,
-                 Object... arguments) throws EntryProcessorException {
-            final IgfsFileInfo info = entry.getValue();
-
-            assert info != null;
-
-            if (info.lockId() != null)
-                return null; // file is already locked.
-
-            IgfsFileInfo newInfo = new IgfsFileInfo(info, newLockId, info.modificationTime());
-
-            entry.setValue(newInfo);
-
-            return newInfo;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeGridUuid(out, newLockId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            newLockId = U.readGridUuid(in);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(LockFileProcessor.class, this);
-        }
-    }
-
-    /**
      * File lock entry processor.
      */
     private static class FileLockProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>, Externalizable {
@@ -3902,6 +3850,69 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Update properties processor.
+     */
+    private static class UpdatePropertiesProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
+        Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Properties to be updated. */
+        private Map<String, String> props;
+
+        /**
+         * Constructor.
+         */
+        public UpdatePropertiesProcessor() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param props Properties.
+         */
+        public UpdatePropertiesProcessor(Map<String, String> props) {
+            this.props = props;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> entry, Object... args)
+            throws EntryProcessorException {
+            IgfsFileInfo oldInfo = entry.getValue();
+
+            Map<String, String> tmp = oldInfo.properties();
+
+            tmp = tmp == null ? new GridLeanMap<String, String>(props.size()) : new GridLeanMap<>(tmp);
+
+            for (Map.Entry<String, String> e : props.entrySet()) {
+                if (e.getValue() == null)
+                    // Remove properties with 'null' values.
+                    tmp.remove(e.getKey());
+                else
+                    // Add/overwrite property.
+                    tmp.put(e.getKey(), e.getValue());
+            }
+
+            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, tmp);
+
+            entry.setValue(newInfo);
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeStringMap(out, props);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            props = U.readStringMap(in);
+        }
+    }
+
+    /**
      * Update times entry processor.
      */
     private static class UpdateTimesProcessor implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,

http://git-wip-us.apache.org/repos/asf/ignite/blob/dda1f0ed/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index f3d35f4..72a2bee 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -171,33 +171,30 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
             assertNull("Unexpected stored properties: " + info, info.properties().get(key1));
             assertNull("Unexpected stored properties: " + info, info.properties().get(key2));
 
-            info = mgr.updateProperties(ROOT_ID, fileId, fileName, F.asMap(key1, "1"));
+            info = mgr.updateProperties(fileId, F.asMap(key1, "1"));
 
             assertEquals("Unexpected stored properties: " + info, "1", info.properties().get(key1));
 
-            info = mgr.updateProperties(ROOT_ID, fileId, fileName, F.asMap(key2, "2"));
+            info = mgr.updateProperties(fileId, F.asMap(key2, "2"));
 
            // assertEquals("Unexpected stored properties: " + info, F.asMap(key1, "1", key2, "2"), info.properties());
             assertEquals("Unexpected stored properties: " + info, "1", info.properties().get(key1));
             assertEquals("Unexpected stored properties: " + info, "2", info.properties().get(key2));
 
-            info = mgr.updateProperties(ROOT_ID, fileId, fileName, F.<String, String>asMap(key1, null));
+            info = mgr.updateProperties(fileId, F.<String, String>asMap(key1, null));
 
             assertEquals("Unexpected stored properties: " + info, "2", info.properties().get(key2));
 
-            info = mgr.updateProperties(ROOT_ID, fileId, fileName, F.<String, String>asMap(key2, null));
+            info = mgr.updateProperties(fileId, F.<String, String>asMap(key2, null));
 
             assertNull("Unexpected stored properties: " + info, info.properties().get(key1));
             assertNull("Unexpected stored properties: " + info, info.properties().get(key2));
-
-            assertNull(mgr.updateProperties(ROOT_ID, fileId, "not_exists", F.<String, String>asMap(key2, null)));
         }
 
         mgr.softDelete(new IgfsPath("/dir"), true);
         mgr.softDelete(new IgfsPath("/file"), false);
 
-        assertNull(mgr.updateProperties(ROOT_ID, dir.id(), "dir", F.asMap("p", "7")));
-        assertNull(mgr.updateProperties(ROOT_ID, file.id(), "file", F.asMap("q", "8")));
+        assertNull(mgr.updateProperties(dir.id(), F.asMap("p", "7")));
     }
 
     private IgfsFileInfo mkdirsAndGetInfo(String path) throws IgniteCheckedException {
@@ -309,8 +306,8 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         expectsRenameFail("/a/k", "/a/b/", "Failed to perform move because destination already " +
             "contains entry with the same name existing file");
 
-        mgr.delete(a.id(), "k", k.id());
-        mgr.delete(b.id(), "k", z.id());
+        mgr.delete(a.id(), "k", z.id());
+        mgr.delete(b.id(), "k", k.id());
 
         System.out.println("/: " + mgr.directoryListing(ROOT_ID));
         System.out.println("a: " + mgr.directoryListing(a.id()));
@@ -339,9 +336,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
                 mgr.directoryListing(ROOT_ID));
 
-        assertEquals(
-            F.asMap("b", new IgfsListingEntry(b),
-                "f2", new IgfsListingEntry(f2)),
+        assertEquals(F.asMap("b", new IgfsListingEntry(b), "f2", new IgfsListingEntry(f2)),
             mgr.directoryListing(a.id()));
 
         assertEmpty(mgr.directoryListing(b.id()));
@@ -423,7 +418,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         Class<? extends Throwable> cls, @Nullable String msg) {
         assertThrows(log, new Callable() {
             @Override public Object call() throws Exception {
-                return mgr.updateProperties(null, fileId, "file", props);
+                return mgr.updateProperties(fileId, props);
             }
         }, cls, msg);
     }