You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/09/23 17:45:44 UTC

[29/30] ignite git commit: IGNITE-586: Fixed IGFS rename problem causing corrupted file system structure.

IGNITE-586: Fixed IGFS rename problem causing corrupted file system structure.


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

Branch: refs/heads/ignite-843
Commit: b3bcf4aeecf9aa1bd6d19e94c8da7b09741f5410
Parents: 0a41ae5
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Sep 23 13:45:01 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Sep 23 13:45:01 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteFileSystem.java     |   2 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../internal/processors/igfs/IgfsImpl.java      |  87 ++------
 .../processors/igfs/IgfsMetaManager.java        | 193 ++++++++++++++++--
 .../processors/igfs/IgfsAbstractSelfTest.java   | 201 ++++++++++++++-----
 .../igfs/IgfsClientCacheSelfTest.java           |  15 +-
 .../igfs/IgfsMetaManagerSelfTest.java           | 106 +++++-----
 .../testsuites/IgniteHadoopTestSuite.java       |  19 +-
 8 files changed, 409 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
index b02d0f1..a187a90 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
@@ -451,7 +451,7 @@ public interface IgniteFileSystem extends IgniteAsyncSupport {
      * @return File information for specified path or {@code null} if such path does not exist.
      * @throws IgniteException In case of error.
      */
-    public IgfsFile info(IgfsPath path) throws IgniteException;
+    @Nullable public IgfsFile info(IgfsPath path) throws IgniteException;
 
     /**
      * Gets used space in bytes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 44a3fa9..6ac2b64 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -436,7 +436,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @return {@code this} for chaining.
      */
     public CacheConfiguration<K, V> setName(String name) {
-        A.ensure(name == null || !name.isEmpty(), "Name cannot be null or empty.");
+        A.ensure(name == null || !name.isEmpty(), "Name cannot be empty.");
 
         this.name = name;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/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 695db38..0dd0307 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
@@ -472,8 +472,7 @@ public final class IgfsImpl implements IgfsEx {
     @SuppressWarnings("ConstantConditions")
     @Override public IgfsStatus globalSpace() {
         return safeOp(new Callable<IgfsStatus>() {
-            @Override
-            public IgfsStatus call() throws Exception {
+            @Override public IgfsStatus call() throws Exception {
                 IgniteBiTuple<Long, Long> space = igfsCtx.kernalContext().grid().compute().execute(
                     new IgfsGlobalSpaceTask(name()), null);
 
@@ -560,7 +559,7 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFile info(final IgfsPath path) {
+    @Override @Nullable public IgfsFile info(final IgfsPath path) {
         A.notNull(path, "path");
 
         return safeOp(new Callable<IgfsFile>() {
@@ -692,64 +691,12 @@ public final class IgfsImpl implements IgfsEx {
                     return null;
                 }
 
-                IgfsPath destParent = dest.parent();
+                IgfsFileInfo info = meta.move(src, dest);
 
-                // Resolve source file info.
-                FileDescriptor srcDesc = getFileDescriptor(src);
+                int evtTyp = info.isFile() ? EVT_IGFS_FILE_RENAMED : EVT_IGFS_DIR_RENAMED;
 
-                // File not found.
-                if (srcDesc == null || srcDesc.parentId == null) {
-                    if (mode == PRIMARY)
-                        checkConflictWithPrimary(src);
-
-                    throw new IgfsPathNotFoundException("Failed to rename (source path not found): " + src);
-                }
-
-                String srcFileName = src.name();
-
-                // Resolve destination file info.
-                FileDescriptor destDesc = getFileDescriptor(dest);
-
-                String destFileName;
-
-                boolean newDest = destDesc == null;
-
-                if (newDest) {
-                    assert destParent != null;
-
-                    // Use parent directory for destination parent and destination path name as destination name.
-                    destDesc = getFileDescriptor(destParent);
-
-                    // Destination directory doesn't exist.
-                    if (destDesc == null)
-                        throw new IgfsPathNotFoundException("Failed to rename (destination directory does not " +
-                            "exist): " + dest);
-
-                    destFileName = dest.name();
-                }
-                else
-                    // Use destination directory for destination parent and source path name as destination name.
-                    destFileName = srcFileName;
-
-                // Can move only into directory, but not into file.
-                if (destDesc.isFile)
-                    throw new IgfsParentNotDirectoryException("Failed to rename (destination is not a directory): "
-                        + dest);
-
-                meta.move(srcDesc.fileId, srcFileName, srcDesc.parentId, destFileName, destDesc.fileId);
-
-                if (srcDesc.isFile) { // Renamed a file.
-                    if (evts.isRecordable(EVT_IGFS_FILE_RENAMED))
-                        evts.record(new IgfsEvent(
-                            src,
-                            newDest ? dest : new IgfsPath(dest, destFileName),
-                            localNode(),
-                            EVT_IGFS_FILE_RENAMED));
-                }
-                else { // Renamed a directory.
-                    if (evts.isRecordable(EVT_IGFS_DIR_RENAMED))
-                        evts.record(new IgfsEvent(src, dest, localNode(), EVT_IGFS_DIR_RENAMED));
-                }
+                if (evts.isRecordable(evtTyp))
+                    evts.record(new IgfsEvent(src, info.path(), localNode(), evtTyp));
 
                 return null;
             }
@@ -967,8 +914,7 @@ public final class IgfsImpl implements IgfsEx {
                 }
 
                 return F.viewReadOnly(files, new C1<String, IgfsPath>() {
-                    @Override
-                    public IgfsPath apply(String e) {
+                    @Override public IgfsPath apply(String e) {
                         return new IgfsPath(path, e);
                     }
                 });
@@ -981,8 +927,7 @@ public final class IgfsImpl implements IgfsEx {
         A.notNull(path, "path");
 
         return safeOp(new Callable<Collection<IgfsFile>>() {
-            @Override
-            public Collection<IgfsFile> call() throws Exception {
+            @Override public Collection<IgfsFile> call() throws Exception {
                 if (log.isDebugEnabled())
                     log.debug("List directory details: " + path);
 
@@ -1058,8 +1003,7 @@ public final class IgfsImpl implements IgfsEx {
         A.ensure(seqReadsBeforePrefetch >= 0, "seqReadsBeforePrefetch >= 0");
 
         return safeOp(new Callable<IgfsInputStreamAdapter>() {
-            @Override
-            public IgfsInputStreamAdapter call() throws Exception {
+            @Override public IgfsInputStreamAdapter call() throws Exception {
                 if (log.isDebugEnabled())
                     log.debug("Open file for reading [path=" + path + ", bufSize=" + bufSize + ']');
 
@@ -1146,8 +1090,7 @@ public final class IgfsImpl implements IgfsEx {
         A.ensure(bufSize >= 0, "bufSize >= 0");
 
         return safeOp(new Callable<IgfsOutputStream>() {
-            @Override
-            public IgfsOutputStream call() throws Exception {
+            @Override public IgfsOutputStream call() throws Exception {
                 if (log.isDebugEnabled())
                     log.debug("Open file for writing [path=" + path + ", bufSize=" + bufSize + ", overwrite=" +
                         overwrite + ", props=" + props + ']');
@@ -1250,8 +1193,7 @@ public final class IgfsImpl implements IgfsEx {
         A.ensure(bufSize >= 0, "bufSize >= 0");
 
         return safeOp(new Callable<IgfsOutputStream>() {
-            @Override
-            public IgfsOutputStream call() throws Exception {
+            @Override public IgfsOutputStream call() throws Exception {
                 if (log.isDebugEnabled())
                     log.debug("Open file for appending [path=" + path + ", bufSize=" + bufSize + ", create=" + create +
                         ", props=" + props + ']');
@@ -1373,8 +1315,7 @@ public final class IgfsImpl implements IgfsEx {
         A.ensure(len >= 0, "len >= 0");
 
         return safeOp(new Callable<Collection<IgfsBlockLocation>>() {
-            @Override
-            public Collection<IgfsBlockLocation> call() throws Exception {
+            @Override public Collection<IgfsBlockLocation> call() throws Exception {
                 if (log.isDebugEnabled())
                     log.debug("Get affinity for file block [path=" + path + ", start=" + start + ", len=" + len + ']');
 
@@ -1407,8 +1348,7 @@ public final class IgfsImpl implements IgfsEx {
     /** {@inheritDoc} */
     @Override public IgfsMetrics metrics() {
         return safeOp(new Callable<IgfsMetrics>() {
-            @Override
-            public IgfsMetrics call() throws Exception {
+            @Override public IgfsMetrics call() throws Exception {
                 IgfsPathSummary sum = new IgfsPathSummary();
 
                 summary0(ROOT_ID, sum);
@@ -1587,6 +1527,7 @@ public final class IgfsImpl implements IgfsEx {
      */
     @Nullable private FileDescriptor getFileDescriptor(IgfsPath path) throws IgniteCheckedException {
         List<IgniteUuid> ids = meta.fileIds(path);
+
         IgfsFileInfo fileInfo = meta.info(ids.get(ids.size() - 1));
 
         if (fileInfo == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/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 5611f33..d283b64 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
@@ -26,13 +26,16 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
@@ -584,23 +587,29 @@ public class IgfsMetaManager extends IgfsManager {
         assert validTxState(true);
         assert fileIds != null && fileIds.length > 0;
 
-        // Always sort file IDs participating in transaction to escape cache transaction deadlocks.
         Arrays.sort(fileIds);
 
-        // Wrap array as collection (1) to escape superfluous check in projection and (2) to check assertions.
-        Collection<IgniteUuid> keys = Arrays.asList(fileIds);
+        return lockIds(Arrays.asList(fileIds));
+    }
 
+    /**
+     * Lock file IDs.
+     * @param fileIds File IDs (sorted).
+     * @return Map with lock info.
+     * @throws IgniteCheckedException If failed.
+     */
+    private Map<IgniteUuid, IgfsFileInfo> lockIds(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
         if (log.isDebugEnabled())
-            log.debug("Locking file ids: " + keys);
+            log.debug("Locking file ids: " + fileIds);
 
         // Lock files and get their infos.
-        Map<IgniteUuid, IgfsFileInfo> map = id2InfoPrj.getAll(keys);
+        Map<IgniteUuid, IgfsFileInfo> map = id2InfoPrj.getAll(fileIds);
 
         if (log.isDebugEnabled())
-            log.debug("Locked file ids: " + keys);
+            log.debug("Locked file ids: " + fileIds);
 
         // Force root ID always exist in cache.
-        if (keys.contains(ROOT_ID) && !map.containsKey(ROOT_ID)) {
+        if (fileIds.contains(ROOT_ID) && !map.containsKey(ROOT_ID)) {
             IgfsFileInfo info = new IgfsFileInfo();
 
             id2InfoPrj.putIfAbsent(ROOT_ID, info);
@@ -807,27 +816,131 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Move or rename file.
+     * Move routine.
      *
-     * @param fileId File ID to move or rename.
-     * @param srcFileName Original file name in the parent's listing.
-     * @param srcParentId Parent directory ID.
-     * @param destFileName New file name in the parent's listing after moving.
-     * @param destParentId New parent directory ID.
-     * @throws IgniteCheckedException If failed.
+     * @param srcPath Source path.
+     * @param dstPath Destinatoin path.
+     * @return File info of renamed entry.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public void move(IgniteUuid fileId, String srcFileName, IgniteUuid srcParentId, String destFileName,
-        IgniteUuid destParentId) throws IgniteCheckedException {
+    public IgfsFileInfo move(IgfsPath srcPath, IgfsPath dstPath) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 assert validTxState(false);
 
+                // 1. First get source and destination path IDs.
+                List<IgniteUuid> srcPathIds = fileIds(srcPath);
+                List<IgniteUuid> dstPathIds = fileIds(dstPath);
+
+                final Set<IgniteUuid> allIds = new TreeSet<>(new Comparator<IgniteUuid>() {
+                    @Override
+                    public int compare(IgniteUuid u1, IgniteUuid u2) {
+                        if (u1 == u2)
+                            return 0;
+
+                        if (u1 == null)
+                            return -1;
+
+                        return u1.compareTo(u2);
+                    }
+                });
+
+                allIds.addAll(srcPathIds);
+
+                final IgniteUuid dstLeafId = dstPathIds.get(dstPathIds.size() - 1);
+
+                if (dstLeafId == null) {
+                    // Delete null entry for the unexisting destination element:
+                    dstPathIds.remove(dstPathIds.size() - 1);
+                }
+
+                allIds.addAll(dstPathIds);
+
+                if (allIds.remove(null)) {
+                    throw new IgfsPathNotFoundException("Failed to perform move because some path component was " +
+                            "not found. [src=" + srcPath + ", dst=" + dstPath + ']');
+                }
+
+                // 2. Start transaction.
                 IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
-                    moveNonTx(fileId, srcFileName, srcParentId, destFileName, destParentId);
+                    // 3. Obtain the locks.
+                    final Map<IgniteUuid, IgfsFileInfo> allInfos = lockIds(allIds);
+
+                    // 4. Verify integrity of source directory.
+                    if (!verifyPathIntegrity(srcPath, srcPathIds, allInfos)) {
+                        throw new IgfsPathNotFoundException("Failed to perform move because source directory " +
+                            "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']');
+                    }
+
+                    // 5. Verify integrity of destination directory.
+                    final IgfsPath dstDirPath = dstLeafId != null ? dstPath : dstPath.parent();
+
+                    if (!verifyPathIntegrity(dstDirPath, dstPathIds, allInfos)) {
+                        throw new IgfsPathNotFoundException("Failed to perform move because destination directory " +
+                            "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']');
+                    }
+
+                    // 6. Calculate source and destination targets which will be changed.
+                    IgniteUuid srcTargetId = srcPathIds.get(srcPathIds.size() - 2);
+                    IgfsFileInfo srcTargetInfo = allInfos.get(srcTargetId);
+                    String srcName = srcPath.name();
+
+                    IgniteUuid dstTargetId;
+                    IgfsFileInfo dstTargetInfo;
+                    String dstName;
+
+                    if (dstLeafId != null) {
+                        // Destination leaf exists. Check if it is an empty directory.
+                        IgfsFileInfo dstLeafInfo = allInfos.get(dstLeafId);
+
+                        assert dstLeafInfo != null;
+
+                        if (dstLeafInfo.isDirectory()) {
+                            // Destination is a directory.
+                            dstTargetId = dstLeafId;
+                            dstTargetInfo = dstLeafInfo;
+                            dstName = srcPath.name();
+                        }
+                        else {
+                            // Error, destination is existing file.
+                            throw new IgfsPathAlreadyExistsException("Failed to perform move " +
+                                "because destination points to " +
+                                "existing file [src=" + srcPath + ", dst=" + dstPath + ']');
+                        }
+                    }
+                    else {
+                        // Destination leaf doesn't exist, so we operate on parent.
+                        dstTargetId = dstPathIds.get(dstPathIds.size() - 1);
+                        dstTargetInfo = allInfos.get(dstTargetId);
+                        dstName = dstPath.name();
+                    }
+
+                    assert dstTargetInfo != null;
+                    assert dstTargetInfo.isDirectory();
+
+                    // 7. Last check: does destination target already have listing entry with the same name?
+                    if (dstTargetInfo.listing().containsKey(dstName)) {
+                        throw new IgfsPathAlreadyExistsException("Failed to perform move because destination already " +
+                            "contains entry with the same name existing file [src=" + srcPath +
+                            ", dst=" + dstPath + ']');
+                    }
+
+                    // 8. Actual move: remove from source parent and add to destination target.
+                    IgfsListingEntry entry = srcTargetInfo.listing().get(srcName);
+
+                    id2InfoPrj.invoke(srcTargetId, new UpdateListing(srcName, entry, true));
+                    id2InfoPrj.invoke(dstTargetId, new UpdateListing(dstName, entry, false));
 
                     tx.commit();
+
+                    IgfsPath realNewPath = new IgfsPath(dstDirPath, dstName);
+
+                    IgfsFileInfo moved = allInfos.get(srcPathIds.get(srcPathIds.size() - 1));
+
+                    // Set the new path to the info to simplify event creation:
+                    return IgfsFileInfo.builder(moved).path(realNewPath).build();
                 }
                 finally {
                     tx.close();
@@ -838,9 +951,49 @@ public class IgfsMetaManager extends IgfsManager {
             }
         }
         else
-            throw new IllegalStateException("Failed to move file system entry because Grid is stopping [fileId=" +
-                fileId + ", srcFileName=" + srcFileName + ", srcParentId=" + srcParentId + ", destFileName=" +
-                destFileName + ", destParentId=" + destParentId + ']');
+            throw new IllegalStateException("Failed to perform move because Grid is stopping [srcPath=" +
+                srcPath + ", dstPath=" + dstPath + ']');
+    }
+
+    /**
+     * Verify path integrity.
+     *
+     * @param path Path to verify.
+     * @param expIds Expected IDs for this path. Might contain additional elements, e.g. because they were created
+     *     on a child path.
+     * @param infos Locked infos.
+     * @return
+     */
+    private static boolean verifyPathIntegrity(IgfsPath path, List<IgniteUuid> expIds,
+        Map<IgniteUuid, IgfsFileInfo> infos) {
+        List<String> pathParts = path.components();
+
+        assert pathParts.size() < expIds.size();
+
+        for (int i = 0; i < pathParts.size(); i++) {
+            IgniteUuid parentId = expIds.get(i);
+
+            // If parent ID is null, it doesn't exist.
+            if (parentId != null) {
+                IgfsFileInfo parentInfo = infos.get(parentId);
+
+                // If parent info is null, it doesn't exist.
+                if (parentInfo != null) {
+                    IgfsListingEntry childEntry = parentInfo.listing().get(pathParts.get(i));
+
+                    // If expected child exists.
+                    if (childEntry != null) {
+                        // If child ID matches expected ID.
+                        if (F.eq(childEntry.fileId(), expIds.get(i + 1)))
+                            continue;
+                    }
+                }
+            }
+
+            return false;
+        }
+
+        return true;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/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 0a1e626..cfa99ff 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
@@ -43,6 +43,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
@@ -59,13 +60,17 @@ import org.apache.ignite.igfs.IgfsOutputStream;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
@@ -101,6 +106,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /** Concurrent operations count. */
     protected static final int OPS_CNT = 16;
 
+    /** Seed. */
+    protected static final long SEED = System.currentTimeMillis();
+
     /** Amount of blocks to prefetch. */
     protected static final int PREFETCH_BLOCKS = 1;
 
@@ -203,11 +211,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /**
      * Data chunk.
      *
-     * @param length Length.
+     * @param len Length.
      * @return Data chunk.
      */
-    static byte[] createChunk(int length) {
-        byte[] chunk = new byte[length];
+    static byte[] createChunk(int len) {
+        byte[] chunk = new byte[len];
 
         for (int i = 0; i < chunk.length; i++)
             chunk[i] = (byte)i;
@@ -224,6 +232,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         igfs = (IgfsImpl) ignite.fileSystem("igfs");
     }
 
+    /**
+     * Creates secondary file system stack.
+     *
+     * @return The secondary file system.
+     * @throws Exception On error.
+     */
     protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
         Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
             SECONDARY_REST_CFG);
@@ -845,17 +859,16 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /**
      * Ensure that formatting is not propagated to the secondary file system.
      *
-     * TODO: IGNITE-586.
-     *
      * @throws Exception If failed.
      */
     @SuppressWarnings("ConstantConditions")
     public void testFormat() throws Exception {
-        // Test works too long and fails.
-        fail("https://issues.apache.org/jira/browse/IGNITE-586");
+        final GridCacheAdapter<IgfsBlockKey, byte[]> dataCache = getDataCache(igfs);
 
-        IgniteKernal grid = (IgniteKernal)G.ignite("grid");
-        GridCacheAdapter cache = grid.internalCache("dataCache");
+        assert dataCache != null;
+
+        int size0 = dataCache.size(new CachePeekMode[] {CachePeekMode.ALL});
+        assert size0 == 0 : "Initial data cache size = " + size0;
 
         if (dual)
             create(igfsSecondary, paths(DIR, SUBDIR, DIR_NEW, SUBDIR_NEW), paths(FILE, FILE_NEW));
@@ -873,23 +886,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         assert igfs.info(FILE).length() == 10 * 1024 * 1024;
 
-        int size = cache.size();
-        int primarySize = cache.primarySize();
-        int primaryKeySetSize = cache.primaryKeySet().size();
-
-        int primaryPartSize = 0;
-
-        for (int p : cache.affinity().primaryPartitions(grid.localNode())) {
-            Set set = cache.entrySet(p);
-
-            if (set != null)
-                primaryPartSize += set.size();
-        }
-
-        assert size > 0;
-        assert primarySize > 0;
-        assert primarySize == primaryKeySetSize;
-        assert primarySize == primaryPartSize;
+        assert dataCache.size(new CachePeekMode[] {CachePeekMode.ALL}) > 0;
 
         igfs.format();
 
@@ -903,27 +900,26 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         // Ensure entries deletion in the primary file system.
         checkNotExist(igfs, DIR, SUBDIR, FILE);
 
-        int sizeNew = cache.size();
-        int primarySizeNew = cache.primarySize();
-        int primaryKeySetSizeNew = cache.primaryKeySet().size();
-
-        int primaryPartSizeNew = 0;
-
-        for (int p : cache.affinity().primaryPartitions(grid.localNode())) {
-            Set set = cache.entrySet(p);
-
-            if (set != null) {
-                for (Object entry : set)
-                    System.out.println(entry);
-
-                primaryPartSizeNew += set.size();
+        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return dataCache.size(new CachePeekMode[] {CachePeekMode.ALL}) == 0;
+                } catch (IgniteCheckedException ice) {
+                    throw new IgniteException(ice);
+                }
+            }
+        }, 10_000)) {
+            Set<GridCacheEntryEx> set = dataCache.allEntries();
+
+            for (GridCacheEntryEx e: set) {
+                X.println("deleted = " + e.deleted());
+                X.println("detached = " + e.detached());
+                X.println("info = " + e.info());
+                X.println("k = " + e.key() + ", v = " + e.valueBytes());
             }
-        }
 
-        assert sizeNew == 0;
-        assert primarySizeNew == 0;
-        assert primaryKeySetSizeNew == 0;
-        assert primaryPartSizeNew == 0;
+            assert false;
+        }
     }
 
     /**
@@ -1885,14 +1881,18 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testDeadlocksRename() throws Exception {
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
+                info(">>>>>> Start deadlock test.");
+
                 checkDeadlocks(5, 2, 2, 2, OPS_CNT, 0, 0, 0, 0);
+
+                info(">>>>>> End deadlock test.");
             }
             finally {
-                info(">>>>>> Start deadlock test");
+                info(">>>>>> Start cleanup.");
 
                 clear(igfs, igfsSecondary);
 
-                info(">>>>>> End deadlock test");
+                info(">>>>>> End cleanup.");
             }
         }
     }
@@ -1903,6 +1903,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksDelete() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1515");
+
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
                 checkDeadlocks(5, 2, 2, 2, 0, OPS_CNT, 0, 0, 0);
@@ -1967,6 +1969,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocks() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1515");
+
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
                 checkDeadlocks(5, 2, 2, 2, OPS_CNT, OPS_CNT, OPS_CNT, OPS_CNT, OPS_CNT);
@@ -2038,7 +2042,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         }
 
         // Now as we have all paths defined, plan operations on them.
-        final Random rand = new Random(U.currentTimeMillis());
+        final Random rand = new Random(SEED);
 
         int totalOpCnt = renCnt + delCnt + updateCnt + mkdirsCnt + createCnt;
 
@@ -2194,7 +2198,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
             threads.add(new Thread(r));
         }
 
-        // Create folder structure.
+        // Create file/directory structure.
         for (int i = 0; i < lvlCnt; i++) {
             int lvl = i + 1;
 
@@ -2252,6 +2256,14 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         }
     }
 
+    /**
+     * Creates specified files/directories
+     *
+     * @param uni The file system to operate on.
+     * @param dirs The directories to create.
+     * @param files The files to create.
+     * @throws Exception On error.
+     */
     @SuppressWarnings("EmptyTryBlock")
     public void create(UniversalFileSystemAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception {
         if (dirs != null) {
@@ -2646,6 +2658,34 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Gets the data cache instance for this IGFS instance.
+     *
+     * @param igfs The IGFS unstance.
+     * @return The data cache.
+     */
+    protected static GridCacheAdapter<IgfsBlockKey, byte[]> getDataCache(IgniteFileSystem igfs) {
+        String dataCacheName = igfs.configuration().getDataCacheName();
+
+        IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
+
+        return ((IgniteKernal)igniteEx).internalCache(dataCacheName);
+    }
+
+    /**
+     * Gets meta cache.
+     *
+     * @param igfs The IGFS instance.
+     * @return The data cache.
+     */
+    protected static GridCacheAdapter<IgniteUuid, IgfsFileInfo> getMetaCache(IgniteFileSystem igfs) {
+        String dataCacheName = igfs.configuration().getMetaCacheName();
+
+        IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
+
+        return ((IgniteKernal)igniteEx).internalCache(dataCacheName);
+    }
+
+    /**
      * Clear particular IGFS.
      *
      * @param igfs IGFS.
@@ -2674,6 +2714,69 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         // Clear igfs.
         igfs.format();
+
+        final IgniteFileSystem igfs0 = igfs;
+
+        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return isEmpty(igfs0);
+            }
+        }, 10_000L)) {
+            dumpCache("MetaCache" , getMetaCache(igfs));
+
+            dumpCache("DataCache" , getDataCache(igfs));
+
+            assert false;
+        }
+    }
+
+    /**
+     * Dumps given cache for diagnostic purposes.
+     *
+     * @param cacheName Name.
+     * @param cache The cache.
+     */
+    private static void dumpCache(String cacheName, GridCacheAdapter<?,?> cache) {
+        X.println("=============================== " + cacheName + " cache dump: ");
+
+        Set<GridCacheEntryEx> set = cache.entries();
+
+        for (GridCacheEntryEx e: set)
+            X.println("Lost " + cacheName + " entry = " + e);
+    }
+
+    /**
+     * Answers if the given IGFS is empty.
+     *
+     * @param igfs IGFS to operate on.
+     * @return True if IGFS is empty.
+     */
+    private static boolean isEmpty(IgniteFileSystem igfs) {
+        GridCacheAdapter dataCache = getDataCache(igfs);
+
+        assert dataCache != null;
+
+        int size1 = dataCache.size();
+
+        if (size1 > 0) {
+            X.println("Data cache size = " + size1);
+
+            return false;
+        }
+
+        GridCacheAdapter metaCache = getMetaCache(igfs);
+
+        assert metaCache != null;
+
+        int size2 = metaCache.size();
+
+        if (size2 > 2) {
+            X.println("Meta cache size = " + size2);
+
+            return false;
+        }
+
+        return true;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
index f2394fc..8e8eac1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
@@ -47,9 +47,6 @@ public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
     /** Data cache name. */
     private static final String DATA_CACHE_NAME = null;
 
-    /** Regular cache name. */
-    private static final String CACHE_NAME = "cache";
-
     /**
      * Constructor.
      */
@@ -61,9 +58,9 @@ public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
     @Override protected void beforeTestsStarted() throws Exception {
         igfsSecondaryFileSystem = createSecondaryFileSystemStack();
 
-        Ignite ignite1 = G.start(getConfiguration(getTestGridName(1)));
+        Ignite ignitePrimary = G.start(getConfiguration(getTestGridName(1)));
 
-        igfs = (IgfsImpl) ignite1.fileSystem("igfs");
+        igfs = (IgfsImpl) ignitePrimary.fileSystem("igfs");
     }
 
     /**{@inheritDoc} */
@@ -86,8 +83,10 @@ public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
     protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setCacheConfiguration(cacheConfiguration(META_CACHE_NAME), cacheConfiguration(DATA_CACHE_NAME),
-            cacheConfiguration(CACHE_NAME));
+        cfg.setCacheConfiguration(
+            cacheConfiguration(META_CACHE_NAME),
+            cacheConfiguration(DATA_CACHE_NAME)
+        );
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
@@ -117,7 +116,7 @@ public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
      * @return Cache configuration.
      */
     protected CacheConfiguration cacheConfiguration(String cacheName) {
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+        CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
 
         cacheCfg.setName(cacheName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/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 75423f1..206c9fe 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
@@ -152,8 +152,6 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEquals(F.asMap("dir", new IgfsListingEntry(dir), "file", new IgfsListingEntry(file)),
             mgr.directoryListing(ROOT_ID));
 
-        //IgfsFileInfo tmp = mgr.info(dir.id());
-
         assertEquals(dir, mgr.info(dir.id()));
         assertEquals(file, mgr.info(file.id()));
 
@@ -215,6 +213,9 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         IgfsFileInfo a = new IgfsFileInfo(true, null);
         IgfsFileInfo b = new IgfsFileInfo(true, null);
+        IgfsFileInfo k = new IgfsFileInfo(true, null);
+        IgfsFileInfo z = new IgfsFileInfo(true, null);
+
         IgfsFileInfo f1 = new IgfsFileInfo(400, null, false, null);
         IgfsFileInfo f2 = new IgfsFileInfo(new IgfsFileInfo(400, null, false, null), 0);
         IgfsFileInfo f3 = new IgfsFileInfo(new IgfsFileInfo(400, null, false, null), 200000L);
@@ -223,6 +224,8 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertNull(mgr.putIfAbsent(ROOT_ID, "a", a));
         assertNull(mgr.putIfAbsent(ROOT_ID, "f1", f1));
         assertNull(mgr.putIfAbsent(a.id(), "b", b));
+        assertNull(mgr.putIfAbsent(a.id(), "k", z));
+        assertNull(mgr.putIfAbsent(b.id(), "k", k));
         assertNull(mgr.putIfAbsent(a.id(), "f2", f2));
         assertNull(mgr.putIfAbsent(b.id(), "f3", f3));
 
@@ -232,15 +235,15 @@ 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), "k", new IgfsListingEntry(z)),
             mgr.directoryListing(a.id()));
 
-        assertEquals(F.asMap("f3", new IgfsListingEntry(f3)), mgr.directoryListing(b.id()));
+        assertEquals(F.asMap("f3", new IgfsListingEntry(f3),
+            "k", new IgfsListingEntry(k)), mgr.directoryListing(b.id()));
 
         // Validate empty files listings.
-        for (IgfsFileInfo info : Arrays.asList(f1, f2, f3)) {
+        for (IgfsFileInfo info : Arrays.asList(f1, f2, f3))
             assertEmpty(mgr.directoryListing(info.id()));
-        }
 
         // Validate 'file info' operations.
         for (IgfsFileInfo info : Arrays.asList(rootInfo, a, b, f1, f2, f3)) {
@@ -279,51 +282,44 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEquals(Arrays.asList(ROOT_ID, a.id(), b.id(), null), mgr.fileIds(new IgfsPath("/a/b/f6")));
         assertEquals(Arrays.asList(ROOT_ID, null, null, null, null), mgr.fileIds(new IgfsPath("/f7/a/b/f6")));
 
-        // Validate 'rename' operation.
-        final IgniteUuid rndId = IgniteUuid.randomUuid();
-
         // One of participated files does not exist in cache.
-        expectsRenameFail(ROOT_ID, "b", rndId, "b2", rndId, "Failed to lock source directory (not found?)");
-        expectsRenameFail(b.id(), "b", rndId, "b2", rndId, "Failed to lock source directory (not found?)");
-        expectsRenameFail(ROOT_ID, "b", ROOT_ID, "b2", rndId, "Failed to lock destination directory (not found?)");
-        expectsRenameFail(b.id(), "b", ROOT_ID, "b2", rndId, "Failed to lock destination directory (not found?)");
-        expectsRenameFail(rndId, "b", ROOT_ID, "b2", ROOT_ID, "Failed to lock target file (not found?)");
-        expectsRenameFail(rndId, "b", b.id(), "b2", b.id(), "Failed to lock target file (not found?)");
-
-        // Target file ID differ from the file ID resolved from the source directory for source file name.
-        expectsRenameFail(b.id(), "a", ROOT_ID, "q", ROOT_ID, "Failed to remove file name from the source directory");
-        expectsRenameFail(f1.id(), "a", ROOT_ID, "q", ROOT_ID, "Failed to remove file name from the source directory");
-        expectsRenameFail(f2.id(), "a", ROOT_ID, "q", ROOT_ID, "Failed to remove file name from the source directory");
-        expectsRenameFail(f3.id(), "a", ROOT_ID, "q", ROOT_ID, "Failed to remove file name from the source directory");
-
-        // Invalid source file name (not found).
-        expectsRenameFail(a.id(), "u1", ROOT_ID, "q", ROOT_ID, "Failed to remove file name from the source");
-        expectsRenameFail(a.id(), "u2", ROOT_ID, "q", ROOT_ID, "Failed to remove file name from the source");
-        expectsRenameFail(a.id(), "u3", ROOT_ID, "q", ROOT_ID, "Failed to remove file name from the source");
-
-        // Invalid destination file - already exists.
-        expectsRenameFail(a.id(), "a", ROOT_ID, "f1", ROOT_ID, "Failed to add file name into the destination");
-        expectsRenameFail(f2.id(), "f2", a.id(), "f1", ROOT_ID, "Failed to add file name into the destination");
-        expectsRenameFail(f3.id(), "f3", b.id(), "f1", ROOT_ID, "Failed to add file name into the destination");
-        expectsRenameFail(b.id(), "b", a.id(), "f2", a.id(), "Failed to add file name into the destination");
+        expectsRenameFail("/b8", "/b2", "Failed to perform move because some path component was not found.");
+
+        expectsRenameFail("/a", "/b/b8", "Failed to perform move because some path component was not found.");
+
+        expectsRenameFail("/a/f2", "/a/b/f3", "Failed to perform move because destination points to existing file");
+
+        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());
 
         System.out.println("/: " + mgr.directoryListing(ROOT_ID));
         System.out.println("a: " + mgr.directoryListing(a.id()));
         System.out.println("b: " + mgr.directoryListing(b.id()));
         System.out.println("f3: " + mgr.directoryListing(f3.id()));
 
-        mgr.move(a.id(), "a", ROOT_ID, "a2", ROOT_ID);
-        mgr.move(b.id(), "b", a.id(), "b2", a.id());
+        //mgr.move(a.id(), "a", ROOT_ID, "a2", ROOT_ID);
+        mgr.move(path("/a"), path("/a2"));
+        //mgr.move(b.id(), "b", a.id(), "b2", a.id());
+        mgr.move(path("/a2/b"), path("/a2/b2"));
 
         assertNotNull(mgr.info(b.id()));
 
-        mgr.move(f3.id(), "f3", b.id(), "f3-2", a.id());
+        //mgr.move(f3.id(), "f3", b.id(), "f3-2", a.id());
+        mgr.move(path("/a2/b2/f3"), path("/a2/b2/f3-2"));
 
         assertNotNull(mgr.info(b.id()));
 
-        mgr.move(f3.id(), "f3-2", a.id(), "f3", b.id());
-        mgr.move(b.id(), "b2", a.id(), "b", a.id());
-        mgr.move(a.id(), "a2", ROOT_ID, "a", ROOT_ID);
+        //mgr.move(f3.id(), "f3-2", a.id(), "f3", b.id());
+        mgr.move(path("/a2/b2/f3-2"), path("/a2/b2/f3"));
+
+        //mgr.move(b.id(), "b2", a.id(), "b", a.id());
+        mgr.move(path("/a2/b2"), path("/a2/b"));
+
+        //mgr.move(a.id(), "a2", ROOT_ID, "a", ROOT_ID);
+        mgr.move(path("/a2"), path("/a"));
 
         // Validate 'remove' operation.
         for (int i = 0; i < 100; i++) {
@@ -345,7 +341,9 @@ 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()));
@@ -393,6 +391,16 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Utility method to make IgfsPath.
+     *
+     * @param p The String path.
+     * @return The IgfsPath object.
+     */
+    private static IgfsPath path(String p) {
+        return new IgfsPath(p);
+    }
+
+    /**
      * Validate passed map is empty.
      *
      * @param map Map to validate it is empty.
@@ -411,8 +419,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
     private void expectsUpdatePropertiesFail(@Nullable final IgniteUuid fileId, @Nullable final Map<String, String> props,
         Class<? extends Throwable> cls, @Nullable String msg) {
         assertThrows(log, new Callable() {
-            @Override
-            public Object call() throws Exception {
+            @Override public Object call() throws Exception {
                 return mgr.updateProperties(null, fileId, "file", props);
             }
         }, cls, msg);
@@ -440,25 +447,18 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
     /**
      * Test expected failures for 'move file' operation.
      *
-     * @param fileId File ID to rename.
-     * @param srcFileName Original file name in the parent's listing.
-     * @param srcParentId Source parent directory ID.
-     * @param destFileName New file name in the parent's listing after renaming.
-     * @param destParentId Destination parent directory ID.
      * @param msg Failure message if expected exception was not thrown.
      */
-    private void expectsRenameFail(final IgniteUuid fileId, final String srcFileName, final IgniteUuid srcParentId,
-        final String destFileName, final IgniteUuid destParentId, @Nullable String msg) {
+    private void expectsRenameFail(final String src, final String dst, @Nullable String msg) {
         Throwable err = assertThrowsInherited(log, new Callable() {
-            @Override
-            public Object call() throws Exception {
-                mgr.move(fileId, srcFileName, srcParentId, destFileName, destParentId);
+            @Override public Object call() throws Exception {
+                mgr.move(new IgfsPath(src), new IgfsPath(dst));
 
                 return null;
             }
-        }, IgniteCheckedException.class, msg);
+        }, IgfsException.class, msg);
 
-        assertTrue("Unexpected cause: " + err.getCause(), err.getCause() instanceof IgfsException);
+        assertTrue("Unexpected cause: " + err, err instanceof IgfsException);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3bcf4ae/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 23f85d2..0216f4b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -61,7 +61,6 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobTrackerSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceEmbeddedSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopSerializationWrapperSelfTest;
-import org.apache.ignite.internal.processors.hadoop.HadoopSortingExternalTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopSortingTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapperSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskExecutionSelfTest;
@@ -73,8 +72,6 @@ import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopCo
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopHashMapSelfTest;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipListSelfTest;
 import org.apache.ignite.internal.processors.hadoop.shuffle.streams.HadoopDataStreamSelfTest;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutionSelfTest;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunicationSelfTest;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -97,6 +94,14 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite");
 
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfs20FileSystemLoopbackPrimarySelfTest.class.getName())));
+
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualSyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualAsyncSelfTest.class.getName())));
+
+        suite.addTest(new TestSuite(ldr.loadClass(Hadoop1OverIgfsDualSyncTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(Hadoop1OverIgfsDualAsyncTest.class.getName())));
+
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.class.getName())));
@@ -115,11 +120,6 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemHandshakeSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfs20FileSystemLoopbackPrimarySelfTest.class.getName())));
-
-        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualSyncSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualAsyncSelfTest.class.getName())));
-
         suite.addTest(IgfsEventsTestSuite.suiteNoarchOnly());
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopFileSystemsTest.class.getName())));
@@ -163,9 +163,6 @@ public class IgniteHadoopTestSuite extends TestSuite {
         suite.addTest(new TestSuite(ldr.loadClass(HadoopCommandLineTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSecondaryFileSystemConfigurationTest.class.getName())));
-
-        suite.addTest(new TestSuite(ldr.loadClass(Hadoop1OverIgfsDualSyncTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(Hadoop1OverIgfsDualAsyncTest.class.getName())));
         return suite;
     }