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 2015/10/15 15:52:07 UTC

[01/10] ignite git commit: IGNITE-1590: Reworked create and append operations to match overall design.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1093-2 9aa613055 -> 4907c5482


http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/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 cc89fd1..b290303 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
@@ -63,7 +63,6 @@ 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;
@@ -102,10 +101,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     protected static final long BLOCK_SIZE = 32 * 1024 * 1024;
 
     /** Default repeat count. */
-    protected static final int REPEAT_CNT = 10;
+    protected static final int REPEAT_CNT = 5; // Diagnostic: up to 500; Regression: 5
 
     /** Concurrent operations count. */
-    protected static final int OPS_CNT = 32;
+    protected static final int OPS_CNT = 16;
 
     /** Renames count. */
     protected static final int RENAME_CNT = OPS_CNT;
@@ -122,6 +121,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /** Create count. */
     protected static final int CREATE_CNT = OPS_CNT;
 
+    /** Time to wait until the caches get empty after format. */
+    private static final long CACHE_EMPTY_TIMEOUT = 30_000L;
+
     /** Seed to generate random numbers. */
     protected static final long SEED = System.currentTimeMillis();
 
@@ -761,6 +763,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         create(igfs, null, new IgfsPath[] { new IgfsPath("/d/f") }); // "f" is a file.
         checkExist(igfs, igfsSecondary, new IgfsPath("/d/f"));
+        assertTrue(igfs.info(new IgfsPath("/d/f")).isFile());
 
         try {
             igfs.mkdirs(new IgfsPath("/d/f"), null);
@@ -770,6 +773,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         catch (IgfsParentNotDirectoryException ignore) {
             // No-op.
         }
+        catch (IgfsException ignore) {
+            // Currently Ok for Hadoop fs:
+            if (!getClass().getSimpleName().startsWith("Hadoop"))
+                throw ignore;
+        }
 
         try {
             igfs.mkdirs(new IgfsPath("/d/f/something/else"), null);
@@ -779,6 +787,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         catch (IgfsParentNotDirectoryException ignore) {
             // No-op.
         }
+        catch (IgfsException ignore) {
+            // Currently Ok for Hadoop fs:
+            if (!getClass().getSimpleName().startsWith("Hadoop"))
+                throw ignore;
+        }
 
         create(igfs, paths(DIR, SUBDIR), null);
 
@@ -1026,8 +1039,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
                 try {
                     is = igfs.open(FILE);
-                }
-                finally {
+                } finally {
                     U.closeQuiet(is);
                 }
 
@@ -1041,12 +1053,84 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings({"ConstantConditions", "EmptyTryBlock", "UnusedDeclaration"})
     public void testCreate() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
         createFile(igfs.asSecondary(), FILE, true, chunk);
 
         checkFile(igfs, igfsSecondary, FILE, chunk);
+
+        try (IgfsOutputStream os = igfs.create(new IgfsPath("/r"), false)) {
+            checkExist(igfs, igfsSecondary, new IgfsPath("/r"));
+            assert igfs.info(new IgfsPath("/r")).isFile();
+        }
+
+        try (IgfsOutputStream os = igfs.create(new IgfsPath("/k/l"), false)) {
+            checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+            assert igfs.info(new IgfsPath("/k/l")).isFile();
+        }
+
+        try {
+            try (IgfsOutputStream os = igfs.create(new IgfsPath("/k/l"), false)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+
+        checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+        assert igfs.info(new IgfsPath("/k/l")).isFile();
+
+        try {
+            try (IgfsOutputStream os = igfs.create(new IgfsPath("/k/l/m"), true)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+        checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m"));
+        checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+        assert igfs.info(new IgfsPath("/k/l")).isFile();
+
+        try {
+            try (IgfsOutputStream os = igfs.create(new IgfsPath("/k/l/m/n/o/p"), true)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+        checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m"));
+        checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+        assert igfs.info(new IgfsPath("/k/l")).isFile();
+
+        igfs.mkdirs(new IgfsPath("/x/y"), null);
+        try {
+            try (IgfsOutputStream os = igfs.create(new IgfsPath("/x/y"), true)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+
+        checkExist(igfs, igfsSecondary, new IgfsPath("/x/y"));
+        assert igfs.info(new IgfsPath("/x/y")).isDirectory();
+
+        try (IgfsOutputStream os = igfs.create(new IgfsPath("/x/y/f"), false)) {
+            assert igfs.info(new IgfsPath("/x/y/f")).isFile();
+        }
+
+        try (IgfsOutputStream os = igfs.create(new IgfsPath("/x/y/z/f"), false)) {
+            assert igfs.info(new IgfsPath("/x/y/z/f")).isFile();
+        }
+
+        try (IgfsOutputStream os = igfs.create(new IgfsPath("/x/y/z/t/f"), false)) {
+            assert igfs.info(new IgfsPath("/x/y/z/t/f")).isFile();
+        }
+
+        try (IgfsOutputStream os = igfs.create(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f"), false)) {
+            assert igfs.info(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f")).isFile();
+        }
     }
 
     /**
@@ -1078,8 +1162,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
                 try {
                     os1 = igfs.create(FILE, true);
                     os2 = igfs.create(FILE, true);
-                }
-                finally {
+                } finally {
                     U.closeQuiet(os1);
                     U.closeQuiet(os2);
                 }
@@ -1141,26 +1224,47 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testCreateDeleteNoClose() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                IgfsOutputStream os = null;
+        IgfsOutputStream os = null;
 
-                try {
-                    os = igfs.create(FILE, true);
+        IgniteUuid id = null;
+
+        try {
+            os = igfs.create(FILE, false);
+
+            id = igfs.context().meta().fileId(FILE);
+
+            assert id != null;
+
+            boolean del = igfs.delete(FILE, false);
+
+            assertTrue(del);
+            assertFalse(igfs.exists(FILE));
+            // The id still exists in meta cache since
+            // it is locked for writing and just moved to TRASH.
+            // Delete worker cannot delete it for that reason:
+            assertTrue(igfs.context().meta().exists(id));
+
+            os.write(chunk);
 
-                    igfs.format();
+            os.close();
+        }
+        finally {
+            U.closeQuiet(os);
+        }
 
-                    os.write(chunk);
+        final IgniteUuid id0 = id;
 
-                    os.close();
+        // Delete worker should delete the file once its output stream is finally closed:
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return !igfs.context().meta().exists(id0);
                 }
-                finally {
-                    U.closeQuiet(os);
+                catch (IgniteCheckedException ice) {
+                    throw new IgniteException(ice);
                 }
-
-                return null;
             }
-        }, IOException.class, "File was concurrently deleted: " + FILE);
+        }, 5_000L);
     }
 
     /**
@@ -1171,29 +1275,47 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testCreateDeleteParentNoClose() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                IgfsOutputStream os = null;
+        IgfsOutputStream os = null;
 
-                try {
-                    os = igfs.create(FILE, true);
+        IgniteUuid id = null;
+
+        try {
+            os = igfs.create(FILE, false);
+
+            id = igfs.context().meta().fileId(FILE);
 
-                    IgniteUuid id = igfs.context().meta().fileId(FILE);
+            assert id != null;
 
-                    igfs.delete(SUBDIR, true); // Since GG-4911 we allow deletes in this case.
+            boolean del = igfs.delete(SUBDIR, true);
 
-                    while (igfs.context().meta().exists(id))
-                        U.sleep(100);
+            assertTrue(del);
+            assertFalse(igfs.exists(FILE));
+            // The id still exists in meta cache since
+            // it is locked for writing and just moved to TRASH.
+            // Delete worker cannot delete it for that reason:
+            assertTrue(igfs.context().meta().exists(id));
 
-                    os.close();
+            os.write(chunk);
+
+            os.close();
+        }
+        finally {
+            U.closeQuiet(os);
+        }
+
+        final IgniteUuid id0 = id;
+
+        // Delete worker should delete the file once its output stream is finally closed:
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return !igfs.context().meta().exists(id0);
                 }
-                finally {
-                    U.closeQuiet(os);
+                catch (IgniteCheckedException ice) {
+                    throw new IgniteException(ice);
                 }
-
-                return null;
             }
-        }, IOException.class, "File was concurrently deleted: " + FILE);
+        }, 5_000L);
     }
 
     /**
@@ -1221,6 +1343,41 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Checks simple write.
+     *
+     * @throws Exception On error.
+     */
+    public void testSimpleWrite() throws Exception {
+        IgfsPath path = new IgfsPath("/file1");
+
+        IgfsOutputStream os = igfs.create(path, 128, true/*overwrite*/, null, 0, 256, null);
+
+        os.write(chunk);
+
+        os.close();
+
+        assert igfs.exists(path);
+        checkFileContent(igfs, path, chunk);
+
+        os = igfs.create(path, 128, true/*overwrite*/, null, 0, 256, null);
+
+        assert igfs.exists(path);
+
+        os.write(chunk);
+
+        assert igfs.exists(path);
+
+        os.write(chunk);
+
+        assert igfs.exists(path);
+
+        os.close();
+
+        assert igfs.exists(path);
+        checkFileContent(igfs, path, chunk, chunk);
+    }
+
+    /**
      * Ensure consistency of data during file creation.
      *
      * @throws Exception If failed.
@@ -1229,17 +1386,17 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         final AtomicInteger ctr = new AtomicInteger();
         final AtomicReference<Exception> err = new AtomicReference<>();
 
-        int threadCnt = 10;
+        final int threadCnt = 10;
 
         multithreaded(new Runnable() {
             @Override public void run() {
                 int idx = ctr.incrementAndGet();
 
-                IgfsPath path = new IgfsPath("/file" + idx);
+                final IgfsPath path = new IgfsPath("/file" + idx);
 
                 try {
                     for (int i = 0; i < REPEAT_CNT; i++) {
-                        IgfsOutputStream os = igfs.create(path, 128, true, null, 0, 256, null);
+                        IgfsOutputStream os = igfs.create(path, 128, true/*overwrite*/, null, 0, 256, null);
 
                         os.write(chunk);
 
@@ -1275,11 +1432,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         igfs.create(FILE, false).close();
 
-        int threadCnt = 5;
+        int threadCnt = 50;
 
         IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
-                while (!stop.get()) {
+                while (!stop.get() && err.get() == null) {
                     IgfsOutputStream os = null;
 
                     try {
@@ -1287,27 +1444,43 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
                         os.write(chunk);
 
-                        U.sleep(50);
-
                         os.close();
 
                         createCtr.incrementAndGet();
                     }
-                    catch (IgniteInterruptedCheckedException | IgniteException ignore) {
-                        try {
-                            U.sleep(10);
-                        }
-                        catch (IgniteInterruptedCheckedException ignored) {
-                            // nO-op.
+                    catch (IgniteException e) {
+                        Throwable[] chain = X.getThrowables(e);
+
+                        Throwable cause = chain[chain.length - 1];
+
+                        if (!e.getMessage().startsWith("Failed to overwrite file (file is opened for writing)")
+                                && (cause == null
+                                    || !cause.getMessage().startsWith("Failed to overwrite file (file is opened for writing)"))) {
+
+                            System.out.println("Failed due to IgniteException exception. Cause:");
+                            cause.printStackTrace(System.out);
+
+                            err.compareAndSet(null, e);
                         }
                     }
                     catch (IOException e) {
-                        // We can ignore concurrent deletion exception since we override the file.
-                        if (!e.getMessage().startsWith("File was concurrently deleted"))
-                            err.compareAndSet(null, e);
+                        err.compareAndSet(null, e);
+
+                        Throwable[] chain = X.getThrowables(e);
+
+                        Throwable cause = chain[chain.length - 1];
+
+                        System.out.println("Failed due to IOException exception. Cause:");
+                        cause.printStackTrace(System.out);
                     }
                     finally {
-                        U.closeQuiet(os);
+                        if (os != null)
+                            try {
+                                os.close();
+                            }
+                            catch (IOException ioe) {
+                                throw new IgniteException(ioe);
+                            }
                     }
                 }
             }
@@ -1315,7 +1488,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         long startTime = U.currentTimeMillis();
 
-        while (createCtr.get() < 50 && U.currentTimeMillis() - startTime < 60 * 1000)
+        while (err.get() == null
+                && createCtr.get() < 500
+                && U.currentTimeMillis() - startTime < 60 * 1000)
             U.sleep(100);
 
         stop.set(true);
@@ -1324,8 +1499,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         awaitFileClose(igfs.asSecondary(), FILE);
 
-        if (err.get() != null)
+        if (err.get() != null) {
+            X.println("Test failed: rethrowing first error: " + err.get());
+
             throw err.get();
+        }
 
         checkFileContent(igfs, FILE, chunk);
     }
@@ -1336,13 +1514,131 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAppend() throws Exception {
+        if (dual)
+            fail("Test fails in DUAL modes, see https://issues.apache.org/jira/browse/IGNITE-1631");
+
         create(igfs, paths(DIR, SUBDIR), null);
 
+        assert igfs.exists(SUBDIR);
+
         createFile(igfs, FILE, true, BLOCK_SIZE, chunk);
 
+        checkFile(igfs, igfsSecondary, FILE, chunk);
+
         appendFile(igfs, FILE, chunk);
 
         checkFile(igfs, igfsSecondary, FILE, chunk, chunk);
+
+        // Test create via append:
+        IgfsPath path2 = FILE2;
+
+        IgfsOutputStream os = null;
+
+        try {
+            os = igfs.append(path2, true/*create*/);
+
+            writeFileChunks(os, chunk);
+        }
+        finally {
+            U.closeQuiet(os);
+
+            awaitFileClose(igfs.asSecondary(), path2);
+        }
+
+        try {
+            os = igfs.append(path2, false/*create*/);
+
+            writeFileChunks(os, chunk);
+        }
+        finally {
+            U.closeQuiet(os);
+
+            awaitFileClose(igfs.asSecondary(), path2);
+        }
+
+        checkFile(igfs, igfsSecondary, path2, chunk, chunk);
+
+        // Negative append (create == false):
+        try {
+            try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/should-not-be-created"), false)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+        checkNotExist(igfs, igfsSecondary, new IgfsPath("/d1"));
+
+        // Positive mkdirs via append:
+        try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/k/l"), true)) {
+            checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+            assert igfs.info(new IgfsPath("/k/l")).isFile();
+        }
+
+        // Negative append (file is immediate parent):
+        try {
+            try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/k/l/m"), true)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+        checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m"));
+        checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+        assert igfs.info(new IgfsPath("/k/l")).isFile();
+
+        // Negative append (file is in the parent chain):
+        try {
+            try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/k/l/m/n/o/p"), true)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+        checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m"));
+        checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+        assert igfs.info(new IgfsPath("/k/l")).isFile();
+
+        // Negative append (target is a directory):
+        igfs.mkdirs(new IgfsPath("/x/y"), null);
+        checkExist(igfs, igfsSecondary, new IgfsPath("/x/y"));
+        assert igfs.info(new IgfsPath("/x/y")).isDirectory();
+        try {
+            try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y"), true)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+
+        // Positive append with create
+        try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/f"), true)) {
+            assert igfs.info(new IgfsPath("/x/y/f")).isFile();
+        }
+
+        // Positive append with create & 1 mkdirs:
+        try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/z/f"), true)) {
+            assert igfs.info(new IgfsPath("/x/y/z/f")).isFile();
+        }
+
+        // Positive append with create & 2 mkdirs:
+        try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/z/t/f"), true)) {
+            assert igfs.info(new IgfsPath("/x/y/z/t/f")).isFile();
+        }
+
+        // Positive mkdirs create & many mkdirs:
+        try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f"), true)) {
+            assert igfs.info(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f")).isFile();
+        }
+
+        // Negative mkdirs via append (create == false):
+        try {
+            try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/d1/d2/d3/f"), false)) {}
+
+            fail("Exception expected");
+        } catch (IgniteException e) {
+            // okay
+        }
+        checkNotExist(igfs, igfsSecondary, new IgfsPath("/d1"));
     }
 
     /**
@@ -1447,26 +1743,44 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         createFile(igfs.asSecondary(), FILE, false);
 
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                IgfsOutputStream os = null;
+        IgfsOutputStream os = null;
+        IgniteUuid id = null;
 
-                try {
-                    os = igfs.append(FILE, false);
+        try {
+            id = igfs.context().meta().fileId(FILE);
 
-                    igfs.format();
+            os = igfs.append(FILE, false);
 
-                    os.write(chunk);
+            boolean del = igfs.delete(FILE, false);
 
-                    os.close();
-                }
-                finally {
-                    U.closeQuiet(os);
-                }
+            assertTrue(del);
+            assertFalse(igfs.exists(FILE));
+            assertTrue(igfs.context().meta().exists(id)); // id still exists in meta cache since
+            // it is locked for writing and just moved to TRASH.
+            // Delete worker cannot delete it for that reason.
 
-                return null;
+            os.write(chunk);
+
+            os.close();
+        }
+        finally {
+            U.closeQuiet(os);
+        }
+
+        assert id != null;
+
+        final IgniteUuid id0 = id;
+
+        // Delete worker should delete the file once its output stream is finally closed:
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return !igfs.context().meta().exists(id0);
+                } catch (IgniteCheckedException ice) {
+                    throw new IgniteException(ice);
+                }
             }
-        }, IOException.class, "File was concurrently deleted: " + FILE);
+        }, 5_000L);
     }
 
     /**
@@ -1479,31 +1793,44 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         createFile(igfs.asSecondary(), FILE, false);
 
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                IgfsOutputStream os = null;
+        IgfsOutputStream os = null;
+        IgniteUuid id = null;
 
-                try {
-                    IgniteUuid id = igfs.context().meta().fileId(FILE);
+        try {
+            id = igfs.context().meta().fileId(FILE);
 
-                    os = igfs.append(FILE, false);
+            os = igfs.append(FILE, false);
 
-                    igfs.delete(SUBDIR, true); // Since GG-4911 we allow deletes in this case.
+            boolean del = igfs.delete(SUBDIR, true); // Since GG-4911 we allow deletes in this case.
 
-                    for (int i = 0; i < 100 && igfs.context().meta().exists(id); i++)
-                        U.sleep(100);
+            assertTrue(del);
+            assertFalse(igfs.exists(FILE));
+            assertTrue(igfs.context().meta().exists(id)); // id still exists in meta cache since
+            // it is locked for writing and just moved to TRASH.
+            // Delete worker cannot delete it for that reason.
 
-                    os.write(chunk);
+            os.write(chunk);
 
-                    os.close();
-                }
-                finally {
-                    U.closeQuiet(os);
-                }
+            os.close();
+        }
+        finally {
+            U.closeQuiet(os);
+        }
 
-                return null;
+        assert id != null;
+
+        final IgniteUuid id0 = id;
+
+        // Delete worker should delete the file once its output stream is finally closed:
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return !igfs.context().meta().exists(id0);
+                } catch (IgniteCheckedException ice) {
+                    throw new IgniteException(ice);
+                }
             }
-        }, IOException.class, "File was concurrently deleted: " + FILE);
+        }, 5_000L);
     }
 
     /**
@@ -1594,11 +1921,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         igfs.create(FILE, false).close();
 
-        int threadCnt = 5;
+        int threadCnt = 50;
 
         IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
-                while (!stop.get()) {
+                while (!stop.get() && err.get() == null) {
                     IgfsOutputStream os = null;
 
                     try {
@@ -1606,25 +1933,31 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
                         os.write(chunk);
 
-                        U.sleep(50);
-
                         os.close();
 
                         chunksCtr.incrementAndGet();
                     }
-                    catch (IgniteInterruptedCheckedException | IgniteException ignore) {
-                        try {
-                            U.sleep(10);
-                        }
-                        catch (IgniteInterruptedCheckedException ignored) {
-                            // nO-op.
-                        }
+                    catch (IgniteException e) {
+                        Throwable[] chain = X.getThrowables(e);
+
+                        Throwable cause = chain[chain.length - 1];
+
+                        if (!e.getMessage().startsWith("Failed to open file (file is opened for writing)")
+                                && (cause == null
+                                || !cause.getMessage().startsWith("Failed to open file (file is opened for writing)")))
+                            err.compareAndSet(null, e);
                     }
                     catch (IOException e) {
                         err.compareAndSet(null, e);
                     }
                     finally {
-                        U.closeQuiet(os);
+                        if (os != null)
+                            try {
+                                os.close();
+                            }
+                            catch (IOException ioe) {
+                                throw new IgniteException(ioe);
+                            }
                     }
                 }
             }
@@ -1632,7 +1965,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         long startTime = U.currentTimeMillis();
 
-        while (chunksCtr.get() < 50 && U.currentTimeMillis() - startTime < 60 * 1000)
+        while (err.get() == null
+                && chunksCtr.get() < 50 && U.currentTimeMillis() - startTime < 60 * 1000)
             U.sleep(100);
 
         stop.set(true);
@@ -1641,8 +1975,11 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         awaitFileClose(igfs.asSecondary(), FILE);
 
-        if (err.get() != null)
+        if (err.get() != null) {
+            X.println("Test failed: rethrowing first error: " + err.get());
+
             throw err.get();
+        }
 
         byte[][] data = new byte[chunksCtr.get()][];
 
@@ -2019,8 +2356,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksCreate() throws Exception {
-        assert false : "https://issues.apache.org/jira/browse/IGNITE-1590";
-
         checkDeadlocksRepeat(5, 2, 2, 2, 0, 0, 0, 0, CREATE_CNT);
     }
 
@@ -2030,8 +2365,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocks() throws Exception {
-        assert false : "https://issues.apache.org/jira/browse/IGNITE-1590";
-
         checkDeadlocksRepeat(5, 2, 2, 2,  RENAME_CNT, DELETE_CNT, UPDATE_CNT, MKDIRS_CNT, CREATE_CNT);
     }
 
@@ -2057,6 +2390,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
             try {
                 checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt,
                     updateCnt, mkdirsCnt, createCnt);
+
+                if (i % 10 == 0)
+                    X.println(" - " + i);
             }
             finally {
                 clear(igfs, igfsSecondary);
@@ -2679,7 +3015,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
                     read = is.read(buf);
 
-                    assert read == chunk.length : "Chunk #" + chunkIdx + " was not read fully.";
+                    assert read == chunk.length : "Chunk #" + chunkIdx + " was not read fully:" +
+                            " read=" + read + ", expected=" + chunk.length;
                     assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + uni.name() + ", chunkIdx=" + chunkIdx +
                         ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']';
 
@@ -2800,19 +3137,57 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         // Clear igfs.
         igfs.format();
 
-        final IgniteFileSystem igfs0 = igfs;
+        int prevDifferentSize = Integer.MAX_VALUE; // Previous different size.
+        int size;
+        int constCnt = 0, totalCnt = 0;
+        final int constThreshold = 20;
+        final long sleepPeriod = 500L;
+        final long totalThreshold = CACHE_EMPTY_TIMEOUT / sleepPeriod;
 
-        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return isEmpty(igfs0);
+        while (true) {
+            size = sumCacheSize(igfs);
+
+            if (size <= 2)
+                return; // Caches are cleared, we're done. (2 because ROOT & TRASH always exist).
+
+            X.println("Sum size: " + size);
+
+            if (size > prevDifferentSize) {
+                X.println("Summary cache size has grown unexpectedly: size=" + size + ", prevSize=" + prevDifferentSize);
+
+                break;
             }
-        }, 10_000L)) {
-            dumpCache("MetaCache" , getMetaCache(igfs));
 
-            dumpCache("DataCache" , getDataCache(igfs));
+            if (totalCnt > totalThreshold) {
+                X.println("Timeout exceeded.");
 
-            assert false;
+                break;
+            }
+
+            if (size == prevDifferentSize) {
+                constCnt++;
+
+                if (constCnt == constThreshold) {
+                    X.println("Summary cache size stays unchanged for too long: size=" + size);
+
+                    break;
+                }
+            } else {
+                constCnt = 0;
+
+                prevDifferentSize = size; // renew;
+            }
+
+            Thread.sleep(sleepPeriod);
+
+            totalCnt++;
         }
+
+        dumpCache("MetaCache" , getMetaCache(igfs));
+
+        dumpCache("DataCache" , getDataCache(igfs));
+
+        fail("Caches are not empty.");
     }
 
     /**
@@ -2831,37 +3206,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * Answers if the given IGFS is empty.
-     *
-     * @param igfs IGFS to operate on.
-     * @return True if IGFS is empty.
+     * Gets summary IGFS cache size.
+     * @param igfs The IGFS to measure.
+     * @return data cache size + meta cache size.
      */
-    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;
+    private static int sumCacheSize(IgniteFileSystem igfs) {
+        return getMetaCache(igfs).size() + getDataCache(igfs).size();
     }
 
     /**
@@ -2880,4 +3230,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         // Clear the filesystem.
         uni.format();
     }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        clear(igfs, igfsSecondary);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
index 6266ab4..84462fd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
@@ -165,7 +165,9 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
     public void testDataStoring() throws Exception {
         for (int i = 0; i < 10; i++) {
             IgfsPath path = new IgfsPath();
-            IgfsFileInfo info = new IgfsFileInfo(200, null, false, null);
+
+            IgfsFileInfo info = new IgfsFileInfo(200, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
+                    false, null);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
 
@@ -246,7 +248,9 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
         for (int i = 0; i < 10; i++) {
             IgfsPath path = new IgfsPath();
-            IgfsFileInfo info = new IgfsFileInfo(blockSize, null, false, null);
+
+            IgfsFileInfo info = new IgfsFileInfo(blockSize, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
+                false, null);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
 
@@ -333,7 +337,10 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
         for (int i = 0; i < 10; i++) {
             IgfsPath path = new IgfsPath();
-            IgfsFileInfo info = new IgfsFileInfo(blockSize, null, false, null);
+
+            IgfsFileInfo info =
+                new IgfsFileInfo(blockSize, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
+                    false, null);
 
             IgfsFileAffinityRange range = new IgfsFileAffinityRange();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/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 4072636..df519ed 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
@@ -26,7 +26,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
 import org.apache.ignite.igfs.IgfsPath;
@@ -46,7 +45,6 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.ROOT_ID;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
 import static org.apache.ignite.testframework.GridTestUtils.assertThrowsInherited;
-import static org.apache.ignite.testframework.GridTestUtils.getFieldValue;
 
 /**
  * {@link IgfsMetaManager} test case.
@@ -143,17 +141,20 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
     public void testUpdateProperties() throws Exception {
         assertEmpty(mgr.directoryListing(ROOT_ID));
 
-        IgfsFileInfo dir = new IgfsFileInfo(true, null);
-        IgfsFileInfo file = new IgfsFileInfo(new IgfsFileInfo(400, null, false, null), 1);
+        assertTrue(mgr.mkdirs(new IgfsPath("/dir"), IgfsImpl.DFLT_DIR_META));
+        assertNotNull(mgr.create(new IgfsPath("/file"), false, false, null, 400, null, false, null));
 
-        assertNull(mgr.putIfAbsent(ROOT_ID, "dir", dir));
-        assertNull(mgr.putIfAbsent(ROOT_ID, "file", file));
+        IgfsListingEntry dirEntry = mgr.directoryListing(ROOT_ID).get("dir");
+        assertNotNull(dirEntry);
+        assertTrue(dirEntry.isDirectory());
+        IgfsFileInfo dir = mgr.info(dirEntry.fileId());
 
-        assertEquals(F.asMap("dir", new IgfsListingEntry(dir), "file", new IgfsListingEntry(file)),
-            mgr.directoryListing(ROOT_ID));
+        IgfsListingEntry fileEntry = mgr.directoryListing(ROOT_ID).get("file");
+        assertNotNull(fileEntry);
+        assertTrue(!fileEntry.isDirectory());
+        IgfsFileInfo file = mgr.info(fileEntry.fileId());
 
-        assertEquals(dir, mgr.info(dir.id()));
-        assertEquals(file, mgr.info(file.id()));
+        assertEquals(2, mgr.directoryListing(ROOT_ID).size());
 
         for (IgniteBiTuple<IgniteUuid, String> tup: Arrays.asList(F.t(dir.id(), "dir"), F.t(file.id(), "file"))) {
             IgniteUuid fileId = tup.get1();
@@ -167,38 +168,63 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
             IgfsFileInfo info = mgr.info(fileId);
 
-            assertNull("Expects empty properties are not stored: " + info, getFieldValue(info, "props"));
-            assertEquals("Expects empty properties are not stored: " + info, Collections.<String, String>emptyMap(),
-                info.properties());
+            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"));
 
-            assertEquals("Unexpected stored properties: " + info, F.asMap(key1, "1"), info.properties());
+            assertEquals("Unexpected stored properties: " + info, "1", info.properties().get(key1));
 
             info = mgr.updateProperties(ROOT_ID, fileId, fileName, F.asMap(key2, "2"));
 
-            assertEquals("Unexpected stored properties: " + info, F.asMap(key1, "1", key2, "2"), info.properties());
+           // 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));
 
-            assertEquals("Unexpected stored properties: " + info, F.asMap(key2, "2"), info.properties());
+            assertEquals("Unexpected stored properties: " + info, "2", info.properties().get(key2));
 
             info = mgr.updateProperties(ROOT_ID, fileId, fileName, F.<String, String>asMap(key2, null));
 
-            assertNull("Expects empty properties are not stored: " + info, getFieldValue(info, "props"));
-            assertEquals("Expects empty properties are not stored: " + info, Collections.<String, String>emptyMap(),
-                info.properties());
+            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.removeIfEmpty(ROOT_ID, "dir", dir.id(), new IgfsPath("/dir"), true);
-        mgr.removeIfEmpty(ROOT_ID, "file", file.id(), new IgfsPath("/file"), true);
+        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")));
     }
 
+    private IgfsFileInfo mkdirsAndGetInfo(String path) throws IgniteCheckedException {
+        IgfsPath p = path(path);
+
+        mgr.mkdirs(p, IgfsImpl.DFLT_DIR_META);
+
+        IgniteUuid id = mgr.fileId(p);
+
+        IgfsFileInfo info = mgr.info(id);
+
+        assert info.isDirectory();
+
+        return info;
+    }
+
+    private IgfsFileInfo createFileAndGetInfo(String path) throws IgniteCheckedException {
+        IgfsPath p = path(path);
+
+        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = mgr.create(p, false, false, null, 400, null, false, null);
+
+        assert t2 != null;
+        assert !t2.get1().isDirectory();
+
+        return t2.get1();
+    }
+
     /**
      * Test file system structure in meta-cache.
      *
@@ -206,31 +232,22 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
      */
     public void testStructure() throws Exception {
         IgfsFileInfo rootInfo = new IgfsFileInfo();
+
         // Test empty structure.
         assertEmpty(mgr.directoryListing(ROOT_ID));
         assertEquals(rootInfo, mgr.info(ROOT_ID));
         assertEquals(F.asMap(ROOT_ID, rootInfo), mgr.infos(Arrays.asList(ROOT_ID)));
 
-        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);
+        // Directories:
+        IgfsFileInfo a = mkdirsAndGetInfo("/a");
+        IgfsFileInfo b = mkdirsAndGetInfo("/a/b");
+        IgfsFileInfo k = mkdirsAndGetInfo("/a/b/k");
+        IgfsFileInfo z = mkdirsAndGetInfo("/a/k");
 
-        // Validate 'add file' operation.
-        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));
-
-        assertEquals(b.id(), mgr.putIfAbsent(a.id(), "b", f3));
-        expectsPutIfAbsentFail(a.id(), "c", f3, "Failed to add file details into cache");
+        // Files:
+        IgfsFileInfo f1 = createFileAndGetInfo("/f1");
+        IgfsFileInfo f2 = createFileAndGetInfo("/a/f2");
+        IgfsFileInfo f3 = createFileAndGetInfo("/a/b/f3");
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -315,25 +332,12 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         mgr.move(path("/a2"), path("/a"));
 
-        // Validate 'remove' operation.
-        for (int i = 0; i < 100; i++) {
-            // One of participants doesn't exist.
-            assertNull(mgr.removeIfEmpty(ROOT_ID, "a", IgniteUuid.randomUuid(), new IgfsPath("/a"), true));
-            assertNull(mgr.removeIfEmpty(IgniteUuid.randomUuid(), "a", IgniteUuid.randomUuid(),
-                new IgfsPath("/" + IgniteUuid.randomUuid() + "/a"), true));
-        }
-
-        expectsRemoveFail(ROOT_ID, "a", a.id(), new IgfsPath("/a"),
-            "Failed to remove file (directory is not empty)");
-        expectsRemoveFail(a.id(), "b", b.id(), new IgfsPath("/a/b"),
-            "Failed to remove file (directory is not empty)");
-        assertNull(mgr.removeIfEmpty(ROOT_ID, "a", f1.id(), new IgfsPath("/a"), true));
-        assertNull(mgr.removeIfEmpty(a.id(), "b", f1.id(), new IgfsPath("/a/b"), true));
+        IgniteUuid del = mgr.softDelete(path("/a/b/f3"), false);
 
-        assertEquals(f3, mgr.removeIfEmpty(b.id(), "f3", f3.id(), new IgfsPath("/a/b/f3"), true));
+        assertEquals(f3.id(), del);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
-            mgr.directoryListing(ROOT_ID));
+                mgr.directoryListing(ROOT_ID));
 
         assertEquals(
             F.asMap("b", new IgfsListingEntry(b),
@@ -342,7 +346,9 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         assertEmpty(mgr.directoryListing(b.id()));
 
-        assertEquals(b, mgr.removeIfEmpty(a.id(), "b", b.id(), new IgfsPath("/a/b"), true));
+        //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);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -362,7 +368,8 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEquals(f2.id(), newF2.id());
         assertNotSame(f2, newF2);
 
-        assertEquals(newF2, mgr.removeIfEmpty(a.id(), "f2", f2.id(), new IgfsPath("/a/f2"), true));
+        del = mgr.softDelete(path("/a/f2"), false);
+        assertEquals(f2.id(), del);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -370,14 +377,16 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEmpty(mgr.directoryListing(a.id()));
         assertEmpty(mgr.directoryListing(b.id()));
 
-        assertEquals(f1, mgr.removeIfEmpty(ROOT_ID, "f1", f1.id(), new IgfsPath("/f1"), true));
+        del = mgr.softDelete(path("/f1"), false);
+        assertEquals(f1.id(), del);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a)), mgr.directoryListing(ROOT_ID));
 
         assertEmpty(mgr.directoryListing(a.id()));
         assertEmpty(mgr.directoryListing(b.id()));
 
-        assertEquals(a, mgr.removeIfEmpty(ROOT_ID, "a", a.id(), new IgfsPath("/a"), true));
+        del = mgr.softDelete(path("/a"), false);
+        assertEquals(a.id(), del);
 
         assertEmpty(mgr.directoryListing(ROOT_ID));
         assertEmpty(mgr.directoryListing(a.id()));
@@ -420,25 +429,6 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * Test expected failures for 'add file' operation.
-     *
-     * @param parentId Parent file ID.
-     * @param fileName New file name in the parent's listing.
-     * @param fileInfo New file initial details.
-     * @param msg Failure message if expected exception was not thrown.
-     */
-    private void expectsPutIfAbsentFail(final IgniteUuid parentId, final String fileName, final IgfsFileInfo fileInfo,
-        @Nullable String msg) {
-        Throwable err = assertThrows(log, new Callable() {
-            @Override public Object call() throws Exception {
-                return mgr.putIfAbsent(parentId, fileName, fileInfo);
-            }
-        }, IgniteCheckedException.class, msg);
-
-        assertTrue("Unexpected cause: " + err.getCause(), err.getCause() instanceof IgfsException);
-    }
-
-    /**
      * Test expected failures for 'move file' operation.
      *
      * @param msg Failure message if expected exception was not thrown.
@@ -454,26 +444,4 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         assertTrue("Unexpected cause: " + err, err instanceof IgfsException);
     }
-
-    /**
-     * Test expected failures for 'remove file' operation.
-     *
-     * @param parentId Parent file ID to remove file from.
-     * @param fileName File name in the parent's listing.
-     * @param fileId File ID to remove.
-     * @param path Removed file path.
-     * @param msg Failure message if expected exception was not thrown.
-     */
-    private void expectsRemoveFail(final IgniteUuid parentId, final String fileName, final IgniteUuid fileId,
-        final IgfsPath path, @Nullable String msg) {
-        Throwable err = assertThrows(log, new Callable() {
-            @Nullable @Override public Object call() throws Exception {
-                mgr.removeIfEmpty(parentId, fileName, fileId, path, true);
-
-                return null;
-            }
-        }, IgniteCheckedException.class, msg);
-
-        assertTrue("Unexpected cause: " + err.getCause(), err.getCause() instanceof IgfsDirectoryNotEmptyException);
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
index 9c4d832..c6853ae 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
@@ -626,17 +626,17 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
     /** @throws Exception If failed. */
     public void testCreateOpenAppend() throws Exception {
         // Error - path points to root directory.
-        assertCreateFails("/", false, "Failed to resolve parent directory");
+        assertCreateFails("/", false, "Failed to create file (path points to an existing directory)");
 
         // Create directories.
         igfs.mkdirs(path("/A/B1/C1"));
 
         // Error - path points to directory.
         for (String path : Arrays.asList("/A", "/A/B1", "/A/B1/C1")) {
-            assertCreateFails(path, false, "Failed to create file (file already exists)");
-            assertCreateFails(path, true, "Failed to create file (path points to a directory)");
-            assertAppendFails(path, false, "Failed to open file (not a file)");
-            assertAppendFails(path, true, "Failed to open file (not a file)");
+            assertCreateFails(path, false, "Failed to create file (path points to an existing directory)");
+            assertCreateFails(path, true, "Failed to create file (path points to an existing directory)");
+            assertAppendFails(path, false, "Failed to open file (path points to an existing directory)");
+            assertAppendFails(path, true, "Failed to open file (path points to an existing directory)");
             assertOpenFails(path, "Failed to open file (not a file)");
         }
 
@@ -653,7 +653,7 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
             assertEquals(text1, create(path, false, text1));
 
             // Error - file already exists.
-            assertCreateFails(path, false, "Failed to create file (file already exists)");
+            assertCreateFails(path, false, "Failed to create file (file already exists and overwrite flag is false)");
 
             // Overwrite existent.
             assertEquals(text2, create(path, true, text2));

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index 954a011..cc5e1ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -66,6 +66,12 @@ public class IgniteIgfsTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite FS Test Suite For Platform Independent Tests");
 
+        suite.addTest(new TestSuite(IgfsPrimarySelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryOffheapTieredSelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryOffheapValuesSelfTest.class));
+        suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class));
+        suite.addTest(new TestSuite(IgfsDualAsyncSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsSizeSelfTest.class));
         suite.addTest(new TestSuite(IgfsAttributesSelfTest.class));
         suite.addTest(new TestSuite(IgfsFileInfoSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
index efe95be..ea65464 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
@@ -64,11 +64,6 @@ public abstract class Hadoop1DualAbstractTest extends IgfsDualAbstractSelfTest {
         super(mode);
     }
 
-    /** {@inheritDoc} */
-    @Override public void testMkdirs() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1620");
-    }
-
     /**
      * Creates secondary filesystems.
      * @return IgfsSecondaryFileSystem

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
index c938571..b3afd22 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
@@ -897,7 +897,7 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         os.close();
 
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
+        GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return fs.create(new Path(fsHome, dir), EnumSet.of(CreateFlag.APPEND),
                     Options.CreateOpts.perms(FsPermission.getDefault()));
@@ -919,8 +919,7 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
             Options.CreateOpts.perms(FsPermission.getDefault()));
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
+            @Override public Object call() throws Exception {
                 return fs.create(file, EnumSet.of(CreateFlag.APPEND),
                     Options.CreateOpts.perms(FsPermission.getDefault()));
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
index 2626ebb..d368955 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
@@ -1096,7 +1096,7 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         os.close();
 
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
+        GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return fs.append(new Path(fsHome, dir), 1024);
             }


[05/10] ignite git commit: IGNITE-825: Disabled test remastered and restored.

Posted by sb...@apache.org.
IGNITE-825: Disabled test remastered and restored.


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

Branch: refs/heads/ignite-1093-2
Commit: 52a733f6bdc463b1b5527b29a0817a85bf1ed7bd
Parents: 49a5cc5
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Oct 15 13:12:52 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Oct 15 13:12:52 2015 +0300

----------------------------------------------------------------------
 .../HadoopIgfs20FileSystemAbstractSelfTest.java | 22 ++++++++++++++------
 1 file changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/52a733f6/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
index b3afd22..9c301c9 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
@@ -1124,10 +1124,10 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
         is.close();
     }
 
-    /** @throws Exception If failed. */
+    /**
+     * @throws Exception If failed.
+     */
     public void testRenameDirectoryIfDstPathExists() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-825"); 
-        
         Path fsHome = new Path(primaryFsUri);
         Path srcDir = new Path(fsHome, "/tmp/");
         Path dstDir = new Path(fsHome, "/tmpNew/");
@@ -1142,11 +1142,21 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         os.close();
 
-        fs.rename(srcDir, dstDir);
+        try {
+            fs.rename(srcDir, dstDir);
+
+            fail("FileAlreadyExistsException expected.");
+        }
+        catch (FileAlreadyExistsException ignore) {
+            // No-op.
+        }
 
+        // Check all the files stay unchanged:
         assertPathExists(fs, dstDir);
-        assertPathExists(fs, new Path(fsHome, "/tmpNew/tmp"));
-        assertPathExists(fs, new Path(fsHome, "/tmpNew/tmp/file1"));
+        assertPathExists(fs, new Path(dstDir, "file2"));
+
+        assertPathExists(fs, srcDir);
+        assertPathExists(fs, new Path(srcDir, "file1"));
     }
 
     /** @throws Exception If failed. */


[07/10] ignite git commit: IGNITE-1168 Fixed typo.

Posted by sb...@apache.org.
IGNITE-1168 Fixed typo.


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

Branch: refs/heads/ignite-1093-2
Commit: 3a29b97371946e50da8d6352e39428673e9c74ca
Parents: 541ba40
Author: Andrey <an...@gridgain.com>
Authored: Thu Oct 15 17:53:16 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Thu Oct 15 17:53:16 2015 +0700

----------------------------------------------------------------------
 .../processors/rest/JettyRestProcessorAbstractSelfTest.java      | 4 ++--
 .../rest/protocols/http/jetty/GridJettyRestHandler.java          | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3a29b973/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index bb6e67e..c413bbd 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -1212,7 +1212,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         params.put("cmd", GridRestCommand.EXECUTE_SCAN_QUERY.key());
         params.put("pageSize", "10");
         params.put("cacheName", "person");
-        params.put("classname", ScanFilter.class.getName());
+        params.put("className", ScanFilter.class.getName());
 
         String ret = content(params);
 
@@ -1236,7 +1236,7 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         params.put("cmd", GridRestCommand.EXECUTE_SCAN_QUERY.key());
         params.put("pageSize", "10");
         params.put("cacheName", "person");
-        params.put("classname", ScanFilter.class.getName() + 1);
+        params.put("className", ScanFilter.class.getName() + 1);
 
         String ret = content(params);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3a29b973/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
index 48a60a9..5f2c4ba 100644
--- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
+++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
@@ -600,7 +600,7 @@ public class GridJettyRestHandler extends AbstractHandler {
 
                 restReq0.cacheName((String)params.get("cacheName"));
 
-                restReq0.className((String)params.get("classname"));
+                restReq0.className((String)params.get("className"));
 
                 restReq0.queryType(RestQueryRequest.QueryType.SCAN);
 


[10/10] ignite git commit: 1093

Posted by sb...@apache.org.
1093


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

Branch: refs/heads/ignite-1093-2
Commit: 4907c5482f378110d0bd730c4288f2872d99933d
Parents: 4c9792e
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Oct 15 16:51:55 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Oct 15 16:51:55 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      | 26 +++++++-------------
 1 file changed, 9 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4907c548/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 00c00c5..4d95894 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -143,9 +143,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /** */
     private final Queue<Runnable> rebalancingQueue = new ConcurrentLinkedDeque8<>();
 
-    /** */
-    private final AtomicReference<Integer> rebalancingQueueOwning = new AtomicReference<>(0);
-
     /**
      * Partition map futures.
      * This set also contains already completed exchange futures to address race conditions when coordinator
@@ -1160,6 +1157,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
             int cnt = 0;
 
+            IgniteInternalFuture asyncStartFut = null;
+
             while (!isCancelled()) {
                 GridDhtPartitionsExchangeFuture exchFut = null;
 
@@ -1326,9 +1325,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             }
                         }
 
-                        while (!rebalancingQueueOwning.compareAndSet(0, 1)) {
-                            U.sleep(10); // Wait for thread stop.
-                        }
+                        if (asyncStartFut != null)
+                            asyncStartFut.get(); // Wait for thread stop.
 
                         if (marsR != null || !rebalancingQueue.isEmpty()) {
                             if (futQ.isEmpty()) {
@@ -1337,6 +1335,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                 if (marsR != null)
                                     marsR.run();//Marshaller cache rebalancing launches in sync way.
 
+                                final GridFutureAdapter fut = new GridFutureAdapter();
+
+                                asyncStartFut = fut;
+
                                 cctx.kernalContext().closure().callLocalSafe(new GPC<Boolean>() {
                                     @Override public Boolean call() {
                                         try {
@@ -1350,27 +1352,17 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                             }
                                         }
                                         finally {
-                                            boolean res = rebalancingQueueOwning.compareAndSet(1, 0);
-
-                                            assert res;
+                                            fut.onDone();
                                         }
                                     }
                                 }, /*system pool*/ true);
                             }
                             else {
                                 U.log(log, "Obsolete exchange, skipping rebalancing [top=" + exchFut.topologyVersion() + "]");
-
-                                boolean res = rebalancingQueueOwning.compareAndSet(1, 0);
-
-                                assert res;
                             }
                         }
                         else {
                             U.log(log, "Nothing scheduled, skipping rebalancing [top=" + exchFut.topologyVersion() + "]");
-
-                            boolean res = rebalancingQueueOwning.compareAndSet(1, 0);
-
-                            assert res;
                         }
                     }
                 }


[02/10] ignite git commit: IGNITE-1590: Reworked create and append operations to match overall design.

Posted by sb...@apache.org.
IGNITE-1590: Reworked create and append operations to match overall design.


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

Branch: refs/heads/ignite-1093-2
Commit: 962fcce3acbecd028c4787a6255fedcdcbdf9db1
Parents: 6844370
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Oct 14 15:59:57 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Oct 14 15:59:57 2015 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDataManager.java        |   2 -
 .../processors/igfs/IgfsDeleteWorker.java       | 102 ++-
 .../internal/processors/igfs/IgfsImpl.java      | 164 +---
 .../processors/igfs/IgfsMetaManager.java        | 897 ++++++++++++-------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   2 +
 .../internal/processors/igfs/IgfsUtils.java     |  23 +
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   6 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   | 639 ++++++++++---
 .../igfs/IgfsDataManagerSelfTest.java           |  13 +-
 .../igfs/IgfsMetaManagerSelfTest.java           | 170 ++--
 .../processors/igfs/IgfsProcessorSelfTest.java  |  12 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   6 +
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   5 -
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   2 +-
 15 files changed, 1289 insertions(+), 759 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index b1b51f9..125d728 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -580,8 +580,6 @@ public class IgfsDataManager extends IgfsManager {
      * @return Delete future that will be completed when file is actually erased.
      */
     public IgniteInternalFuture<Object> delete(IgfsFileInfo fileInfo) {
-        //assert validTxState(any); // Allow this method call for any transaction state.
-
         if (!fileInfo.isFile()) {
             if (log.isDebugEnabled())
                 log.debug("Cannot delete content of not-data file: " + fileInfo);

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
index 98672e8..95a6a5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
@@ -27,12 +27,10 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
-import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.LT;
@@ -62,9 +60,6 @@ public class IgfsDeleteWorker extends IgfsThread {
     /** Data manager. */
     private final IgfsDataManager data;
 
-    /** Event manager. */
-    private final GridEventStorageManager evts;
-
     /** Logger. */
     private final IgniteLogger log;
 
@@ -96,8 +91,6 @@ public class IgfsDeleteWorker extends IgfsThread {
         meta = igfsCtx.meta();
         data = igfsCtx.data();
 
-        evts = igfsCtx.kernalContext().event();
-
         String igfsName = igfsCtx.igfs().name();
 
         topic = F.isEmpty(igfsName) ? TOPIC_IGFS : TOPIC_IGFS.topic(igfsName);
@@ -147,6 +140,9 @@ public class IgfsDeleteWorker extends IgfsThread {
         }
     }
 
+    /**
+     * Cancels the worker.
+     */
     void cancel() {
         cancelled = true;
 
@@ -218,7 +214,8 @@ public class IgfsDeleteWorker extends IgfsThread {
 
             if (info != null) {
                 if (info.isDirectory()) {
-                    deleteDirectory(TRASH_ID, id);
+                    if (!deleteDirectoryContents(TRASH_ID, id))
+                        return false;
 
                     if (meta.delete(TRASH_ID, name, id))
                         return true;
@@ -226,19 +223,22 @@ public class IgfsDeleteWorker extends IgfsThread {
                 else {
                     assert info.isFile();
 
+                    // Lock the file with special lock Id to prevent concurrent writing:
+                    IgfsFileInfo lockedInfo = meta.lock(id, true);
+
+                    if (lockedInfo == null)
+                        return false; // File is locked, we cannot delete it.
+
+                    assert id.equals(lockedInfo.id());
+
                     // Delete file content first.
                     // In case this node crashes, other node will re-delete the file.
-                    data.delete(info).get();
+                    data.delete(lockedInfo).get();
 
                     boolean ret = meta.delete(TRASH_ID, name, id);
 
-                    if (evts.isRecordable(EVT_IGFS_FILE_PURGED)) {
-                        if (info.path() != null)
-                            evts.record(new IgfsEvent(info.path(),
-                                igfsCtx.kernalContext().discovery().localNode(), EVT_IGFS_FILE_PURGED));
-                        else
-                            LT.warn(log, null, "Removing file without path info: " + info);
-                    }
+                    if (info.path() != null)
+                        IgfsUtils.sendEvents(igfsCtx.kernalContext(), info.path(), EVT_IGFS_FILE_PURGED);
 
                     return ret;
                 }
@@ -253,9 +253,10 @@ public class IgfsDeleteWorker extends IgfsThread {
      *
      * @param parentId Parent ID.
      * @param id Entry id.
+     * @return true iff all the items in the directory were deleted (directory is seen to be empty).
      * @throws IgniteCheckedException If delete failed for some reason.
      */
-    private void deleteDirectory(IgniteUuid parentId, IgniteUuid id) throws IgniteCheckedException {
+    private boolean deleteDirectoryContents(IgniteUuid parentId, final IgniteUuid id) throws IgniteCheckedException {
         assert parentId != null;
         assert id != null;
 
@@ -265,47 +266,50 @@ public class IgfsDeleteWorker extends IgfsThread {
             if (info != null) {
                 assert info.isDirectory();
 
-                Map<String, IgfsListingEntry> listing = info.listing();
+                final Map<String, IgfsListingEntry> listing = info.listing();
 
                 if (listing.isEmpty())
-                    return; // Directory is empty.
+                    return true; // Directory is empty.
 
-                Map<String, IgfsListingEntry> delListing;
+                final Map<String, IgfsListingEntry> delListing = new HashMap<>(MAX_DELETE_BATCH, 1.0f);
 
-                if (listing.size() <= MAX_DELETE_BATCH)
-                    delListing = listing;
-                else {
-                    delListing = new HashMap<>(MAX_DELETE_BATCH, 1.0f);
+                final GridCompoundFuture<Object, ?> fut = new GridCompoundFuture<>();
 
-                    int i = 0;
+                int failedFiles = 0;
 
-                    for (Map.Entry<String, IgfsListingEntry> entry : listing.entrySet()) {
-                        delListing.put(entry.getKey(), entry.getValue());
+                for (final Map.Entry<String, IgfsListingEntry> entry : listing.entrySet()) {
+                    if (cancelled)
+                        return false;
 
-                        if (++i == MAX_DELETE_BATCH)
-                            break;
+                    if (entry.getValue().isDirectory()) {
+                        if (deleteDirectoryContents(id, entry.getValue().fileId())) // *** Recursive call.
+                            delListing.put(entry.getKey(), entry.getValue());
+                        else
+                            failedFiles++;
                     }
-                }
+                    else {
+                        IgfsFileInfo fileInfo = meta.info(entry.getValue().fileId());
 
-                GridCompoundFuture<Object, ?> fut = new GridCompoundFuture<>();
+                        if (fileInfo != null) {
+                            assert fileInfo.isFile();
 
-                // Delegate to child folders.
-                for (IgfsListingEntry entry : delListing.values()) {
-                    if (!cancelled) {
-                        if (entry.isDirectory())
-                            deleteDirectory(id, entry.fileId());
-                        else {
-                            IgfsFileInfo fileInfo = meta.info(entry.fileId());
+                            IgfsFileInfo lockedInfo = meta.lock(fileInfo.id(), true);
+
+                            if (lockedInfo == null)
+                                // File is already locked:
+                                failedFiles++;
+                            else {
+                                assert IgfsMetaManager.DELETE_LOCK_ID.equals(lockedInfo.lockId());
 
-                            if (fileInfo != null) {
-                                assert fileInfo.isFile();
+                                fut.add(data.delete(lockedInfo));
 
-                                fut.add(data.delete(fileInfo));
+                                delListing.put(entry.getKey(), entry.getValue());
                             }
                         }
                     }
-                    else
-                        return;
+
+                    if (delListing.size() == MAX_DELETE_BATCH)
+                        break;
                 }
 
                 fut.markInitialized();
@@ -318,17 +322,21 @@ public class IgfsDeleteWorker extends IgfsThread {
                     // This future can be cancelled only due to IGFS shutdown.
                     cancelled = true;
 
-                    return;
+                    return false;
                 }
 
                 // Actual delete of folder content.
                 Collection<IgniteUuid> delIds = meta.delete(id, delListing);
 
-                if (delListing == listing && delListing.size() == delIds.size())
-                    break; // All entries were deleted.
+                if (listing.size() == delIds.size())
+                    return true; // All entries were deleted.
+
+                if (listing.size() == delListing.size() + failedFiles)
+                    // All the files were tried, no reason to continue the loop:
+                    return false;
             }
             else
-                break; // Entry was deleted concurrently.
+                return true; // Directory entry was deleted concurrently.
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/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 fa3a955..0d5cda3 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
@@ -60,7 +60,6 @@ import org.apache.ignite.igfs.IgfsMetrics;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsOutputStream;
 import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.igfs.IgfsPathAlreadyExistsException;
 import org.apache.ignite.igfs.IgfsPathIsDirectoryException;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
 import org.apache.ignite.igfs.IgfsPathSummary;
@@ -97,7 +96,6 @@ import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_RENAMED;
 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_CREATED;
 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;
@@ -112,7 +110,6 @@ import static org.apache.ignite.igfs.IgfsMode.PROXY;
 import static org.apache.ignite.internal.GridTopic.TOPIC_IGFS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS;
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.ROOT_ID;
-import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.TRASH_ID;
 
 /**
  * Cache-based IGFS implementation.
@@ -122,7 +119,7 @@ public final class IgfsImpl implements IgfsEx {
     private static final String PERMISSION_DFLT_VAL = "0777";
 
     /** Default directory metadata. */
-    private static final Map<String, String> DFLT_DIR_META = F.asMap(PROP_PERMISSION, PERMISSION_DFLT_VAL);
+    static final Map<String, String> DFLT_DIR_META = F.asMap(PROP_PERMISSION, PERMISSION_DFLT_VAL);
 
     /** Handshake message. */
     private final IgfsPaths secondaryPaths;
@@ -740,14 +737,9 @@ public final class IgfsImpl implements IgfsEx {
                 }
 
                 // Record event if needed.
-                if (res && desc != null) {
-                    if (desc.isFile) {
-                        if (evts.isRecordable(EVT_IGFS_FILE_DELETED))
-                            evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_DELETED));
-                    }
-                    else if (evts.isRecordable(EVT_IGFS_DIR_DELETED))
-                        evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_DIR_DELETED));
-                }
+                if (res && desc != null)
+                    IgfsUtils.sendEvents(igfsCtx.kernalContext(), path,
+                            desc.isFile ? EVT_IGFS_FILE_DELETED : EVT_IGFS_DIR_DELETED);
 
                 return res;
             }
@@ -928,8 +920,7 @@ public final class IgfsImpl implements IgfsEx {
                     IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, desc.info(),
                         cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, desc.reader(), metrics);
 
-                    if (evts.isRecordable(EVT_IGFS_FILE_OPENED_READ))
-                        evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_OPENED_READ));
+                    IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ);
 
                     return os;
                 }
@@ -949,8 +940,7 @@ public final class IgfsImpl implements IgfsEx {
                 IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, info,
                     cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, null, metrics);
 
-                if (evts.isRecordable(EVT_IGFS_FILE_OPENED_READ))
-                    evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_OPENED_READ));
+                IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ);
 
                 return os;
             }
@@ -1004,7 +994,7 @@ public final class IgfsImpl implements IgfsEx {
                     log.debug("Open file for writing [path=" + path + ", bufSize=" + bufSize + ", overwrite=" +
                         overwrite + ", props=" + props + ']');
 
-                IgfsMode mode = resolveMode(path);
+                final IgfsMode mode = resolveMode(path);
 
                 IgfsFileWorkerBatch batch;
 
@@ -1021,71 +1011,28 @@ public final class IgfsImpl implements IgfsEx {
                     IgfsEventAwareOutputStream os = new IgfsEventAwareOutputStream(path, desc.info(), desc.parentId(),
                         bufSize == 0 ? cfg.getStreamBufferSize() : bufSize, mode, batch);
 
-                    if (evts.isRecordable(EVT_IGFS_FILE_OPENED_WRITE))
-                        evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_OPENED_WRITE));
+                    IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_WRITE);
 
                     return os;
                 }
 
-                // Re-create parents when working in PRIMARY mode. In DUAL mode this is done by MetaManager.
-                IgfsPath parent = path.parent();
-
-                // Create missing parent directories if necessary.
-                if (parent != null)
-                    mkdirs(parent, props);
-
-                List<IgniteUuid> ids = meta.fileIds(path);
-
-                // Resolve parent ID for file.
-                IgniteUuid parentId = ids.size() >= 2 ? ids.get(ids.size() - 2) : null;
-
-                if (parentId == null)
-                    throw new IgfsPathNotFoundException("Failed to resolve parent directory: " + parent);
-
-                String fileName = path.name();
-
-                // Constructs new file info.
-                IgfsFileInfo info = new IgfsFileInfo(cfg.getBlockSize(), affKey, evictExclude(path, true), props);
-
-                // Add new file into tree structure.
-                while (true) {
-                    IgniteUuid oldId = meta.putIfAbsent(parentId, fileName, info);
-
-                    if (oldId == null)
-                        break;
-
-                    if (!overwrite)
-                        throw new IgfsPathAlreadyExistsException("Failed to create file (file already exists): " +
-                            path);
-
-                    IgfsFileInfo oldInfo = meta.info(oldId);
-
-                    assert oldInfo != null;
-
-                    if (oldInfo.isDirectory())
-                        throw new IgfsPathAlreadyExistsException("Failed to create file (path points to a " +
-                            "directory): " + path);
+                final Map<String, String> dirProps, fileProps;
 
-                    // Remove old file from the tree.
-                    // Only one file is deleted, so we use internal data streamer.
-                    deleteFile(path, new FileDescriptor(parentId, fileName, oldId, oldInfo.isFile()), false);
+                if (props == null) {
+                    dirProps = DFLT_DIR_META;
 
-                    if (evts.isRecordable(EVT_IGFS_FILE_DELETED))
-                        evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_DELETED));
+                    fileProps = null;
                 }
+                else
+                    dirProps = fileProps = new HashMap<>(props);
 
-                if (evts.isRecordable(EVT_IGFS_FILE_CREATED))
-                    evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_CREATED));
+                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(path, false/*append*/, overwrite, dirProps,
+                    cfg.getBlockSize(), affKey, evictExclude(path, true), fileProps);
 
-                info = meta.lock(info.id());
+                assert t2 != null;
 
-                IgfsEventAwareOutputStream os = new IgfsEventAwareOutputStream(path, info, parentId,
+                return new IgfsEventAwareOutputStream(path, t2.get1(), t2.get2(),
                     bufSize == 0 ? cfg.getStreamBufferSize() : bufSize, mode, null);
-
-                if (evts.isRecordable(EVT_IGFS_FILE_OPENED_WRITE))
-                    evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_OPENED_WRITE));
-
-                return os;
             }
         });
     }
@@ -1107,7 +1054,7 @@ public final class IgfsImpl implements IgfsEx {
                     log.debug("Open file for appending [path=" + path + ", bufSize=" + bufSize + ", create=" + create +
                         ", props=" + props + ']');
 
-                IgfsMode mode = resolveMode(path);
+                final IgfsMode mode = resolveMode(path);
 
                 IgfsFileWorkerBatch batch;
 
@@ -1124,46 +1071,39 @@ public final class IgfsImpl implements IgfsEx {
                         bufSize == 0 ? cfg.getStreamBufferSize() : bufSize, mode, batch);
                 }
 
-                List<IgniteUuid> ids = meta.fileIds(path);
+                final List<IgniteUuid> ids = meta.fileIds(path);
 
-                IgfsFileInfo info = meta.info(ids.get(ids.size() - 1));
+                final IgniteUuid id = ids.get(ids.size() - 1);
 
-                // Resolve parent ID for the file.
-                IgniteUuid parentId = ids.size() >= 2 ? ids.get(ids.size() - 2) : null;
-
-                if (info == null) {
+                if (id == null) {
                     if (!create) {
                         checkConflictWithPrimary(path);
 
                         throw new IgfsPathNotFoundException("File not found: " + path);
                     }
+                }
 
-                    if (parentId == null)
-                        throw new IgfsPathNotFoundException("Failed to resolve parent directory: " + path.parent());
-
-                    info = new IgfsFileInfo(cfg.getBlockSize(), /**affinity key*/null, evictExclude(path, true), props);
+                // Prevent attempt to append to ROOT in early stage:
+                if (ids.size() == 1)
+                    throw new IgfsPathIsDirectoryException("Failed to open file (not a file): " + path);
 
-                    IgniteUuid oldId = meta.putIfAbsent(parentId, path.name(), info);
+                final Map<String, String> dirProps, fileProps;
 
-                    if (oldId != null)
-                        info = meta.info(oldId);
+                if (props == null) {
+                    dirProps = DFLT_DIR_META;
 
-                    if (evts.isRecordable(EVT_IGFS_FILE_CREATED))
-                        evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_CREATED));
+                    fileProps = null;
                 }
+                else
+                    dirProps = fileProps = new HashMap<>(props);
 
-                assert info != null;
-
-                if (!info.isFile())
-                    throw new IgfsPathIsDirectoryException("Failed to open file (not a file): " + path);
-
-                info = meta.lock(info.id());
+                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = meta.create(path, true/*append*/, false/*overwrite*/,
+                    dirProps, cfg.getBlockSize(), null/*affKey*/, evictExclude(path, true), fileProps);
 
-                if (evts.isRecordable(EVT_IGFS_FILE_OPENED_WRITE))
-                    evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_OPENED_WRITE));
+                assert t2 != null;
 
-                return new IgfsEventAwareOutputStream(path, info, parentId, bufSize == 0 ?
-                    cfg.getStreamBufferSize() : bufSize, mode, null);
+                return new IgfsEventAwareOutputStream(path, t2.get1(), t2.get2(),
+                        bufSize == 0 ? cfg.getStreamBufferSize() : bufSize, mode, null);
             }
         });
     }
@@ -1451,30 +1391,6 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /**
-     * Remove file from the file system (structure and data).
-     *
-     * @param path Path of the deleted file.
-     * @param desc Detailed file descriptor to remove.
-     * @param rmvLocked Whether to remove this entry in case it is has explicit lock.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void deleteFile(IgfsPath path, FileDescriptor desc, boolean rmvLocked) throws IgniteCheckedException {
-        IgniteUuid parentId = desc.parentId;
-        IgniteUuid fileId = desc.fileId;
-
-        if (parentId == null || ROOT_ID.equals(fileId)) {
-            assert parentId == null && ROOT_ID.equals(fileId) : "Invalid file descriptor: " + desc;
-
-            return; // Never remove the root directory!
-        }
-
-        if (TRASH_ID.equals(fileId))
-            return; // Never remove trash directory.
-
-        meta.removeIfEmpty(parentId, desc.fileName, fileId, path, rmvLocked);
-    }
-
-    /**
      * Check whether IGFS with the same name exists among provided attributes.
      *
      * @param attrs Attributes.
@@ -2005,13 +1921,13 @@ public final class IgfsImpl implements IgfsEx {
     /**
      * Perform IGFS operation in safe context.
      *
-     * @param action Action.
+     * @param act Action.
      * @return Result.
      */
-    private <T> T safeOp(Callable<T> action) {
+    private <T> T safeOp(Callable<T> act) {
         if (enterBusy()) {
             try {
-                return action.call();
+                return act.call();
             }
             catch (Exception e) {
                 throw IgfsUtils.toIgfsException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/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 927067a..c016e46 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
@@ -37,8 +37,11 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.UUID;
 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;
 import org.apache.ignite.IgniteException;
@@ -46,6 +49,7 @@ import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.FileSystemConfiguration;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.igfs.IgfsConcurrentModificationException;
 import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException;
@@ -73,9 +77,10 @@ import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
@@ -83,9 +88,8 @@ import org.jetbrains.annotations.Nullable;
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_CREATED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_RENAMED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CREATED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_PURGED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_RENAMED;
+import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.ROOT_ID;
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.TRASH_ID;
 import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.builder;
@@ -97,6 +101,9 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  */
 @SuppressWarnings("all")
 public class IgfsMetaManager extends IgfsManager {
+    /** Lock Id used to lock files being deleted from TRASH. This is a global constant. */
+    static final IgniteUuid DELETE_LOCK_ID = new IgniteUuid(new UUID(0L, 0L), 0L);
+
     /** Comparator for Id sorting. */
     private static final Comparator<IgniteUuid> PATH_ID_SORTING_COMPARATOR
             = new Comparator<IgniteUuid>() {
@@ -295,7 +302,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File ID.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private IgniteUuid fileId(IgniteUuid parentId, String fileName, boolean skipTx) throws IgniteCheckedException {
+    @Nullable private IgniteUuid fileId(IgniteUuid parentId, String fileName, boolean skipTx)
+        throws IgniteCheckedException {
         IgfsListingEntry entry = directoryListing(parentId, skipTx).get(fileName);
 
         if (entry == null) {
@@ -464,9 +472,9 @@ public class IgfsMetaManager extends IgfsManager {
      *
      * @param fileId File ID to lock.
      * @return Locked file info or {@code null} if file cannot be locked or doesn't exist.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteCheckedException If the file with such id does not exist, or on another failure.
      */
-    public IgfsFileInfo lock(IgniteUuid fileId) throws IgniteCheckedException {
+    public @Nullable IgfsFileInfo lock(IgniteUuid fileId, boolean isDeleteLock) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 assert validTxState(false);
@@ -479,14 +487,19 @@ public class IgfsMetaManager extends IgfsManager {
                     IgfsFileInfo oldInfo = info(fileId);
 
                     if (oldInfo == null)
-                        throw new IgniteCheckedException("Failed to lock file (file not found): " + fileId);
+                        return null;
 
-                    IgfsFileInfo newInfo = lockInfo(oldInfo);
+                    if (oldInfo.lockId() != null)
+                        return null; // The file is already locked, we cannot lock it.
 
-                    boolean put = metaCache.put(fileId, newInfo);
+                    IgfsFileInfo newInfo = lockInfo(oldInfo, isDeleteLock);
+
+                    boolean put = metaCache.replace(fileId, oldInfo, newInfo);
 
                     assert put : "Value was not stored in cache [fileId=" + fileId + ", newInfo=" + newInfo + ']';
 
+                    assert newInfo.id().equals(oldInfo.id()); // Same id.
+
                     tx.commit();
 
                     return newInfo;
@@ -510,26 +523,26 @@ public class IgfsMetaManager extends IgfsManager {
      * Set lock on file info.
      *
      * @param info File info.
-     * @return New file info with lock set.
+     * @return New file info with lock set, or null if the info passed in is already locked.
      * @throws IgniteCheckedException In case lock is already set on that file.
      */
-    public IgfsFileInfo lockInfo(IgfsFileInfo info) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                assert info != null;
+    private static @Nullable IgfsFileInfo lockInfo(IgfsFileInfo info, boolean isDeleteLock) {
+         assert info != null;
 
-                if (info.lockId() != null)
-                    throw new IgniteCheckedException("Failed to lock file (file is being concurrently written) [fileId=" +
-                        info.id() + ", lockId=" + info.lockId() + ']');
+         if (info.lockId() != null)
+             return null; // Null return value indicates that the file is already locked.
 
-                return new IgfsFileInfo(info, IgniteUuid.randomUuid(), info.modificationTime());
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to get lock info because Grid is stopping: " + info);
+         return new IgfsFileInfo(info, composeLockId(isDeleteLock), info.modificationTime());
+    }
+
+    /**
+     * Gets a new lock id.
+     *
+     * @param isDeleteLock if this is special delete lock.
+     * @return The new lock id.
+     */
+    private static IgniteUuid composeLockId(boolean isDeleteLock) {
+        return isDeleteLock ? DELETE_LOCK_ID : IgniteUuid.randomUuid();
     }
 
     /**
@@ -556,23 +569,28 @@ public class IgfsMetaManager extends IgfsManager {
                 try {
                     IgfsUtils.doInTransactionWithRetries(metaCache, new IgniteOutClosureX<Void>() {
                         @Override public Void applyx() throws IgniteCheckedException {
+                            assert validTxState(true);
+
                             IgniteUuid fileId = info.id();
 
                             // Lock file ID for this transaction.
                             IgfsFileInfo oldInfo = info(fileId);
 
                             if (oldInfo == null)
-                                throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not found): " + fileId));
+                                throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not " +
+                                    "found): " + fileId));
 
                             if (!info.lockId().equals(oldInfo.lockId()))
-                                throw new IgniteCheckedException("Failed to unlock file (inconsistent file lock ID) [fileId=" + fileId +
-                                    ", lockId=" + info.lockId() + ", actualLockId=" + oldInfo.lockId() + ']');
+                                throw new IgniteCheckedException("Failed to unlock file (inconsistent file lock ID) " +
+                                    "[fileId=" + fileId + ", lockId=" + info.lockId() + ", actualLockId=" +
+                                    oldInfo.lockId() + ']');
 
                             IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, null, modificationTime);
 
                             boolean put = metaCache.put(fileId, newInfo);
 
-                            assert put : "Value was not stored in cache [fileId=" + fileId + ", newInfo=" + newInfo + ']';
+                            assert put : "Value was not stored in cache [fileId=" + fileId + ", newInfo=" + newInfo
+                                    + ']';
 
                             return null;
                         }
@@ -668,7 +686,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @param id The id to check.
      * @throws IgniteCheckedException On error.
      */
-    private void addInfoIfNeeded(Collection<IgniteUuid> fileIds, Map<IgniteUuid, IgfsFileInfo> map, IgniteUuid id) throws IgniteCheckedException {
+    private void addInfoIfNeeded(Collection<IgniteUuid> fileIds, Map<IgniteUuid, IgfsFileInfo> map, IgniteUuid id)
+        throws IgniteCheckedException {
         assert validTxState(true);
 
         if (fileIds.contains(id) && !map.containsKey(id)) {
@@ -773,7 +792,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Directory listing for the specified file.*
      * @throws IgniteCheckedException If failed.
      */
-    private Map<String, IgfsListingEntry> directoryListing(IgniteUuid fileId, boolean skipTx) throws IgniteCheckedException {
+    private Map<String, IgfsListingEntry> directoryListing(IgniteUuid fileId, boolean skipTx)
+        throws IgniteCheckedException {
         assert fileId != null;
 
         IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) :
@@ -783,48 +803,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Add file into file system structure.
-     *
-     * @param parentId Parent file ID.
-     * @param fileName File name in the parent's listing.
-     * @param newFileInfo File info to store in the parent's listing.
-     * @return File id already stored in meta cache or {@code null} if passed file info was stored.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgniteUuid putIfAbsent(IgniteUuid parentId, String fileName, IgfsFileInfo newFileInfo)
-        throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                assert validTxState(false);
-                assert parentId != null;
-                assert fileName != null;
-                assert newFileInfo != null;
-
-                IgniteUuid res = null;
-
-                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
-
-                try {
-                    res = putIfAbsentNonTx(parentId, fileName, newFileInfo);
-
-                    tx.commit();
-                }
-                finally {
-                    tx.close();
-                }
-
-                return res;
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to put file because Grid is stopping [parentId=" + parentId +
-                ", fileName=" + fileName + ", newFileInfo=" + newFileInfo + ']');
-    }
-
-    /**
      * Add file into file system structure. Do not create new transaction expecting that the one already exists.
      *
      * @param parentId Parent file ID.
@@ -845,7 +823,8 @@ public class IgfsMetaManager extends IgfsManager {
         IgfsFileInfo parentInfo = info(parentId);
 
         if (parentInfo == null)
-            throw fsException(new IgfsPathNotFoundException("Failed to lock parent directory (not found): " + parentId));
+            throw fsException(new IgfsPathNotFoundException("Failed to lock parent directory (not found): " +
+                parentId));
 
         if (!parentInfo.isDirectory())
             throw fsException(new IgfsPathIsNotDirectoryException("Parent file is not a directory: " + parentInfo));
@@ -1122,124 +1101,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Remove file from the file system structure.
-     *
-     * @param parentId Parent file ID.
-     * @param fileName New file name in the parent's listing.
-     * @param fileId File ID to remove.
-     * @param path Path of the deleted file.
-     * @param rmvLocked Whether to remove this entry in case it is has explicit lock.
-     * @return The last actual file info or {@code null} if such file no more exist.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable public IgfsFileInfo removeIfEmpty(IgniteUuid parentId, String fileName, IgniteUuid fileId,
-        IgfsPath path, boolean rmvLocked)
-        throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                assert validTxState(false);
-
-                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
-
-                try {
-                    if (parentId != null)
-                        lockIds(parentId, fileId, TRASH_ID);
-                    else
-                        lockIds(fileId, TRASH_ID);
-
-                    IgfsFileInfo fileInfo = removeIfEmptyNonTx(parentId, fileName, fileId, path, rmvLocked);
-
-                    tx.commit();
-
-                    delWorker.signal();
-
-                    return fileInfo;
-                }
-                finally {
-                    tx.close();
-                }
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to remove file system entry because Grid is stopping [parentId=" +
-                parentId + ", fileName=" + fileName + ", fileId=" + fileId + ", path=" + path + ']');
-    }
-
-    /**
-     * Remove file from the file system structure in existing transaction.
-     *
-     * @param parentId Parent file ID.
-     * @param fileName New file name in the parent's listing.
-     * @param fileId File ID to remove.
-     * @param path Path of the deleted file.
-     * @param rmvLocked Whether to remove this entry in case it has explicit lock.
-     * @return The last actual file info or {@code null} if such file no more exist.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable private IgfsFileInfo removeIfEmptyNonTx(@Nullable IgniteUuid parentId, String fileName, IgniteUuid fileId,
-        IgfsPath path, boolean rmvLocked)
-        throws IgniteCheckedException {
-        assert validTxState(true);
-        assert parentId != null;
-        assert fileName != null;
-        assert fileId != null;
-        assert !ROOT_ID.equals(fileId);
-
-        if (log.isDebugEnabled())
-            log.debug("Remove file: [parentId=" + parentId + ", fileName= " + fileName + ", fileId=" + fileId + ']');
-
-        // Safe gets because locks are obtained in removeIfEmpty.
-        IgfsFileInfo fileInfo = id2InfoPrj.get(fileId);
-        IgfsFileInfo parentInfo = id2InfoPrj.get(parentId);
-
-        if (fileInfo == null || parentInfo == null) {
-            if (parentInfo != null) { // fileInfo == null
-                IgfsListingEntry entry = parentInfo.listing().get(fileName);
-
-                // If file info does not exists but listing entry exists, throw inconsistent exception.
-                if (entry != null && entry.fileId().equals(fileId))
-                    throw new IgniteCheckedException("Failed to remove file (file system is in inconsistent state) " +
-                        "[fileInfo=" + fileInfo + ", fileName=" + fileName + ", fileId=" + fileId + ']');
-            }
-
-            return null; // Parent directory or removed file cannot be locked (not found?).
-        }
-
-        assert parentInfo.isDirectory();
-
-        if (!rmvLocked && fileInfo.lockId() != null)
-            throw fsException("Failed to remove file (file is opened for writing) [fileName=" +
-                fileName + ", fileId=" + fileId + ", lockId=" + fileInfo.lockId() + ']');
-
-        // Validate own directory listing.
-        if (fileInfo.isDirectory()) {
-            Map<String, IgfsListingEntry> listing = fileInfo.listing();
-
-            if (!F.isEmpty(listing))
-                throw fsException(new IgfsDirectoryNotEmptyException("Failed to remove file (directory is not empty)" +
-                    " [fileId=" + fileId + ", listing=" + listing + ']'));
-        }
-
-        // Validate file in the parent listing.
-        IgfsListingEntry listingEntry = parentInfo.listing().get(fileName);
-
-        if (listingEntry == null || !listingEntry.fileId().equals(fileId))
-            return null;
-
-        // Actual remove.
-        softDeleteNonTx(parentId, fileName, fileId);
-
-        // Update a file info of the removed file with a file path,
-        // which will be used by delete worker for event notifications.
-        id2InfoPrj.invoke(fileId, new UpdatePath(path));
-
-        return builder(fileInfo).path(path).build();
-    }
-
-    /**
      * Deletes (moves to TRASH) all elements under the root folder.
      *
      * @return The new Id if the artificially created folder containing all former root
@@ -1528,6 +1389,9 @@ public class IgfsMetaManager extends IgfsManager {
                             IgfsFileInfo entryInfo = locks.get(entryId);
 
                             if (entryInfo != null) {
+                                // File must be locked for deletion:
+                                assert entryInfo.isDirectory() || DELETE_LOCK_ID.equals(entryInfo.lockId());
+
                                 // Delete only files or empty folders.
                                 if (entryInfo.isFile() || entryInfo.isDirectory() && entryInfo.listing().isEmpty()) {
                                     id2InfoPrj.getAndRemove(entryId);
@@ -1588,6 +1452,14 @@ public class IgfsMetaManager extends IgfsManager {
 
                     Map<IgniteUuid, IgfsFileInfo> infos = lockIds(parentId, id);
 
+                    IgfsFileInfo victim = infos.get(id);
+
+                    if (victim == null)
+                        return res;
+
+                    assert victim.isDirectory() || DELETE_LOCK_ID.equals(victim.lockId()) :
+                            " isDir: " + victim.isDirectory() + ", lockId: " + victim.lockId();
+
                     // Proceed only in case both parent and child exist.
                     if (infos.containsKey(parentId) && infos.containsKey(id)) {
                         IgfsFileInfo parentInfo = infos.get(parentId);
@@ -1599,7 +1471,9 @@ public class IgfsMetaManager extends IgfsManager {
                         if (listingEntry != null)
                             id2InfoPrj.invoke(parentId, new UpdateListing(name, listingEntry, true));
 
-                        id2InfoPrj.getAndRemove(id);
+                        IgfsFileInfo deleted = id2InfoPrj.getAndRemove(id);
+
+                        assert victim.id().equals(deleted.id());
 
                         res = true;
                     }
@@ -1885,66 +1759,34 @@ public class IgfsMetaManager extends IgfsManager {
         assert props != null;
         assert validTxState(false);
 
-        List<String> components;
-        SortedSet<IgniteUuid> idSet;
-        IgfsPath existingPath;
+        DirectoryChainBuilder b = null;
 
         while (true) {
             if (busyLock.enterBusy()) {
                 try {
-                    // Take the ids in *path* order out of transaction:
-                    final List<IgniteUuid> idList = fileIds(path);
-
-                    idSet = new TreeSet<IgniteUuid>(PATH_ID_SORTING_COMPARATOR);
-
-                    idSet.add(ROOT_ID);
-
-                    components = path.components();
-
-                    // Store all the non-null ids in the set & construct existing path in one loop:
-                    existingPath = path.root();
-
-                    assert idList.size() == components.size() + 1;
-
-                    // Find the lowermost existing id:
-                    IgniteUuid parentId = ROOT_ID;
-
-                    for (int i = 1; i < idList.size(); i++) {
-                        IgniteUuid id = idList.get(i);
-
-                        if (id == null)
-                            break;
-
-                        parentId = id;
-
-                        boolean added = idSet.add(id);
-
-                        assert added;
-
-                        existingPath = new IgfsPath(existingPath, components.get(i - 1));
-                    }
+                    b = new DirectoryChainBuilder(path, props, props);
 
                     // Start TX.
                     IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                     try {
-                        final Map<IgniteUuid, IgfsFileInfo> lockedInfos = lockIds(idSet);
+                        final Map<IgniteUuid, IgfsFileInfo> lockedInfos = lockIds(b.idSet);
 
                         // If the path was changed, we close the current Tx and repeat the procedure again
                         // starting from taking the path ids.
-                        if (verifyPathIntegrity(existingPath, idList, lockedInfos)) {
+                        if (verifyPathIntegrity(b.existingPath, b.idList, lockedInfos)) {
                             // Locked path okay, trying to proceed with the remainder creation.
-                            IgfsFileInfo parentInfo = lockedInfos.get(parentId);
+                            IgfsFileInfo lowermostExistingInfo = lockedInfos.get(b.lowermostExistingId);
 
                             // Check only the lowermost directory in the existing directory chain
                             // because others are already checked in #verifyPathIntegrity() above.
-                            if (!parentInfo.isDirectory())
+                            if (!lowermostExistingInfo.isDirectory())
                                 throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
                                     "element is not a directory)");
 
-                            if (idSet.size() == components.size() + 1) {
-                                assert existingPath.equals(path);
-                                assert lockedInfos.size() == idSet.size();
+                            if (b.existingIdCnt == b.components.size() + 1) {
+                                assert b.existingPath.equals(path);
+                                assert lockedInfos.size() == b.existingIdCnt;
 
                                 // The target directory already exists, nothing to do.
                                 // (The fact that all the path consisns of directories is already checked above).
@@ -1952,48 +1794,15 @@ public class IgfsMetaManager extends IgfsManager {
                                 return false;
                             }
 
-                            Map<String, IgfsListingEntry> parentListing = parentInfo.listing();
+                            Map<String, IgfsListingEntry> parentListing = lowermostExistingInfo.listing();
 
-                            String shortName = components.get(idSet.size() - 1);
+                            String shortName = b.components.get(b.existingIdCnt - 1);
 
                             IgfsListingEntry entry = parentListing.get(shortName);
 
                             if (entry == null) {
-                                IgfsFileInfo childInfo = null;
-
-                                String childName = null;
-
-                                IgfsFileInfo newDirInfo;
-
-                                // This loop creates the missing directory chain from the bottom to the top:
-                                for (int i = components.size() - 1; i >= idSet.size() - 1; i--) {
-                                    // Required entry does not exist.
-                                    // Create new directory info:
-                                    if (childName == null) {
-                                        assert childInfo == null;
-
-                                        newDirInfo = new IgfsFileInfo(true, props);
-                                    }
-                                    else {
-                                        assert childInfo != null;
-
-                                        newDirInfo = new IgfsFileInfo(Collections.singletonMap(childName,
-                                            new IgfsListingEntry(childInfo)), props);
-                                    }
-
-                                    boolean put = id2InfoPrj.putIfAbsent(newDirInfo.id(), newDirInfo);
-
-                                    assert put; // Because we used a new id that should be unique.
+                                b.doBuild();
 
-                                    childInfo = newDirInfo;
-                                    childName = components.get(i);
-                                }
-
-                                // Now link the newly created directory chain to the lowermost existing parent:
-                                id2InfoPrj.invoke(parentId,
-                                    new UpdateListing(childName, new IgfsListingEntry(childInfo), false));
-
-                                // We're close to finish:
                                 tx.commit();
 
                                 break;
@@ -2022,17 +1831,11 @@ public class IgfsMetaManager extends IgfsManager {
             }
             else
                 throw new IllegalStateException("Failed to mkdir because Grid is stopping. [path=" + path + ']');
-        } // retry loop
-
-        if (evts.isRecordable(EVT_IGFS_DIR_CREATED)) {
-            IgfsPath createdPath = existingPath;
+        }
 
-            for (int i = idSet.size() - 1; i < components.size(); i++) {
-                createdPath = new IgfsPath(createdPath, components.get(i));
+        assert b != null;
 
-                evts.record(new IgfsEvent(createdPath, locNode, EVT_IGFS_DIR_CREATED));
-            }
-        }
+        b.sendEvents();
 
         return true;
     }
@@ -2135,6 +1938,8 @@ public class IgfsMetaManager extends IgfsManager {
 
                         @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
                             IgfsFileInfo> infos) throws Exception {
+                            assert validTxState(true);
+
                             assert !infos.isEmpty();
 
                             // Determine the first existing parent.
@@ -2186,7 +1991,7 @@ public class IgfsMetaManager extends IgfsManager {
                                     "the secondary file system because the path points to a directory: " + path);
 
                             IgfsFileInfo newInfo = new IgfsFileInfo(status.blockSize(), status.length(), affKey,
-                                IgniteUuid.randomUuid(), igfsCtx.igfs().evictExclude(path, false), status.properties());
+                                composeLockId(false), igfsCtx.igfs().evictExclude(path, false), status.properties());
 
                             // Add new file info to the listing optionally removing the previous one.
                             IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), path.name(), newInfo);
@@ -2194,6 +1999,13 @@ public class IgfsMetaManager extends IgfsManager {
                             if (oldId != null) {
                                 IgfsFileInfo oldInfo = info(oldId);
 
+                                assert oldInfo != null; // Otherwise cache is in inconsistent state.
+
+                                // The contact is that we cannot overwrite a file locked for writing:
+                                if (oldInfo.lockId() != null)
+                                    throw fsException("Failed to overwrite file (file is opened for writing) [path=" +
+                                        path + ", fileId=" + oldId + ", lockId=" + oldInfo.lockId() + ']');
+
                                 id2InfoPrj.remove(oldId); // Remove the old one.
                                 id2InfoPrj.put(newInfo.id(), newInfo); // Put the new one.
 
@@ -2203,29 +2015,6 @@ public class IgfsMetaManager extends IgfsManager {
                                     new UpdateListing(path.name(), new IgfsListingEntry(newInfo), false));
 
                                 IgniteInternalFuture<?> delFut = igfsCtx.data().delete(oldInfo);
-
-                                // Record PURGE event if needed.
-                                if (evts.isRecordable(EVT_IGFS_FILE_PURGED)) {
-                                    delFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                                        @Override public void apply(IgniteInternalFuture<?> t) {
-                                            try {
-                                                t.get(); // Ensure delete succeeded.
-
-                                                evts.record(new IgfsEvent(path, locNode, EVT_IGFS_FILE_PURGED));
-                                            }
-                                            catch (IgniteCheckedException e) {
-                                                LT.warn(log, e, "Old file deletion failed in DUAL mode [path=" + path +
-                                                    ", simpleCreate=" + simpleCreate + ", props=" + props +
-                                                    ", overwrite=" + overwrite + ", bufferSize=" + bufSize +
-                                                    ", replication=" + replication + ", blockSize=" + blockSize + ']');
-                                            }
-                                        }
-                                    });
-                                }
-
-                                // Record DELETE event if needed.
-                                if (evts.isRecordable(EVT_IGFS_FILE_DELETED))
-                                    pendingEvts.add(new IgfsEvent(path, locNode, EVT_IGFS_FILE_DELETED));
                             }
 
                             // Record CREATE event if needed.
@@ -2287,7 +2076,9 @@ public class IgfsMetaManager extends IgfsManager {
 
                         @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
                             IgfsFileInfo> infos) throws Exception {
-                            IgfsFileInfo info = infos.get(path);
+                            assert validTxState(true);
+
+                            final IgfsFileInfo info = infos.get(path);
 
                             if (info.isDirectory())
                                 throw fsException("Failed to open output stream to the file in the " +
@@ -2314,12 +2105,22 @@ public class IgfsMetaManager extends IgfsManager {
                                 }
                             }
 
+                            if (info.lockId() != null) {
+                                throw fsException("Failed to open file (file is opened for writing) [path=" +
+                                    path + ", fileId=" + info.id() + ", lockId=" + info.lockId() + ']');
+                            }
+
                             // Set lock and return.
-                            info = lockInfo(info);
+                            IgfsFileInfo lockedInfo = lockInfo(info, false);
+
+                            assert lockedInfo != null; // We checked the lock above.
 
-                            metaCache.put(info.id(), info);
+                            boolean put = metaCache.put(info.id(), lockedInfo);
 
-                            return new IgfsSecondaryOutputStreamDescriptor(infos.get(path.parent()).id(), info, out);
+                            assert put;
+
+                            return new IgfsSecondaryOutputStreamDescriptor(infos.get(path.parent()).id(),
+                                lockedInfo, out);
                         }
 
                         @Override public IgfsSecondaryOutputStreamDescriptor onFailure(@Nullable Exception err)
@@ -2329,8 +2130,8 @@ public class IgfsMetaManager extends IgfsManager {
                             U.error(log, "File append in DUAL mode failed [path=" + path + ", bufferSize=" + bufSize +
                                 ']', err);
 
-                            throw new IgniteCheckedException("Failed to append to the file due to secondary file system " +
-                                "exception: " + path, err);
+                            throw new IgniteCheckedException("Failed to append to the file due to secondary file " +
+                                "system exception: " + path, err);
                         }
                     };
 
@@ -2438,8 +2239,8 @@ public class IgfsMetaManager extends IgfsManager {
                         }
 
                         @Override public IgfsFileInfo onFailure(@Nullable Exception err) throws IgniteCheckedException {
-                            throw new IgniteCheckedException("Failed to synchronize path due to secondary file system " +
-                                "exception: " + path, err);
+                            throw new IgniteCheckedException("Failed to synchronize path due to secondary file " +
+                                "system exception: " + path, err);
                         }
                     };
 
@@ -2517,8 +2318,8 @@ public class IgfsMetaManager extends IgfsManager {
                         U.error(log, "Directory creation in DUAL mode failed [path=" + path + ", properties=" + props +
                             ']', err);
 
-                        throw new IgniteCheckedException("Failed to create the path due to secondary file system exception: " +
-                            path, err);
+                        throw new IgniteCheckedException("Failed to create the path due to secondary file system " +
+                            "exception: " + path, err);
                     }
                 };
 
@@ -2685,8 +2486,8 @@ public class IgfsMetaManager extends IgfsManager {
                         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);
+                        throw new IgniteCheckedException("Failed to delete the path due to secondary file system " +
+                            "exception: ", err);
                     }
                 };
 
@@ -2713,8 +2514,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Update file info.
      * @throws IgniteCheckedException If update failed.
      */
-    public IgfsFileInfo updateDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, final Map<String, String> props)
-        throws IgniteCheckedException {
+    public IgfsFileInfo updateDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
+        final Map<String, String> props) throws IgniteCheckedException {
         assert fs != null;
         assert path != null;
         assert props != null && !props.isEmpty();
@@ -2740,8 +2541,8 @@ public class IgfsMetaManager extends IgfsManager {
                         U.error(log, "Path update in DUAL mode failed [path=" + path + ", properties=" + props + ']',
                             err);
 
-                        throw new IgniteCheckedException("Failed to update the path due to secondary file system exception: " +
-                            path, err);
+                        throw new IgniteCheckedException("Failed to update the path due to secondary file system " +
+                            "exception: " + path, err);
                     }
                 };
 
@@ -2805,8 +2606,8 @@ public class IgfsMetaManager extends IgfsManager {
 
                 if (status != null) {
                     if (!status.isDirectory() && !curPath.equals(endPath))
-                        throw new IgniteCheckedException("Failed to create path the locally because secondary file system " +
-                            "directory structure was modified concurrently and the path is not a directory as " +
+                        throw new IgniteCheckedException("Failed to create path the locally because secondary file " +
+                            "system directory structure was modified concurrently and the path is not a directory as " +
                             "expected: " + curPath);
                 }
                 else {
@@ -3084,7 +2885,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @return {@code True} if value was stored in cache, {@code false} otherwise.
      * @throws IgniteCheckedException If operation failed.
      */
-    private <K, V> boolean putx(IgniteInternalCache<K, V> cache, K key, IgniteClosure<V, V> c) throws IgniteCheckedException {
+    private <K, V> boolean putx(IgniteInternalCache<K, V> cache, K key, IgniteClosure<V, V> c)
+        throws IgniteCheckedException {
         assert validTxState(true);
 
         V oldVal = cache.get(key);
@@ -3549,4 +3351,455 @@ public class IgfsMetaManager extends IgfsManager {
             return S.toString(UpdatePath.class, this);
         }
     }
+
+    /**
+     * Create a new file.
+     *
+     * @param path Path.
+     * @param bufSize Buffer size.
+     * @param overwrite Overwrite flag.
+     * @param affKey Affinity key.
+     * @param replication Replication factor.
+     * @param props Properties.
+     * @param simpleCreate Whether new file should be created in secondary FS using create(Path, boolean) method.
+     * @return Tuple containing the created file info and its parent id.
+     */
+    IgniteBiTuple<IgfsFileInfo, IgniteUuid> create(
+        final IgfsPath path,
+        final boolean append,
+        final boolean overwrite,
+        Map<String, String> dirProps,
+        final int blockSize,
+        final @Nullable IgniteUuid affKey,
+        final boolean evictExclude,
+        @Nullable Map<String, String> fileProps) throws IgniteCheckedException {
+        assert validTxState(false);
+        assert path != null;
+
+        final String name = path.name();
+
+        DirectoryChainBuilder b = null;
+
+        while (true) {
+            if (busyLock.enterBusy()) {
+                try {
+                    b = new DirectoryChainBuilder(path, dirProps, fileProps) {
+                        /** {@inheritDoc} */
+                        @Override protected IgfsFileInfo buildLeaf() {
+                            return new IgfsFileInfo(blockSize, 0L, affKey, composeLockId(false),
+                                 evictExclude, leafProps);
+                        }
+                    };
+
+                    // Start Tx:
+                    IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+
+                    try {
+                        if (overwrite)
+                            // Lock also the TRASH directory because in case of overwrite we
+                            // may need to delete the old file:
+                            b.idSet.add(TRASH_ID);
+
+                        final Map<IgniteUuid, IgfsFileInfo> lockedInfos = lockIds(b.idSet);
+
+                        assert !overwrite || lockedInfos.get(TRASH_ID) != null; // TRASH must exist at this point.
+
+                        // If the path was changed, we close the current Tx and repeat the procedure again
+                        // starting from taking the path ids.
+                        if (verifyPathIntegrity(b.existingPath, b.idList, lockedInfos)) {
+                            // Locked path okay, trying to proceed with the remainder creation.
+                            final IgfsFileInfo lowermostExistingInfo = lockedInfos.get(b.lowermostExistingId);
+
+                            if (b.existingIdCnt == b.components.size() + 1) {
+                                // Full requestd path exists.
+
+                                assert b.existingPath.equals(path);
+                                assert lockedInfos.size() ==
+                                        (overwrite ? b.existingIdCnt + 1/*TRASH*/ : b.existingIdCnt);
+
+                                if (lowermostExistingInfo.isDirectory()) {
+                                    throw new IgfsPathAlreadyExistsException("Failed to "
+                                            + (append ? "open" : "create") + " file (path points to an " +
+                                        "existing directory): " + path);
+                                }
+                                else {
+                                    // This is a file.
+                                    assert lowermostExistingInfo.isFile();
+
+                                    final IgniteUuid parentId = b.idList.get(b.idList.size() - 2);
+
+                                    final IgniteUuid lockId = lowermostExistingInfo.lockId();
+
+                                    if (append) {
+                                        if (lockId != null)
+                                            throw fsException("Failed to open file (file is opened for writing) "
+                                                + "[fileName=" + name + ", fileId=" + lowermostExistingInfo.id()
+                                                + ", lockId=" + lockId + ']');
+
+                                        IgniteUuid newLockId = composeLockId(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());
+
+                                        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(lockedInfo, parentId);
+
+                                        tx.commit();
+
+                                        IgfsUtils.sendEvents(igfsCtx.kernalContext(), path,
+                                                EventType.EVT_IGFS_FILE_OPENED_WRITE);
+
+                                        return t2;
+                                    }
+                                    else if (overwrite) {
+                                        // Delete existing file, but fail if it is locked:
+                                        if (lockId != null)
+                                            throw fsException("Failed to overwrite file (file is opened for writing) " +
+                                                    "[fileName=" + name + ", fileId=" + lowermostExistingInfo.id()
+                                                    + ", lockId=" + lockId + ']');
+
+                                        final IgfsListingEntry deletedEntry = lockedInfos.get(parentId).listing()
+                                                .get(name);
+
+                                        assert deletedEntry != null;
+
+                                        id2InfoPrj.invoke(parentId, new UpdateListing(name, deletedEntry, true));
+
+                                        // Add listing entry into the destination parent listing.
+                                        id2InfoPrj.invoke(TRASH_ID, new UpdateListing(
+                                                lowermostExistingInfo.id().toString(), deletedEntry, false));
+
+                                        // 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));
+
+                                        // Make a new locked info:
+                                        final IgfsFileInfo newFileInfo = new IgfsFileInfo(cfg.getBlockSize(), 0L,
+                                            affKey, composeLockId(false), evictExclude, fileProps);
+
+                                        assert newFileInfo.lockId() != null; // locked info should be created.
+
+                                        boolean put = id2InfoPrj.putIfAbsent(newFileInfo.id(), newFileInfo);
+
+                                        assert put;
+
+                                        id2InfoPrj.invoke(parentId,
+                                                new UpdateListing(name, new IgfsListingEntry(newFileInfo), false));
+
+                                        IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(newFileInfo, parentId);
+
+                                        tx.commit();
+
+                                        delWorker.signal();
+
+                                        IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_WRITE);
+
+                                        return t2;
+                                    }
+                                    else {
+                                        throw new IgfsPathAlreadyExistsException("Failed to create file (file " +
+                                            "already exists and overwrite flag is false): " + path);
+                                    }
+                                }
+                            }
+
+                            // The full requested path does not exist.
+
+                            // Check only the lowermost directory in the existing directory chain
+                            // because others are already checked in #verifyPathIntegrity() above.
+                            if (!lowermostExistingInfo.isDirectory())
+                                throw new IgfsParentNotDirectoryException("Failed to " + (append ? "open" : "create" )
+                                    + " file (parent element is not a directory)");
+
+                            Map<String, IgfsListingEntry> parentListing = lowermostExistingInfo.listing();
+
+                            final String uppermostFileToBeCreatedName = b.components.get(b.existingIdCnt - 1);
+
+                            final IgfsListingEntry entry = parentListing.get(uppermostFileToBeCreatedName);
+
+                            if (entry == null) {
+                                b.doBuild();
+
+                                assert b.leafInfo != null;
+                                assert b.leafParentId != null;
+
+                                IgniteBiTuple<IgfsFileInfo, IgniteUuid> t2 = new T2<>(b.leafInfo, b.leafParentId);
+
+                                tx.commit();
+
+                                b.sendEvents();
+
+                                return t2;
+                            }
+
+                            // Another thread concurrently created file or directory in the path with
+                            // the name we need.
+                        }
+                    }
+                    finally {
+                        tx.close();
+                    }
+                }
+                finally {
+                    busyLock.leaveBusy();
+                }
+            } else
+                throw new IllegalStateException("Failed to mkdir because Grid is stopping. [path=" + path + ']');
+        }
+    }
+
+    /** File chain builder. */
+    private class DirectoryChainBuilder {
+        /** The requested path to be created. */
+        protected final IgfsPath path;
+
+        /** Full path components. */
+        protected final List<String> components;
+
+        /** The list of ids. */
+        protected final List<IgniteUuid> idList;
+
+        /** The set of ids. */
+        protected final SortedSet<IgniteUuid> idSet;
+
+        /** The middle node properties. */
+        protected final Map<String, String> middleProps;
+
+        /** The leaf node properties. */
+        protected final Map<String, String> leafProps;
+
+        /** The lowermost exsiting path id. */
+        protected final IgniteUuid lowermostExistingId;
+
+        /** The existing path. */
+        protected final IgfsPath existingPath;
+
+        /** The created leaf info. */
+        protected IgfsFileInfo leafInfo;
+
+        /** The leaf parent id. */
+        protected IgniteUuid leafParentId;
+
+        /** The number of existing ids. */
+        protected final int existingIdCnt;
+
+        /**
+         * Creates the builder and performa all the initial calculations.
+         */
+        protected DirectoryChainBuilder(IgfsPath path,
+                 Map<String,String> middleProps, Map<String,String> leafProps) 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;
+            // Store all the non-null ids in the set & construct existing path in one loop:
+            IgfsPath existingPath = path.root();
+
+            assert idList.size() == components.size() + 1;
+
+            // Find the lowermost existing id:
+            IgniteUuid lowermostExistingId = null;
+
+            int idIdx = 0;
+
+            for (IgniteUuid id: idList) {
+                if (id == null)
+                    break;
+
+                lowermostExistingId = id;
+
+                boolean added = idSet.add(id);
+
+                assert added : "Not added id = " + id;
+
+                if (idIdx >= 1) // skip root.
+                    existingPath = new IgfsPath(existingPath, components.get(idIdx - 1));
+
+                idIdx++;
+            }
+
+            assert idSet.contains(ROOT_ID);
+
+            this.lowermostExistingId = lowermostExistingId;
+
+            this.existingPath = existingPath;
+
+            this.existingIdCnt = idSet.size();
+        }
+
+        /**
+         * 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()  {
+            return new IgfsFileInfo(true, leafProps);
+        }
+
+        /**
+         * Links newly created chain to existing parent.
+         */
+        final void linkBuiltChainToExistingParent(String childName, IgfsFileInfo childInfo)
+                throws IgniteCheckedException {
+            assert childInfo != null;
+
+            id2InfoPrj.invoke(lowermostExistingId,
+                    new UpdateListing(childName, new IgfsListingEntry(childInfo), false));
+        }
+
+        /**
+         * Does the main portion of job building the renmaining path.
+         */
+        public final void doBuild() throws IgniteCheckedException {
+            IgfsFileInfo childInfo = null;
+
+            String childName = null;
+
+            IgfsFileInfo newLeafInfo;
+            IgniteUuid parentId = null;
+
+            // This loop creates the missing directory chain from the bottom to the top:
+            for (int i = components.size() - 1; i >= existingIdCnt - 1; i--) {
+                // Required entry does not exist.
+                // Create new directory info:
+                if (childName == null) {
+                    assert childInfo == null;
+
+                    newLeafInfo = buildLeaf();
+
+                    assert newLeafInfo != null;
+
+                    leafInfo = newLeafInfo;
+                }
+                else {
+                    assert childInfo != null;
+
+                    newLeafInfo = buildMiddleNode(childName, childInfo);
+
+                    assert newLeafInfo != null;
+
+                    if (parentId == null)
+                        parentId = newLeafInfo.id();
+                }
+
+                boolean put = id2InfoPrj.putIfAbsent(newLeafInfo.id(), newLeafInfo);
+
+                assert put; // Because we used a new id that should be unique.
+
+                childInfo = newLeafInfo;
+
+                childName = components.get(i);
+            }
+
+            if (parentId == null)
+                parentId = lowermostExistingId;
+
+            leafParentId = parentId;
+
+            // Now link the newly created directory chain to the lowermost existing parent:
+            linkBuiltChainToExistingParent(childName, childInfo);
+        }
+
+        /**
+         * Sends events.
+         */
+        public final void sendEvents() {
+            if (evts.isRecordable(EVT_IGFS_DIR_CREATED)) {
+                IgfsPath createdPath = existingPath;
+
+                for (int i = existingPath.components().size(); i < components.size() - 1; i++) {
+                    createdPath = new IgfsPath(createdPath, components.get(i));
+
+                    IgfsUtils.sendEvents(igfsCtx.kernalContext(), createdPath, EVT_IGFS_DIR_CREATED);
+                }
+            }
+
+            if (leafInfo.isDirectory())
+                IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_DIR_CREATED);
+            else {
+                IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_CREATED);
+                IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_WRITE);
+            }
+        }
+    }
+
+    /**
+     * 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);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
index c297eed..c9225ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
@@ -121,6 +121,8 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
         if (fileInfo.lockId() == null)
             throw new IgfsException("Failed to acquire file lock (concurrently modified?): " + path);
 
+        assert !IgfsMetaManager.DELETE_LOCK_ID.equals(fileInfo.lockId());
+
         this.igfsCtx = igfsCtx;
         meta = igfsCtx.meta();
         data = igfsCtx.data();

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 50ebd56..07fdda4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -21,10 +21,15 @@ import java.lang.reflect.Constructor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.FileSystemConfiguration;
+import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.igfs.IgfsException;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
+import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
@@ -158,4 +163,22 @@ public class IgfsUtils {
         throw new IgniteCheckedException("Failed to perform operation since max number of attempts " +
             "exceeded. [maxAttempts=" + MAX_CACHE_TX_RETRIES + ']');
     }
+
+
+    /**
+     * Sends a series of event.
+     *
+     * @param path The path of the created file.
+     * @param type The type of event to send.
+     */
+    public static void sendEvents(GridKernalContext kernalCtx, IgfsPath path, int type) {
+        assert kernalCtx != null;
+        assert path != null;
+
+        GridEventStorageManager evts = kernalCtx.event();
+        ClusterNode locNode = kernalCtx.discovery().localNode();
+
+        if (evts.isRecordable(type))
+            evts.record(new IgfsEvent(path, locNode, type));
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/962fcce3/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
index f0f86ec..6ca75a1 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
@@ -683,7 +683,7 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
     public void testSingleFileOverwrite() throws Exception {
         final List<Event> evtList = new ArrayList<>();
 
-        final int evtsCnt = 3 + 4 + 1;
+        final int evtsCnt = 1 + 4 + 1;
 
         final CountDownLatch latch = new CountDownLatch(evtsCnt);
 
@@ -703,7 +703,7 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
 
         igfs.create(file, false).close(); // Will generate create, open and close events.
 
-        igfs.create(file, true).close(); // Will generate same event set + delete and purge events.
+        igfs.create(file, true).close(); // Will generate only OPEN_WRITE & close events.
 
         try {
             igfs.create(file, false).close(); // Won't generate any event.
@@ -732,7 +732,7 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
         assertEquals(0, evt.dataSize());
 
         assertOneToOne(
-            evtList.subList(3, 8),
+            evtList.subList(3, evtsCnt),
             new P1<Event>() {
                 @Override public boolean apply(Event e) {
                     IgfsEvent e0 = (IgfsEvent)e;


[04/10] ignite git commit: CPP common: Disabled incremental linking.

Posted by sb...@apache.org.
CPP common: Disabled incremental linking.


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

Branch: refs/heads/ignite-1093-2
Commit: 49a5cc5d68a62602bb485b342dca9225c0a23193
Parents: 1dc3936
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Oct 15 12:10:59 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Oct 15 12:10:59 2015 +0300

----------------------------------------------------------------------
 modules/platform/src/main/cpp/common/project/vs/common.vcxproj | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/49a5cc5d/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/vs/common.vcxproj b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
index b7cfb8a..c5c790e 100644
--- a/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
+++ b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
@@ -67,7 +67,7 @@
   </ImportGroup>
   <PropertyGroup Label="UserMacros" />
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
-    <LinkIncremental>true</LinkIncremental>
+    <LinkIncremental>false</LinkIncremental>
     <TargetName>ignite.common</TargetName>
     <OutDir>$(SolutionDir)$(Platform)\$(Configuration)\</OutDir>
     <IntDir>$(Platform)\$(Configuration)\</IntDir>
@@ -76,7 +76,7 @@
     <TargetName>ignite.common</TargetName>
     <OutDir>$(SolutionDir)$(Platform)\$(Configuration)\</OutDir>
     <IntDir>$(Platform)\$(Configuration)\</IntDir>
-    <LinkIncremental>true</LinkIncremental>
+    <LinkIncremental>false</LinkIncremental>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <LinkIncremental>false</LinkIncremental>


[03/10] ignite git commit: IGNITE-1612

Posted by sb...@apache.org.
IGNITE-1612


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

Branch: refs/heads/ignite-1093-2
Commit: 1dc3936e6a8384e5b5052757d642ae0914be81be
Parents: 962fcce
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Oct 15 11:43:41 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Oct 15 11:43:41 2015 +0300

----------------------------------------------------------------------
 modules/apache-license-gen/pom.xml | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1dc3936e/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index fa6a1e2..5d14f89 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -40,6 +40,9 @@
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-deploy-plugin</artifactId>
                 <version>2.8.2</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
             </plugin>
         </plugins>
     </build>


[08/10] ignite git commit: 1093

Posted by sb...@apache.org.
1093


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

Branch: refs/heads/ignite-1093-2
Commit: fbc26f702ae203969eacf52655982cfc6ef78e40
Parents: 9aa6130
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Oct 15 15:48:05 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Oct 15 15:48:05 2015 +0300

----------------------------------------------------------------------
 .../rebalancing/GridCacheRebalancingSyncSelfTest.java | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fbc26f70/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
index 2b75adf..5acc797 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
@@ -205,8 +205,6 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         checkData(grid(1), 0, 0);
 
         log.info("Spend " + spend + " seconds to rebalance entries.");
-
-        stopAllGrids();
     }
 
     /**
@@ -265,8 +263,6 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         long spend = (System.currentTimeMillis() - start) / 1000;
 
         log.info("Spend " + spend + " seconds to rebalance entries.");
-
-        stopAllGrids();
     }
 
     /**
@@ -321,6 +317,8 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 //    public void test() throws Exception {
 //        while (true) {
 //            testComplexRebalancing();
+
+//            stopAllGrids();
 //
 //            U.sleep(5000);
 //
@@ -464,8 +462,6 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         checkData(grid(4), 0, 2);
 
         log.info("Spend " + spend + " seconds to rebalance entries.");
-
-        stopAllGrids();
     }
 
     /**
@@ -489,8 +485,6 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         stopGrid(0);
 
         checkData(grid(1), 0, 0);
-
-        stopAllGrids();
     }
 
     /**
@@ -515,6 +509,10 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
         waitForRebalancing(0, 4);
         waitForRebalancing(1, 4);
+    }
+
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
 
         stopAllGrids();
     }


[06/10] ignite git commit: IGNITE-1487: Exceptions on normal execution path avoided.

Posted by sb...@apache.org.
IGNITE-1487: Exceptions on normal execution path avoided.


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

Branch: refs/heads/ignite-1093-2
Commit: 541ba4037dbde7f93ec7951e57fca46ad1a9ec50
Parents: 52a733f
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Oct 15 13:25:11 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Oct 15 13:25:11 2015 +0300

----------------------------------------------------------------------
 .../hadoop/igfs/HadoopIgfsWrapper.java          | 54 +++++++++++---------
 1 file changed, 29 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/541ba403/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
index 857db71..69df381 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
@@ -26,6 +26,10 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.IgniteIllegalStateException;
+import org.apache.ignite.IgniteState;
+import org.apache.ignite.Ignition;
 import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsPath;
@@ -34,11 +38,11 @@ import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse;
 import org.apache.ignite.internal.processors.igfs.IgfsStatus;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.IgniteState.STARTED;
 import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint.LOCALHOST;
 import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED;
 import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM;
@@ -354,30 +358,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
         boolean skipInProc = parameter(conf, PARAM_IGFS_ENDPOINT_NO_EMBED, authority, false);
 
         if (!skipInProc) {
-            IgfsEx igfs = null;
-
-            if (endpoint.grid() == null) {
-                try {
-                    Ignite ignite = G.ignite();
-
-                    igfs = (IgfsEx)ignite.fileSystem(endpoint.igfs());
-                }
-                catch (Exception ignore) {
-                    // No-op.
-                }
-            }
-            else {
-                for (Ignite ignite : G.allGrids()) {
-                    try {
-                        igfs = (IgfsEx)ignite.fileSystem(endpoint.igfs());
-
-                        break;
-                    }
-                    catch (Exception ignore) {
-                        // No-op.
-                    }
-                }
-            }
+            IgfsEx igfs = getIgfsEx(endpoint.grid(), endpoint.igfs());
 
             if (igfs != null) {
                 HadoopIgfsEx hadoop = null;
@@ -540,4 +521,27 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 hadoop.close(force);
         }
     }
+
+    /**
+     * Helper method to find Igfs of the given name in the given Ignite instance.
+     *
+     * @param gridName The name of the grid to check.
+     * @param igfsName The name of Igfs.
+     * @return The file system instance, or null if not found.
+     */
+    private static IgfsEx getIgfsEx(@Nullable String gridName, @Nullable String igfsName) {
+        if (Ignition.state(gridName) == STARTED) {
+            try {
+                for (IgniteFileSystem fs : Ignition.ignite(gridName).fileSystems()) {
+                    if (F.eq(fs.name(), igfsName))
+                        return (IgfsEx)fs;
+                }
+            }
+            catch (IgniteIllegalStateException ignore) {
+                // May happen if the grid state has changed:
+            }
+        }
+
+        return null;
+    }
 }
\ No newline at end of file


[09/10] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1093-2

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-1093-2


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

Branch: refs/heads/ignite-1093-2
Commit: 4c9792e4f32f8b2b8aed17f7282b158176fb4725
Parents: fbc26f7 3a29b97
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Oct 15 16:22:24 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Oct 15 16:22:24 2015 +0300

----------------------------------------------------------------------
 modules/apache-license-gen/pom.xml              |   3 +
 .../JettyRestProcessorAbstractSelfTest.java     |   4 +-
 .../processors/igfs/IgfsDataManager.java        |   2 -
 .../processors/igfs/IgfsDeleteWorker.java       | 102 ++-
 .../internal/processors/igfs/IgfsImpl.java      | 164 +---
 .../processors/igfs/IgfsMetaManager.java        | 897 ++++++++++++-------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   2 +
 .../internal/processors/igfs/IgfsUtils.java     |  23 +
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   6 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   | 639 ++++++++++---
 .../igfs/IgfsDataManagerSelfTest.java           |  13 +-
 .../igfs/IgfsMetaManagerSelfTest.java           | 170 ++--
 .../processors/igfs/IgfsProcessorSelfTest.java  |  12 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   6 +
 .../hadoop/igfs/HadoopIgfsWrapper.java          |  54 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   5 -
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  27 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   2 +-
 .../main/cpp/common/project/vs/common.vcxproj   |   4 +-
 .../http/jetty/GridJettyRestHandler.java        |   2 +-
 20 files changed, 1342 insertions(+), 795 deletions(-)
----------------------------------------------------------------------