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/05 07:58:54 UTC

[1/5] ignite git commit: IGNITE-1573: Fixed IGFS "mkdirs" concurrency problem.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1607 3ed37b267 -> a5103a87b


IGNITE-1573: Fixed IGFS "mkdirs" concurrency problem.


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

Branch: refs/heads/ignite-1607
Commit: a411f94d29b9073457ae2b31f89dd9f702e6bf53
Parents: fd091c8
Author: iveselovskiy <iv...@gridgain.com>
Authored: Fri Oct 2 10:54:33 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Oct 2 10:54:33 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteFileSystem.java     |   2 +
 .../internal/processors/igfs/IgfsFileInfo.java  |  13 +-
 .../internal/processors/igfs/IgfsImpl.java      |  64 +------
 .../processors/igfs/IgfsMetaManager.java        | 180 +++++++++++++++++-
 .../processors/igfs/IgfsAbstractSelfTest.java   | 185 +++++++++++--------
 5 files changed, 300 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
index a187a90..def2965 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
@@ -417,6 +417,8 @@ public interface IgniteFileSystem extends IgniteAsyncSupport {
 
     /**
      * Creates directories under specified path with the specified properties.
+     * Note that the properties are applied only to created directories, but never
+     * updated for existing ones.
      *
      * @param path Path of directories chain to create.
      * @param props Metadata properties to set on created directories.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index 8564500..fa79d80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -124,6 +124,16 @@ public final class IgfsFileInfo implements Externalizable {
     }
 
     /**
+     * Consturcts directory with random ID, provided listing and properties.
+     *
+     * @param listing Listing.
+     * @param props The properties to set for the new directory.
+     */
+    IgfsFileInfo(@Nullable Map<String, IgfsListingEntry> listing, @Nullable Map<String,String> props) {
+        this(true/*dir*/, null, 0, 0, null, listing, props, null, false, System.currentTimeMillis(), false);
+    }
+
+    /**
      * Constructs file info.
      *
      * @param blockSize Block size.
@@ -194,7 +204,7 @@ public final class IgfsFileInfo implements Externalizable {
      * @param evictExclude Evict exclude flag.
      */
     IgfsFileInfo(int blockSize, long len, boolean evictExclude, @Nullable Map<String, String> props) {
-        this(blockSize == 0, // NB The contract is: (blockSize == null) <=> isDirectory()
+        this(blockSize == 0, // NB The contract is: (blockSize == 0) <=> isDirectory()
             null, blockSize, len, null, null, props, null, true, System.currentTimeMillis(), evictExclude);
     }
 
@@ -495,6 +505,7 @@ public final class IgfsFileInfo implements Externalizable {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         id = U.readGridUuid(in);
         blockSize = in.readInt();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/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 d5ba95f..fa3a955 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
@@ -23,6 +23,7 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -58,7 +59,6 @@ import org.apache.ignite.igfs.IgfsInvalidPathException;
 import org.apache.ignite.igfs.IgfsMetrics;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsOutputStream;
-import org.apache.ignite.igfs.IgfsParentNotDirectoryException;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathAlreadyExistsException;
 import org.apache.ignite.igfs.IgfsPathIsDirectoryException;
@@ -93,7 +93,6 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_CREATED;
 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;
@@ -769,71 +768,18 @@ public final class IgfsImpl implements IgfsEx {
                 if (log.isDebugEnabled())
                     log.debug("Make directories: " + path);
 
-                Map<String, String> props0 = props == null ? DFLT_DIR_META : props;
+                final Map<String, String> props0 = props == null ? DFLT_DIR_META : new HashMap<>(props);
 
                 IgfsMode mode = resolveMode(path);
 
-                if (mode != PRIMARY) {
+                if (mode == PRIMARY)
+                    meta.mkdirs(path, props0);
+                else {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
 
                     await(path);
 
                     meta.mkdirsDual(secondaryFs, path, props0);
-
-                    return null;
-                }
-
-                List<IgniteUuid> ids = meta.fileIds(path);
-                List<String> components = path.components();
-
-                assert ids.size() == components.size() + 1 : "Components doesn't contain ROOT element" +
-                    " [ids=" + ids + ", components=" + components + ']';
-
-                IgniteUuid parentId = ROOT_ID;
-
-                IgfsPath curPath = path.root();
-
-                for (int step = 0, size = components.size(); step < size; step++) {
-                    IgniteUuid fileId = ids.get(step + 1); // Skip the first ROOT element.
-
-                    if (fileId == null) {
-                        IgfsFileInfo fileInfo = new IgfsFileInfo(true, props0); // Create new directory.
-
-                        String fileName = components.get(step); // Get current component name.
-
-                        curPath = new IgfsPath(curPath, fileName);
-
-                        try {
-                            // Fails only if parent is not a directory or if modified concurrently.
-                            IgniteUuid oldId = meta.putIfAbsent(parentId, fileName, fileInfo);
-
-                            fileId = oldId == null ? fileInfo.id() : oldId; // Update node ID.
-
-                            if (oldId == null && evts.isRecordable(EVT_IGFS_DIR_CREATED))
-                                evts.record(new IgfsEvent(curPath, localNode(), EVT_IGFS_DIR_CREATED));
-                        }
-                        catch (IgniteCheckedException e) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to create directory [path=" + path + ", parentId=" + parentId +
-                                    ", fileName=" + fileName + ", step=" + step + ", e=" + e.getMessage() + ']');
-
-                            // Check directory with such name already exists.
-                            IgfsFileInfo stored = meta.info(meta.fileId(parentId, fileName));
-
-                            if (stored == null)
-                                throw e;
-
-                            if (!stored.isDirectory())
-                                throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
-                                    "element is not a directory)");
-
-                            fileId = stored.id(); // Update node ID.
-                        }
-                    }
-
-                    assert fileId != null;
-
-                    parentId = fileId;
                 }
 
                 return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/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 bb6404c..927067a 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
@@ -51,6 +51,7 @@ import org.apache.ignite.igfs.IgfsConcurrentModificationException;
 import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsParentNotDirectoryException;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathAlreadyExistsException;
 import org.apache.ignite.igfs.IgfsPathIsDirectoryException;
@@ -673,9 +674,10 @@ public class IgfsMetaManager extends IgfsManager {
         if (fileIds.contains(id) && !map.containsKey(id)) {
             IgfsFileInfo info = new IgfsFileInfo(id);
 
-            assert info.listing() != null;
+            IgfsFileInfo anotherInfo = id2InfoPrj.getAndPutIfAbsent(id, info);
 
-            id2InfoPrj.putIfAbsent(id, info);
+            if (anotherInfo != null)
+                info = anotherInfo;
 
             map.put(id, info);
         }
@@ -1662,7 +1664,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Updated file info or {@code null} if such file ID not found.
      * @throws IgniteCheckedException If operation failed.
      */
-    @Nullable private IgfsFileInfo updatePropertiesNonTx(@Nullable IgniteUuid parentId, IgniteUuid fileId,
+    @Nullable private IgfsFileInfo updatePropertiesNonTx(final @Nullable IgniteUuid parentId, final IgniteUuid fileId,
         String fileName, Map<String, String> props) throws IgniteCheckedException {
         assert fileId != null;
         assert !F.isEmpty(props) : "Expects not-empty file's properties";
@@ -1672,8 +1674,8 @@ public class IgfsMetaManager extends IgfsManager {
             log.debug("Update file properties [fileId=" + fileId + ", props=" + props + ']');
 
         try {
-            IgfsFileInfo oldInfo;
-            IgfsFileInfo parentInfo;
+            final IgfsFileInfo oldInfo;
+            final IgfsFileInfo parentInfo;
 
             // Lock file ID for this transaction.
             if (parentId == null) {
@@ -1690,8 +1692,6 @@ public class IgfsMetaManager extends IgfsManager {
                     return null; // Parent not found.
             }
 
-            assert validTxState(true);
-
             if (oldInfo == null)
                 return null; // File not found.
 
@@ -1724,7 +1724,7 @@ public class IgfsMetaManager extends IgfsManager {
             if (parentId != null) {
                 IgfsListingEntry entry = new IgfsListingEntry(newInfo);
 
-                assert metaCache.get(parentId) != null;
+                assert id2InfoPrj.get(parentId) != null;
 
                 id2InfoPrj.invoke(parentId, new UpdateListing(fileName, entry, false));
             }
@@ -1874,6 +1874,170 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Mkdirs implementation.
+     *
+     * @param path The path to create.
+     * @param props The properties to use for created directories.
+     * @return True iff a directory was created during the operation.
+     * @throws IgniteCheckedException If a non-directory file exists on the requested path, and in case of other errors.
+     */
+    boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
+        assert props != null;
+        assert validTxState(false);
+
+        List<String> components;
+        SortedSet<IgniteUuid> idSet;
+        IgfsPath existingPath;
+
+        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));
+                    }
+
+                    // Start TX.
+                    IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+
+                    try {
+                        final Map<IgniteUuid, IgfsFileInfo> lockedInfos = lockIds(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)) {
+                            // Locked path okay, trying to proceed with the remainder creation.
+                            IgfsFileInfo parentInfo = lockedInfos.get(parentId);
+
+                            // Check only the lowermost directory in the existing directory chain
+                            // because others are already checked in #verifyPathIntegrity() above.
+                            if (!parentInfo.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();
+
+                                // The target directory already exists, nothing to do.
+                                // (The fact that all the path consisns of directories is already checked above).
+                                // Note that properties are not updated in this case.
+                                return false;
+                            }
+
+                            Map<String, IgfsListingEntry> parentListing = parentInfo.listing();
+
+                            String shortName = components.get(idSet.size() - 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.
+
+                                    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;
+                            }
+                            else {
+                                // Another thread created file or directory with the same name.
+                                if (!entry.isDirectory()) {
+                                    // Entry exists, and it is not a directory:
+                                    throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
+                                        "element is not a directory)");
+                                }
+
+                                // If this is a directory, we continue the repeat loop,
+                                // because we cannot lock this directory without
+                                // lock ordering rule violation.
+                            }
+                        }
+                    }
+                    finally {
+                        tx.close();
+                    }
+                }
+                finally {
+                    busyLock.leaveBusy();
+                }
+            }
+            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));
+
+                evts.record(new IgfsEvent(createdPath, locNode, EVT_IGFS_DIR_CREATED));
+            }
+        }
+
+        return true;
+    }
+
+    /**
      * Set sampling flag.
      *
      * @param val Sampling flag state or {@code null} to clear sampling state and mark it as "not set".

http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/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 7e73859..cc89fd1 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
@@ -57,6 +57,7 @@ import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsIpcEndpointType;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsOutputStream;
+import org.apache.ignite.igfs.IgfsParentNotDirectoryException;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
@@ -101,12 +102,27 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     protected static final long BLOCK_SIZE = 32 * 1024 * 1024;
 
     /** Default repeat count. */
-    protected static final int REPEAT_CNT = 5; // Diagnostic: ~100; Regression: 5
+    protected static final int REPEAT_CNT = 10;
 
     /** Concurrent operations count. */
-    protected static final int OPS_CNT = 16; // Diagnostic: ~160; Regression: 16
+    protected static final int OPS_CNT = 32;
 
-    /** Seed. */
+    /** Renames count. */
+    protected static final int RENAME_CNT = OPS_CNT;
+
+    /** Deletes count. */
+    protected static final int DELETE_CNT = OPS_CNT;
+
+    /** Updates count. */
+    protected static final int UPDATE_CNT = OPS_CNT;
+
+    /** Mkdirs count. */
+    protected static final int MKDIRS_CNT = OPS_CNT;
+
+    /** Create count. */
+    protected static final int CREATE_CNT = OPS_CNT;
+
+    /** Seed to generate random numbers. */
     protected static final long SEED = System.currentTimeMillis();
 
     /** Amount of blocks to prefetch. */
@@ -724,9 +740,46 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ConstantConditions")
     public void testMkdirs() throws Exception {
         Map<String, String> props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info.
 
+        igfs.mkdirs(new IgfsPath("/x"), null);
+        checkExist(igfs, igfsSecondary, new IgfsPath("/x"));
+
+        igfs.mkdirs(new IgfsPath("/k/l"), null);
+        checkExist(igfs, igfsSecondary, new IgfsPath("/k/l"));
+
+        igfs.mkdirs(new IgfsPath("/x/y"), null);
+        checkExist(igfs, igfsSecondary, new IgfsPath("/x/y"));
+
+        igfs.mkdirs(new IgfsPath("/a/b/c/d"), null);
+        checkExist(igfs, igfsSecondary, new IgfsPath("/a/b/c/d"));
+
+        igfs.mkdirs(new IgfsPath("/a/b/c/d/e"), null);
+        checkExist(igfs, igfsSecondary, new IgfsPath("/a/b/c/d/e"));
+
+        create(igfs, null, new IgfsPath[] { new IgfsPath("/d/f") }); // "f" is a file.
+        checkExist(igfs, igfsSecondary, new IgfsPath("/d/f"));
+
+        try {
+            igfs.mkdirs(new IgfsPath("/d/f"), null);
+
+            fail("IgfsParentNotDirectoryException expected.");
+        }
+        catch (IgfsParentNotDirectoryException ignore) {
+            // No-op.
+        }
+
+        try {
+            igfs.mkdirs(new IgfsPath("/d/f/something/else"), null);
+
+            fail("IgfsParentNotDirectoryException expected.");
+        }
+        catch (IgfsParentNotDirectoryException ignore) {
+            // No-op.
+        }
+
         create(igfs, paths(DIR, SUBDIR), null);
 
         igfs.mkdirs(SUBSUBDIR, props);
@@ -747,6 +800,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ConstantConditions")
     public void testMkdirsParentRoot() throws Exception {
         Map<String, String> props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info.
 
@@ -814,6 +868,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ConstantConditions")
     public void testUpdate() throws Exception {
         Map<String, String> props = properties("owner", "group", "0555");
 
@@ -832,6 +887,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
+    @SuppressWarnings("ConstantConditions")
     public void testUpdateParentRoot() throws Exception {
         Map<String, String> props = properties("owner", "group", "0555");
 
@@ -936,6 +992,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         IgfsFile file = igfs.info(new IgfsPath("/"));
 
+        assert file != null;
+
         Map<String,String> props = file.properties();
 
         assertEquals("moo", props.get("foo"));
@@ -1619,8 +1677,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testConcurrentMkdirsDelete() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1541");
-
         for (int i = 0; i < REPEAT_CNT; i++) {
             final CyclicBarrier barrier = new CyclicBarrier(2);
 
@@ -1900,14 +1956,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksRename() throws Exception {
-        for (int i = 0; i < REPEAT_CNT; i++) {
-            try {
-                checkDeadlocks(5, 2, 2, 2, OPS_CNT, 0, 0, 0, 0);
-            }
-            finally {
-                clear(igfs, igfsSecondary);
-            }
-        }
+        checkDeadlocksRepeat(5, 2, 2, 2,  RENAME_CNT, 0, 0, 0, 0);
     }
 
     /**
@@ -1916,14 +1965,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksDelete() throws Exception {
-        for (int i = 0; i < REPEAT_CNT; i++) {
-            try {
-                checkDeadlocks(5, 2, 2, 2, 0, OPS_CNT, 0, 0, 0);
-            }
-            finally {
-                clear(igfs, igfsSecondary);
-            }
-        }
+         checkDeadlocksRepeat(5, 2, 2, 2,  0, DELETE_CNT, 0, 0, 0);
     }
 
     /**
@@ -1932,14 +1974,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksUpdate() throws Exception {
-        for (int i = 0; i < REPEAT_CNT; i++) {
-            try {
-                checkDeadlocks(5, 2, 2, 2, 0, 0, OPS_CNT, 0, 0);
-            }
-            finally {
-                clear(igfs, igfsSecondary);
-            }
-        }
+        checkDeadlocksRepeat(5, 2, 2, 2, 0, 0, UPDATE_CNT, 0, 0);
     }
 
     /**
@@ -1948,14 +1983,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksMkdirs() throws Exception {
-        for (int i = 0; i < REPEAT_CNT; i++) {
-            try {
-                checkDeadlocks(5, 2, 2, 2, 0, 0, 0, OPS_CNT, 0);
-            }
-            finally {
-                clear(igfs, igfsSecondary);
-            }
-        }
+         checkDeadlocksRepeat(5, 2, 2, 2,  0, 0, 0, MKDIRS_CNT, 0);
     }
 
     /**
@@ -1964,15 +1992,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksDeleteRename() throws Exception {
-        for (int i = 0; i < REPEAT_CNT; i++) {
-            try {
-                checkDeadlocks(5, 2, 2, 2,
-                    OPS_CNT, OPS_CNT, 0, 0, 0);
-            }
-            finally {
-                clear(igfs, igfsSecondary);
-            }
-        }
+        checkDeadlocksRepeat(5, 2, 2, 2,  RENAME_CNT, DELETE_CNT, 0, 0, 0);
     }
 
     /**
@@ -1980,18 +2000,17 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
-    public void testDeadlocksDeleteMkdirs() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1541");
+    public void testDeadlocksDeleteMkdirsRename() throws Exception {
+        checkDeadlocksRepeat(5, 2, 2, 2,  RENAME_CNT, DELETE_CNT, 0, MKDIRS_CNT, 0);
+    }
 
-        for (int i = 0; i < REPEAT_CNT; i++) {
-            try {
-                checkDeadlocks(5, 2, 2, 2,
-                     0, OPS_CNT, 0, OPS_CNT, 0);
-            }
-            finally {
-                clear(igfs, igfsSecondary);
-            }
-        }
+    /**
+     * Ensure that deadlocks do not occur during concurrent delete & rename operations.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDeadlocksDeleteMkdirs() throws Exception {
+        checkDeadlocksRepeat(5, 2, 2, 2,  0, DELETE_CNT, 0, MKDIRS_CNT, 0);
     }
 
     /**
@@ -2000,14 +2019,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksCreate() throws Exception {
-        for (int i = 0; i < REPEAT_CNT; i++) {
-            try {
-                checkDeadlocks(5, 2, 2, 2, 0, 0, 0, 0, OPS_CNT);
-            }
-            finally {
-                clear(igfs, igfsSecondary);
-            }
-        }
+        assert false : "https://issues.apache.org/jira/browse/IGNITE-1590";
+
+        checkDeadlocksRepeat(5, 2, 2, 2, 0, 0, 0, 0, CREATE_CNT);
     }
 
     /**
@@ -2016,16 +2030,33 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocks() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1541");
+        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);
+    }
+
+    /**
+     * Invokes {@link #checkDeadlocks(int, int, int, int, int, int, int, int, int)} for
+     *  {@link #REPEAT_CNT} times.
+     *
+     * @param lvlCnt Total levels in folder hierarchy.
+     * @param childrenDirPerLvl How many children directories to create per level.
+     * @param childrenFilePerLvl How many children file to create per level.
+     * @param primaryLvlCnt How many levels will exist in the primary file system before check start.
+     * @param renCnt How many renames to perform.
+     * @param delCnt How many deletes to perform.
+     * @param updateCnt How many updates to perform.
+     * @param mkdirsCnt How many directory creations to perform.
+     * @param createCnt How many file creations to perform.
+     * @throws Exception If failed.
+     */
+    private void checkDeadlocksRepeat(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl,
+        int primaryLvlCnt, int renCnt, int delCnt,
+        int updateCnt, int mkdirsCnt, int createCnt) throws Exception {
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
-                checkDeadlocks(5, 2, 2, 2,
-                    OPS_CNT, // rename
-                    OPS_CNT, // delete
-                    OPS_CNT, // update
-                    OPS_CNT, // mkdirs
-                    OPS_CNT); // create
+                checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt,
+                    updateCnt, mkdirsCnt, createCnt);
             }
             finally {
                 clear(igfs, igfsSecondary);
@@ -2050,9 +2081,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     @SuppressWarnings("ConstantConditions")
-    public void checkDeadlocks(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl,
-        int primaryLvlCnt, int renCnt, int delCnt,
-        int updateCnt, int mkdirsCnt, int createCnt) throws Exception {
+    private void checkDeadlocks(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl,
+        int primaryLvlCnt, int renCnt, int delCnt, int updateCnt, int mkdirsCnt, int createCnt) throws Exception {
         assert childrenDirPerLvl > 0;
 
         // First define file system structure.
@@ -2096,7 +2126,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         // Now as we have all paths defined, plan operations on them.
         final Random rand = new Random(SEED);
 
-        int totalOpCnt = renCnt + delCnt + updateCnt + mkdirsCnt + createCnt;
+        final int totalOpCnt = renCnt + delCnt + updateCnt + mkdirsCnt + createCnt;
+
+        if (totalOpCnt == 0)
+            throw new RuntimeException("Operations count is zero.");
 
         final CyclicBarrier barrier = new CyclicBarrier(totalOpCnt);
 


[3/5] ignite git commit: Apache Ignite License Generator Module Readme

Posted by sb...@apache.org.
Apache Ignite License Generator Module Readme


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

Branch: refs/heads/ignite-1607
Commit: 9702b52f8ade6f2ee970d8cd77a71911106f3f21
Parents: 7b54cbd
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Oct 2 17:05:31 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Oct 2 17:05:31 2015 +0300

----------------------------------------------------------------------
 modules/apache-license-gen/README.txt | 33 ++++++++++++++++++++++++++++++
 1 file changed, 33 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9702b52f/modules/apache-license-gen/README.txt
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/README.txt b/modules/apache-license-gen/README.txt
new file mode 100644
index 0000000..a0c027e
--- /dev/null
+++ b/modules/apache-license-gen/README.txt
@@ -0,0 +1,33 @@
+Apache Ignite License Generator Module
+------------------------------
+
+Apache Ignite License Generator module is a custom maven resource plugin.
+It generates /license/{module name}-licenses.txt file contains list of module's non transitive dependencies.
+Apache Ignite binary distribution contains all non transitive dependencies of it's modules.
+Set of modules included to binary distribution may vary as well as their dependencies list, versions and licenses.
+Automatic generation of /license/{module name}-licenses.txt file guarantee that binary distribution gives user
+actual information about licenses used by Apache Ignite's modules.
+
+Note that in case dependency provided under Apache License 2.0 only in will not appear inside generated file.
+
+To use Apache Ignite License Generator Module in your project please add following to pom.xml:
+
+<plugin><!-- generates dependencies licenses -->
+     <groupId>org.apache.maven.plugins</groupId>
+     <artifactId>maven-remote-resources-plugin</artifactId>
+     <executions>
+         <execution>
+             <id>ignite-dependencies</id>
+             <goals>
+                 <goal>process</goal>
+             </goals>
+             <configuration>
+                 <resourceBundles>
+                     <resourceBundle>org.apache.ignite:ignite-apache-license-gen:${project.version}</resourceBundle>
+                 </resourceBundles>
+                 <excludeTransitive>true</excludeTransitive>
+             </configuration>
+         </execution>
+     </executions>
+ </plugin>
+


[2/5] ignite git commit: ignite-1534 Fixed races in dynamic cache start exchange ordering.

Posted by sb...@apache.org.
ignite-1534 Fixed races in dynamic cache start exchange ordering.


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

Branch: refs/heads/ignite-1607
Commit: 7b54cbd7499cd498b04e821dfa3b572bd94debec
Parents: a411f94
Author: sboikov <sb...@gridgain.com>
Authored: Fri Oct 2 11:19:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Oct 2 11:19:06 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |   2 +-
 .../cache/DynamicCacheDescriptor.java           |  17 ++
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../processors/cache/GridCacheMvccManager.java  |  20 ++-
 .../GridCachePartitionExchangeManager.java      |  72 ++-------
 .../processors/cache/GridCacheProcessor.java    |  26 ++--
 .../cache/distributed/dht/GridDhtGetFuture.java |   4 +-
 .../dht/GridPartitionedGetFuture.java           |   5 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   9 +-
 .../distributed/near/GridNearGetFuture.java     |   2 +
 .../cache/IgniteCachePutAllRestartTest.java     |   4 +-
 .../CacheGetFutureHangsSelfTest.java            | 156 +++++++++----------
 .../distributed/IgniteCacheCreatePutTest.java   | 125 +++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   5 +
 14 files changed, 284 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index b694523..a6f5f08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -550,7 +550,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         gridStartTime = getSpi().getGridStartTime();
 
                     updateTopologyVersionIfGreater(new AffinityTopologyVersion(locNode.order()),
-                        new DiscoCache(localNode(), getSpi().getRemoteNodes()));
+                        new DiscoCache(localNode(), F.view(topSnapshot, F.remoteNodes(locNode.id()))));
 
                     startLatch.countDown();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 24df7e4..b100a31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -68,6 +68,9 @@ public class DynamicCacheDescriptor {
     /** */
     private AffinityTopologyVersion startTopVer;
 
+    /** */
+    private boolean rcvdOnDiscovery;
+
     /**
      * @param ctx Context.
      * @param cacheCfg Cache configuration.
@@ -236,6 +239,20 @@ public class DynamicCacheDescriptor {
         this.updatesAllowed = updatesAllowed;
     }
 
+    /**
+     * @return {@code True} if received in discovery data.
+     */
+    public boolean receivedOnDiscovery() {
+        return rcvdOnDiscovery;
+    }
+
+    /**
+     * @param rcvdOnDiscovery {@code True} if received in discovery data.
+     */
+    public void receivedOnDiscovery(boolean rcvdOnDiscovery) {
+        this.rcvdOnDiscovery = rcvdOnDiscovery;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DynamicCacheDescriptor.class, this, "cacheName", U.maskName(cacheCfg.getName()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 5385dec..3a1cee6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1848,7 +1848,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         boolean deserializePortable,
         boolean cpy) {
         assert key != null;
-        assert val != null;
+        assert val != null || skipVals;
 
         if (!keepCacheObjects) {
             Object key0 = key.value(cacheObjCtx, false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index dd51da2..0960c9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -391,13 +391,14 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     /**
      * @param futVer Future ID.
      * @param fut Future.
+     * @return {@code False} if future was forcibly completed with error.
      */
-    public void addAtomicFuture(GridCacheVersion futVer, GridCacheAtomicFuture<?> fut) {
+    public boolean addAtomicFuture(GridCacheVersion futVer, GridCacheAtomicFuture<?> fut) {
         IgniteInternalFuture<?> old = atomicFuts.put(futVer, fut);
 
         assert old == null : "Old future is not null [futVer=" + futVer + ", fut=" + fut + ", old=" + old + ']';
 
-        onFutureAdded(fut);
+        return onFutureAdded(fut);
     }
 
     /**
@@ -529,12 +530,21 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
 
     /**
      * @param fut Future.
+     * @return {@code False} if future was forcibly completed with error.
      */
-    private void onFutureAdded(IgniteInternalFuture<?> fut) {
-        if (stopping)
+    private boolean onFutureAdded(IgniteInternalFuture<?> fut) {
+        if (stopping) {
             ((GridFutureAdapter)fut).onDone(stopError());
-        else if (cctx.kernalContext().clientDisconnected())
+
+            return false;
+        }
+        else if (cctx.kernalContext().clientDisconnected()) {
             ((GridFutureAdapter)fut).onDone(disconnectedError(null));
+
+            return false;
+        }
+
+        return true;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/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 3e77e0d..adc2174 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
@@ -105,18 +105,12 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /** Partition resend timeout after eviction. */
     private final long partResendTimeout = getLong(IGNITE_PRELOAD_RESEND_TIMEOUT, DFLT_PRELOAD_RESEND_TIMEOUT);
 
-    /** Latch which completes after local exchange future is created. */
-    private GridFutureAdapter<?> locExchFut;
-
     /** */
     private final ReadWriteLock busyLock = new ReentrantReadWriteLock();
 
     /** Last partition refresh. */
     private final AtomicLong lastRefresh = new AtomicLong(-1);
 
-    /** Pending futures. */
-    private final Queue<GridDhtPartitionsExchangeFuture> pendingExchangeFuts = new ConcurrentLinkedQueue<>();
-
     /** */
     @GridToStringInclude
     private ExchangeWorker exchWorker;
@@ -229,31 +223,18 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 }
 
                 if (exchId != null) {
-                    // Start exchange process.
-                    pendingExchangeFuts.add(exchFut);
+                    if (log.isDebugEnabled())
+                        log.debug("Discovery event (will start exchange): " + exchId);
 
                     // Event callback - without this callback future will never complete.
                     exchFut.onEvent(exchId, e);
 
+                    // Start exchange process.
+                    addFuture(exchFut);
+                }
+                else {
                     if (log.isDebugEnabled())
-                        log.debug("Discovery event (will start exchange): " + exchId);
-
-                    locExchFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                        @Override public void apply(IgniteInternalFuture<?> t) {
-                            if (!enterBusy())
-                                return;
-
-                            try {
-                                // Unwind in the order of discovery events.
-                                for (GridDhtPartitionsExchangeFuture f = pendingExchangeFuts.poll(); f != null;
-                                    f = pendingExchangeFuts.poll())
-                                    addFuture(f);
-                            }
-                            finally {
-                                leaveBusy();
-                            }
-                        }
-                    });
+                        log.debug("Do not start exchange for discovery event: " + evt);
                 }
             }
             finally {
@@ -266,8 +247,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     @Override protected void start0() throws IgniteCheckedException {
         super.start0();
 
-        locExchFut = new GridFutureAdapter<>();
-
         exchWorker = new ExchangeWorker();
 
         cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED,
@@ -328,12 +307,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (reconnect)
             reconnectExchangeFut = new GridFutureAdapter<>();
 
-        new IgniteThread(cctx.gridName(), "exchange-worker", exchWorker).start();
-
-        onDiscoveryEvent(cctx.localNodeId(), fut);
+        exchWorker.futQ.addFirst(fut);
 
-        // Allow discovery events to get processed.
-        locExchFut.onDone();
+        new IgniteThread(cctx.gridName(), "exchange-worker", exchWorker).start();
 
         if (reconnect) {
             fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
@@ -382,8 +358,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 }
             }
 
-            for (GridCacheContext cacheCtx : cctx.cacheContexts())
-                cacheCtx.preloader().onInitialExchangeComplete(null);
+            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                if (cacheCtx.startTopologyVersion() == null)
+                    cacheCtx.preloader().onInitialExchangeComplete(null);
+            }
 
             if (log.isDebugEnabled())
                 log.debug("Finished waiting for initial exchange: " + fut.exchangeId());
@@ -414,12 +392,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         for (AffinityReadyFuture f : readyFuts.values())
             f.onDone(stopErr);
 
-        for (GridDhtPartitionsExchangeFuture f : pendingExchangeFuts)
-            f.onDone(stopErr);
-
-        if (locExchFut != null)
-            locExchFut.onDone(stopErr);
-
         U.cancel(exchWorker);
 
         if (log.isDebugEnabled())
@@ -583,22 +555,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * @param nodeId New node ID.
-     * @param fut Exchange future.
-     */
-    void onDiscoveryEvent(UUID nodeId, GridDhtPartitionsExchangeFuture fut) {
-        if (!enterBusy())
-            return;
-
-        try {
-            addFuture(fut);
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /**
      * @param evt Discovery event.
      * @return Affinity topology version.
      */
@@ -1033,7 +989,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         U.warn(log, "Pending exchange futures:");
 
-        for (GridDhtPartitionsExchangeFuture fut : pendingExchangeFuts)
+        for (GridDhtPartitionsExchangeFuture fut : exchWorker.futQ)
             U.warn(log, ">>> " + fut);
 
         ExchangeFutureSet exchFuts = this.exchFuts;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 6c13399..daa4475 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -805,7 +805,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 boolean loc = desc.locallyConfigured();
 
-                if (loc || CU.affinityNode(locNode, filter)) {
+                if (loc || (desc.receivedOnDiscovery() && CU.affinityNode(locNode, filter))) {
                     CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
                     CachePluginManager pluginMgr = desc.pluginManager();
@@ -1958,7 +1958,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                             if (req.initiatingNodeId() == null)
                                 desc.staticallyConfigured(true);
 
-                            registeredCaches.put(maskNull(req.cacheName()), desc);
+                            desc.receivedOnDiscovery(true);
+
+                            DynamicCacheDescriptor old = registeredCaches.put(maskNull(req.cacheName()), desc);
+
+                            assert old == null : old;
 
                             ctx.discovery().setCacheFilter(
                                 req.cacheName(),
@@ -2474,10 +2478,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     }
                 }
                 else {
-                    if (req.clientStartOnly()) {
-                        assert req.initiatingNodeId() != null : req;
+                    assert req.initiatingNodeId() != null : req;
+
+                    // Cache already exists, exchange is needed only if client cache should be created.
+                    ClusterNode node = ctx.discovery().node(req.initiatingNodeId());
 
-                        needExchange = ctx.discovery().addClientNode(req.cacheName(),
+                    boolean clientReq = node != null &&
+                        !ctx.discovery().cacheAffinityNode(node, req.cacheName());
+
+                    if (req.clientStartOnly()) {
+                        needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
                             req.initiatingNodeId(),
                             req.nearCacheConfiguration() != null);
                     }
@@ -2488,12 +2498,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                                     "(a cache with the same name is already started): " + U.maskName(req.cacheName())));
                         }
                         else {
-                            // Cache already exists, exchange is needed only if client cache should be created.
-                            ClusterNode node = ctx.discovery().node(req.initiatingNodeId());
-
-                            boolean clientReq = node != null &&
-                                !ctx.discovery().cacheAffinityNode(node, req.cacheName());
-
                             needExchange = clientReq && ctx.discovery().addClientNode(req.cacheName(),
                                 req.initiatingNodeId(),
                                 req.nearCacheConfiguration() != null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index 76aaf72..a67b1de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -447,8 +447,8 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
 
                             if (v == null)
                                 it.remove();
-                            else if (!skipVals)
-                                info.value((CacheObject)v);
+                            else
+                                info.value(skipVals ? null : (CacheObject)v);
                         }
 
                         return infos;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 0202c53..abbe7b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -587,8 +587,11 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
         if (keysSize != 0) {
             Map<K, V> map = new GridLeanMap<>(keysSize);
 
-            for (GridCacheEntryInfo info : infos)
+            for (GridCacheEntryInfo info : infos) {
+                assert skipVals == (info.value() == null);
+
                 cctx.addResult(map, info.key(), info.value(), skipVals, false, deserializePortable, false);
+            }
 
             return map;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index fb2c5ad..41df53a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -825,8 +825,13 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
 
                 futVer = cctx.versions().next(topVer);
 
-                if (storeFuture())
-                    cctx.mvcc().addAtomicFuture(futVer, GridNearAtomicUpdateFuture.this);
+                if (storeFuture()) {
+                    if (!cctx.mvcc().addAtomicFuture(futVer, GridNearAtomicUpdateFuture.this)) {
+                        assert isDone() : GridNearAtomicUpdateFuture.this;
+
+                        return;
+                    }
+                }
 
                 // Assign version on near node in CLOCK ordering mode even if fastMap is false.
                 if (updVer == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index a7875f6..d9763f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -703,6 +703,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     CacheObject val = info.value();
                     KeyCacheObject key = info.key();
 
+                    assert skipVals == (info.value() == null);
+
                     cctx.addResult(map, key, val, skipVals, false, deserializePortable, false);
                 }
                 catch (GridCacheEntryRemovedException ignore) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePutAllRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePutAllRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePutAllRestartTest.java
index fc14085..ae99926 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePutAllRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePutAllRestartTest.java
@@ -163,12 +163,12 @@ public class IgniteCachePutAllRestartTest extends GridCommonAbstractTest {
 
             info("Running iteration on the node [idx=" + node + ", nodeId=" + ignite.cluster().localNode().id() + ']');
 
+            final IgniteCache<Integer, Integer> cache = ignite.cache(CACHE_NAME);
+
             IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     Thread.currentThread().setName("put-thread");
 
-                    IgniteCache<Integer, Integer> cache = ignite.cache(CACHE_NAME);
-
                     Random rnd = new Random();
 
                     long endTime = System.currentTimeMillis() + 60_000;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheGetFutureHangsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheGetFutureHangsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheGetFutureHangsSelfTest.java
index 8e8447e..e8622aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheGetFutureHangsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheGetFutureHangsSelfTest.java
@@ -18,23 +18,21 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
+import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReferenceArray;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
 
@@ -45,22 +43,14 @@ public class CacheGetFutureHangsSelfTest extends GridCommonAbstractTest {
     /** Grid count. */
     private static final int GRID_CNT = 8;
 
-    /** Grids. */
-    private static Ignite[] grids;
+    /** */
+    private AtomicReferenceArray<Ignite> nodes;
 
-    /** Ids. */
-    private static String[] ids;
-
-    /** Flags. */
-    private static AtomicBoolean[] flags;
-
-    /** Futs. */
-    private static Collection<IgniteInternalFuture> futs;
-
-    /** Alive grids. */
-    private static Set<Integer> aliveGrids;
+    /** */
+    private volatile boolean done;
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
@@ -81,17 +71,27 @@ public class CacheGetFutureHangsSelfTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 5 * 60_000;
+    }
+
     /**
      * @throws Exception If failed.
      */
-    public void testFailover() throws Exception {
-        int cnt = 10;
+    public void testContainsKeyFailover() throws Exception {
+        int cnt = 3;
 
         for (int i = 0; i < cnt; i++) {
             try {
-                U.debug("*** Iteration " + (i + 1) + '/' + cnt);
-
-                init();
+                log.info("Iteration: " + (i + 1) + '/' + cnt);
 
                 doTestFailover();
             }
@@ -102,54 +102,34 @@ public class CacheGetFutureHangsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Initializes test.
-     */
-    private void init() {
-        grids = new Ignite[GRID_CNT + 1];
-
-        ids = new String[GRID_CNT + 1];
-
-        aliveGrids = new HashSet<>();
-
-        flags = new AtomicBoolean[GRID_CNT + 1];
-
-        futs = new ArrayList<>();
-    }
-
-    /**
      * Executes one test iteration.
+     * @throws Exception If failed.
      */
     private void doTestFailover() throws Exception {
         try {
-            for (int i = 0; i < GRID_CNT + 1; i++) {
-                final IgniteEx grid = startGrid(i);
+            done = false;
 
-                grids[i] = grid;
+            nodes = new AtomicReferenceArray<>(GRID_CNT);
 
-                ids[i] = grid.localNode().id().toString();
+            startGridsMultiThreaded(GRID_CNT, false);
 
-                aliveGrids.add(i);
+            for (int i = 0; i < GRID_CNT ; i++)
+                assertTrue(nodes.compareAndSet(i, null, ignite(i)));
 
-                flags[i] = new AtomicBoolean();
-            }
+            List<IgniteInternalFuture> futs = new ArrayList<>();
 
             for (int i = 0; i < GRID_CNT + 1; i++) {
-                final int gridIdx = i;
-
                 futs.add(multithreadedAsync(new Runnable() {
                     @Override public void run() {
-                        IgniteCache cache = grids[gridIdx].cache(null);
+                        T2<Ignite, Integer> ignite;
 
-                        while (!flags[gridIdx].get()) {
-                            int idx = ThreadLocalRandom.current().nextInt(GRID_CNT + 1);
+                        while ((ignite = randomNode()) != null) {
+                            IgniteCache<Object, Object> cache = ignite.get1().cache(null);
 
-                            String id = ids[idx];
+                            for (int i = 0; i < 100; i++)
+                                cache.containsKey(ThreadLocalRandom.current().nextInt(100_000));
 
-                            if (id != null /*&& grids[gridIdx] != null*/) {
-                                //U.debug("!!! Grid containsKey start " + gridIdx);
-                                cache.containsKey(id);
-                                //U.debug("!!! Grid containsKey finished " + gridIdx);
-                            }
+                            assertTrue(nodes.compareAndSet(ignite.get2(), null, ignite.get1()));
 
                             try {
                                 Thread.sleep(ThreadLocalRandom.current().nextLong(50));
@@ -163,18 +143,15 @@ public class CacheGetFutureHangsSelfTest extends GridCommonAbstractTest {
 
                 futs.add(multithreadedAsync(new Runnable() {
                     @Override public void run() {
-                        IgniteCache cache = grids[gridIdx].cache(null);
+                        T2<Ignite, Integer> ignite;
 
-                        while (!flags[gridIdx].get()) {
-                            int idx = ThreadLocalRandom.current().nextInt(GRID_CNT + 1);
+                        while ((ignite = randomNode()) != null) {
+                            IgniteCache<Object, Object> cache = ignite.get1().cache(null);
 
-                            String id = ids[idx];
+                            for (int i = 0; i < 100; i++)
+                                cache.put(ThreadLocalRandom.current().nextInt(100_000), UUID.randomUUID());
 
-                            if (id != null /*&& grids[gridIdx] != null*/) {
-                                //U.debug("!!! Grid put start " + gridIdx);
-                                cache.put(id, UUID.randomUUID());
-                                //U.debug("!!! Grid put finished " + gridIdx);
-                            }
+                            assertTrue(nodes.compareAndSet(ignite.get2(), null, ignite.get1()));
 
                             try {
                                 Thread.sleep(ThreadLocalRandom.current().nextLong(50));
@@ -187,35 +164,50 @@ public class CacheGetFutureHangsSelfTest extends GridCommonAbstractTest {
                 }, 1, "put-thread-" + i));
             }
 
-            while (aliveGrids.size() > 1) {
-                final int gridToKill = ThreadLocalRandom.current().nextInt(GRID_CNT) + 1;
+            try {
+                int aliveGrids = GRID_CNT;
 
-                if (gridToKill > 0 && grids[gridToKill] != null) {
-                    U.debug("!!! Trying to kill grid " + gridToKill);
+                while (aliveGrids > 0) {
+                    T2<Ignite, Integer> ignite = randomNode();
 
-                    //synchronized (mons[gridToKill]) {
-                        U.debug("!!! Grid stop start " + gridToKill);
+                    assert ignite != null;
 
-                        grids[gridToKill].close();
+                    Ignite ignite0 = ignite.get1();
 
-                        aliveGrids.remove(gridToKill);
+                    log.info("Stop node: " + ignite0.name());
 
-                        grids[gridToKill] = null;
+                    ignite0.close();
 
-                        flags[gridToKill].set(true);
+                    log.info("Node stop finished: " + ignite0.name());
 
-                        U.debug("!!! Grid stop finished " + gridToKill);
-                    //}
+                    aliveGrids--;
                 }
             }
+            finally {
+                done = true;
+            }
 
-            Thread.sleep(ThreadLocalRandom.current().nextLong(100));
+            for (IgniteInternalFuture fut : futs)
+                fut.get();
         }
         finally {
-            flags[0].set(true);
+            done = true;
+        }
+    }
 
-            for (IgniteInternalFuture fut : futs)
-                fut.get();
+    /**
+     * @return Random node and its index.
+     */
+    @Nullable private T2<Ignite, Integer> randomNode() {
+        while (!done) {
+            int idx = ThreadLocalRandom.current().nextInt(GRID_CNT);
+
+            Ignite ignite = nodes.get(idx);
+
+            if (ignite != null && nodes.compareAndSet(idx, ignite, null))
+                return new T2<>(ignite, idx);
         }
+
+        return null;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
new file mode 100644
index 0000000..8b3d9d3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteCacheCreatePutTest extends GridCommonAbstractTest {
+    /** Grid count. */
+    private static final int GRID_CNT = 3;
+
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        discoSpi.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        OptimizedMarshaller marsh = new OptimizedMarshaller();
+        marsh.setRequireSerializable(false);
+
+        cfg.setMarshaller(marsh);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName("cache*");
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 3 * 60 * 1000L;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartNodes() throws Exception {
+        long stopTime = System.currentTimeMillis() + 2 * 60_000;
+
+        try {
+            int iter = 0;
+
+            while (System.currentTimeMillis() < stopTime) {
+                log.info("Iteration: " + iter++);
+
+                try {
+                    final AtomicInteger idx = new AtomicInteger();
+
+                    GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                        @Override
+                        public Void call() throws Exception {
+                            int node = idx.getAndIncrement();
+
+                            Ignite ignite = startGrid(node);
+
+                            IgniteCache<Object, Object> cache = ignite.getOrCreateCache("cache1");
+
+                            assertNotNull(cache);
+
+                            for (int i = 0; i < 100; i++)
+                                cache.put(i, i);
+
+                            return null;
+                        }
+                    }, GRID_CNT, "start");
+                }
+                finally {
+                    stopAllGrids();
+                }
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b54cbd7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index f8c9d26..b89bffd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -77,8 +77,10 @@ import org.apache.ignite.internal.processors.cache.IgniteStartCacheInTransaction
 import org.apache.ignite.internal.processors.cache.IgniteStartCacheInTransactionSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteSystemCacheOnClientTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheAffinityEarlyTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheGetFutureHangsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheNoValueClassOnServerNodeTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheCreatePutMultiNodeSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheCreatePutTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtTxPreloadSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheLockFailoverSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheMultiTxLockSelfTest;
@@ -205,6 +207,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteDynamicCacheStartNoExchangeTimeoutTest.class);
         suite.addTestSuite(CacheAffinityEarlyTest.class);
         suite.addTestSuite(IgniteCacheCreatePutMultiNodeSelfTest.class);
+        suite.addTestSuite(IgniteCacheCreatePutTest.class);
 
         suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class);
 
@@ -278,6 +281,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CrossCacheLockTest.class);
         suite.addTestSuite(IgniteCrossCacheTxSelfTest.class);
 
+        suite.addTestSuite(CacheGetFutureHangsSelfTest.class);
+
         return suite;
     }
 }
\ No newline at end of file


[4/5] ignite git commit: Test Failed

Posted by sb...@apache.org.
Test Failed


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

Branch: refs/heads/ignite-1607
Commit: 8616889556edf4aef532389c2639555facfc710d
Parents: 9702b52
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Oct 2 17:35:20 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Oct 2 17:35:20 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridNearCacheTxNodeFailureSelfTest.java      | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86168895/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
index 5735182..ca23646 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNearCacheTxNodeFailureSelfTest.java
@@ -28,4 +28,8 @@ public class GridNearCacheTxNodeFailureSelfTest extends GridCacheTxNodeFailureSe
     @Override protected CacheConfiguration cacheConfiguration(String gridName) {
         return super.cacheConfiguration(gridName).setNearConfiguration(new NearCacheConfiguration());
     }
+
+    @Override public void testPrimaryNodeFailureBackupCommitImplicit(){
+        fail("https://issues.apache.org/jira/browse/IGNITE-1611");
+    }
 }


[5/5] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1607

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


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

Branch: refs/heads/ignite-1607
Commit: a5103a87b9ef9f50f6b039f2eeddef7ad36f1f19
Parents: 3ed37b2 8616889
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 5 08:58:16 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 5 08:58:16 2015 +0300

----------------------------------------------------------------------
 modules/apache-license-gen/README.txt           |  33 ++++
 .../org/apache/ignite/IgniteFileSystem.java     |   2 +
 .../discovery/GridDiscoveryManager.java         |   2 +-
 .../cache/DynamicCacheDescriptor.java           |  17 ++
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../processors/cache/GridCacheMvccManager.java  |  20 +-
 .../GridCachePartitionExchangeManager.java      |  72 ++------
 .../processors/cache/GridCacheProcessor.java    |  26 +--
 .../cache/distributed/dht/GridDhtGetFuture.java |   4 +-
 .../dht/GridPartitionedGetFuture.java           |   5 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   9 +-
 .../distributed/near/GridNearGetFuture.java     |   2 +
 .../internal/processors/igfs/IgfsFileInfo.java  |  13 +-
 .../internal/processors/igfs/IgfsImpl.java      |  64 +------
 .../processors/igfs/IgfsMetaManager.java        | 180 +++++++++++++++++-
 .../cache/IgniteCachePutAllRestartTest.java     |   4 +-
 .../CacheGetFutureHangsSelfTest.java            | 156 ++++++++--------
 .../distributed/IgniteCacheCreatePutTest.java   | 125 +++++++++++++
 .../dht/GridNearCacheTxNodeFailureSelfTest.java |   4 +
 .../processors/igfs/IgfsAbstractSelfTest.java   | 185 +++++++++++--------
 .../testsuites/IgniteCacheTestSuite4.java       |   5 +
 21 files changed, 621 insertions(+), 309 deletions(-)
----------------------------------------------------------------------