You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/06/09 10:14:43 UTC

ignite git commit: IGNITE-3287: Implemented.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3287 [created] 66fb92230


IGNITE-3287: Implemented.


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

Branch: refs/heads/ignite-3287
Commit: 66fb922300e9dff9a1fde192c93c142f3d51aa62
Parents: ec5706f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Jun 9 13:14:27 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Jun 9 13:14:27 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDeleteResult.java       |  62 +++++
 .../internal/processors/igfs/IgfsImpl.java      |  30 +--
 .../processors/igfs/IgfsMetaManager.java        | 265 ++++++-------------
 .../internal/processors/igfs/IgfsPathIds.java   |  10 +
 .../processors/igfs/IgfsAbstractSelfTest.java   |  14 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |  20 +-
 6 files changed, 177 insertions(+), 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/66fb9223/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java
new file mode 100644
index 0000000..ff42762
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Result of deletion in IGFS.
+ */
+public class IgfsDeleteResult {
+    /** Success flag. */
+    private final boolean success;
+
+    /** Entry info. */
+    private final IgfsEntryInfo info;
+
+    /**
+     * Constructor.
+     *
+     * @param success Success flag.
+     * @param info Entry info.
+     */
+    public IgfsDeleteResult(boolean success, @Nullable IgfsEntryInfo info) {
+        this.success = success;
+        this.info = info;
+    }
+
+    /**
+     * @return Success flag.
+     */
+    public boolean success() {
+        return success;
+    }
+
+    /**
+     * @return Entry info.
+     */
+    public IgfsEntryInfo info() {
+        return info;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsDeleteResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/66fb9223/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 bc26ace..0808619 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
@@ -104,7 +104,6 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_READ;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_WRITE;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
@@ -723,38 +722,21 @@ public final class IgfsImpl implements IgfsEx {
                 if (IgfsPath.SLASH.equals(path.toString()))
                     return false;
 
-                IgfsMode mode = resolveMode(path);
-
                 Set<IgfsMode> childrenModes = modeRslvr.resolveChildrenModes(path);
 
-                boolean res = false;
-
-                FileDescriptor desc = getFileDescriptor(path);
-
-                if (childrenModes.contains(PRIMARY)) {
-                    if (desc != null) {
-                        IgniteUuid deletedId = meta.softDelete(path, recursive);
-
-                        res = deletedId != null;
-                    }
-                    else if (mode == PRIMARY)
-                        checkConflictWithPrimary(path);
-                }
-
-                if (childrenModes.contains(DUAL_SYNC) || childrenModes.contains(DUAL_ASYNC)) {
-                    assert secondaryFs != null;
+                boolean dual = childrenModes.contains(DUAL_SYNC) ||childrenModes.contains(DUAL_ASYNC);
 
+                if (dual)
                     await(path);
 
-                    res |= meta.deleteDual(secondaryFs, path, recursive);
-                }
+                IgfsDeleteResult res = meta.softDelete(path, recursive, dual ? secondaryFs : null);
 
                 // Record event if needed.
-                if (res && desc != null)
+                if (res.success() && res.info() != null)
                     IgfsUtils.sendEvents(igfsCtx.kernalContext(), path,
-                            desc.isFile ? EVT_IGFS_FILE_DELETED : EVT_IGFS_DIR_DELETED);
+                        res.info().isFile() ? EVT_IGFS_FILE_DELETED : EVT_IGFS_DIR_DELETED);
 
-                return res;
+                return res.success();
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/66fb9223/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 91bb334..2fb6066 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
@@ -1150,153 +1150,129 @@ public class IgfsMetaManager extends IgfsManager {
      *
      * @param path Path.
      * @param recursive Recursive flag.
+     * @param secondaryFs Secondary file system (optional).
      * @return ID of an entry located directly under the trash directory.
      * @throws IgniteCheckedException If failed.
      */
-    IgniteUuid softDelete(final IgfsPath path, final boolean recursive) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                validTxState(false);
-
-                IgfsPathIds pathIds = pathIds(path);
-
-                // Continue only if the whole path present.
-                if (!pathIds.allExists())
-                    return null; // A fragment of the path no longer exists.
-
-                IgniteUuid victimId = pathIds.lastId();
-                String victimName = pathIds.lastPart();
-
-                if (IgfsUtils.isRootId(victimId))
-                    throw new IgfsException("Cannot remove root directory");
-
-                // Prepare IDs to lock.
-                SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
-
-                pathIds.addExistingIds(allIds, relaxed);
-
-                IgniteUuid trashId = IgfsUtils.randomTrashId();
+    IgfsDeleteResult softDelete(final IgfsPath path, final boolean recursive, @Nullable IgfsSecondaryFileSystem secondaryFs)
+        throws IgniteCheckedException {
+        while (true) {
+            if (busyLock.enterBusy()) {
+                try {
+                    validTxState(false);
 
-                allIds.add(trashId);
+                    IgfsPathIds pathIds = pathIds(path);
 
-                try (IgniteInternalTx tx = startTx()) {
-                    // Lock participants.
-                    Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
+                    boolean relaxed0 = relaxed;
 
-                    // Ensure that all participants are still in place.
-                    if (!pathIds.verifyIntegrity(lockInfos, relaxed))
-                        return null;
+                    if (!pathIds.allExists()) {
+                        if (secondaryFs == null)
+                            // Return early if target path doesn't exist and we do not have secondary file system.
+                            return new IgfsDeleteResult(false, null);
+                        else
+                            // If path is missing partially, we need more serious locking for DUAL mode.
+                            relaxed0 = false;
+                    }
 
-                    IgfsEntryInfo victimInfo = lockInfos.get(victimId);
+                    IgniteUuid victimId = pathIds.lastId();
+                    String victimName = pathIds.lastPart();
 
-                    // Cannot delete non-empty directory if recursive flag is not set.
-                    if (!recursive && victimInfo.hasChildren())
-                        throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
-                            "empty and recursive flag is not set).");
+                    if (IgfsUtils.isRootId(victimId))
+                        throw new IgfsException("Cannot remove root directory");
 
-                    // Prepare trash data.
-                    IgfsEntryInfo trashInfo = lockInfos.get(trashId);
-                    final String trashName = victimId.toString();
+                    // Prepare IDs to lock.
+                    SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                    assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " +
-                        "destination directory (file already exists) [destName=" + trashName + ']';
+                    pathIds.addExistingIds(allIds, relaxed0);
 
-                    IgniteUuid parentId = pathIds.lastParentId();
-                    IgfsEntryInfo parentInfo = lockInfos.get(parentId);
+                    IgniteUuid trashId = IgfsUtils.randomTrashId();
 
-                    transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName);
+                    allIds.add(trashId);
 
-                    tx.commit();
+                    try (IgniteInternalTx tx = startTx()) {
+                        // Lock participants.
+                        Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
 
-                    signalDeleteWorker();
+                        if (!pathIds.allExists()) {
+                            // We need to ensure that the last locked info is not linked with expected child.
+                            // Otherwise there was some ocncurrent file system update and we have to re-try.
+                            assert secondaryFs != null;
 
-                    return victimId;
-                }
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to perform soft delete because Grid is " +
-                "stopping [path=" + path + ']');
-    }
+                            // Find the last locked index
+                            IgfsEntryInfo lastLockedInfo = null;
+                            int lastLockedIdx = -1;
 
-    /**
-     * Move path to the trash directory in existing transaction.
-     *
-     * @param parentId Parent ID.
-     * @param name Path name.
-     * @param id Path ID.
-     * @param trashId Trash ID.
-     * @return ID of an entry located directly under the trash directory.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("RedundantCast")
-    @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id,
-        IgniteUuid trashId) throws IgniteCheckedException {
-        validTxState(true);
+                            while (lastLockedIdx < pathIds.lastExistingIndex()) {
+                                IgfsEntryInfo nextInfo = lockInfos.get(pathIds.id(lastLockedIdx + 1));
 
-        IgniteUuid resId;
+                                if (nextInfo != null) {
+                                    lastLockedInfo = nextInfo;
+                                    lastLockedIdx++;
+                                }
+                                else
+                                    break;
+                            }
 
-        if (parentId == null) {
-            // Handle special case when we deleting root directory.
-            assert IgfsUtils.ROOT_ID.equals(id);
+                            assert lastLockedIdx < pathIds.count();
 
-            IgfsEntryInfo rootInfo = getInfo(IgfsUtils.ROOT_ID);
+                            if (lastLockedInfo != null) {
+                                String part = pathIds.part(lastLockedIdx + 1);
 
-            if (rootInfo == null)
-                return null; // Root was never created.
+                                if (lastLockedInfo.listing().containsKey(part))
+                                    continue;
+                            }
+                        }
 
-            // Ensure trash directory existence.
-            createSystemDirectoryIfAbsent(trashId);
+                        // Ensure that all participants are still in place.
+                        if (!pathIds.allExists() || !pathIds.verifyIntegrity(lockInfos, relaxed0)) {
+                            // For DUAL mode we will try to update the underlying FS still. Note we do that inside TX.
+                            if (secondaryFs != null) {
+                                boolean res = secondaryFs.delete(path, recursive);
 
-            Map<String, IgfsListingEntry> rootListing = rootInfo.listing();
+                                return new IgfsDeleteResult(res, null);
+                            }
+                            else
+                                return new IgfsDeleteResult(false, null);
+                        }
 
-            if (!rootListing.isEmpty()) {
-                IgniteUuid[] lockIds = new IgniteUuid[rootInfo.listing().size()];
+                        IgfsEntryInfo victimInfo = lockInfos.get(victimId);
 
-                int i = 0;
+                        // Cannot delete non-empty directory if recursive flag is not set.
+                        if (!recursive && victimInfo.hasChildren())
+                            throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
+                                "empty and recursive flag is not set).");
 
-                for (IgfsListingEntry entry : rootInfo.listing().values())
-                    lockIds[i++] = entry.fileId();
+                        // Prepare trash data.
+                        IgfsEntryInfo trashInfo = lockInfos.get(trashId);
+                        final String trashName = victimId.toString();
 
-                // Lock children IDs in correct order.
-                lockIds(lockIds);
+                        assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " +
+                            "destination directory (file already exists) [destName=" + trashName + ']';
 
-                // Construct new info and move locked entries from root to it.
-                Map<String, IgfsListingEntry> transferListing = new HashMap<>();
+                        IgniteUuid parentId = pathIds.lastParentId();
+                        IgfsEntryInfo parentInfo = lockInfos.get(parentId);
 
-                transferListing.putAll(rootListing);
+                        // Propagate call to the secondary file system.
+                        if (secondaryFs != null && !secondaryFs.delete(path, recursive))
+                            return new IgfsDeleteResult(false, null);
 
-                IgfsEntryInfo newInfo = IgfsUtils.createDirectory(
-                    IgniteUuid.randomUuid(),
-                    transferListing,
-                    (Map<String,String>)null
-                );
+                        transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName);
 
-                createNewEntry(newInfo, trashId, newInfo.id().toString());
+                        tx.commit();
 
-                // Remove listing entries from root.
-                for (Map.Entry<String, IgfsListingEntry> entry : transferListing.entrySet())
-                    id2InfoPrj.invoke(IgfsUtils.ROOT_ID,
-                        new IgfsMetaDirectoryListingRemoveProcessor(entry.getKey(), entry.getValue().fileId()));
+                        signalDeleteWorker();
 
-                resId = newInfo.id();
+                        return new IgfsDeleteResult(true, victimInfo);
+                    }
+                }
+                finally {
+                    busyLock.leaveBusy();
+                }
             }
             else
-                resId = null;
+                throw new IllegalStateException("Failed to perform soft delete because Grid is " +
+                    "stopping [path=" + path + ']');
         }
-        else {
-            // Ensure trash directory existence.
-            createSystemDirectoryIfAbsent(trashId);
-
-            moveNonTx(id, name, parentId, id.toString(), trashId);
-
-            resId = id;
-        }
-
-        return resId;
     }
 
     /**
@@ -2273,71 +2249,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Delete path in DUAL mode.
-     *
-     * @param fs Secondary file system.
-     * @param path Path to update.
-     * @param recursive Recursive flag.
-     * @return Operation result.
-     * @throws IgniteCheckedException If delete failed.
-     */
-    public boolean deleteDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, final boolean recursive)
-        throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                assert fs != null;
-                assert path != null;
-
-                final IgniteUuid trashId = IgfsUtils.randomTrashId();
-
-                SynchronizationTask<Boolean> task = new SynchronizationTask<Boolean>() {
-                    @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.
-
-                        if (!fs.delete(path, recursive))
-                            return false; // Delete failed remotely.
-
-                        if (path.parent() != null) {
-                            assert infos.containsKey(path.parent());
-
-                            softDeleteNonTx(infos.get(path.parent()).id(), path.name(), info.id(), trashId);
-                        }
-                        else {
-                            assert IgfsUtils.ROOT_ID.equals(info.id());
-
-                            softDeleteNonTx(null, path.name(), info.id(), trashId);
-                        }
-
-                        return true; // No additional handling is required.
-                    }
-
-                    @Override public Boolean onFailure(@Nullable Exception err) throws IgniteCheckedException {
-                        U.error(log, "Path delete in DUAL mode failed [path=" + path + ", recursive=" + recursive + ']',
-                            err);
-
-                        throw new IgniteCheckedException("Failed to delete the path due to secondary file system " +
-                            "exception: ", err);
-                    }
-                };
-
-                Boolean res = synchronizeAndExecute(task, fs, false, Collections.singleton(trashId), path);
-
-                signalDeleteWorker();
-
-                return res;
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to delete in DUAL mode because Grid is stopping: " + path);
-    }
-
-    /**
      * Update path in DUAL mode.
      *
      * @param fs Secondary file system.

http://git-wip-us.apache.org/repos/asf/ignite/blob/66fb9223/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 446495e..b710ba2 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
@@ -256,6 +256,16 @@ public class IgfsPathIds {
     }
 
     /**
+     * Get ID at the give index.
+     *
+     * @param idx Index.
+     * @return ID.
+     */
+    public IgniteUuid id(int idx) {
+        return idx <= lastExistingIdx ? ids[idx] : surrogateId(idx);
+    }
+
+    /**
      * Get surrogate ID at the given index.
      *
      * @param idx Index.

http://git-wip-us.apache.org/repos/asf/ignite/blob/66fb9223/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 261a494..ad8c9e3 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
@@ -2899,11 +2899,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException {
-        for (IgfsPath path : paths) {
-            assert igfs.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + igfs.name() +
-                ", path=" + path + ']';
+        for (IgfsPath path : paths)
             assert igfs.exists(path) : "Path doesn't exist [igfs=" + igfs.name() + ", path=" + path + ']';
-        }
     }
 
     /**
@@ -2952,11 +2949,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     protected void checkNotExist(IgfsImpl igfs, IgfsPath... paths) throws Exception {
-        for (IgfsPath path : paths) {
-            assert igfs.context().meta().fileId(path) == null : "Path exists [igfs=" + igfs.name() + ", path=" +
-                path + ']';
+        for (IgfsPath path : paths)
             assert !igfs.exists(path) : "Path exists [igfs=" + igfs.name() + ", path=" + path + ']';
-        }
     }
 
     /**
@@ -2968,10 +2962,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      */
     protected void checkNotExist(UniversalFileSystemAdapter uni, IgfsPath... paths) throws Exception {
         IgfsEx ex = uni.unwrap(IgfsEx.class);
+
         for (IgfsPath path : paths) {
             if (ex != null)
-                assert ex.context().meta().fileId(path) == null : "Path exists [igfs=" + ex.name() + ", path=" +
-                    path + ']';
+                assert !ex.exists(path) : "Path exists [igfs=" + ex.name() + ", path=" + path + ']';
 
             assert !uni.exists(path.toString()) : "Path exists [igfs=" + uni.name() + ", path=" + path + ']';
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/66fb9223/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 039bf8d..8b88157 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
@@ -191,8 +191,8 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
             assertNull("Unexpected stored properties: " + info, info.properties().get(key2));
         }
 
-        mgr.softDelete(new IgfsPath("/dir"), true);
-        mgr.softDelete(new IgfsPath("/file"), false);
+        mgr.softDelete(new IgfsPath("/dir"), true, null);
+        mgr.softDelete(new IgfsPath("/file"), false, null);
 
         assertNull(mgr.updateProperties(dir.id(), F.asMap("p", "7")));
     }
@@ -328,9 +328,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         mgr.move(path("/a2"), path("/a"));
 
-        IgniteUuid del = mgr.softDelete(path("/a/b/f3"), false);
-
-        assertEquals(f3.id(), del);
+        mgr.softDelete(path("/a/b/f3"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
                 mgr.directoryListing(ROOT_ID));
@@ -341,8 +339,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEmpty(mgr.directoryListing(b.id()));
 
         //assertEquals(b, mgr.removeIfEmpty(a.id(), "b", b.id(), new IgfsPath("/a/b"), true));
-        del = mgr.softDelete(path("/a/b"), false);
-        assertEquals(b.id(), del);
+        mgr.softDelete(path("/a/b"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -351,8 +348,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         assertEmpty(mgr.directoryListing(b.id()));
 
-        del = mgr.softDelete(path("/a/f2"), false);
-        assertEquals(f2.id(), del);
+        mgr.softDelete(path("/a/f2"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -360,16 +356,14 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEmpty(mgr.directoryListing(a.id()));
         assertEmpty(mgr.directoryListing(b.id()));
 
-        del = mgr.softDelete(path("/f1"), false);
-        assertEquals(f1.id(), del);
+        mgr.softDelete(path("/f1"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a)), mgr.directoryListing(ROOT_ID));
 
         assertEmpty(mgr.directoryListing(a.id()));
         assertEmpty(mgr.directoryListing(b.id()));
 
-        del = mgr.softDelete(path("/a"), false);
-        assertEquals(a.id(), del);
+        mgr.softDelete(path("/a"), false, null);
 
         assertEmpty(mgr.directoryListing(ROOT_ID));
         assertEmpty(mgr.directoryListing(a.id()));