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

[01/17] ignite git commit: ignite-4044 always authenticate local node

Repository: ignite
Updated Branches:
  refs/heads/ignite-4154-opt2 6cad38471 -> 6d6bfdc35


ignite-4044  always authenticate local node


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

Branch: refs/heads/ignite-4154-opt2
Commit: bac0cba7fddd412dfbff98163afbc15d81d5e0d4
Parents: cdae2ab
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Thu Nov 10 09:02:41 2016 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Thu Nov 10 09:02:41 2016 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 112 +++++++++++++++----
 1 file changed, 89 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bac0cba7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 55e5c89..0de787d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -815,33 +815,24 @@ class ServerImpl extends TcpDiscoveryImpl {
         SecurityCredentials locCred = (SecurityCredentials)locNode.getAttributes()
             .get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
 
+        boolean auth = false;
+
+        if (spi.nodeAuth != null && spi.nodeAuth.isGlobalNodeAuthentication()) {
+            localAuthentication(locCred);
+
+            auth = true;
+        }
+
         // Marshal credentials for backward compatibility and security.
-        marshalCredentials(locNode);
+        marshalCredentials(locNode, locCred);
 
         while (true) {
             if (!sendJoinRequestMessage()) {
                 if (log.isDebugEnabled())
                     log.debug("Join request message has not been sent (local node is the first in the topology).");
 
-                if (spi.nodeAuth != null) {
-                    // Authenticate local node.
-                    try {
-                        SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred);
-
-                        if (subj == null)
-                            throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
-
-                        Map<String, Object> attrs = new HashMap<>(locNode.attributes());
-
-                        attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj));
-                        attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
-
-                        locNode.setAttributes(attrs);
-                    }
-                    catch (IgniteException | IgniteCheckedException e) {
-                        throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
-                    }
-                }
+                if (!auth && spi.nodeAuth != null)
+                    localAuthentication(locCred);
 
                 locNode.order(1);
                 locNode.internalOrder(1);
@@ -920,6 +911,8 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
         }
 
+        locNode.attributes().remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
+
         assert locNode.order() != 0;
         assert locNode.internalOrder() != 0;
 
@@ -928,6 +921,33 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /**
+     * Authenticate local node.
+     *
+     * @param locCred Local security credentials for authentication.
+     * @throws IgniteSpiException If any error occurs.
+     */
+    private void localAuthentication(SecurityCredentials locCred){
+        assert spi.nodeAuth != null;
+        assert locCred != null;
+
+        try {
+            SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred);
+
+            if (subj == null)
+                throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
+
+            Map<String, Object> attrs = new HashMap<>(locNode.attributes());
+
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj));
+
+            locNode.setAttributes(attrs);
+
+        } catch (IgniteException | IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
+        }
+    }
+
+    /**
      * Tries to send join request message to a random node presenting in topology.
      * Address is provided by {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} and message is
      * sent to first node connection succeeded to.
@@ -1241,15 +1261,15 @@ class ServerImpl extends TcpDiscoveryImpl {
      * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
      *
      * @param node Node to marshall credentials for.
+     * @param cred Credentials for marshall.
      * @throws IgniteSpiException If marshalling failed.
      */
-    private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
+    private void marshalCredentials(TcpDiscoveryNode node, SecurityCredentials cred) throws IgniteSpiException {
         try {
             // Use security-unsafe getter.
             Map<String, Object> attrs = new HashMap<>(node.getAttributes());
 
-            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
-                U.marshal(spi.marshaller(), attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, spi.marshaller().marshal(cred));
 
             node.setAttributes(attrs);
         }
@@ -3906,6 +3926,52 @@ class ServerImpl extends TcpDiscoveryImpl {
                         if (top != null && !top.isEmpty()) {
                             spi.gridStartTime = msg.gridStartTime();
 
+                            if (spi.nodeAuth != null && spi.nodeAuth.isGlobalNodeAuthentication()) {
+                                TcpDiscoveryAbstractMessage authFail =
+                                    new TcpDiscoveryAuthFailedMessage(locNodeId, spi.locHost);
+
+                                try {
+                                    ClassLoader cl = U.resolveClassLoader(spi.ignite().configuration());
+
+                                    byte[] rmSubj = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT);
+                                    byte[] locSubj = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT);
+
+                                    SecurityContext rmCrd = spi.marshaller().unmarshal(rmSubj, cl);
+                                    SecurityContext locCrd = spi.marshaller().unmarshal(locSubj, cl);
+
+                                    if (!permissionsEqual(locCrd.subject().permissions(),
+                                        rmCrd.subject().permissions())) {
+                                        // Node has not pass authentication.
+                                        LT.warn(log,
+                                            null,
+                                            "Failed to authenticate local node " +
+                                                "(local authentication result is different from rest of topology) " +
+                                                "[nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']',
+                                            "Authentication failed [nodeId=" + U.id8(node.id()) +
+                                                ", addrs=" + U.addressesAsString(node) + ']');
+
+                                        joinRes.set(authFail);
+
+                                        spiState = AUTH_FAILED;
+
+                                        mux.notifyAll();
+
+                                        return;
+                                    }
+                                }
+                                catch (IgniteCheckedException e) {
+                                    U.error(log, "Failed to verify node permissions consistency (will drop the node): " + node, e);
+
+                                    joinRes.set(authFail);
+
+                                    spiState = AUTH_FAILED;
+
+                                    mux.notifyAll();
+
+                                    return;
+                                }
+                            }
+
                             for (TcpDiscoveryNode n : top) {
                                 assert n.internalOrder() < node.internalOrder() :
                                     "Invalid node [topNode=" + n + ", added=" + node + ']';


[07/17] ignite git commit: IGNITE-4100: Improved README and DEVNOTES for CPP.

Posted by sb...@apache.org.
IGNITE-4100: Improved README and DEVNOTES for CPP.


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

Branch: refs/heads/ignite-4154-opt2
Commit: eac5f0c05b85b620f979bd73d5043ae521b6f2f4
Parents: ef9d6cf
Author: Igor Sapego <is...@gridgain.com>
Authored: Thu Nov 10 21:13:13 2016 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Thu Nov 10 21:13:13 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/DEVNOTES.txt        | 10 +++++++---
 modules/platforms/cpp/README.txt          | 10 +++++++---
 modules/platforms/cpp/examples/README.txt | 16 +++++++++-------
 3 files changed, 23 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/eac5f0c0/modules/platforms/cpp/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/DEVNOTES.txt b/modules/platforms/cpp/DEVNOTES.txt
index 924b0d8..5d0b2eb 100644
--- a/modules/platforms/cpp/DEVNOTES.txt
+++ b/modules/platforms/cpp/DEVNOTES.txt
@@ -3,7 +3,9 @@ Apache Ignite C++ Build Instructions
 Here you can find instruction on how to build Apache Ignite C++ core library and
 stand-alone node binary. To build examples you need to build and install core Apache
 Ignite library then refer to $IGNITE_HOME/platforms/cpp/example/README.txt for
-futher instructions.
+further instructions.
+
+For details on ODBC driver installation and usage please refer to odbc/README.txt.
 
 Building on Linux With Autotools
 ----------------------------------
@@ -15,7 +17,9 @@ Common Requirements:
  * JAVA_HOME environment variable must be set pointing to Java installation directory.
  * IGNITE_HOME environment variable must be set to Ignite installation directory.
  * To build ODBC driver you need to install ODBC Driver Manager on you system.
-   Apache Ignite has been tested with UnixODBC.
+   Apache Ignite has been tested with UnixODBC. Make sure that you install development
+   version of the driver manager which contains header files that are necessary for
+   building a driver.
 
 Building the Apache Ignite C++ components:
  * Navigate to the directory $IGNITE_HOME/platforms/cpp
@@ -37,7 +41,7 @@ use configure script with the following arguments:
 With the config like that you should not have any ignite dependencies as jvm.dll installed
 to build ODBC driver.
 
-NOTE: You may want to use "make install" command to install Ignite libraries and heasders
+NOTE: You may want to use "make install" command to install Ignite libraries and headers
 for your operation system. Note however that this action may require superuser privileges.
 
 Building on Windows with Visual Studio (tm)

http://git-wip-us.apache.org/repos/asf/ignite/blob/eac5f0c0/modules/platforms/cpp/README.txt
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/README.txt b/modules/platforms/cpp/README.txt
index dbde9f0..b1d7eef 100644
--- a/modules/platforms/cpp/README.txt
+++ b/modules/platforms/cpp/README.txt
@@ -2,7 +2,7 @@ Apache Ignite C++
 ==================================
 
 Apache Ignite C++ provides data grid functionality.
-Using Apache Ignite C++ APIs you can execute perform concurrent operations on
+Using Apache Ignite C++ APIs you can perform concurrent operations on
 the data stored in cache.
 
 Apache Ignite C++ can access cluster and share data with .Net and
@@ -11,14 +11,18 @@ Java applications using binary object format.
 Support for the following will be added in next releases:
  * ACID transactions management.
  * Distributed locks.
- * Async operations.
+ * Asynchronous operations.
  * Cache SQL continuous queries.
  * Event listening.
  * Compute grid functionality.
 
 Full source code is provided. Users should build the library for intended platform.
+
 For build instructions please refer to DEVNOTES.txt.
 
+For details on ODBC driver installation and usage please refer to
+$IGNITE_HOME/platforms/cpp/odbc/README.txt.
+
 Linux info
 ==============
 
@@ -49,7 +53,7 @@ Files list:
 
  * ignite.exe - executable to start standalone Ignite C++ node.
  * ignite.core.dll - Ignite C++ API library.
- * odbc.dll - Ignite ODBC driver.
+ * ignite.odbc.dll - Ignite ODBC driver.
  
 Development:
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/eac5f0c0/modules/platforms/cpp/examples/README.txt
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/README.txt b/modules/platforms/cpp/examples/README.txt
index 29df732..91e4869 100644
--- a/modules/platforms/cpp/examples/README.txt
+++ b/modules/platforms/cpp/examples/README.txt
@@ -6,8 +6,13 @@ Common requirements
  * Java Development Kit (JDK) must be installed: https://java.com/en/download/index.jsp
  * JAVA_HOME environment variable must be set pointing to Java installation directory.
  * IGNITE_HOME environment variable must be set to Ignite installation directory.
- * Ignite must be build and packaged using Maven. You can use the followin Maven command: mvn clean package -DskipTests
- * Apache Ignite C++ must be built according to instructions for your platform.
+ * Ignite must be built and packaged using Maven. You can use the following Maven command: mvn clean package -DskipTests
+ * Apache Ignite C++ must be built according to instructions for your platform. Refer to
+   $IGNITE_HOME/platforms/cpp/DEVNOTES.txt for instructions.
+ * For odbc-example additionally ODBC Driver Manager must be present and installed on your platform and
+   Apache Ignite ODBC driver must be built and installed according to instructions for your platform. Refer to
+   $IGNITE_HOME/platforms/cpp/DEVNOTES.txt for build instructions and to $IGNITE_HOME/platforms/cpp/odbc/README.txt.
+   for installation instructions.
 
 Running examples on Linux
 ----------------------------------
@@ -22,11 +27,8 @@ To build examples execute the following commands one by one from examples root d
 
 As a result executables will appear in every example's directory.
 
-Before running examples ensure that:
- * LD_LIBRARY_PATH environment variable is set and pointing to a directory with "libjvm.so" library. Typically this
-   library is located in $JAVA_HOME/jre/lib/amd64/server directory.
- * For odbc-example additionaly ODBC Driver Manager must be present and installed on your platform and
-   Apache Ignite ODBC driver must be built and installed according to instructions for your platform.
+Before running examples ensure that LD_LIBRARY_PATH environment variable is set and pointing to a directory with
+"libjvm.so" library. Typically this library is located in $JAVA_HOME/jre/lib/amd64/server directory.
 
 Running examples on Windows
 ----------------------------------


[10/17] ignite git commit: IGNITE-4145: Fixes "No query result found for request" exception when running multiple queries concurrently. This closes #1218.

Posted by sb...@apache.org.
IGNITE-4145: Fixes "No query result found for request" exception when running multiple queries concurrently. This closes #1218.


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

Branch: refs/heads/ignite-4154-opt2
Commit: a70f0bac3ac2487b8ab58598ad921daa952b485f
Parents: 53876d3
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Nov 11 13:03:40 2016 +0300
Committer: Andrey V. Mashenkov <an...@gmail.com>
Committed: Fri Nov 11 13:03:40 2016 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/GridMergeIndex.java        | 49 +++++++++++-----
 .../IgniteCacheQueryMultiThreadedSelfTest.java  | 59 ++++++++++++++++++++
 2 files changed, 93 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a70f0bac/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
index 3914bd7..7ac2ee3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
@@ -177,36 +177,50 @@ public abstract class GridMergeIndex extends BaseIndex {
     public final void addPage(GridResultPage page) {
         int pageRowsCnt = page.rowsInPage();
 
-        if (pageRowsCnt != 0)
-            addPage0(page);
-
         Counter cnt = remainingRows.get(page.source());
 
+        // RemainingRowsCount should be updated before page adding to avoid race
+        // in GridMergeIndexUnsorted cursor iterator
+        int remainingRowsCount;
+
         int allRows = page.response().allRows();
 
         if (allRows != -1) { // Only the first page contains allRows count and is allowed to init counter.
-            assert !cnt.initialized : "Counter is already initialized.";
+            assert cnt.state == State.UNINITIALIZED : "Counter is already initialized.";
+
+            remainingRowsCount = cnt.addAndGet(allRows - pageRowsCnt);
 
-            cnt.addAndGet(allRows);
             expRowsCnt.addAndGet(allRows);
 
+            // Add page before setting initialized flag to avoid race condition with adding last page
+            if (pageRowsCnt > 0)
+                addPage0(page);
+
             // We need this separate flag to handle case when the first source contains only one page
             // and it will signal that all remaining counters are zero and fetch is finished.
-            cnt.initialized = true;
+            cnt.state = State.INITIALIZED;
         }
+        else {
+            remainingRowsCount = cnt.addAndGet(-pageRowsCnt);
 
-        if (cnt.addAndGet(-pageRowsCnt) == 0) { // Result can be negative in case of race between messages, it is ok.
-            boolean last = true;
+            if (pageRowsCnt > 0)
+                addPage0(page);
+        }
 
-            for (Counter c : remainingRows.values()) { // Check all the sources.
-                if (c.get() != 0 || !c.initialized) {
-                    last = false;
+        if (remainingRowsCount == 0) { // Result can be negative in case of race between messages, it is ok.
+            if (cnt.state == State.UNINITIALIZED)
+                return;
 
-                    break;
-                }
+            // Guarantee that finished state possible only if counter is zero and all pages was added
+            cnt.state = State.FINISHED;
+
+            for (Counter c : remainingRows.values()) { // Check all the sources.
+                if (c.state != State.FINISHED)
+                    return;
             }
 
-            if (last && lastSubmitted.compareAndSet(false, true)) {
+            if (lastSubmitted.compareAndSet(false, true)) {
+                // Add page-marker that last page was added
                 addPage0(new GridResultPage(null, page.source(), null) {
                     @Override public boolean isLast() {
                         return true;
@@ -426,11 +440,16 @@ public abstract class GridMergeIndex extends BaseIndex {
         }
     }
 
+    /** */
+    enum State {
+        UNINITIALIZED, INITIALIZED, FINISHED
+    }
+
     /**
      * Counter with initialization flag.
      */
     private static class Counter extends AtomicInteger {
         /** */
-        volatile boolean initialized;
+        volatile State state = State.UNINITIALIZED;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a70f0bac/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
index be644e2..efa6bd6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
@@ -21,6 +21,7 @@ import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
@@ -34,6 +35,7 @@ import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -730,6 +732,63 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
     }
 
     /**
+     * SqlFieldsQuery paging mechanics stress test
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"TooBroadScope"})
+    public void testMultiThreadedSqlFieldsQuery() throws Throwable {
+        int threadCnt = 16;
+        final int keyCnt = 1100; // set resultSet size bigger than page size
+        final int logMod = 5000;
+
+        final Ignite g = grid(0);
+
+        // Put test values into cache.
+        final IgniteCache<Integer, TestValue> c = g.cache(null);
+
+        for (int i = 0; i < keyCnt; i++)
+            c.put(i, new TestValue(i));
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        final AtomicBoolean done = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = multithreadedAsync(
+            new CAX() {
+                @Override public void applyx() throws IgniteCheckedException {
+                    int iter = 0;
+
+                    while (!done.get() && !Thread.currentThread().isInterrupted()) {
+                        iter++;
+
+                        List<List<?>> entries =
+                            c.query(new SqlFieldsQuery("SELECT * from TestValue").setPageSize(100)).getAll();
+
+                        assert entries != null;
+
+                        assertEquals("Entries count is not as expected on iteration: " + iter, keyCnt, entries.size());
+
+                        if (cnt.incrementAndGet() % logMod == 0) {
+                            GridCacheQueryManager<Object, Object> qryMgr =
+                                ((IgniteKernal)g).internalCache().context().queries();
+
+                            assert qryMgr != null;
+
+                            qryMgr.printMemoryStats();
+                        }
+                    }
+                }
+            }, threadCnt);
+
+        Thread.sleep(DURATION);
+
+        done.set(true);
+
+        fut.get();
+    }
+
+    /**
      * Test value.
      */
     private static class TestValue implements Serializable {


[16/17] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-1.6.11' into ignite-4154-opt2

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/community/ignite-1.6.11' into ignite-4154-opt2

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
#	modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java


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

Branch: refs/heads/ignite-4154-opt2
Commit: ffd35415d9aa276142d7f1b9e6cb13ad6378694e
Parents: 166552f 7128a39
Author: sboikov <sb...@gridgain.com>
Authored: Fri Nov 11 15:50:57 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Nov 11 15:50:57 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteDataStreamer.java   |   2 +-
 .../internal/ComputeTaskInternalFuture.java     |  11 +
 .../processors/cache/GridCacheContext.java      |   8 +
 .../processors/cache/GridCacheMapEntry.java     |   5 +-
 .../processors/cache/GridCacheMvccManager.java  |  77 +++
 .../GridCachePartitionExchangeManager.java      |   5 +
 .../cache/GridCacheSharedContext.java           |   1 +
 .../datastreamer/DataStreamProcessor.java       | 104 +++-
 .../datastreamer/DataStreamerImpl.java          | 603 ++++++++++++++-----
 .../ignite/internal/util/GridLogThrottle.java   |  29 +-
 .../internal/util/future/GridFutureAdapter.java |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 260 ++++----
 .../cache/IgniteCacheDynamicStopSelfTest.java   |  48 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |   2 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 251 +++++++-
 ...ncurrentGridStartSelfTestAllowOverwrite.java |  30 +
 ...cingDelayedPartitionMapExchangeSelfTest.java |   8 +-
 .../GridCacheRebalancingSyncSelfTest.java       |  18 +-
 ...ComputeJobExecutionErrorToLogManualTest.java |  88 +++
 .../DataStreamProcessorSelfTest.java            |   4 +-
 .../datastreamer/DataStreamerImplSelfTest.java  | 170 ++++--
 .../DataStreamerMultiThreadedSelfTest.java      |   2 -
 .../datastreamer/DataStreamerTimeoutTest.java   |  92 ++-
 .../junits/common/GridCommonAbstractTest.java   |  11 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 modules/hadoop/pom.xml                          |  49 ++
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   2 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../hadoop/delegate/HadoopDelegateUtils.java    |   4 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |   3 +-
 ...KerberosHadoopFileSystemFactorySelfTest.java |   3 +-
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   2 +-
 ...adoopIgfsSecondaryFileSystemTestAdapter.java |   3 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../query/h2/GridH2ResultSetIterator.java       |  62 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   4 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    |  10 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  34 +-
 .../query/h2/twostep/GridMergeIndex.java        |  49 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../cache/CacheSqlQueryValueCopySelfTest.java   | 226 +++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |  59 ++
 .../query/h2/sql/GridQueryParsingTest.java      |  11 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   2 +
 modules/platforms/cpp/DEVNOTES.txt              |  10 +-
 modules/platforms/cpp/README.txt                |  10 +-
 modules/platforms/cpp/examples/README.txt       |  16 +-
 47 files changed, 1887 insertions(+), 523 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ffd35415/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------


[12/17] ignite git commit: debug

Posted by sb...@apache.org.
debug


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

Branch: refs/heads/ignite-4154-opt2
Commit: 166552f1eb68c98cfd39798db514960e0c4180a1
Parents: 6cad384
Author: sboikov <sb...@gridgain.com>
Authored: Fri Nov 11 15:14:11 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Nov 11 15:14:11 2016 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 66 ++++++++++++++++----
 .../tcp/internal/TcpDiscoveryStatistics.java    |  2 +-
 .../TcpDiscoveryJoinRequestMessage.java         |  2 +
 3 files changed, 56 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/166552f1/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index b0812fb..b614263 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -129,6 +129,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRedirectToClient;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -171,15 +172,14 @@ class ServerImpl extends TcpDiscoveryImpl {
     private static final int ENSURED_MSG_HIST_SIZE = getInteger(IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE, 512);
 
     /** */
-    private static final boolean SEND_JOIN_REQ_DIRECTLY = getBoolean("SEND_JOIN_REQ_DIRECTLY", false);
+    private static final boolean SEND_JOIN_REQ_DIRECTLY = getBoolean("SEND_JOIN_REQ_DIRECTLY", true);
 
     /** */
     private static final IgniteProductVersion CUSTOM_MSG_ALLOW_JOINING_FOR_VERIFIED_SINCE =
         IgniteProductVersion.fromString("1.5.0");
 
     /** */
-    private final ThreadPoolExecutor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>());
+    private ThreadPoolExecutor utilityPool;
 
     /** Nodes ring. */
     @GridToStringExclude
@@ -301,6 +301,13 @@ class ServerImpl extends TcpDiscoveryImpl {
             spiState = DISCONNECTED;
         }
 
+        utilityPool = new IgniteThreadPoolExecutor("disco-pool",
+            spi.ignite().name(),
+            0,
+            1,
+            2000,
+            new LinkedBlockingQueue<Runnable>());
+
         if (debugMode) {
             if (!log.isInfoEnabled())
                 throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
@@ -1440,6 +1447,9 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             TcpDiscoveryNode coord = resolveCoordinator();
 
+            if (!locNode.equals(coord) || locNode.order() != 1)
+                return;
+
             log.info("Discovery SPI statistics [statistics=" + spi.stats + ", spiState=" + spiStateCopy() +
                 ", coord=" + coord +
                 ", next=" + (msgWorker != null ? msgWorker.next : "N/A") +
@@ -3751,20 +3761,29 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     "[msg=" + msg + ", crd=" + crd + ", next=" + next + ']');
                             }
 
+                            log.info("Will send join request directly to coordinator " +
+                                "[cnt=" + joiningNodes.size() + ", msg=" + msg + ", crd=" + crd + ", next=" + next + ']');
+
                             utilityPool.submit(new Runnable() {
                                 @Override public void run() {
                                     IgniteSpiException sndErr = null;
                                     Integer res = null;
 
+                                    TcpDiscoveryJoinRequestMessage msg0 =
+                                        new TcpDiscoveryJoinRequestMessage(msg.node(), msg.discoveryData());
+
                                     try {
-                                        res = trySendMessageDirectly(crd, msg);
+                                        res = trySendMessageDirectly(crd, msg0);
 
                                         if (F.eq(RES_OK, res)) {
                                             if (log.isDebugEnabled()) {
                                                 log.debug("Sent join request directly to coordinator " +
-                                                    "[msg=" + msg + ", crd=" + crd + ']');
+                                                    "[msg=" + msg0 + ", crd=" + crd + ']');
                                             }
 
+                                            log.info("Sent join request directly to coordinator " +
+                                                "[msg=" + msg0 + ", crd=" + crd + ']');
+
                                             return;
                                         }
                                     }
@@ -3774,10 +3793,14 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                     if (log.isDebugEnabled()) {
                                         log.debug("Failed to send join request to coordinator, will process from " +
-                                            "message worker [msg=" + msg + ", crd=" + crd + ", err=" + sndErr +
+                                            "message worker [msg=" + msg0 + ", crd=" + crd + ", err=" + sndErr +
                                             ", res=" + res + ']');
                                     }
 
+                                    log.info("Failed to send join request to coordinator, will process from " +
+                                        "message worker [msg=" + msg0 + ", crd=" + crd + ", err=" + sndErr +
+                                        ", res=" + res + ']');
+
                                     msg.directSendFailed(true);
 
                                     msgWorker.addMessage(msg);
@@ -5542,6 +5565,9 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             ClientMessageWorker clientMsgWrk = null;
 
+            TcpDiscoveryAbstractMessage msg = null;
+            Exception sockE = null;
+
             try {
                 InputStream in;
 
@@ -5602,7 +5628,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     // Restore timeout.
                     sock.setSoTimeout(timeout);
 
-                    TcpDiscoveryAbstractMessage msg = spi.readMessage(sock, in, spi.netTimeout);
+                    msg = spi.readMessage(sock, in, spi.netTimeout);
 
                     // Ping.
                     if (msg instanceof TcpDiscoveryPingRequest) {
@@ -5707,6 +5733,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
                 catch (IOException e) {
+                    sockE = e;
+
                     if (log.isDebugEnabled())
                         U.error(log, "Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
 
@@ -5734,6 +5762,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
                 catch (IgniteCheckedException e) {
+                    sockE = e;
+
                     if (log.isDebugEnabled())
                         U.error(log, "Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
 
@@ -5763,8 +5793,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 while (!isInterrupted()) {
                     try {
-                        TcpDiscoveryAbstractMessage msg = U.unmarshal(spi.marshaller(), in,
-                            U.resolveClassLoader(spi.ignite().configuration()));
+                        msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration()));
 
                         msg.senderNodeId(nodeId);
 
@@ -5791,9 +5820,13 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                 if (clientMsgWrk != null && ok)
                                     continue;
-                                else
+                                else {
+                                    if (locNode.order() == 1)
+                                        log.info("Processed join request, close connection [msg=" + msg + ']');
+
                                     // Direct join request - no need to handle this socket anymore.
                                     break;
+                                }
                             }
                         }
                         else if (msg instanceof TcpDiscoveryClientReconnectMessage) {
@@ -5966,6 +5999,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                             processClientHeartbeatMessage(heartbeatMsg);
                     }
                     catch (IgniteCheckedException e) {
+                        sockE = e;
+
                         if (log.isDebugEnabled())
                             U.error(log, "Caught exception on message read [sock=" + sock +
                                 ", locNodeId=" + locNodeId + ", rmtNodeId=" + nodeId + ']', e);
@@ -5993,6 +6028,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                         return;
                     }
                     catch (IOException e) {
+                        sockE = e;
+
                         if (log.isDebugEnabled())
                             U.error(log, "Caught exception on message read [sock=" + sock + ", locNodeId=" + locNodeId +
                                 ", rmtNodeId=" + nodeId + ']', e);
@@ -6016,6 +6053,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
             }
             finally {
+                if (locNode.order() == 1)
+                    log.error("Close sock [msg=" + msg + ", err=" + sockE + ']', sockE);
+
                 if (clientMsgWrk != null) {
                     if (log.isDebugEnabled())
                         log.debug("Client connection failed [sock=" + sock + ", locNodeId=" + locNodeId +
@@ -6060,11 +6100,11 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             TcpDiscoverySpiState state = spiStateCopy();
 
-            long socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
+            long sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
                 spi.getSocketTimeout();
 
             if (state == CONNECTED) {
-                spi.writeToSocket(msg, sock, RES_OK, socketTimeout);
+                spi.writeToSocket(msg, sock, RES_OK, sockTimeout);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + RES_OK + ']');
@@ -6101,7 +6141,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     // Local node is stopping. Remote node should try next one.
                     res = RES_CONTINUE_JOIN;
 
-                spi.writeToSocket(msg, sock, res, socketTimeout);
+                spi.writeToSocket(msg, sock, res, sockTimeout);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + res + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/166552f1/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
index 9e73632..7c6025d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
@@ -687,6 +687,6 @@ public class TcpDiscoveryStatistics {
 
     /** {@inheritDoc} */
     @Override public synchronized String toString() {
-        return S.toString(TcpDiscoveryStatistics.class, this);
+        return "Stats [created=" + sockReadersCreated + ", rmvd=" + sockReadersRmv + ']';//S.toString(TcpDiscoveryStatistics.class, this);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/166552f1/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
index 202c29c..4422919 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.spi.discovery.tcp.messages;
 
 import java.util.Map;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
@@ -37,6 +38,7 @@ public class TcpDiscoveryJoinRequestMessage extends TcpDiscoveryAbstractMessage
     private final Map<Integer, byte[]> discoData;
 
     /** */
+    @GridToStringExclude
     private transient boolean directSndFailed;
 
     /**


[11/17] ignite git commit: IGNITE-4053: Moved task error output from console to logger. This closes #1160.

Posted by sb...@apache.org.
IGNITE-4053: Moved task error output from console to logger. This closes #1160.


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

Branch: refs/heads/ignite-4154-opt2
Commit: 8bb8bdda2e846dcc92a2fd449e64d7594b2700ed
Parents: a70f0ba
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Fri Nov 11 15:01:14 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Nov 11 15:01:14 2016 +0300

----------------------------------------------------------------------
 .../internal/ComputeTaskInternalFuture.java     | 11 +++
 .../internal/util/future/GridFutureAdapter.java | 12 ++-
 ...ComputeJobExecutionErrorToLogManualTest.java | 88 ++++++++++++++++++++
 3 files changed, 109 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8bb8bdda/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java
index d511480..d6c54d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.compute.ComputeJobSibling;
 import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.compute.ComputeTaskFuture;
@@ -59,6 +60,9 @@ public class ComputeTaskInternalFuture<R> extends GridFutureAdapter<R> {
     @GridToStringExclude
     private ComputeFuture<R> userFut;
 
+    /** */
+    private transient IgniteLogger log;
+
     /**
      * @param ses Task session instance.
      * @param ctx Kernal context.
@@ -71,6 +75,8 @@ public class ComputeTaskInternalFuture<R> extends GridFutureAdapter<R> {
         this.ctx = ctx;
 
         userFut = new ComputeFuture<>(this);
+
+        log = ctx.log(ComputeTaskInternalFuture.class);
     }
 
     /**
@@ -247,6 +253,11 @@ public class ComputeTaskInternalFuture<R> extends GridFutureAdapter<R> {
         return S.toString(ComputeTaskInternalFuture.class, this, "super", super.toString());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteLogger logger() {
+        return log;
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8bb8bdda/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index ea7a202..2cd534e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -21,6 +21,7 @@ import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.AbstractQueuedSynchronizer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -263,11 +264,11 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
             lsnr.apply(this);
         }
         catch (IllegalStateException e) {
-            U.error(null, "Failed to notify listener (is grid stopped?) [fut=" + this +
+            U.error(logger(), "Failed to notify listener (is grid stopped?) [fut=" + this +
                 ", lsnr=" + lsnr + ", err=" + e.getMessage() + ']', e);
         }
         catch (RuntimeException | Error e) {
-            U.error(null, "Failed to notify listener: " + lsnr, e);
+            U.error(logger(), "Failed to notify listener: " + lsnr, e);
 
             throw e;
         }
@@ -413,6 +414,13 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
         return s == INIT ? "INIT" : s == CANCELLED ? "CANCELLED" : "DONE";
     }
 
+    /**
+     * @return Logger instance.
+     */
+    @Nullable public IgniteLogger logger() {
+        return null;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridFutureAdapter.class, this, "state", state());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8bb8bdda/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java
new file mode 100644
index 0000000..691bc9f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.compute;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteRunnable;
+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.junits.common.GridCommonAbstractTest;
+
+/**
+ * Manual test to reproduce IGNITE-4053
+ */
+public class GridComputeJobExecutionErrorToLogManualTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int GRID_CNT = 2;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(GRID_CNT, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If fails.
+     */
+    public void testRuntimeException() throws Exception {
+        Ignite ignite = grid(0);
+
+        IgniteCompute async = ignite.compute().withAsync();
+        async.run(new IgniteRunnable() {
+            @Override public void run() {
+                try {
+                    Thread.sleep(500);
+                }
+                catch (InterruptedException e) {
+                    // No-op.
+                }
+            }
+        });
+
+        async.future().listen(new IgniteInClosure<IgniteFuture<Object>>() {
+            @Override public void apply(IgniteFuture<Object> future) {
+                throw new RuntimeException();
+            }
+        });
+    }
+}
\ No newline at end of file


[09/17] ignite git commit: Fixed classnames.properties generation for ignite-hadoop module.

Posted by sb...@apache.org.
Fixed classnames.properties generation for ignite-hadoop module.


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

Branch: refs/heads/ignite-4154-opt2
Commit: 53876d3f1b3e561702fc0cd8c6ddd078d6c7dce6
Parents: e159fde
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Nov 11 10:26:09 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Nov 11 10:26:09 2016 +0700

----------------------------------------------------------------------
 modules/hadoop/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/53876d3f/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 98f8576..f5bc081 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -168,7 +168,7 @@
 #
                                     ]]>
                                 </argument>
-                                <argument>org.apache.ignite</argument>
+                                <argument>org.apache.ignite.hadoop:org.apache.ignite.internal.processors.hadoop</argument>
                             </arguments>
                         </configuration>
                     </execution>


[14/17] ignite git commit: ignite-4154 Optimize amount of data stored in discovery history Discovery history optimizations: - remove discarded message for discovery pending messages - remove duplicated data from TcpDiscoveryNodeAddedMessage.oldNodesDisco

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
index a4ff04b..90d6242 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
@@ -22,14 +22,19 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.UUID;
+
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.NotNull;
@@ -48,6 +53,10 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     private Map<Integer, GridDhtPartitionFullMap> parts;
 
     /** */
+    @GridDirectMap(keyType = Integer.class, valueType = Integer.class)
+    private Map<Integer, Integer> dupPartsData;
+
+    /** */
     private byte[] partsBytes;
 
     /** Partitions update counters. */
@@ -61,6 +70,10 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     /** Topology version. */
     private AffinityTopologyVersion topVer;
 
+    /** */
+    @GridDirectTransient
+    private transient boolean compress;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -84,6 +97,13 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     }
 
     /**
+     * @param compress {@code True} if it is possible to use compression for message.
+     */
+    public void compress(boolean compress) {
+        this.compress = compress;
+    }
+
+    /**
      * @return Local partitions.
      */
     public Map<Integer, GridDhtPartitionFullMap> partitions() {
@@ -92,14 +112,34 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
     /**
      * @param cacheId Cache ID.
+     * @return {@code True} if message contains full map for given cache.
+     */
+    public boolean containsCache(int cacheId) {
+        return parts != null && parts.containsKey(cacheId);
+    }
+
+    /**
+     * @param cacheId Cache ID.
      * @param fullMap Full partitions map.
+     * @param dupDataCache Optional ID of cache with the same partition state map.
      */
-    public void addFullPartitionsMap(int cacheId, GridDhtPartitionFullMap fullMap) {
+    public void addFullPartitionsMap(int cacheId, GridDhtPartitionFullMap fullMap, @Nullable Integer dupDataCache) {
         if (parts == null)
             parts = new HashMap<>();
 
-        if (!parts.containsKey(cacheId))
+        if (!parts.containsKey(cacheId)) {
             parts.put(cacheId, fullMap);
+
+            if (dupDataCache != null) {
+                assert compress;
+                assert parts.containsKey(dupDataCache);
+
+                if (dupPartsData == null)
+                    dupPartsData = new HashMap<>();
+
+                dupPartsData.put(cacheId, dupDataCache);
+            }
+        }
     }
 
     /**
@@ -132,11 +172,38 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
 
-        if (parts != null && partsBytes == null)
-            partsBytes = U.marshal(ctx, parts);
+        boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null);
+
+        if (marshal) {
+            byte[] partsBytes0 = null;
+            byte[] partCntrsBytes0 = null;
+
+            if (parts != null && partsBytes == null)
+                partsBytes0 = U.marshal(ctx, parts);
 
-        if (partCntrs != null && partCntrsBytes == null)
-            partCntrsBytes = U.marshal(ctx, partCntrs);
+            if (partCntrs != null && partCntrsBytes == null)
+                partCntrsBytes0 = U.marshal(ctx, partCntrs);
+
+            if (compress) {
+                assert !compressed();
+
+                try {
+                    byte[] partsBytesZip = U.zip(partsBytes0);
+                    byte[] partCntrsBytesZip = U.zip(partCntrsBytes0);
+
+                    partsBytes0 = partsBytesZip;
+                    partCntrsBytes0 = partCntrsBytesZip;
+
+                    compressed(true);
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(ctx.logger(getClass()), "Failed to compress partitions data: " + e, e);
+                }
+            }
+
+            partsBytes = partsBytes0;
+            partCntrsBytes = partCntrsBytes0;
+        }
     }
 
     /**
@@ -157,14 +224,49 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(ctx, ldr);
 
-        if (partsBytes != null && parts == null)
-            parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        if (partsBytes != null && parts == null) {
+            if (compressed())
+                parts = U.unmarshalZip(ctx.marshaller(), partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+            else
+                parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+
+            if (dupPartsData != null) {
+                assert parts != null;
+
+                for (Map.Entry<Integer, Integer> e : dupPartsData.entrySet()) {
+                    GridDhtPartitionFullMap map1 = parts.get(e.getKey());
+                    GridDhtPartitionFullMap map2 = parts.get(e.getValue());
+
+                    assert map1 != null : e.getKey();
+                    assert map2 != null : e.getValue();
+                    assert map1.size() == map2.size();
+
+                    for (Map.Entry<UUID, GridDhtPartitionMap2> e0 : map2.entrySet()) {
+                        GridDhtPartitionMap2 partMap1 = map1.get(e0.getKey());
+
+                        assert partMap1 != null && partMap1.map().isEmpty() : partMap1;
+                        assert !partMap1.hasMovingPartitions() : partMap1;
+
+                        GridDhtPartitionMap2 partMap2 = e0.getValue();
+
+                        assert partMap2 != null;
+
+                        for (Map.Entry<Integer, GridDhtPartitionState> stateEntry : partMap2.entrySet())
+                            partMap1.put(stateEntry.getKey(), stateEntry.getValue());
+                    }
+                }
+            }
+        }
 
         if (parts == null)
             parts = new HashMap<>();
 
-        if (partCntrsBytes != null && partCntrs == null)
-            partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        if (partCntrsBytes != null && partCntrs == null) {
+            if (compressed())
+                partCntrs = U.unmarshalZip(ctx.marshaller(), partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+            else
+                partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        }
 
         if (partCntrs == null)
             partCntrs = new HashMap<>();
@@ -185,19 +287,25 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
         }
 
         switch (writer.state()) {
-            case 5:
+            case 6:
+                if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
                 if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 8:
                 if (!writer.writeByteArray("partsBytes", partsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 9:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -219,7 +327,15 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
             return false;
 
         switch (reader.state()) {
-            case 5:
+            case 6:
+                dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
                 partCntrsBytes = reader.readByteArray("partCntrsBytes");
 
                 if (!reader.isLastRead())
@@ -227,7 +343,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 6:
+            case 8:
                 partsBytes = reader.readByteArray("partsBytes");
 
                 if (!reader.isLastRead())
@@ -235,7 +351,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
-            case 7:
+            case 9:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -255,7 +371,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
index e4356b1..bf08f0a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
@@ -23,12 +23,16 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
@@ -45,6 +49,10 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     @GridDirectTransient
     private Map<Integer, GridDhtPartitionMap2> parts;
 
+    /** */
+    @GridDirectMap(keyType = Integer.class, valueType = Integer.class)
+    private Map<Integer, Integer> dupPartsData;
+
     /** Serialized partitions. */
     private byte[] partsBytes;
 
@@ -59,6 +67,10 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     /** */
     private boolean client;
 
+    /** */
+    @GridDirectTransient
+    private transient boolean compress;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -70,13 +82,16 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
      * @param exchId Exchange ID.
      * @param client Client message flag.
      * @param lastVer Last version.
+     * @param compress {@code True} if it is possible to use compression for message.
      */
     public GridDhtPartitionsSingleMessage(GridDhtPartitionExchangeId exchId,
         boolean client,
-        @Nullable GridCacheVersion lastVer) {
+        @Nullable GridCacheVersion lastVer,
+        boolean compress) {
         super(exchId, lastVer);
 
         this.client = client;
+        this.compress = compress;
     }
 
     /**
@@ -87,16 +102,26 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     }
 
     /**
-     * Adds partition map to this message.
-     *
      * @param cacheId Cache ID to add local partition for.
      * @param locMap Local partition map.
+     * @param dupDataCache Optional ID of cache with the same partition state map.
      */
-    public void addLocalPartitionMap(int cacheId, GridDhtPartitionMap2 locMap) {
+    public void addLocalPartitionMap(int cacheId, GridDhtPartitionMap2 locMap, @Nullable Integer dupDataCache) {
         if (parts == null)
             parts = new HashMap<>();
 
         parts.put(cacheId, locMap);
+
+        if (dupDataCache != null) {
+            assert compress;
+            assert F.isEmpty(locMap.map());
+            assert parts.containsKey(dupDataCache);
+
+            if (dupPartsData == null)
+                dupPartsData = new HashMap<>();
+
+            dupPartsData.put(cacheId, dupDataCache);
+        }
     }
 
     /**
@@ -136,22 +161,77 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
 
-        if (partsBytes == null && parts != null)
-            partsBytes = U.marshal(ctx, parts);
+        boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null);
+
+        if (marshal) {
+            byte[] partsBytes0 = null;
+            byte[] partCntrsBytes0 = null;
+
+            if (parts != null && partsBytes == null)
+                partsBytes0 = U.marshal(ctx, parts);
 
-        if (partCntrsBytes == null && partCntrs != null)
-            partCntrsBytes = U.marshal(ctx, partCntrs);
+            if (partCntrs != null && partCntrsBytes == null)
+                partCntrsBytes0 = U.marshal(ctx, partCntrs);
+
+            if (compress) {
+                assert !compressed();
+
+                try {
+                    byte[] partsBytesZip = U.zip(partsBytes0);
+                    byte[] partCntrsBytesZip = U.zip(partCntrsBytes0);
+
+                    partsBytes0 = partsBytesZip;
+                    partCntrsBytes0 = partCntrsBytesZip;
+
+                    compressed(true);
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(ctx.logger(getClass()), "Failed to compress partitions data: " + e, e);
+                }
+            }
+
+            partsBytes = partsBytes0;
+            partCntrsBytes = partCntrsBytes0;
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(ctx, ldr);
 
-        if (partsBytes != null && parts == null)
-            parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        if (partsBytes != null && parts == null) {
+            if (compressed())
+                parts = U.unmarshalZip(ctx.marshaller(), partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+            else
+                parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        }
+
+        if (partCntrsBytes != null && partCntrs == null) {
+            if (compressed())
+                partCntrs = U.unmarshalZip(ctx.marshaller(), partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+            else
+                partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        }
+
+        if (dupPartsData != null) {
+            assert parts != null;
+
+            for (Map.Entry<Integer, Integer> e : dupPartsData.entrySet()) {
+                GridDhtPartitionMap2 map1 = parts.get(e.getKey());
 
-        if (partCntrsBytes != null && partCntrs == null)
-            partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+                assert map1 != null : e.getKey();
+                assert F.isEmpty(map1.map());
+                assert !map1.hasMovingPartitions();
+
+                GridDhtPartitionMap2 map2 = parts.get(e.getValue());
+
+                assert map2 != null : e.getValue();
+                assert map2.map() != null;
+
+                for (Map.Entry<Integer, GridDhtPartitionState> e0 : map2.map().entrySet())
+                    map1.put(e0.getKey(), e0.getValue());
+            }
+        }
     }
 
     /** {@inheritDoc} */
@@ -169,19 +249,25 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
         }
 
         switch (writer.state()) {
-            case 5:
+            case 6:
                 if (!writer.writeBoolean("client", client))
                     return false;
 
                 writer.incrementState();
 
-            case 6:
+            case 7:
+                if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT))
+                    return false;
+
+                writer.incrementState();
+
+            case 8:
                 if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes))
                     return false;
 
                 writer.incrementState();
 
-            case 7:
+            case 9:
                 if (!writer.writeByteArray("partsBytes", partsBytes))
                     return false;
 
@@ -203,7 +289,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
             return false;
 
         switch (reader.state()) {
-            case 5:
+            case 6:
                 client = reader.readBoolean("client");
 
                 if (!reader.isLastRead())
@@ -211,7 +297,15 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 6:
+            case 7:
+                dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 8:
                 partCntrsBytes = reader.readByteArray("partCntrsBytes");
 
                 if (!reader.isLastRead())
@@ -219,7 +313,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
-            case 7:
+            case 9:
                 partsBytes = reader.readByteArray("partsBytes");
 
                 if (!reader.isLastRead())
@@ -239,7 +333,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 10;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
index a4106af..850b6d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java
@@ -81,11 +81,11 @@ public class GridDhtPartitionsSingleRequest extends GridDhtPartitionsAbstractMes
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 5;
+        return 6;
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridDhtPartitionsSingleRequest.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 09aec81..d6865c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -35,8 +35,8 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment;
 import org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -600,7 +600,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
                     log.debug("Affinity is ready for topology version, will send response [topVer=" + topVer +
                         ", node=" + node + ']');
 
-                GridAffinityAssignment assignment = cctx.affinity().assignment(topVer);
+                AffinityAssignment assignment = cctx.affinity().assignment(topVer);
 
                 boolean newAffMode = node.version().compareTo(CacheAffinitySharedManager.LATE_AFF_ASSIGN_SINCE) >= 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 3a559e7..9fd9b6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -239,7 +239,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                                 GridCacheContext cctx = interCache != null ? interCache.context() : null;
 
                                 if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
-                                    cntrsPerNode.put(ctx.localNodeId(), cctx.topology().updateCounters());
+                                    cntrsPerNode.put(ctx.localNodeId(), cctx.topology().updateCounters(false));
 
                                 routine.handler().updateCounters(topVer, cntrsPerNode, cntrs);
                             }
@@ -1049,7 +1049,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 GridCacheAdapter cache = ctx.cache().internalCache(hnd0.cacheName());
 
                 if (cache != null && !cache.isLocal() && cache.context().userCache())
-                    req.addUpdateCounters(ctx.localNodeId(), cache.context().topology().updateCounters());
+                    req.addUpdateCounters(ctx.localNodeId(), cache.context().topology().updateCounters(false));
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 7b011dd..e0f4a2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.util;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.Externalizable;
@@ -128,6 +130,8 @@ import java.util.logging.Logger;
 import java.util.regex.Pattern;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
+import java.util.zip.ZipInputStream;
+import java.util.zip.ZipOutputStream;
 import javax.management.DynamicMBean;
 import javax.management.JMException;
 import javax.management.MBeanServer;
@@ -9693,6 +9697,32 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @param marsh Marshaller.
+     * @param zipBytes Zip-compressed bytes.
+     * @param clsLdr Class loader to use.
+     * @return Unmarshalled object.
+     * @throws IgniteCheckedException
+     */
+    public static <T> T unmarshalZip(Marshaller marsh, byte[] zipBytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        assert marsh != null;
+        assert zipBytes != null;
+
+        try {
+            ZipInputStream in = new ZipInputStream(new ByteArrayInputStream(zipBytes));
+
+            in.getNextEntry();
+
+            return marsh.unmarshal(in, clsLdr);
+        }
+        catch (IgniteCheckedException e) {
+            throw e;
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
      * Unmarshals object from the input stream using given class loader.
      * This method should not close given input stream.
      * <p/>
@@ -9907,4 +9937,38 @@ public abstract class IgniteUtils {
         if (oldName != curName)
             LOC_IGNITE_NAME.set(oldName);
     }
+
+    /**
+     * @param bytes Byte array to compress.
+     * @return Compressed bytes.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static byte[] zip(@Nullable byte[] bytes) throws IgniteCheckedException {
+        try {
+            if (bytes == null)
+                return null;
+
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+
+            try (ZipOutputStream zos = new ZipOutputStream(bos)) {
+                ZipEntry entry = new ZipEntry("");
+
+                try {
+                    entry.setSize(bytes.length);
+
+                    zos.putNextEntry(entry);
+
+                    zos.write(bytes);
+                }
+                finally {
+                    zos.closeEntry();
+                }
+            }
+
+            return bos.toByteArray();
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index f929121..733d204 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -1049,7 +1049,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
-            TcpDiscoveryAbstractMessage msg = null;
+            TcpDiscoveryAbstractMessage msg;
 
             while (!Thread.currentThread().isInterrupted()) {
                 Socket sock;
@@ -1063,8 +1063,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                         continue;
                     }
 
-                    if (msg == null)
-                        msg = queue.poll();
+                    msg = queue.poll();
 
                     if (msg == null) {
                         mux.wait();
@@ -1121,19 +1120,13 @@ class ClientImpl extends TcpDiscoveryImpl {
                         }
                     }
                 }
-                catch (IOException e) {
+                catch (InterruptedException e) {
                     if (log.isDebugEnabled())
-                        U.error(log, "Failed to send node left message (will stop anyway) " +
-                            "[sock=" + sock + ", msg=" + msg + ']', e);
-
-                    U.closeQuiet(sock);
+                        log.debug("Client socket writer interrupted.");
 
-                    synchronized (mux) {
-                        if (sock == this.sock)
-                            this.sock = null; // Connection has dead.
-                    }
+                    return;
                 }
-                catch (IgniteCheckedException e) {
+                catch (Exception e) {
                     if (spi.getSpiContext().isStopping()) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to send message, node is stopping [msg=" + msg + ", err=" + e + ']');
@@ -1141,7 +1134,12 @@ class ClientImpl extends TcpDiscoveryImpl {
                     else
                         U.error(log, "Failed to send message: " + msg, e);
 
-                    msg = null;
+                    U.closeQuiet(sock);
+
+                    synchronized (mux) {
+                        if (sock == this.sock)
+                            this.sock = null; // Connection has dead.
+                    }
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 0de787d..8814745 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -133,7 +133,7 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_HISTORY_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
@@ -167,7 +167,7 @@ import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusChe
  */
 class ServerImpl extends TcpDiscoveryImpl {
     /** */
-    private static final int ENSURED_MSG_HIST_SIZE = getInteger(IGNITE_DISCOVERY_HISTORY_SIZE, 1024);
+    private static final int ENSURED_MSG_HIST_SIZE = getInteger(IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE, 512);
 
     /** */
     private static final IgniteProductVersion CUSTOM_MSG_ALLOW_JOINING_FOR_VERIFIED_SINCE =
@@ -1479,7 +1479,7 @@ class ServerImpl extends TcpDiscoveryImpl {
     private void prepareNodeAddedMessage(
         TcpDiscoveryAbstractMessage msg,
         UUID destNodeId,
-        @Nullable Collection<TcpDiscoveryAbstractMessage> msgs,
+        @Nullable Collection<PendingMessage> msgs,
         @Nullable IgniteUuid discardMsgId,
         @Nullable IgniteUuid discardCustomMsgId
         ) {
@@ -1506,7 +1506,19 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 nodeAddedMsg.topology(topToSnd);
-                nodeAddedMsg.messages(msgs, discardMsgId, discardCustomMsgId);
+
+                Collection<TcpDiscoveryAbstractMessage> msgs0 = null;
+
+                if (msgs != null) {
+                    msgs0 = new ArrayList<>(msgs.size());
+
+                    for (PendingMessage pendingMsg : msgs) {
+                        if (pendingMsg.msg != null)
+                            msgs0.add(pendingMsg.msg);
+                    }
+                }
+
+                nodeAddedMsg.messages(msgs0, discardMsgId, discardCustomMsgId);
 
                 Map<Long, Collection<ClusterNode>> hist;
 
@@ -1892,7 +1904,10 @@ class ServerImpl extends TcpDiscoveryImpl {
             assert spi.ensured(msg) && msg.verified() : msg;
 
             if (msg instanceof TcpDiscoveryNodeAddedMessage) {
-                TcpDiscoveryNodeAddedMessage addedMsg = (TcpDiscoveryNodeAddedMessage)msg;
+                TcpDiscoveryNodeAddedMessage addedMsg =
+                    new TcpDiscoveryNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
+
+                msg = addedMsg;
 
                 TcpDiscoveryNode node = addedMsg.node();
 
@@ -1910,12 +1925,109 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     addedMsg.clientTopology(top);
                 }
+
+                // Do not need this data for client reconnect.
+                addedMsg.oldNodesDiscoveryData(null);
+            }
+            else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) {
+                TcpDiscoveryNodeAddFinishedMessage addFinishMsg = (TcpDiscoveryNodeAddFinishedMessage)msg;
+
+                if (addFinishMsg.clientDiscoData() != null) {
+                    addFinishMsg = new TcpDiscoveryNodeAddFinishedMessage(addFinishMsg);
+
+                    msg = addFinishMsg;
+
+                    Map<UUID, Map<Integer, byte[]>> discoData = addFinishMsg.clientDiscoData();
+
+                    Set<UUID> replaced = null;
+
+                    for (TcpDiscoveryAbstractMessage msg0 : msgs) {
+                        if (msg0 instanceof TcpDiscoveryNodeAddFinishedMessage) {
+                            Map<UUID, Map<Integer, byte[]>> existingDiscoData =
+                                ((TcpDiscoveryNodeAddFinishedMessage)msg0).clientDiscoData();
+
+                            // Check if already stored message contains the same data to do not store copies multiple times.
+                            if (existingDiscoData != null) {
+                                for (Map.Entry<UUID, Map<Integer, byte[]>> e : discoData.entrySet()) {
+                                    UUID nodeId = e.getKey();
+
+                                    if (F.contains(replaced, nodeId))
+                                        continue;
+
+                                    Map<Integer, byte[]> existingData = existingDiscoData.get(e.getKey());
+
+                                    if (existingData != null && mapsEqual(e.getValue(), existingData)) {
+                                        e.setValue(existingData);
+
+                                        if (replaced == null)
+                                            replaced = new HashSet<>();
+
+                                        boolean add = replaced.add(nodeId);
+
+                                        assert add;
+
+                                        if (replaced.size() == discoData.size())
+                                            break;
+                                    }
+                                }
+
+                                if (replaced != null && replaced.size() == discoData.size())
+                                    break;
+                            }
+                        }
+                    }
+                }
             }
+            else if (msg instanceof TcpDiscoveryNodeLeftMessage)
+                clearClientAddFinished(msg.creatorNodeId());
+            else if (msg instanceof TcpDiscoveryNodeFailedMessage)
+                clearClientAddFinished(((TcpDiscoveryNodeFailedMessage)msg).failedNodeId());
 
             msgs.add(msg);
         }
 
         /**
+         * @param clientId Client node ID.
+         */
+        private void clearClientAddFinished(UUID clientId) {
+            for (TcpDiscoveryAbstractMessage msg : msgs) {
+                if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) {
+                    TcpDiscoveryNodeAddFinishedMessage addFinishMsg = (TcpDiscoveryNodeAddFinishedMessage)msg;
+
+                    if (addFinishMsg.clientDiscoData() != null && clientId.equals(addFinishMsg.nodeId())) {
+                        addFinishMsg.clientDiscoData(null);
+                        addFinishMsg.clientNodeAttributes(null);
+
+                        break;
+                    }
+                }
+            }
+        }
+
+        /**
+         * @param m1 Map 1.
+         * @param m2 Map 2.
+         * @return {@code True} if maps contain the same data.
+         */
+        private boolean mapsEqual(Map<Integer, byte[]> m1, Map<Integer, byte[]> m2) {
+            if (m1 == m2)
+                return true;
+
+            if (m1.size() == m2.size()) {
+                for (Map.Entry<Integer, byte[]> e : m1.entrySet()) {
+                    byte[] data = m2.get(e.getKey());
+
+                    if (!Arrays.equals(e.getValue(), data))
+                        return false;
+                }
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /**
          * Gets messages starting from provided ID (exclusive). If such
          * message is not found, {@code null} is returned (this indicates
          * a failure condition when it was already removed from queue).
@@ -2009,6 +2121,37 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /**
+     *
+     */
+    private static class PendingMessage {
+        /** */
+        TcpDiscoveryAbstractMessage msg;
+
+        /** */
+        final boolean customMsg;
+
+        /** */
+        final IgniteUuid id;
+
+        /**
+         * @param msg Message.
+         */
+        PendingMessage(TcpDiscoveryAbstractMessage msg) {
+            assert msg != null && msg.id() != null : msg;
+
+            this.msg = msg;
+
+            id = msg.id();
+            customMsg = msg instanceof TcpDiscoveryCustomEventMessage;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(PendingMessage.class, this);
+        }
+    }
+
+    /**
      * Pending messages container.
      */
     private static class PendingMessages implements Iterable<TcpDiscoveryAbstractMessage> {
@@ -2016,7 +2159,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         private static final int MAX = 1024;
 
         /** Pending messages. */
-        private final Queue<TcpDiscoveryAbstractMessage> msgs = new ArrayDeque<>(MAX * 2);
+        private final Queue<PendingMessage> msgs = new ArrayDeque<>(MAX * 2);
 
         /** Processed custom message IDs. */
         private Set<IgniteUuid> procCustomMsgs = new GridBoundedLinkedHashSet<>(MAX * 2);
@@ -2024,7 +2167,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         /** Discarded message ID. */
         private IgniteUuid discardId;
 
-        /** Discarded message ID. */
+        /** Discarded custom message ID. */
         private IgniteUuid customDiscardId;
 
         /**
@@ -2034,14 +2177,14 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @param msg Message to add.
          */
         void add(TcpDiscoveryAbstractMessage msg) {
-            msgs.add(msg);
+            msgs.add(new PendingMessage(msg));
 
             while (msgs.size() > MAX) {
-                TcpDiscoveryAbstractMessage polled = msgs.poll();
+                PendingMessage polled = msgs.poll();
 
                 assert polled != null;
 
-                if (polled.id().equals(discardId))
+                if (polled.id.equals(discardId))
                     break;
             }
         }
@@ -2051,6 +2194,7 @@ class ServerImpl extends TcpDiscoveryImpl {
          *
          * @param msgs Message.
          * @param discardId Discarded message ID.
+         * @param customDiscardId Discarded custom event message ID.
          */
         void reset(
             @Nullable Collection<TcpDiscoveryAbstractMessage> msgs,
@@ -2059,8 +2203,10 @@ class ServerImpl extends TcpDiscoveryImpl {
         ) {
             this.msgs.clear();
 
-            if (msgs != null)
-                this.msgs.addAll(msgs);
+            if (msgs != null) {
+                for (TcpDiscoveryAbstractMessage msg : msgs)
+                    this.msgs.add(new PendingMessage(msg));
+            }
 
             this.discardId = discardId;
             this.customDiscardId = customDiscardId;
@@ -2070,12 +2216,52 @@ class ServerImpl extends TcpDiscoveryImpl {
          * Discards message with provided ID and all before it.
          *
          * @param id Discarded message ID.
+         * @param custom {@code True} if discard for {@link TcpDiscoveryCustomEventMessage}.
          */
         void discard(IgniteUuid id, boolean custom) {
             if (custom)
                 customDiscardId = id;
             else
                 discardId = id;
+
+            cleanup();
+        }
+
+        /**
+         *
+         */
+        void cleanup() {
+            Iterator<PendingMessage> msgIt = msgs.iterator();
+
+            boolean skipMsg = discardId != null;
+            boolean skipCustomMsg = customDiscardId != null;
+
+            while (msgIt.hasNext()) {
+                PendingMessage msg = msgIt.next();
+
+                if (msg.customMsg) {
+                    if (skipCustomMsg) {
+                        assert customDiscardId != null;
+
+                        if (F.eq(customDiscardId, msg.id)) {
+                            msg.msg = null;
+
+                            return;
+                        }
+                    }
+                }
+                else {
+                    if (skipMsg) {
+                        assert discardId != null;
+
+                        if (F.eq(discardId, msg.id)) {
+                            msg.msg = null;
+
+                            return;
+                        }
+                    }
+                }
+            }
         }
 
         /**
@@ -2098,7 +2284,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             private boolean skipCustomMsg = customDiscardId != null;
 
             /** Internal iterator. */
-            private Iterator<TcpDiscoveryAbstractMessage> msgIt = msgs.iterator();
+            private Iterator<PendingMessage> msgIt = msgs.iterator();
 
             /** Next message. */
             private TcpDiscoveryAbstractMessage next;
@@ -2136,13 +2322,13 @@ class ServerImpl extends TcpDiscoveryImpl {
                 next = null;
 
                 while (msgIt.hasNext()) {
-                    TcpDiscoveryAbstractMessage msg0 = msgIt.next();
+                    PendingMessage msg0 = msgIt.next();
 
-                    if (msg0 instanceof TcpDiscoveryCustomEventMessage) {
+                    if (msg0.customMsg) {
                         if (skipCustomMsg) {
                             assert customDiscardId != null;
 
-                            if (F.eq(customDiscardId, msg0.id()))
+                            if (F.eq(customDiscardId, msg0.id))
                                 skipCustomMsg = false;
 
                             continue;
@@ -2152,14 +2338,17 @@ class ServerImpl extends TcpDiscoveryImpl {
                         if (skipMsg) {
                             assert discardId != null;
 
-                            if (F.eq(discardId, msg0.id()))
+                            if (F.eq(discardId, msg0.id))
                                 skipMsg = false;
 
                             continue;
                         }
                     }
 
-                    next = msg0;
+                    if (msg0.msg == null)
+                        continue;
+
+                    next = msg0.msg;
 
                     break;
                 }
@@ -2985,9 +3174,9 @@ class ServerImpl extends TcpDiscoveryImpl {
             if (pendingMsgs.msgs.isEmpty())
                 return false;
 
-            for (TcpDiscoveryAbstractMessage pendingMsg : pendingMsgs.msgs) {
-                if (pendingMsg instanceof TcpDiscoveryNodeAddedMessage) {
-                    TcpDiscoveryNodeAddedMessage addMsg = (TcpDiscoveryNodeAddedMessage)pendingMsg;
+            for (PendingMessage pendingMsg : pendingMsgs.msgs) {
+                if (pendingMsg.msg instanceof TcpDiscoveryNodeAddedMessage) {
+                    TcpDiscoveryNodeAddedMessage addMsg = (TcpDiscoveryNodeAddedMessage)pendingMsg.msg;
 
                     if (addMsg.node().id().equals(nodeId) && addMsg.id().compareTo(pendingMsgs.discardId) > 0)
                         return true;
@@ -3901,8 +4090,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
 
                     if (data != null)
-                        spi.onExchange(node.id(), node.id(), data,
-                            U.resolveClassLoader(spi.ignite().configuration()));
+                        spi.onExchange(node.id(), node.id(), data, U.resolveClassLoader(spi.ignite().configuration()));
 
                     msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id()));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
index 1b99a56..80f4565 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
@@ -59,6 +59,17 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
     }
 
     /**
+     * @param msg Message.
+     */
+    public TcpDiscoveryNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage msg) {
+        super(msg);
+
+        nodeId = msg.nodeId;
+        clientDiscoData = msg.clientDiscoData;
+        clientNodeAttrs = msg.clientNodeAttrs;
+    }
+
+    /**
      * Gets ID of the node added.
      *
      * @return ID of the node added.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
index 6f8e14e..bd52c04 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.spi.discovery.tcp.messages;
 
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.UUID;
@@ -234,14 +236,41 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
     }
 
     /**
+     * @param oldNodesDiscoData Discovery data from old nodes.
+     */
+    public void oldNodesDiscoveryData(Map<UUID, Map<Integer, byte[]>> oldNodesDiscoData) {
+        this.oldNodesDiscoData = oldNodesDiscoData;
+    }
+
+    /**
      * @param nodeId Node ID.
      * @param discoData Discovery data to add.
      */
     public void addDiscoveryData(UUID nodeId, Map<Integer, byte[]> discoData) {
         // Old nodes disco data may be null if message
         // makes more than 1 pass due to stopping of the nodes in topology.
-        if (oldNodesDiscoData != null)
-            oldNodesDiscoData.put(nodeId, discoData);
+        if (oldNodesDiscoData != null) {
+            for (Map.Entry<UUID, Map<Integer, byte[]>> existingDataEntry : oldNodesDiscoData.entrySet()) {
+                Map<Integer, byte[]> existingData = existingDataEntry.getValue();
+
+                Iterator<Map.Entry<Integer, byte[]>> it = discoData.entrySet().iterator();
+
+                while (it.hasNext()) {
+                    Map.Entry<Integer, byte[]> discoDataEntry = it.next();
+
+                    byte[] curData = existingData.get(discoDataEntry.getKey());
+
+                    if (Arrays.equals(curData, discoDataEntry.getValue()))
+                        it.remove();
+                }
+
+                if (discoData.isEmpty())
+                    break;
+            }
+
+            if (!discoData.isEmpty())
+                oldNodesDiscoData.put(nodeId, discoData);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
new file mode 100644
index 0000000..ed186ac
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
@@ -0,0 +1,393 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.lang.IgnitePredicate;
+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 CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String AFF1_CACHE1 = "a1c1";
+
+    /** */
+    private static final String AFF1_CACHE2 = "a1c2";
+
+    /** */
+    private static final String AFF2_CACHE1 = "a2c1";
+
+    /** */
+    private static final String AFF2_CACHE2 = "a2c2";
+
+    /** */
+    private static final String AFF3_CACHE1 = "a3c1";
+
+    /** */
+    private static final String AFF4_FILTER_CACHE1 = "a4c1";
+
+    /** */
+    private static final String AFF4_FILTER_CACHE2 = "a4c2";
+
+    /** */
+    private static final String AFF5_FILTER_CACHE1 = "a5c1";
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        TestRecordingCommunicationSpi commSpi = new TestRecordingCommunicationSpi();
+
+        commSpi.record(GridDhtPartitionsSingleMessage.class, GridDhtPartitionsFullMessage.class);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        List<CacheConfiguration> ccfgs = new ArrayList<>();
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF1_CACHE1);
+            ccfg.setAffinity(new RendezvousAffinityFunction());
+            ccfgs.add(ccfg);
+        }
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF1_CACHE2);
+            ccfg.setAffinity(new RendezvousAffinityFunction());
+            ccfgs.add(ccfg);
+        }
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF2_CACHE1);
+            ccfg.setAffinity(new FairAffinityFunction());
+            ccfgs.add(ccfg);
+        }
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF2_CACHE2);
+            ccfg.setAffinity(new FairAffinityFunction());
+            ccfgs.add(ccfg);
+        }
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF3_CACHE1);
+            ccfg.setBackups(3);
+
+            RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false, 64);
+            ccfg.setAffinity(aff);
+
+            ccfgs.add(ccfg);
+        }
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF4_FILTER_CACHE1);
+            ccfg.setNodeFilter(new TestNodeFilter());
+            ccfg.setAffinity(new RendezvousAffinityFunction());
+            ccfgs.add(ccfg);
+        }
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF4_FILTER_CACHE2);
+            ccfg.setNodeFilter(new TestNodeFilter());
+            ccfg.setAffinity(new RendezvousAffinityFunction());
+            ccfgs.add(ccfg);
+        }
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName(AFF5_FILTER_CACHE1);
+            ccfg.setNodeFilter(new TestNodeFilter());
+            ccfg.setAffinity(new FairAffinityFunction());
+            ccfgs.add(ccfg);
+        }
+
+        cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExchangeMessages() throws Exception {
+        ignite(0);
+
+        startGrid(1);
+
+        awaitPartitionMapExchange();
+
+        checkMessages(0, true);
+
+        startGrid(2);
+
+        awaitPartitionMapExchange();
+
+        checkMessages(0, true);
+
+        client = true;
+
+        startGrid(3);
+
+        awaitPartitionMapExchange();
+
+        checkMessages(0, false);
+
+        stopGrid(0);
+
+        awaitPartitionMapExchange();
+
+        checkMessages(1, true);
+    }
+
+    /**
+     * @param crdIdx Coordinator node index.
+     * @param checkSingle {@code True} if need check single messages.
+     */
+    private void checkMessages(int crdIdx, boolean checkSingle) {
+        checkFullMessages(crdIdx);
+
+        if (checkSingle)
+            checkSingleMessages(crdIdx);
+    }
+
+    /**
+     * @param crdIdx Coordinator node index.
+     */
+    private void checkFullMessages(int crdIdx) {
+        TestRecordingCommunicationSpi commSpi0 =
+            (TestRecordingCommunicationSpi)ignite(crdIdx).configuration().getCommunicationSpi();
+
+        List<Object> msgs = commSpi0.recordedMessages(false);
+
+        assertTrue(msgs.size() > 0);
+
+        for (Object msg : msgs) {
+            assertTrue("Unexpected messages: " + msg, msg instanceof GridDhtPartitionsFullMessage);
+
+            checkFullMessage((GridDhtPartitionsFullMessage)msg);
+        }
+    }
+
+    /**
+     * @param crdIdx Coordinator node index.
+     */
+    private void checkSingleMessages(int crdIdx) {
+        int cnt = 0;
+
+        for (Ignite ignite : Ignition.allGrids()) {
+            if (getTestGridName(crdIdx).equals(ignite.name()) || ignite.configuration().isClientMode())
+                continue;
+
+            TestRecordingCommunicationSpi commSpi0 =
+                (TestRecordingCommunicationSpi)ignite.configuration().getCommunicationSpi();
+
+            List<Object> msgs = commSpi0.recordedMessages(false);
+
+            assertTrue(msgs.size() > 0);
+
+            for (Object msg : msgs) {
+                assertTrue("Unexpected messages: " + msg, msg instanceof GridDhtPartitionsSingleMessage);
+
+                checkSingleMessage((GridDhtPartitionsSingleMessage)msg);
+            }
+
+            cnt++;
+        }
+
+        assertTrue(cnt > 0);
+    }
+
+    /**
+     * @param msg Message.
+     */
+    private void checkFullMessage(GridDhtPartitionsFullMessage msg) {
+        Map<Integer, Integer> dupPartsData = GridTestUtils.getFieldValue(msg, "dupPartsData");
+
+        assertNotNull(dupPartsData);
+
+        checkFullMessage(AFF1_CACHE1, AFF1_CACHE2, dupPartsData, msg);
+        checkFullMessage(AFF2_CACHE1, AFF2_CACHE2, dupPartsData, msg);
+        checkFullMessage(AFF4_FILTER_CACHE1, AFF4_FILTER_CACHE2, dupPartsData, msg);
+
+        assertFalse(dupPartsData.containsKey(CU.cacheId(AFF3_CACHE1)));
+        assertFalse(dupPartsData.containsKey(CU.cacheId(AFF5_FILTER_CACHE1)));
+
+        Map<Integer, Map<Integer, Long>> partCntrs = GridTestUtils.getFieldValue(msg, "partCntrs");
+
+        if (partCntrs != null) {
+            for (Map<Integer, Long> cntrs : partCntrs.values())
+                assertTrue(cntrs.isEmpty());
+        }
+    }
+
+    /**
+     * @param msg Message.
+     */
+    private void checkSingleMessage(GridDhtPartitionsSingleMessage msg) {
+        Map<Integer, Integer> dupPartsData = GridTestUtils.getFieldValue(msg, "dupPartsData");
+
+        assertNotNull(dupPartsData);
+
+        checkSingleMessage(AFF1_CACHE1, AFF1_CACHE2, dupPartsData, msg);
+        checkSingleMessage(AFF2_CACHE1, AFF2_CACHE2, dupPartsData, msg);
+        checkSingleMessage(AFF4_FILTER_CACHE1, AFF4_FILTER_CACHE2, dupPartsData, msg);
+
+        assertFalse(dupPartsData.containsKey(CU.cacheId(AFF3_CACHE1)));
+        assertFalse(dupPartsData.containsKey(CU.cacheId(AFF5_FILTER_CACHE1)));
+
+        Map<Integer, Map<Integer, Long>> partCntrs = GridTestUtils.getFieldValue(msg, "partCntrs");
+
+        if (partCntrs != null) {
+            for (Map<Integer, Long> cntrs : partCntrs.values())
+                assertTrue(cntrs.isEmpty());
+        }
+    }
+
+    /**
+     * @param cache1 Cache 1.
+     * @param cache2 Cache 2.
+     * @param dupPartsData Duplicated data map.
+     * @param msg Message.
+     */
+    private void checkFullMessage(String cache1,
+        String cache2,
+        Map<Integer, Integer> dupPartsData,
+        GridDhtPartitionsFullMessage msg)
+    {
+        Integer cacheId;
+        Integer dupCacheId;
+
+        if (dupPartsData.containsKey(CU.cacheId(cache1))) {
+            cacheId = CU.cacheId(cache1);
+            dupCacheId = CU.cacheId(cache2);
+        }
+        else {
+            cacheId = CU.cacheId(cache2);
+            dupCacheId = CU.cacheId(cache1);
+        }
+
+        assertTrue(dupPartsData.containsKey(cacheId));
+        assertEquals(dupCacheId, dupPartsData.get(cacheId));
+        assertFalse(dupPartsData.containsKey(dupCacheId));
+
+        Map<Integer, GridDhtPartitionFullMap> parts = msg.partitions();
+
+        GridDhtPartitionFullMap emptyFullMap = parts.get(cacheId);
+
+        for (GridDhtPartitionMap2 map : emptyFullMap.values())
+            assertEquals(0, map.map().size());
+
+        GridDhtPartitionFullMap fullMap = parts.get(dupCacheId);
+
+        for (GridDhtPartitionMap2 map : fullMap.values())
+            assertFalse(map.map().isEmpty());
+    }
+
+    /**
+     * @param cache1 Cache 1.
+     * @param cache2 Cache 2.
+     * @param dupPartsData Duplicated data map.
+     * @param msg Message.
+     */
+    private void checkSingleMessage(String cache1,
+        String cache2,
+        Map<Integer, Integer> dupPartsData,
+        GridDhtPartitionsSingleMessage msg)
+    {
+        Integer cacheId;
+        Integer dupCacheId;
+
+        if (dupPartsData.containsKey(CU.cacheId(cache1))) {
+            cacheId = CU.cacheId(cache1);
+            dupCacheId = CU.cacheId(cache2);
+        }
+        else {
+            cacheId = CU.cacheId(cache2);
+            dupCacheId = CU.cacheId(cache1);
+        }
+
+        assertTrue(dupPartsData.containsKey(cacheId));
+        assertEquals(dupCacheId, dupPartsData.get(cacheId));
+        assertFalse(dupPartsData.containsKey(dupCacheId));
+
+        Map<Integer, GridDhtPartitionMap2> parts = msg.partitions();
+
+        GridDhtPartitionMap2 emptyMap = parts.get(cacheId);
+
+        assertEquals(0, emptyMap.map().size());
+
+        GridDhtPartitionMap2 map = parts.get(dupCacheId);
+
+        assertFalse(map.map().isEmpty());
+    }
+
+    /**
+     *
+     */
+    private static class TestNodeFilter implements IgnitePredicate<ClusterNode> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            // Do not start cache on coordinator.
+            return node.order() > 1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
index 5dc059b..6c577c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
@@ -630,7 +630,7 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
         if (cacheMode() == LOCAL)
             return;
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkEmpty();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
index 71d1182..3b0c2fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 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;
@@ -72,6 +73,8 @@ public class IgniteCacheGetRestartTest extends GridCommonAbstractTest {
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         Boolean clientMode = client.get();
 
         if (clientMode != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
index 2c47a1c..7b57d5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java
@@ -115,21 +115,21 @@ public class GridCacheRabalancingDelayedPartitionMapExchangeSelfTest extends Gri
         startGrid(2);
         startGrid(3);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         for (int i = 0; i < 2; i++) {
             stopGrid(3);
 
-            awaitPartitionMapExchange(true, true);
+            awaitPartitionMapExchange(true, true, null);
 
             startGrid(3);
 
-            awaitPartitionMapExchange(true, true);
+            awaitPartitionMapExchange(true, true, null);
         }
 
         startGrid(4);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         assert rs.isEmpty();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/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 5716d59..de38952 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
@@ -240,7 +240,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         waitForRebalancing(0, new AffinityTopologyVersion(2, waitMinorVer));
         waitForRebalancing(1, new AffinityTopologyVersion(2, waitMinorVer));
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkPartitionMapExchangeFinished();
 
@@ -250,7 +250,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
         waitForRebalancing(1, 3);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkPartitionMapExchangeFinished();
 
@@ -261,7 +261,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         waitForRebalancing(1, new AffinityTopologyVersion(4, waitMinorVer));
         waitForRebalancing(2, new AffinityTopologyVersion(4, waitMinorVer));
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkPartitionMapExchangeFinished();
 
@@ -271,7 +271,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
         waitForRebalancing(1, 5);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkPartitionMapExchangeFinished();
 
@@ -339,7 +339,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
         concurrentStartFinished = true;
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkSupplyContextMapIsEmpty();
 
@@ -607,7 +607,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         waitForRebalancing(3, 5, 1);
         waitForRebalancing(4, 5, 1);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkSupplyContextMapIsEmpty();
 
@@ -631,7 +631,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         waitForRebalancing(3, 6);
         waitForRebalancing(4, 6);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkSupplyContextMapIsEmpty();
 
@@ -641,7 +641,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         waitForRebalancing(3, 7);
         waitForRebalancing(4, 7);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkSupplyContextMapIsEmpty();
 
@@ -650,7 +650,7 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
         waitForRebalancing(3, 8);
         waitForRebalancing(4, 8);
 
-        awaitPartitionMapExchange(true, true);
+        awaitPartitionMapExchange(true, true, null);
 
         checkPartitionMapExchangeFinished();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
index 87d02a5..cde6b8d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
@@ -41,9 +41,6 @@ public class GridCacheSyncReplicatedPreloadSelfTest extends GridCommonAbstractTe
     /** */
     private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
-    /** */
-    private static final boolean DISCO_DEBUG_MODE = false;
-
     /**
      * Constructs test.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java
index 9b0637e..f3942d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java
@@ -34,7 +34,9 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 public class IgniteCacheSyncRebalanceModeSelfTest extends GridCommonAbstractTest {
     /** Entry count. */
     public static final int CNT = 100_000;
-    public static final String STATIC_CACHE_NAME = "static";
+
+    /** */
+    private static final String STATIC_CACHE_NAME = "static";
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index 1b7fe2b..d2cb710 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -537,7 +537,7 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
 
             Affinity<Object> aff = grid(i).affinity(null);
 
-            Map<Integer, Long> act = grid(i).cachex(null).context().topology().updateCounters();
+            Map<Integer, Long> act = grid(i).cachex(null).context().topology().updateCounters(false);
 
             for (Map.Entry<Integer, Long> e : updCntrs.entrySet()) {
                 if (aff.mapPartitionToPrimaryAndBackups(e.getKey()).contains(grid(i).localNode()))

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java
index 5ecc27a..1259faf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java
@@ -68,6 +68,13 @@ public class IgniteNoCustomEventsOnNodeStart extends GridCommonAbstractTest {
         assertFalse(failed);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 1ce98a5..043208c 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -49,6 +49,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
@@ -114,6 +115,12 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     /** */
     private GridStringLogger strLog;
 
+    /** */
+    private CacheConfiguration[] ccfgs;
+
+    /** */
+    private boolean client;
+
     /**
      * @throws Exception If fails.
      */
@@ -152,7 +159,10 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(spi);
 
-        cfg.setCacheConfiguration();
+        if (ccfgs != null)
+            cfg.setCacheConfiguration(ccfgs);
+        else
+            cfg.setCacheConfiguration();
 
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
 
@@ -194,9 +204,8 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
         }
         else if (gridName.contains("testPingInterruptedOnNodeFailedPingingNode"))
             cfg.setFailureDetectionTimeout(30_000);
-        else if (gridName.contains("testNoRingMessageWorkerAbnormalFailureNormalNode")) {
+        else if (gridName.contains("testNoRingMessageWorkerAbnormalFailureNormalNode"))
             cfg.setFailureDetectionTimeout(3_000);
-        }
         else if (gridName.contains("testNoRingMessageWorkerAbnormalFailureSegmentedNode")) {
             cfg.setFailureDetectionTimeout(6_000);
 
@@ -205,6 +214,8 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
         else if (gridName.contains("testNodeShutdownOnRingMessageWorkerFailureFailedNode"))
             cfg.setGridLogger(strLog = new GridStringLogger());
 
+        cfg.setClientMode(client);
+
         return cfg;
     }
 
@@ -1961,6 +1972,63 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testDuplicatedDiscoveryDataRemoved() throws Exception {
+        try {
+            TestDiscoveryDataDuplicateSpi.checkNodeAdded = false;
+            TestDiscoveryDataDuplicateSpi.checkClientNodeAddFinished = false;
+            TestDiscoveryDataDuplicateSpi.fail = false;
+
+            ccfgs = new CacheConfiguration[5];
+
+            for (int i = 0; i < ccfgs.length; i++) {
+                CacheConfiguration ccfg = new CacheConfiguration();
+
+                ccfg.setName(i == 0 ? null : ("static-cache-" + i));
+
+                ccfgs[i] = ccfg;
+            }
+
+            TestDiscoveryDataDuplicateSpi spi = new TestDiscoveryDataDuplicateSpi();
+
+            nodeSpi.set(spi);
+
+            startGrid(0);
+
+            for (int i = 0; i < 5; i++) {
+                nodeSpi.set(new TestDiscoveryDataDuplicateSpi());
+
+                startGrid(i + 1);
+            }
+
+            client = true;
+
+            Ignite clientNode = startGrid(6);
+
+            assertTrue(clientNode.configuration().isClientMode());
+
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setName("c1");
+
+            clientNode.createCache(ccfg);
+
+            client = false;
+
+            nodeSpi.set(new TestDiscoveryDataDuplicateSpi());
+
+            startGrid(7);
+
+            assertTrue(TestDiscoveryDataDuplicateSpi.checkNodeAdded);
+            assertTrue(TestDiscoveryDataDuplicateSpi.checkClientNodeAddFinished);
+            assertFalse(TestDiscoveryDataDuplicateSpi.fail);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
      * @param nodeName Node name.
      * @throws Exception If failed.
      */
@@ -2015,6 +2083,66 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /**
+     *
+     */
+    private static class TestDiscoveryDataDuplicateSpi extends TcpDiscoverySpi {
+        /** */
+        static volatile boolean fail;
+
+        /** */
+        static volatile boolean checkNodeAdded;
+
+        /** */
+        static volatile boolean checkClientNodeAddFinished;
+
+        /** {@inheritDoc} */
+        @Override protected void writeToSocket(Socket sock, OutputStream out,
+            TcpDiscoveryAbstractMessage msg,
+            long timeout) throws IOException, IgniteCheckedException {
+            if (msg instanceof TcpDiscoveryNodeAddedMessage) {
+                Map<UUID, Map<Integer, byte[]>> discoData = ((TcpDiscoveryNodeAddedMessage)msg).oldNodesDiscoveryData();
+
+                checkDiscoData(discoData, msg);
+            }
+            else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) {
+                Map<UUID, Map<Integer, byte[]>> discoData = ((TcpDiscoveryNodeAddFinishedMessage)msg).clientDiscoData();
+
+                checkDiscoData(discoData, msg);
+            }
+
+            super.writeToSocket(sock, out, msg, timeout);
+        }
+
+        /**
+         * @param discoData Discovery data.
+         * @param msg Message.
+         */
+        private void checkDiscoData(Map<UUID, Map<Integer, byte[]>> discoData, TcpDiscoveryAbstractMessage msg) {
+            if (discoData != null && discoData.size() > 1) {
+                int cnt = 0;
+
+                for (Map.Entry<UUID, Map<Integer, byte[]>> e : discoData.entrySet()) {
+                    Map<Integer, byte[]> map = e.getValue();
+
+                    if (map.containsKey(GridComponent.DiscoveryDataExchangeType.CACHE_PROC.ordinal()))
+                        cnt++;
+                }
+
+                if (cnt > 1) {
+                    fail = true;
+
+                    log.error("Expect cache data only from one node, but actually: " + cnt);
+                }
+
+                if (msg instanceof TcpDiscoveryNodeAddedMessage)
+                    checkNodeAdded = true;
+                else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
+                    checkClientNodeAddFinished = true;
+            }
+        }
+    }
+
 
     /**
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 87509a4..22fa36d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -428,21 +428,42 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      */
     @SuppressWarnings("BusyWait")
     protected void awaitPartitionMapExchange() throws InterruptedException {
-        awaitPartitionMapExchange(false, false);
+        awaitPartitionMapExchange(false, false, null);
     }
 
     /**
      * @param waitEvicts If {@code true} will wait for evictions finished.
      * @param waitNode2PartUpdate If {@code true} will wait for nodes node2part info update finished.
+     * @param nodes Optional nodes.
      * @throws InterruptedException If interrupted.
      */
     @SuppressWarnings("BusyWait")
-    protected void awaitPartitionMapExchange(boolean waitEvicts, boolean waitNode2PartUpdate) throws InterruptedException {
+    protected void awaitPartitionMapExchange(boolean waitEvicts,
+        boolean waitNode2PartUpdate,
+        @Nullable Collection<ClusterNode> nodes)
+        throws InterruptedException {
         long timeout = 30_000;
 
+        long startTime = -1;
+
+        Set<String> names = new HashSet<>();
+
         for (Ignite g : G.allGrids()) {
+            if (nodes != null && !nodes.contains(g.cluster().localNode()))
+                continue;
+
             IgniteKernal g0 = (IgniteKernal)g;
 
+            names.add(g0.configuration().getGridName());
+
+            if (startTime != -1) {
+                if (startTime != g0.context().discovery().gridStartTime())
+                    fail("Found nodes from different clusters, probable some test does not stop nodes " +
+                        "[allNodes=" + names + ']');
+            }
+            else
+                startTime = g0.context().discovery().gridStartTime();
+
             for (IgniteCacheProxy<?, ?> c : g0.context().cache().jcaches()) {
                 CacheConfiguration cfg = c.context().config();
 


[17/17] ignite git commit: ignite-4154

Posted by sb...@apache.org.
ignite-4154


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

Branch: refs/heads/ignite-4154-opt2
Commit: 6d6bfdc35c1a6819c6c1acb72c5a3779dcf895a2
Parents: ffd3541
Author: sboikov <sb...@gridgain.com>
Authored: Fri Nov 11 16:09:37 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Nov 11 16:14:04 2016 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 146 ++++++++++++++++---
 .../tcp/internal/TcpDiscoveryStatistics.java    |   7 +-
 2 files changed, 129 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6d6bfdc3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 8814745..3766c39 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -129,6 +129,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRedirectToClient;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -136,6 +137,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
@@ -174,8 +176,10 @@ class ServerImpl extends TcpDiscoveryImpl {
         IgniteProductVersion.fromString("1.5.0");
 
     /** */
-    private final ThreadPoolExecutor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>());
+    private static final boolean SEND_JOIN_REQ_DIRECTLY = getBoolean("SEND_JOIN_REQ_DIRECTLY", true);
+
+    /** */
+    private IgniteThreadPoolExecutor utilityPool;
 
     /** Nodes ring. */
     @GridToStringExclude
@@ -297,6 +301,13 @@ class ServerImpl extends TcpDiscoveryImpl {
             spiState = DISCONNECTED;
         }
 
+        utilityPool = new IgniteThreadPoolExecutor("disco-pool",
+            spi.ignite().name(),
+            0,
+            1,
+            2000,
+            new LinkedBlockingQueue<Runnable>());
+
         if (debugMode) {
             if (!log.isInfoEnabled())
                 throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
@@ -918,6 +929,8 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         if (log.isDebugEnabled())
             log.debug("Discovery SPI has been connected to topology with order: " + locNode.internalOrder());
+
+        log.info("Node joined topology: " + locNode);
     }
 
     /**
@@ -974,7 +987,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             for (InetSocketAddress addr : addrs) {
                 try {
-                    Integer res = sendMessageDirectly(joinReq, addr);
+                    Integer res = sendMessageDirectly(joinReq, addr, true);
 
                     assert res != null;
 
@@ -1087,13 +1100,15 @@ class ServerImpl extends TcpDiscoveryImpl {
      *
      * @param msg Message to send.
      * @param addr Address to send message to.
+     * @param join {@code True} if sends initial node join request.
      * @return Response read from the recipient or {@code null} if no response is supposed.
      * @throws IgniteSpiException If an error occurs.
      */
-    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr)
+    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr, boolean join)
         throws IgniteSpiException {
         assert msg != null;
         assert addr != null;
+        assert !join || msg instanceof TcpDiscoveryJoinRequestMessage;
 
         Collection<Throwable> errs = null;
 
@@ -1180,7 +1195,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 // Connection has been established, but
                 // join request may not be unmarshalled on remote host.
                 // E.g. due to class not found issue.
-                joinReqSent = msg instanceof TcpDiscoveryJoinRequestMessage;
+                joinReqSent = join;
 
                 int receipt = spi.readReceipt(sock, timeoutHelper.nextTimeoutChunk(ackTimeout0));
 
@@ -3744,8 +3759,79 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (nodeAddedMsg.verified())
                     msgHist.add(nodeAddedMsg);
             }
-            else if (sendMessageToRemotes(msg))
-                sendMessageAcrossRing(msg);
+            else {
+                if (sendMessageToRemotes(msg)) {
+                    if (SEND_JOIN_REQ_DIRECTLY && !msg.directSendFailed()) {
+                        final TcpDiscoveryNode crd = resolveCoordinator();
+
+                        Collection<TcpDiscoveryNode> failedNodes;
+
+                        synchronized (mux) {
+                            failedNodes = U.arrayList(ServerImpl.this.failedNodes.keySet());
+                        }
+
+                        TcpDiscoveryNode next = ring.nextNode(failedNodes);
+
+                        if (crd != null && !crd.equals(next)) {
+                            if (log.isDebugEnabled()) {
+                                log.debug("Will send join request directly to coordinator " +
+                                    "[msg=" + msg + ", crd=" + crd + ", next=" + next + ']');
+                            }
+
+                            log.info("Will send join request directly to coordinator " +
+                                "[cnt=" + joiningNodes.size() + ", msg=" + msg + ", crd=" + crd + ", next=" + next + ']');
+
+                            utilityPool.submit(new Runnable() {
+                                @Override public void run() {
+                                    IgniteSpiException sndErr = null;
+                                    Integer res = null;
+
+                                    TcpDiscoveryJoinRequestMessage msg0 =
+                                        new TcpDiscoveryJoinRequestMessage(msg.node(), msg.discoveryData());
+
+                                    try {
+                                        res = trySendMessageDirectly(crd, msg0);
+
+                                        if (F.eq(RES_OK, res)) {
+                                            if (log.isDebugEnabled()) {
+                                                log.debug("Sent join request directly to coordinator " +
+                                                    "[msg=" + msg0 + ", crd=" + crd + ']');
+                                            }
+
+                                            log.info("Sent join request directly to coordinator " +
+                                                "[msg=" + msg0 + ", crd=" + crd + ']');
+
+                                            return;
+                                        }
+                                    }
+                                    catch (IgniteSpiException e) {
+                                        sndErr = e;
+                                    }
+
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("Failed to send join request to coordinator, will process from " +
+                                            "message worker [msg=" + msg0 + ", crd=" + crd + ", err=" + sndErr +
+                                            ", res=" + res + ']');
+                                    }
+
+                                    log.info("Failed to send join request to coordinator, will process from " +
+                                        "message worker [msg=" + msg0 + ", crd=" + crd + ", err=" + sndErr +
+                                        ", res=" + res + ']');
+
+                                    msg.directSendFailed(true);
+
+                                    msgWorker.addMessage(msg);
+                                }
+                            });
+
+                            return;
+                        }
+                    }
+
+                    sendMessageAcrossRing(msg);
+                }
+
+            }
         }
 
         /**
@@ -3780,7 +3866,7 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @param msg Message.
          * @throws IgniteSpiException Last failure if all attempts failed.
          */
-        private void trySendMessageDirectly(TcpDiscoveryNode node, TcpDiscoveryAbstractMessage msg)
+        private Integer trySendMessageDirectly(TcpDiscoveryNode node, TcpDiscoveryAbstractMessage msg)
             throws IgniteSpiException {
             if (node.isClient()) {
                 TcpDiscoveryNode routerNode = ring.node(node.clientRouterNodeId());
@@ -3801,25 +3887,21 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     worker.addMessage(msg);
 
-                    return;
+                    return null;
                 }
 
-                trySendMessageDirectly(routerNode, msg);
-
-                return;
+                return trySendMessageDirectly(routerNode, msg);
             }
 
             IgniteSpiException ex = null;
 
             for (InetSocketAddress addr : spi.getNodeAddresses(node, U.sameMacs(locNode, node))) {
                 try {
-                    sendMessageDirectly(msg, addr);
+                    Integer res = sendMessageDirectly(msg, addr, false);
 
                     node.lastSuccessfulAddress(addr);
 
-                    ex = null;
-
-                    break;
+                    return res;
                 }
                 catch (IgniteSpiException e) {
                     ex = e;
@@ -3828,6 +3910,8 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             if (ex != null)
                 throw ex;
+
+            return null;
         }
 
         /**
@@ -5544,6 +5628,9 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             ClientMessageWorker clientMsgWrk = null;
 
+            TcpDiscoveryAbstractMessage msg = null;
+            Exception sockE = null;
+
             try {
                 InputStream in;
 
@@ -5604,7 +5691,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     // Restore timeout.
                     sock.setSoTimeout(timeout);
 
-                    TcpDiscoveryAbstractMessage msg = spi.readMessage(sock, in, spi.netTimeout);
+                    msg = spi.readMessage(sock, in, spi.netTimeout);
 
                     // Ping.
                     if (msg instanceof TcpDiscoveryPingRequest) {
@@ -5709,6 +5796,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
                 catch (IOException e) {
+                    sockE = e;
+
                     if (log.isDebugEnabled())
                         U.error(log, "Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
 
@@ -5736,6 +5825,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
                 catch (IgniteCheckedException e) {
+                    sockE = e;
+
                     if (log.isDebugEnabled())
                         U.error(log, "Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
 
@@ -5765,8 +5856,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 while (!isInterrupted()) {
                     try {
-                        TcpDiscoveryAbstractMessage msg = U.unmarshal(spi.marshaller(), in,
-                            U.resolveClassLoader(spi.ignite().configuration()));
+                        msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration()));
 
                         msg.senderNodeId(nodeId);
 
@@ -5793,9 +5883,12 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                 if (clientMsgWrk != null && ok)
                                     continue;
-                                else
+                                else {
+                                    log.info("Processed join request, close connection [msg=" + msg + ']');
+
                                     // Direct join request - no need to handle this socket anymore.
                                     break;
+                                }
                             }
                         }
                         else if (msg instanceof TcpDiscoveryClientReconnectMessage) {
@@ -5968,6 +6061,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                             processClientHeartbeatMessage(heartbeatMsg);
                     }
                     catch (IgniteCheckedException e) {
+                        sockE = e;
+
                         if (log.isDebugEnabled())
                             U.error(log, "Caught exception on message read [sock=" + sock +
                                 ", locNodeId=" + locNodeId + ", rmtNodeId=" + nodeId + ']', e);
@@ -5995,6 +6090,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                         return;
                     }
                     catch (IOException e) {
+                        sockE = e;
+
                         if (log.isDebugEnabled())
                             U.error(log, "Caught exception on message read [sock=" + sock + ", locNodeId=" + locNodeId +
                                 ", rmtNodeId=" + nodeId + ']', e);
@@ -6018,6 +6115,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
             }
             finally {
+                if (locNode.order() == 1)
+                    log.info("Close sock [readers=" + spi.stats.socketReaders() + ", msg=" + msg + ", err=" + sockE + ']');
+
                 if (clientMsgWrk != null) {
                     if (log.isDebugEnabled())
                         log.debug("Client connection failed [sock=" + sock + ", locNodeId=" + locNodeId +
@@ -6062,11 +6162,11 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             TcpDiscoverySpiState state = spiStateCopy();
 
-            long socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
+            long sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
                 spi.getSocketTimeout();
 
             if (state == CONNECTED) {
-                spi.writeToSocket(msg, sock, RES_OK, socketTimeout);
+                spi.writeToSocket(msg, sock, RES_OK, sockTimeout);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + RES_OK + ']');
@@ -6103,7 +6203,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     // Local node is stopping. Remote node should try next one.
                     res = RES_CONTINUE_JOIN;
 
-                spi.writeToSocket(msg, sock, res, socketTimeout);
+                spi.writeToSocket(msg, sock, res, sockTimeout);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + res + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d6bfdc3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
index 7c6025d..441aa46 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
@@ -628,6 +628,10 @@ public class TcpDiscoveryStatistics {
         return sockReadersCreated;
     }
 
+    public synchronized int socketReaders() {
+        return sockReadersCreated - sockReadersRmv;
+    }
+
     /**
      * Gets socket readers removed count.
      *
@@ -687,6 +691,7 @@ public class TcpDiscoveryStatistics {
 
     /** {@inheritDoc} */
     @Override public synchronized String toString() {
-        return "Stats [created=" + sockReadersCreated + ", rmvd=" + sockReadersRmv + ']';//S.toString(TcpDiscoveryStatistics.class, this);
+        //return "Stats [created=" + sockReadersCreated + ", rmvd=" + sockReadersRmv + ']';
+        return S.toString(TcpDiscoveryStatistics.class, this);
     }
 }
\ No newline at end of file


[08/17] ignite git commit: Fixed classnames.properties generation for ignite-hadoop module.

Posted by sb...@apache.org.
Fixed classnames.properties generation for ignite-hadoop module.


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

Branch: refs/heads/ignite-4154-opt2
Commit: e159fdee0152a6df1bbf4dafc3d18a284e86a57e
Parents: eac5f0c
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Nov 11 09:39:32 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Nov 11 09:39:32 2016 +0700

----------------------------------------------------------------------
 modules/hadoop/pom.xml | 49 +++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 49 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e159fdee/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index a3f40e5..98f8576 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -127,6 +127,55 @@
     <build>
         <plugins>
             <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>exec-maven-plugin</artifactId>
+                <version>1.3.2</version>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.ignite</groupId>
+                        <artifactId>ignite-tools</artifactId>
+                        <version>${project.version}</version>
+                    </dependency>
+                </dependencies>
+                <executions>
+                    <execution>
+                        <phase>process-classes</phase>
+                        <goals>
+                            <goal>java</goal>
+                        </goals>
+                        <configuration>
+                            <includePluginDependencies>true</includePluginDependencies>
+                            <mainClass>org.apache.ignite.tools.classgen.ClassesGenerator</mainClass>
+                            <arguments>
+                                <argument>${project.basedir}/target/classes</argument>
+                                <argument>
+                                    <![CDATA[
+#
+# 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.
+#
+                                    ]]>
+                                </argument>
+                                <argument>org.apache.ignite</argument>
+                            </arguments>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-jar-plugin</artifactId>
                 <version>2.2</version>


[04/17] ignite git commit: IGNITE-500 CacheLoadingConcurrentGridStartSelfTest fails (DataStreamer data loss at unstable topology in !allowOverwrite mode fixed)

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
index c1251ae..3d3f146 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
@@ -125,8 +125,6 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
                 ignite.getOrCreateCache(cacheConfiguration());
 
             try (final DataStreamerImpl dataLdr = (DataStreamerImpl)ignite.dataStreamer(null)) {
-                dataLdr.maxRemapCount(0);
-
                 Random rnd = new Random();
 
                 long endTime = U.currentTimeMillis() + 15_000;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java
index 4e981b7..766aa84 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java
@@ -17,11 +17,13 @@
 package org.apache.ignite.internal.processors.datastreamer;
 
 import java.util.Collection;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteDataStreamerTimeoutException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
@@ -36,7 +38,6 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
  * Test timeout for Data streamer.
  */
 public class DataStreamerTimeoutTest extends GridCommonAbstractTest {
-
     /** Cache name. */
     public static final String CACHE_NAME = "cacheName";
 
@@ -46,6 +47,9 @@ public class DataStreamerTimeoutTest extends GridCommonAbstractTest {
     /** Amount of entries. */
     public static final int ENTRY_AMOUNT = 100;
 
+    /** Fail on. */
+    private static volatile int failOn;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -76,6 +80,8 @@ public class DataStreamerTimeoutTest extends GridCommonAbstractTest {
      * @throws Exception If fail.
      */
     public void testTimeoutOnCloseMethod() throws Exception {
+        failOn = 1;
+
         Ignite ignite = startGrid(1);
 
         boolean thrown = false;
@@ -85,12 +91,10 @@ public class DataStreamerTimeoutTest extends GridCommonAbstractTest {
             ldr.receiver(new TestDataReceiver());
             ldr.perNodeBufferSize(ENTRY_AMOUNT);
 
-            for (int i=0; i < ENTRY_AMOUNT; i++)
+            for (int i = 0; i < ENTRY_AMOUNT; i++)
                 ldr.addData(i, i);
-
         }
-        catch (IgniteDataStreamerTimeoutException e) {
-            assertEquals(e.getMessage(), "Data streamer exceeded timeout on flush.");
+        catch (CacheException | IgniteDataStreamerTimeoutException e) {
             thrown = true;
         }
         finally {
@@ -102,40 +106,68 @@ public class DataStreamerTimeoutTest extends GridCommonAbstractTest {
 
     /**
      * Test timeout on {@code DataStreamer.close()} method
+     *
      * @throws Exception If fail.
      */
-    public void testTimeoutOnAddDataMethod() throws Exception {
-        Ignite ignite = startGrid(1);
+    public void testTimeoutOnAddData() throws Exception {
+        failOn = 1;
 
-        boolean thrown = false;
+        int processed = timeoutOnAddData();
 
-        IgniteDataStreamer ldr = ignite.dataStreamer(CACHE_NAME);
+        assertTrue(processed == (failOn + 1) || processed == failOn);
 
-        try {
-            ldr.timeout(TIMEOUT);
-            ldr.receiver(new TestDataReceiver());
-            ldr.perNodeBufferSize(ENTRY_AMOUNT/2);
-            ldr.perNodeParallelOperations(1);
+        failOn = ENTRY_AMOUNT / 2;
 
-            try {
-                for (int i=0; i < ENTRY_AMOUNT; i++)
-                    ldr.addData(i, i);
-            }
-            catch (IgniteDataStreamerTimeoutException e) {
-                assertEquals(e.getMessage(), "Data streamer exceeded timeout when starts parallel operation.");
+        processed = timeoutOnAddData();
+
+        assertTrue(processed == (failOn + 1) || processed == failOn);
 
+        failOn = ENTRY_AMOUNT;
+
+        processed = timeoutOnAddData();
+
+        assertTrue(processed == (failOn + 1) || processed == failOn);
+    }
+
+    /**
+     *
+     */
+    private int timeoutOnAddData() throws Exception {
+        boolean thrown = false;
+        int processed = 0;
+
+        try {
+            Ignite ignite = startGrid(1);
+
+            try (IgniteDataStreamer ldr = ignite.dataStreamer(CACHE_NAME)) {
+                ldr.timeout(TIMEOUT);
+                ldr.receiver(new TestDataReceiver());
+                ldr.perNodeBufferSize(1);
+                ldr.perNodeParallelOperations(1);
+                ((DataStreamerImpl)ldr).maxRemapCount(0);
+
+                try {
+                    for (int i = 0; i < ENTRY_AMOUNT; i++) {
+                        ldr.addData(i, i);
+
+                        processed++;
+                    }
+                }
+                catch (IllegalStateException e) {
+                    // No-op.
+                }
+            }
+            catch (CacheException | IgniteDataStreamerTimeoutException e) {
                 thrown = true;
             }
-
         }
         finally {
-            if (thrown)
-                ldr.close(true);
-
             stopAllGrids();
         }
 
         assertTrue(thrown);
+
+        return processed;
     }
 
     /**
@@ -143,16 +175,14 @@ public class DataStreamerTimeoutTest extends GridCommonAbstractTest {
      */
     private static class TestDataReceiver implements StreamReceiver {
 
-        /** Is first. */
-        boolean isFirst = true;
+        /** Count. */
+        private final AtomicInteger cnt = new AtomicInteger();
 
         /** {@inheritDoc} */
-        @Override public void receive(IgniteCache cache, Collection collection) throws IgniteException {
+        @Override public void receive(IgniteCache cache, Collection col) throws IgniteException {
             try {
-                if (isFirst)
+                if (cnt.incrementAndGet() == failOn)
                     U.sleep(2 * TIMEOUT);
-
-                isFirst = false;
             }
             catch (IgniteInterruptedCheckedException e) {
                 throw new IgniteException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index dc412a9..0663903 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheIncrementTxTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTestAllowOverwrite;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLockReleaseNodeLeaveTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionNotLoadedEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedNearDisabledTxMultiThreadedSelfTest;
@@ -214,6 +215,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(FairAffinityFunctionBackupFilterSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedPreloadLifecycleSelfTest.class));
         suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTest.class));
+        suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.class));
         suite.addTest(new TestSuite(GridCacheDhtPreloadDelayedSelfTest.class));
         suite.addTest(new TestSuite(GridPartitionedBackupLoadSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedLoadCacheSelfTest.class));


[02/17] ignite git commit: Fixed javadoc.

Posted by sb...@apache.org.
Fixed javadoc.


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

Branch: refs/heads/ignite-4154-opt2
Commit: 26daa57ca82254d68ac04a7b33223c6eb5ade0e4
Parents: bac0cba
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 10 11:17:29 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 10 11:17:29 2016 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/26daa57c/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
index 39170ea..783a113 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
@@ -99,7 +99,7 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable {
     }
 
     /**
-     * @return
+     * @return {@code True} if need use trace logging for this message (to reduce amount of logging with debug level).
      */
     public boolean traceLogLevel() {
         return false;


[15/17] ignite git commit: ignite-4154 Optimize amount of data stored in discovery history Discovery history optimizations: - remove discarded message for discovery pending messages - remove duplicated data from TcpDiscoveryNodeAddedMessage.oldNodesDisco

Posted by sb...@apache.org.
ignite-4154 Optimize amount of data stored in discovery history
Discovery history optimizations:
- remove discarded message for discovery pending messages
- remove duplicated data from TcpDiscoveryNodeAddedMessage.oldNodesDiscoData
- do not store unnecessary data in discovery EnsuredMessageHistory
- use special property for EnsuredMessageHistory size instead of IGNITE_DISCOVERY_HISTORY_SIZE
Affinity history optimizations:
- do not store calculated primary/backup maps in history
- try save the same assignments instance for caches with similar affinity
Exchange messages optimizations:
- do not send duplicated partition state maps for caches with similar affinity
- use zip compression for data sent in exchange messages


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

Branch: refs/heads/ignite-4154-opt2
Commit: 7128a395085b60e86436f807b4bdbca83627d41a
Parents: 8bb8bdd
Author: sboikov <sb...@gridgain.com>
Authored: Fri Nov 11 15:29:38 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Nov 11 15:29:38 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   4 +
 .../processors/affinity/AffinityAssignment.java |  88 +++++
 .../affinity/GridAffinityAssignment.java        |   8 +-
 .../affinity/GridAffinityAssignmentCache.java   |  35 +-
 .../affinity/GridAffinityProcessor.java         |  89 ++++-
 .../processors/affinity/GridAffinityUtils.java  |   8 +-
 .../affinity/HistoryAffinityAssignment.java     | 169 ++++++++
 .../cache/CacheAffinitySharedManager.java       |  57 ++-
 .../cache/DynamicCacheChangeBatch.java          |   7 +
 .../cache/GridCacheAffinityManager.java         |   6 +-
 .../GridCachePartitionExchangeManager.java      | 284 ++++++++++++--
 .../processors/cache/GridCacheProcessor.java    |   5 +-
 .../dht/GridClientPartitionTopology.java        |  33 +-
 .../dht/GridDhtPartitionTopology.java           |   3 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  31 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |  18 +
 .../dht/preloader/GridDhtPartitionMap2.java     |  53 ++-
 .../GridDhtPartitionsAbstractMessage.java       |  40 +-
 .../GridDhtPartitionsExchangeFuture.java        |  84 +---
 .../preloader/GridDhtPartitionsFullMessage.java | 150 ++++++-
 .../GridDhtPartitionsSingleMessage.java         | 132 ++++++-
 .../GridDhtPartitionsSingleRequest.java         |   4 +-
 .../dht/preloader/GridDhtPreloader.java         |   4 +-
 .../continuous/GridContinuousProcessor.java     |   4 +-
 .../ignite/internal/util/IgniteUtils.java       |  64 +++
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  26 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 234 +++++++++--
 .../TcpDiscoveryNodeAddFinishedMessage.java     |  11 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |  33 +-
 ...CacheExchangeMessageDuplicatedStateTest.java | 393 +++++++++++++++++++
 .../cache/IgniteCachePeekModesAbstractTest.java |   2 +-
 .../distributed/IgniteCacheGetRestartTest.java  |   3 +
 ...cingDelayedPartitionMapExchangeSelfTest.java |   8 +-
 .../GridCacheRebalancingSyncSelfTest.java       |  18 +-
 .../GridCacheSyncReplicatedPreloadSelfTest.java |   3 -
 .../IgniteCacheSyncRebalanceModeSelfTest.java   |   4 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |   2 +-
 .../IgniteNoCustomEventsOnNodeStart.java        |   7 +
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 134 ++++++-
 .../junits/common/GridCommonAbstractTest.java   |  25 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   3 +
 .../cache/IgniteCacheOffheapEvictQueryTest.java |   7 +
 ...lientQueryReplicatedNodeRestartSelfTest.java |   7 +
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |   7 +
 .../query/h2/sql/GridQueryParsingTest.java      |  11 +-
 .../src/test/config/incorrect-store-cache.xml   |   2 +
 .../src/test/config/jdbc-pojo-store-builtin.xml |   3 +
 .../src/test/config/jdbc-pojo-store-obj.xml     |   3 +
 modules/spring/src/test/config/node.xml         |   2 +
 modules/spring/src/test/config/node1.xml        |   2 +
 .../test/config/pojo-incorrect-store-cache.xml  |   2 +
 modules/spring/src/test/config/store-cache.xml  |   2 +
 modules/spring/src/test/config/store-cache1.xml |   2 +
 53 files changed, 2061 insertions(+), 275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index ab6403f..a75027b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -383,6 +383,10 @@ public final class IgniteSystemProperties {
     /** Maximum size for discovery messages history. */
     public static final String IGNITE_DISCOVERY_HISTORY_SIZE = "IGNITE_DISCOVERY_HISTORY_SIZE";
 
+    /** Maximum number of discovery message history used to support client reconnect. */
+    public static final String IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE =
+        "IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE";
+
     /** Number of cache operation retries in case of topology exceptions. */
     public static final String IGNITE_CACHE_RETRIES_COUNT = "IGNITE_CACHE_RETRIES_COUNT";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java
new file mode 100644
index 0000000..06207d3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java
@@ -0,0 +1,88 @@
+/*
+ * 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.affinity;
+
+import org.apache.ignite.cluster.ClusterNode;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Cached affinity calculations.
+ */
+public interface AffinityAssignment {
+    /**
+     * @return {@code True} if related discovery event did not not cause affinity assignment change and
+     *    this assignment is just reference to the previous one.
+     */
+    public boolean clientEventChange();
+
+    /**
+     * @return Affinity assignment computed by affinity function.
+     */
+    public List<List<ClusterNode>> idealAssignment();
+
+    /**
+     * @return Affinity assignment.
+     */
+    public List<List<ClusterNode>> assignment();
+
+    /**
+     * @return Topology version.
+     */
+    public AffinityTopologyVersion topologyVersion();
+
+    /**
+     * Get affinity nodes for partition.
+     *
+     * @param part Partition.
+     * @return Affinity nodes.
+     */
+    public List<ClusterNode> get(int part);
+
+    /**
+     * Get affinity node IDs for partition.
+     *
+     * @param part Partition.
+     * @return Affinity nodes IDs.
+     */
+    public HashSet<UUID> getIds(int part);
+
+    /**
+     * @return Nodes having primary partitions assignments.
+     */
+    public Set<ClusterNode> primaryPartitionNodes();
+
+    /**
+     * Get primary partitions for specified node ID.
+     *
+     * @param nodeId Node ID to get primary partitions for.
+     * @return Primary partitions for specified node ID.
+     */
+    public Set<Integer> primaryPartitions(UUID nodeId);
+
+    /**
+     * Get backup partitions for specified node ID.
+     *
+     * @param nodeId Node ID to get backup partitions for.
+     * @return Backup partitions for specified node ID.
+     */
+    public Set<Integer> backupPartitions(UUID nodeId);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
index 568e4e8..35130a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
@@ -32,7 +32,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 /**
  * Cached affinity calculations.
  */
-public class GridAffinityAssignment implements Serializable {
+public class GridAffinityAssignment implements AffinityAssignment, Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -86,7 +86,7 @@ public class GridAffinityAssignment implements Serializable {
 
         this.topVer = topVer;
         this.assignment = assignment;
-        this.idealAssignment = idealAssignment;
+        this.idealAssignment = idealAssignment.equals(assignment) ? assignment : idealAssignment;
 
         primary = new HashMap<>();
         backup = new HashMap<>();
@@ -274,10 +274,10 @@ public class GridAffinityAssignment implements Serializable {
         if (o == this)
             return true;
 
-        if (o == null || getClass() != o.getClass())
+        if (o == null || !(o instanceof AffinityAssignment))
             return false;
 
-        return topVer.equals(((GridAffinityAssignment)o).topVer);
+        return topVer.equals(((AffinityAssignment)o).topologyVersion());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index a81b34d..a388c7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -78,7 +78,7 @@ public class GridAffinityAssignmentCache {
     private final int partsCnt;
 
     /** Affinity calculation results cache: topology version => partition => nodes. */
-    private final ConcurrentNavigableMap<AffinityTopologyVersion, GridAffinityAssignment> affCache;
+    private final ConcurrentNavigableMap<AffinityTopologyVersion, HistoryAffinityAssignment> affCache;
 
     /** */
     private List<List<ClusterNode>> idealAssignment;
@@ -107,6 +107,9 @@ public class GridAffinityAssignmentCache {
     /** Full history size. */
     private final AtomicInteger fullHistSize = new AtomicInteger();
 
+    /** */
+    private final Object similarAffKey;
+
     /**
      * Constructs affinity cached calculations.
      *
@@ -127,6 +130,7 @@ public class GridAffinityAssignmentCache {
     {
         assert ctx != null;
         assert aff != null;
+        assert nodeFilter != null;
 
         this.ctx = ctx;
         this.aff = aff;
@@ -142,6 +146,17 @@ public class GridAffinityAssignmentCache {
         partsCnt = aff.partitions();
         affCache = new ConcurrentSkipListMap<>();
         head = new AtomicReference<>(new GridAffinityAssignment(AffinityTopologyVersion.NONE));
+
+        similarAffKey = ctx.affinity().similaryAffinityKey(aff, nodeFilter, backups, partsCnt);
+
+        assert similarAffKey != null;
+    }
+
+    /**
+     * @return Key to find caches with similar affinity.
+     */
+    public Object similarAffinityKey() {
+        return similarAffKey;
     }
 
     /**
@@ -170,7 +185,7 @@ public class GridAffinityAssignmentCache {
 
         GridAffinityAssignment assignment = new GridAffinityAssignment(topVer, affAssignment, idealAssignment);
 
-        affCache.put(topVer, assignment);
+        affCache.put(topVer, new HistoryAffinityAssignment(assignment));
         head.set(assignment);
 
         for (Map.Entry<AffinityTopologyVersion, AffinityReadyFuture> entry : readyFuts.entrySet()) {
@@ -300,7 +315,7 @@ public class GridAffinityAssignmentCache {
 
         GridAffinityAssignment assignmentCpy = new GridAffinityAssignment(topVer, aff);
 
-        affCache.put(topVer, assignmentCpy);
+        affCache.put(topVer, new HistoryAffinityAssignment(assignmentCpy));
         head.set(assignmentCpy);
 
         for (Map.Entry<AffinityTopologyVersion, AffinityReadyFuture> entry : readyFuts.entrySet()) {
@@ -328,7 +343,7 @@ public class GridAffinityAssignmentCache {
      * @return Affinity assignment.
      */
     public List<List<ClusterNode>> assignments(AffinityTopologyVersion topVer) {
-        GridAffinityAssignment aff = cachedAffinity(topVer);
+        AffinityAssignment aff = cachedAffinity(topVer);
 
         return aff.assignment();
     }
@@ -427,7 +442,7 @@ public class GridAffinityAssignmentCache {
      * @param topVer Topology version.
      * @return Cached affinity.
      */
-    public GridAffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) {
+    public AffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) {
         if (topVer.equals(AffinityTopologyVersion.NONE))
             topVer = lastVersion();
         else
@@ -435,7 +450,7 @@ public class GridAffinityAssignmentCache {
 
         assert topVer.topologyVersion() >= 0 : topVer;
 
-        GridAffinityAssignment cache = head.get();
+        AffinityAssignment cache = head.get();
 
         if (!cache.topologyVersion().equals(topVer)) {
             cache = affCache.get(topVer);
@@ -463,7 +478,7 @@ public class GridAffinityAssignmentCache {
      * @return {@code True} if primary changed or required affinity version not found in history.
      */
     public boolean primaryChanged(int part, AffinityTopologyVersion startVer, AffinityTopologyVersion endVer) {
-        GridAffinityAssignment aff = affCache.get(startVer);
+        AffinityAssignment aff = affCache.get(startVer);
 
         if (aff == null)
             return false;
@@ -475,7 +490,7 @@ public class GridAffinityAssignmentCache {
 
         ClusterNode primary = nodes.get(0);
 
-        for (GridAffinityAssignment assignment : affCache.tailMap(startVer, false).values()) {
+        for (AffinityAssignment assignment : affCache.tailMap(startVer, false).values()) {
             List<ClusterNode> nodes0 = assignment.assignment().get(part);
 
             if (nodes0.isEmpty())
@@ -549,10 +564,10 @@ public class GridAffinityAssignmentCache {
         }
 
         if (rmvCnt > 0) {
-            Iterator<GridAffinityAssignment> it = affCache.values().iterator();
+            Iterator<HistoryAffinityAssignment> it = affCache.values().iterator();
 
             while (it.hasNext() && rmvCnt > 0) {
-                GridAffinityAssignment aff0 = it.next();
+                AffinityAssignment aff0 = it.next();
 
                 it.remove();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
index 1726d02..b9182ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -385,10 +386,16 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
             }
 
             try {
+                AffinityAssignment assign0 = cctx.affinity().assignment(topVer);
+
+                GridAffinityAssignment assign = assign0 instanceof GridAffinityAssignment ?
+                    (GridAffinityAssignment)assign0 :
+                    new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment());
+
                 AffinityInfo info = new AffinityInfo(
                     cctx.config().getAffinity(),
                     cctx.config().getAffinityMapper(),
-                    new GridAffinityAssignment(topVer, cctx.affinity().assignment(topVer)),
+                    assign,
                     cctx.cacheObjectContext());
 
                 IgniteInternalFuture<AffinityInfo> old = affMap.putIfAbsent(key, new GridFinishedFuture<>(info));
@@ -562,6 +569,20 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
         return nodes.iterator().next();
     }
 
+    /**
+     * @param aff Affinity function.
+     * @param nodeFilter Node class.
+     * @param backups Number of backups.
+     * @param parts Number of partitions.
+     * @return Key to find caches with similar affinity.
+     */
+    public Object similaryAffinityKey(AffinityFunction aff,
+        IgnitePredicate<ClusterNode> nodeFilter,
+        int backups,
+        int parts) {
+        return new SimilarAffinityKey(aff.getClass(), nodeFilter.getClass(), backups, parts);
+    }
+
     /** {@inheritDoc} */
     @Override public void printMemoryStats() {
         X.println(">>>");
@@ -960,4 +981,70 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
             return aff;
         }
     }
+
+    /**
+     *
+     */
+    private static class SimilarAffinityKey {
+        /** */
+        private final int backups;
+
+        /** */
+        private final Class<?> affFuncCls;
+
+        /** */
+        private final Class<?> filterCls;
+
+        /** */
+        private final int partsCnt;
+
+        /** */
+        private final int hash;
+
+        /**
+         * @param affFuncCls Affinity function class.
+         * @param filterCls Node filter class.
+         * @param backups Number of backups.
+         * @param partsCnt Number of partitions.
+         */
+        SimilarAffinityKey(Class<?> affFuncCls, Class<?> filterCls, int backups, int partsCnt) {
+            this.backups = backups;
+            this.affFuncCls = affFuncCls;
+            this.filterCls = filterCls;
+            this.partsCnt = partsCnt;
+
+            int hash = backups;
+            hash = 31 * hash + affFuncCls.hashCode();
+            hash = 31 * hash + filterCls.hashCode();
+            hash= 31 * hash + partsCnt;
+
+            this.hash = hash;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return hash;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (o == this)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            SimilarAffinityKey key = (SimilarAffinityKey)o;
+
+            return backups == key.backups &&
+                affFuncCls == key.affFuncCls &&
+                filterCls == key.filterCls &&
+                partsCnt == key.partsCnt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(SimilarAffinityKey.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
index c24dd2d..abd5292 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
@@ -180,10 +180,16 @@ class GridAffinityUtils {
 
             cctx.affinity().affinityReadyFuture(topVer).get();
 
+            AffinityAssignment assign0 = cctx.affinity().assignment(topVer);
+
+            GridAffinityAssignment assign = assign0 instanceof GridAffinityAssignment ?
+                (GridAffinityAssignment)assign0 :
+                new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment());
+
             return F.t(
                 affinityMessage(ctx, cctx.config().getAffinity()),
                 affinityMessage(ctx, cctx.config().getAffinityMapper()),
-                new GridAffinityAssignment(topVer, cctx.affinity().assignment(topVer)));
+                assign);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java
new file mode 100644
index 0000000..e502dd5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java
@@ -0,0 +1,169 @@
+/*
+ * 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.affinity;
+
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ *
+ */
+public class HistoryAffinityAssignment implements AffinityAssignment {
+    /** */
+    private final AffinityTopologyVersion topVer;
+
+    /** */
+    private final List<List<ClusterNode>> assignment;
+
+    /** */
+    private final List<List<ClusterNode>> idealAssignment;
+
+    /** */
+    private final boolean clientEvtChange;
+
+    /**
+     * @param assign Assignment.
+     */
+    public HistoryAffinityAssignment(GridAffinityAssignment assign) {
+        this.topVer = assign.topologyVersion();
+        this.assignment = assign.assignment();
+        this.idealAssignment = assign.idealAssignment();
+        this.clientEvtChange = assign.clientEventChange();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean clientEventChange() {
+        return clientEvtChange;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<List<ClusterNode>> idealAssignment() {
+        return idealAssignment;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<List<ClusterNode>> assignment() {
+        return assignment;
+    }
+
+    /** {@inheritDoc} */
+    @Override public AffinityTopologyVersion topologyVersion() {
+        return topVer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<ClusterNode> get(int part) {
+        assert part >= 0 && part < assignment.size() : "Affinity partition is out of range" +
+            " [part=" + part + ", partitions=" + assignment.size() + ']';
+
+        return assignment.get(part);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HashSet<UUID> getIds(int part) {
+        assert part >= 0 && part < assignment.size() : "Affinity partition is out of range" +
+            " [part=" + part + ", partitions=" + assignment.size() + ']';
+
+        List<ClusterNode> nodes = assignment.get(part);
+
+        HashSet<UUID> ids = U.newHashSet(nodes.size());
+
+        for (int i = 0; i < nodes.size(); i++)
+            ids.add(nodes.get(i).id());
+
+        return ids;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<ClusterNode> primaryPartitionNodes() {
+        Set<ClusterNode> res = new HashSet<>();
+
+        for (int p = 0; p < assignment.size(); p++) {
+            List<ClusterNode> nodes = assignment.get(p);
+
+            if (!F.isEmpty(nodes))
+                res.add(nodes.get(0));
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Integer> primaryPartitions(UUID nodeId) {
+        Set<Integer> res = new HashSet<>();
+
+        for (int p = 0; p < assignment.size(); p++) {
+            List<ClusterNode> nodes = assignment.get(p);
+
+            if (!F.isEmpty(nodes) && nodes.get(0).id().equals(nodeId))
+                res.add(p);
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<Integer> backupPartitions(UUID nodeId) {
+        Set<Integer> res = new HashSet<>();
+
+        for (int p = 0; p < assignment.size(); p++) {
+            List<ClusterNode> nodes = assignment.get(p);
+
+            for (int i = 1; i < nodes.size(); i++) {
+                ClusterNode node = nodes.get(i);
+
+                if (node.id().equals(nodeId)) {
+                    res.add(p);
+
+                    break;
+                }
+            }
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return topVer.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("SimplifiableIfStatement")
+    @Override public boolean equals(Object o) {
+        if (o == this)
+            return true;
+
+        if (o == null || !(o instanceof AffinityAssignment))
+            return false;
+
+        return topVer.equals(((AffinityAssignment)o).topologyVersion());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HistoryAffinityAssignment.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 1aedf4e..2890887 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -127,7 +127,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param node Event node.
      * @param topVer Topology version.
      */
-    public void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
+    void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) {
         if (type == EVT_NODE_JOINED && node.isLocal()) {
             // Clean-up in case of client reconnect.
             registeredCaches.clear();
@@ -153,7 +153,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param msg Customer message.
      * @return {@code True} if minor topology version should be increased.
      */
-    public boolean onCustomEvent(CacheAffinityChangeMessage msg) {
+    boolean onCustomEvent(CacheAffinityChangeMessage msg) {
         assert lateAffAssign : msg;
 
         if (msg.exchangeId() != null) {
@@ -219,7 +219,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param top Topology.
      * @param checkCacheId Cache ID.
      */
-    public void checkRebalanceState(GridDhtPartitionTopology top, Integer checkCacheId) {
+    void checkRebalanceState(GridDhtPartitionTopology top, Integer checkCacheId) {
         if (!lateAffAssign)
             return;
 
@@ -508,6 +508,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         assert assignment != null;
 
+        final Map<Object, List<List<ClusterNode>>> affCache = new HashMap<>();
+
         forAllCaches(crd, new IgniteInClosureX<GridAffinityAssignmentCache>() {
             @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
                 List<List<ClusterNode>> idealAssignment = aff.idealAssignment();
@@ -527,7 +529,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 else
                     newAssignment = idealAssignment;
 
-                aff.initialize(topVer, newAssignment);
+                aff.initialize(topVer, cachedAssignment(aff, newAssignment, affCache));
             }
         });
     }
@@ -562,6 +564,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         final Map<Integer, IgniteUuid> deploymentIds = msg.cacheDeploymentIds();
 
+        final Map<Object, List<List<ClusterNode>>> affCache = new HashMap<>();
+
         forAllCaches(crd, new IgniteInClosureX<GridAffinityAssignmentCache>() {
             @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException {
                 AffinityTopologyVersion affTopVer = aff.lastVersion();
@@ -602,7 +606,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                         assignment.set(part, nodes);
                     }
 
-                    aff.initialize(topVer, assignment);
+                    aff.initialize(topVer, cachedAssignment(aff, assignment, affCache));
                 }
                 else
                     aff.clientEventTopologyChange(exchFut.discoveryEvent(), topVer);
@@ -1206,6 +1210,8 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         throws IgniteCheckedException {
         AffinityTopologyVersion topVer = fut.topologyVersion();
 
+        final Map<Object, List<List<ClusterNode>>> affCache = new HashMap<>();
+
         if (!crd) {
             for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
                 if (cacheCtx.isLocal())
@@ -1213,7 +1219,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
                 boolean latePrimary = cacheCtx.rebalanceEnabled();
 
-                initAffinityOnNodeJoin(fut, cacheCtx.affinity().affinityCache(), null, latePrimary);
+                initAffinityOnNodeJoin(fut, cacheCtx.affinity().affinityCache(), null, latePrimary, affCache);
             }
 
             return null;
@@ -1227,7 +1233,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
                     boolean latePrimary = cache.rebalanceEnabled;
 
-                    initAffinityOnNodeJoin(fut, cache.affinity(), waitRebalanceInfo, latePrimary);
+                    initAffinityOnNodeJoin(fut, cache.affinity(), waitRebalanceInfo, latePrimary, affCache);
                 }
             });
 
@@ -1240,12 +1246,14 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param aff Affinity.
      * @param rebalanceInfo Rebalance information.
      * @param latePrimary If {@code true} delays primary assignment if it is not owner.
+     * @param affCache Already calculated assignments (to reduce data stored in history).
      * @throws IgniteCheckedException If failed.
      */
     private void initAffinityOnNodeJoin(GridDhtPartitionsExchangeFuture fut,
         GridAffinityAssignmentCache aff,
         WaitRebalanceInfo rebalanceInfo,
-        boolean latePrimary)
+        boolean latePrimary,
+        Map<Object, List<List<ClusterNode>>> affCache)
         throws IgniteCheckedException
     {
         assert lateAffAssign;
@@ -1292,7 +1300,26 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         if (newAssignment == null)
             newAssignment = idealAssignment;
 
-        aff.initialize(fut.topologyVersion(), newAssignment);
+        aff.initialize(fut.topologyVersion(), cachedAssignment(aff, newAssignment, affCache));
+    }
+
+    /**
+     * @param aff Assignment cache.
+     * @param assign Assignment.
+     * @param affCache Assignments already calculated for other caches.
+     * @return Assignment.
+     */
+    private List<List<ClusterNode>> cachedAssignment(GridAffinityAssignmentCache aff,
+        List<List<ClusterNode>> assign,
+        Map<Object, List<List<ClusterNode>>> affCache) {
+        List<List<ClusterNode>> assign0 = affCache.get(aff.similarAffinityKey());
+
+        if (assign0 != null && assign0.equals(assign))
+            assign = assign0;
+        else
+            affCache.put(aff.similarAffinityKey(), assign);
+
+        return assign;
     }
 
     /**
@@ -1367,7 +1394,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @return Affinity assignment.
      * @throws IgniteCheckedException If failed.
      */
-    public Map<Integer, Map<Integer, List<UUID>>> initAffinityOnNodeLeft0(final GridDhtPartitionsExchangeFuture fut)
+    private Map<Integer, Map<Integer, List<UUID>>> initAffinityOnNodeLeft0(final GridDhtPartitionsExchangeFuture fut)
         throws IgniteCheckedException {
         final AffinityTopologyVersion topVer = fut.topologyVersion();
 
@@ -1554,7 +1581,9 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
          * @param aff Affinity cache.
          * @param initAff Existing affinity cache.
          */
-        public CacheHolder(boolean rebalanceEnabled, GridAffinityAssignmentCache aff, @Nullable GridAffinityAssignmentCache initAff) {
+        CacheHolder(boolean rebalanceEnabled,
+            GridAffinityAssignmentCache aff,
+            @Nullable GridAffinityAssignmentCache initAff) {
             this.aff = aff;
 
             if (initAff != null)
@@ -1606,7 +1635,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     /**
      * Created cache is started on coordinator.
      */
-    class CacheHolder1 extends CacheHolder {
+    private class CacheHolder1 extends CacheHolder {
         /** */
         private final GridCacheContext cctx;
 
@@ -1614,7 +1643,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
          * @param cctx Cache context.
          * @param initAff Current affinity.
          */
-        public CacheHolder1(GridCacheContext cctx, @Nullable GridAffinityAssignmentCache initAff) {
+        CacheHolder1(GridCacheContext cctx, @Nullable GridAffinityAssignmentCache initAff) {
             super(cctx.rebalanceEnabled(), cctx.affinity().affinityCache(), initAff);
 
             assert !cctx.isLocal() : cctx.name();
@@ -1651,7 +1680,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
     /**
      * Created if cache is not started on coordinator.
      */
-    static class CacheHolder2 extends CacheHolder {
+    private static class CacheHolder2 extends CacheHolder {
         /** */
         private final GridCacheSharedContext cctx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index e10e5aa..4dcff9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -62,6 +62,13 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     }
 
     /**
+     * @param id Message ID.
+     */
+    public void id(IgniteUuid id) {
+        this.id = id;
+    }
+
+    /**
      * @return Collection of change requests.
      */
     public Collection<DynamicCacheChangeRequest> requests() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index 71ae5c9..c6e7ee6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -25,8 +25,8 @@ import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -48,7 +48,7 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
     private static final AffinityTopologyVersion LOC_CACHE_TOP_VER = new AffinityTopologyVersion(1);
 
     /** */
-    public static final String FAILED_TO_FIND_CACHE_ERR_MSG = "Failed to find cache (cache was not started " +
+    private static final String FAILED_TO_FIND_CACHE_ERR_MSG = "Failed to find cache (cache was not started " +
         "yet or cache was already stopped): ";
 
     /** Affinity cached function. */
@@ -265,7 +265,7 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @param topVer Topology version.
      * @return Affinity assignment.
      */
-    public GridAffinityAssignment assignment(AffinityTopologyVersion topVer) {
+    public AffinityAssignment assignment(AffinityTopologyVersion topVer) {
         if (cctx.isLocal())
             topVer = LOC_CACHE_TOP_VER;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/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 00d2d16..503b334 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
@@ -44,7 +44,9 @@ import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
@@ -56,6 +58,7 @@ import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
@@ -64,6 +67,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsAbstractMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
@@ -71,6 +75,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridListSet;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -80,6 +85,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.GPC;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -531,8 +537,23 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (top != null)
             return top;
 
+        Object affKey = null;
+
+        DynamicCacheDescriptor desc = cctx.cache().cacheDescriptor(cacheId);
+
+        if (desc != null) {
+            CacheConfiguration ccfg = desc.cacheConfiguration();
+
+            AffinityFunction aff = ccfg.getAffinity();
+
+            affKey = cctx.kernalContext().affinity().similaryAffinityKey(aff,
+                ccfg.getNodeFilter(),
+                ccfg.getBackups(),
+                aff.partitions());
+        }
+
         GridClientPartitionTopology old = clientTops.putIfAbsent(cacheId,
-            top = new GridClientPartitionTopology(cctx, cacheId, exchFut));
+            top = new GridClientPartitionTopology(cctx, cacheId, exchFut, affKey));
 
         return old != null ? old : top;
     }
@@ -761,40 +782,16 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @param nodes Nodes.
      * @return {@code True} if message was sent, {@code false} if node left grid.
      */
-    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes) {
-        GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(null, null, AffinityTopologyVersion.NONE);
-
-        boolean useOldApi = false;
-
-        for (ClusterNode node : nodes) {
-            if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0)
-                useOldApi = true;
-        }
-
-        for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-            if (!cacheCtx.isLocal() && cacheCtx.started()) {
-                GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
-
-                if (useOldApi) {
-                    locMap = new GridDhtPartitionFullMap(locMap.nodeId(),
-                        locMap.nodeOrder(),
-                        locMap.updateSequence(),
-                        locMap);
-                }
-
-                m.addFullPartitionsMap(cacheCtx.cacheId(), locMap);
-            }
-        }
-
-        // It is important that client topologies be added after contexts.
-        for (GridClientPartitionTopology top : cctx.exchange().clientTopologies())
-            m.addFullPartitionsMap(top.cacheId(), top.partitionMap(true));
+    private boolean sendAllPartitions(Collection<ClusterNode> nodes) {
+        GridDhtPartitionsFullMessage m = createPartitionsFullMessage(nodes, null, null, true);
 
         if (log.isDebugEnabled())
             log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
 
         for (ClusterNode node : nodes) {
             try {
+                assert !node.equals(cctx.localNode());
+
                 cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
             }
             catch (ClusterTopologyCheckedException ignore) {
@@ -811,31 +808,140 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * @param node Node.
-     * @param id ID.
+     * @param nodes Target nodes.
+     * @param exchId Non-null exchange ID if message is created for exchange.
+     * @param lastVer Last version.
+     * @param compress {@code True} if it is possible to use compression for message.
+     * @return Message.
      */
-    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) {
-        GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
-            cctx.kernalContext().clientNode(),
-            cctx.versions().last());
+    public GridDhtPartitionsFullMessage createPartitionsFullMessage(Collection<ClusterNode> nodes,
+        @Nullable GridDhtPartitionExchangeId exchId,
+        @Nullable GridCacheVersion lastVer,
+        boolean compress) {
+        GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(exchId,
+                lastVer,
+                exchId != null ? exchId.topologyVersion() : AffinityTopologyVersion.NONE);
+
+        boolean useOldApi = false;
+
+        if (nodes != null) {
+            for (ClusterNode node : nodes) {
+                if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0) {
+                    useOldApi = true;
+                    compress = false;
+
+                    break;
+                }
+                else if (node.version().compareToIgnoreTimestamp(GridDhtPartitionsAbstractMessage.PART_MAP_COMPRESS_SINCE) < 0)
+                    compress = false;
+            }
+        }
+
+        m.compress(compress);
+
+        Map<Object, T2<Integer, GridDhtPartitionFullMap>> dupData = new HashMap<>();
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
             if (!cacheCtx.isLocal()) {
-                GridDhtPartitionMap2 locMap = cacheCtx.topology().localPartitionMap();
+                boolean ready;
 
-                if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0)
-                    locMap = new GridDhtPartitionMap(locMap.nodeId(), locMap.updateSequence(), locMap.map());
+                if (exchId != null) {
+                    AffinityTopologyVersion startTopVer = cacheCtx.startTopologyVersion();
+
+                    ready = startTopVer == null || startTopVer.compareTo(exchId.topologyVersion()) <= 0;
+                }
+                else
+                    ready = cacheCtx.started();
+
+                if (ready) {
+                    GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
 
-                m.addLocalPartitionMap(cacheCtx.cacheId(), locMap);
+                    if (useOldApi) {
+                        locMap = new GridDhtPartitionFullMap(locMap.nodeId(),
+                            locMap.nodeOrder(),
+                            locMap.updateSequence(),
+                            locMap);
+                    }
+
+                    addFullPartitionsMap(m,
+                        dupData,
+                        compress,
+                        cacheCtx.cacheId(),
+                        locMap,
+                        cacheCtx.affinity().affinityCache().similarAffinityKey());
+
+                    if (exchId != null)
+                        m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true));
+                }
             }
         }
 
-        for (GridClientPartitionTopology top : clientTops.values()) {
-            GridDhtPartitionMap2 locMap = top.localPartitionMap();
+        // It is important that client topologies be added after contexts.
+        for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) {
+            GridDhtPartitionFullMap map = top.partitionMap(true);
+
+            addFullPartitionsMap(m,
+                dupData,
+                compress,
+                top.cacheId(),
+                map,
+                top.similarAffinityKey());
+
+            if (exchId != null)
+                m.addPartitionUpdateCounters(top.cacheId(), top.updateCounters(true));
+        }
+
+        return m;
+    }
 
-            m.addLocalPartitionMap(top.cacheId(), locMap);
+    /**
+     * @param m Message.
+     * @param dupData Duplicated data map.
+     * @param compress {@code True} if need check for duplicated partition state data.
+     * @param cacheId Cache ID.
+     * @param map Map to add.
+     * @param affKey Cache affinity key.
+     */
+    private void addFullPartitionsMap(GridDhtPartitionsFullMessage m,
+        Map<Object, T2<Integer, GridDhtPartitionFullMap>> dupData,
+        boolean compress,
+        Integer cacheId,
+        GridDhtPartitionFullMap map,
+        Object affKey) {
+        Integer dupDataCache = null;
+
+        if (compress && affKey != null && !m.containsCache(cacheId)) {
+            T2<Integer, GridDhtPartitionFullMap> state0 = dupData.get(affKey);
+
+            if (state0 != null && state0.get2().partitionStateEquals(map)) {
+                GridDhtPartitionFullMap map0 = new GridDhtPartitionFullMap(map.nodeId(),
+                    map.nodeOrder(),
+                    map.updateSequence());
+
+                for (Map.Entry<UUID, GridDhtPartitionMap2> e : map.entrySet())
+                    map0.put(e.getKey(), e.getValue().emptyCopy());
+
+                map = map0;
+
+                dupDataCache = state0.get1();
+            }
+            else
+                dupData.put(affKey, new T2<>(cacheId, map));
         }
 
+        m.addFullPartitionsMap(cacheId, map, dupDataCache);
+    }
+
+    /**
+     * @param node Node.
+     * @param id ID.
+     */
+    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) {
+        GridDhtPartitionsSingleMessage m = createPartitionsSingleMessage(node,
+            id,
+            cctx.kernalContext().clientNode(),
+            false);
+
         if (log.isDebugEnabled())
             log.debug("Sending local partitions [nodeId=" + node.id() + ", msg=" + m + ']');
 
@@ -853,6 +959,98 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
+     * @param targetNode Target node.
+     * @param exchangeId ID.
+     * @param clientOnlyExchange Client exchange flag.
+     * @param sndCounters {@code True} if need send partition update counters.
+     * @return Message.
+     */
+    public GridDhtPartitionsSingleMessage createPartitionsSingleMessage(ClusterNode targetNode,
+        @Nullable GridDhtPartitionExchangeId exchangeId,
+        boolean clientOnlyExchange,
+        boolean sndCounters)
+    {
+        boolean compress =
+            targetNode.version().compareToIgnoreTimestamp(GridDhtPartitionsSingleMessage.PART_MAP_COMPRESS_SINCE) >= 0;
+
+        GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(exchangeId,
+            clientOnlyExchange,
+            cctx.versions().last(),
+            compress);
+
+        Map<Object, T2<Integer,Map<Integer, GridDhtPartitionState>>> dupData = new HashMap<>();
+
+        for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+            if (!cacheCtx.isLocal()) {
+                GridDhtPartitionMap2 locMap = cacheCtx.topology().localPartitionMap();
+
+                if (targetNode.version().compareTo(GridDhtPartitionMap2.SINCE) < 0)
+                    locMap = new GridDhtPartitionMap(locMap.nodeId(), locMap.updateSequence(), locMap.map());
+
+                addPartitionMap(m,
+                    dupData,
+                    compress,
+                    cacheCtx.cacheId(),
+                    locMap,
+                    cacheCtx.affinity().affinityCache().similarAffinityKey());
+
+                if (sndCounters)
+                    m.partitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true));
+            }
+        }
+
+        for (GridClientPartitionTopology top : clientTops.values()) {
+            if (m.partitions() != null && m.partitions().containsKey(top.cacheId()))
+                continue;
+
+            GridDhtPartitionMap2 locMap = top.localPartitionMap();
+
+            addPartitionMap(m,
+                dupData,
+                compress,
+                top.cacheId(),
+                locMap,
+                top.similarAffinityKey());
+
+            if (sndCounters)
+                m.partitionUpdateCounters(top.cacheId(), top.updateCounters(true));
+        }
+
+        return m;
+    }
+
+    /**
+     * @param m Message.
+     * @param dupData Duplicated data map.
+     * @param compress {@code True} if need check for duplicated partition state data.
+     * @param cacheId Cache ID.
+     * @param map Map to add.
+     * @param affKey Cache affinity key.
+     */
+    private void addPartitionMap(GridDhtPartitionsSingleMessage m,
+        Map<Object, T2<Integer, Map<Integer, GridDhtPartitionState>>> dupData,
+        boolean compress,
+        Integer cacheId,
+        GridDhtPartitionMap2 map,
+        Object affKey) {
+        Integer dupDataCache = null;
+
+        if (compress) {
+            T2<Integer, Map<Integer, GridDhtPartitionState>> state0 = dupData.get(affKey);
+
+            if (state0 != null && state0.get2().equals(map.map())) {
+                dupDataCache = state0.get1();
+
+                map = map.emptyCopy();
+            }
+            else
+                dupData.put(affKey, new T2<>(cacheId, map.map()));
+        }
+
+        m.addLocalPartitionMap(cacheId, map, dupDataCache);
+    }
+
+    /**
      * @param nodeId Cause node ID.
      * @param topVer Topology version.
      * @param evt Event type.
@@ -869,7 +1067,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @param affChangeMsg Affinity change message.
      * @return Exchange future.
      */
-    GridDhtPartitionsExchangeFuture exchangeFuture(GridDhtPartitionExchangeId exchId,
+    private GridDhtPartitionsExchangeFuture exchangeFuture(GridDhtPartitionExchangeId exchId,
         @Nullable DiscoveryEvent discoEvt,
         @Nullable Collection<DynamicCacheChangeRequest> reqs,
         @Nullable CacheAffinityChangeMessage affChangeMsg) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/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 fd6abbd..5e777fd 100755
--- 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
@@ -1958,8 +1958,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 req.template(true);
 
-                req.deploymentId(desc.deploymentId());
-
                 reqs.add(req);
             }
 
@@ -1972,6 +1970,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         batch.clientReconnect(reconnect);
 
+        // Reset random batch ID so that serialized batches with the same descriptors will be exactly the same.
+        batch.id(null);
+
         return batch;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 58933b7..5efb317 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -61,6 +61,9 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     /** Flag to control amount of output for full map. */
     private static final boolean FULL_MAP_DEBUG = false;
 
+    /** */
+    private static final Long ZERO = 0L;
+
     /** Cache shared context. */
     private GridCacheSharedContext cctx;
 
@@ -97,18 +100,24 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     /** Partition update counters. */
     private Map<Integer, Long> cntrMap = new HashMap<>();
 
+    /** */
+    private final Object similarAffKey;
+
     /**
      * @param cctx Context.
      * @param cacheId Cache ID.
      * @param exchFut Exchange ID.
+     * @param similarAffKey Key to find caches with similar affinity.
      */
     public GridClientPartitionTopology(
         GridCacheSharedContext cctx,
         int cacheId,
-        GridDhtPartitionsExchangeFuture exchFut
+        GridDhtPartitionsExchangeFuture exchFut,
+        Object similarAffKey
     ) {
         this.cctx = cctx;
         this.cacheId = cacheId;
+        this.similarAffKey = similarAffKey;
 
         topVer = exchFut.topologyVersion();
 
@@ -125,6 +134,13 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /**
+     * @return Key to find caches with similar affinity.
+     */
+    @Nullable public Object similarAffinityKey() {
+        return similarAffKey;
+    }
+
+    /**
      * @return Full map string representation.
      */
     @SuppressWarnings( {"ConstantConditions"})
@@ -873,11 +889,22 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public Map<Integer, Long> updateCounters() {
+    @Override public Map<Integer, Long> updateCounters(boolean skipZeros) {
         lock.readLock().lock();
 
         try {
-            return new HashMap<>(cntrMap);
+            if (skipZeros) {
+                Map<Integer, Long> res = U.newHashMap(cntrMap.size());
+
+                for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
+                    if (!e.getValue().equals(ZERO))
+                        res.put(e.getKey(), e.getValue());
+                }
+
+                return res;
+            }
+            else
+                return new HashMap<>(cntrMap);
         }
         finally {
             lock.readLock().unlock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index 6e9b907..4ae4e47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -220,9 +220,10 @@ public interface GridDhtPartitionTopology {
         @Nullable Map<Integer, Long> cntrMap);
 
     /**
+     * @param skipZeros If {@code true} then filters out zero counters.
      * @return Partition update counters.
      */
-    public Map<Integer, Long> updateCounters();
+    public Map<Integer, Long> updateCounters(boolean skipZeros);
 
     /**
      * @param part Partition to own.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 50f7f0f..f3751ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -35,8 +35,8 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
@@ -71,6 +71,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     /** Flag to control amount of output for full map. */
     private static final boolean FULL_MAP_DEBUG = false;
 
+    /** */
+    private static final Long ZERO = 0L;
+
     /** Context. */
     private final GridCacheContext<?, ?> cctx;
 
@@ -859,7 +862,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public List<ClusterNode> nodes(int p, AffinityTopologyVersion topVer) {
-        GridAffinityAssignment affAssignment = cctx.affinity().assignment(topVer);
+        AffinityAssignment affAssignment = cctx.affinity().assignment(topVer);
 
         List<ClusterNode> affNodes = affAssignment.get(p);
 
@@ -1500,11 +1503,26 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public Map<Integer, Long> updateCounters() {
+    @Override public Map<Integer, Long> updateCounters(boolean skipZeros) {
         lock.readLock().lock();
 
         try {
-            Map<Integer, Long> res = new HashMap<>(cntrMap);
+            Map<Integer, Long> res;
+
+            if (skipZeros) {
+                res = U.newHashMap(cntrMap.size());
+
+                for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
+                    Long cntr = e.getValue();
+
+                    if (ZERO.equals(cntr))
+                        continue;
+
+                    res.put(e.getKey(), cntr);
+                }
+            }
+            else
+                res = new HashMap<>(cntrMap);
 
             for (int i = 0; i < locParts.length; i++) {
                 GridDhtLocalPartition part = locParts[i];
@@ -1513,7 +1531,10 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     continue;
 
                 Long cntr0 = res.get(part.id());
-                Long cntr1 = part.updateCounter();
+                long cntr1 = part.updateCounter();
+
+                if (skipZeros && cntr1 == 0L)
+                    continue;
 
                 if (cntr0 == null || cntr1 > cntr0)
                     res.put(part.id(), cntr1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
index 498d492..8f5ad17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
@@ -158,6 +158,24 @@ public class GridDhtPartitionFullMap extends HashMap<UUID, GridDhtPartitionMap2>
     }
 
     /**
+     * @param fullMap Map.
+     * @return {@code True} if this map and given map contain the same data.
+     */
+    public boolean partitionStateEquals(GridDhtPartitionFullMap fullMap) {
+        if (size() != fullMap.size())
+            return false;
+
+        for (Map.Entry<UUID, GridDhtPartitionMap2> e : entrySet()) {
+            GridDhtPartitionMap2 m = fullMap.get(e.getKey());
+
+            if (m == null || !m.map().equals(e.getValue().map()))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
      * @param updateSeq New update sequence value.
      * @return Old update sequence value.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java
index 15b5a2e..ce36a11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java
@@ -61,27 +61,24 @@ public class GridDhtPartitionMap2 implements Comparable<GridDhtPartitionMap2>, E
     private volatile int moving;
 
     /**
-     * @param nodeId Node ID.
-     * @param updateSeq Update sequence number.
+     * Empty constructor required for {@link Externalizable}.
      */
-    public GridDhtPartitionMap2(UUID nodeId, long updateSeq) {
-        assert nodeId != null;
-        assert updateSeq > 0;
-
-        this.nodeId = nodeId;
-        this.updateSeq = updateSeq;
-
-        map = new HashMap<>();
+    public GridDhtPartitionMap2() {
+        // No-op.
     }
 
     /**
      * @param nodeId Node ID.
      * @param updateSeq Update sequence number.
+     * @param top Topology version.
      * @param m Map to copy.
      * @param onlyActive If {@code true}, then only active states will be included.
      */
-    public GridDhtPartitionMap2(UUID nodeId, long updateSeq, AffinityTopologyVersion top,
-        Map<Integer, GridDhtPartitionState> m, boolean onlyActive) {
+    public GridDhtPartitionMap2(UUID nodeId,
+        long updateSeq,
+        AffinityTopologyVersion top,
+        Map<Integer, GridDhtPartitionState> m,
+        boolean onlyActive) {
         assert nodeId != null;
         assert updateSeq > 0;
 
@@ -100,10 +97,33 @@ public class GridDhtPartitionMap2 implements Comparable<GridDhtPartitionMap2>, E
     }
 
     /**
-     * Empty constructor required for {@link Externalizable}.
+     * @param nodeId Node ID.
+     * @param updateSeq Update sequence number.
+     * @param top Topology version.
+     * @param map Map.
+     * @param moving Number of moving partitions.
      */
-    public GridDhtPartitionMap2() {
-        // No-op.
+    private GridDhtPartitionMap2(UUID nodeId,
+        long updateSeq,
+        AffinityTopologyVersion top,
+        Map<Integer, GridDhtPartitionState> map,
+        int moving) {
+        this.nodeId = nodeId;
+        this.updateSeq = updateSeq;
+        this.top = top;
+        this.map = map;
+        this.moving = moving;
+    }
+
+    /**
+     * @return Copy with empty partition state map.
+     */
+    public GridDhtPartitionMap2 emptyCopy() {
+        return new GridDhtPartitionMap2(nodeId,
+            updateSeq,
+            top,
+            U.<Integer, GridDhtPartitionState>newHashMap(0),
+            0);
     }
 
     /**
@@ -277,9 +297,8 @@ public class GridDhtPartitionMap2 implements Comparable<GridDhtPartitionMap2>, E
         long ver = in.readLong();
         int minorVer = in.readInt();
 
-        if (ver != 0) {
+        if (ver != 0)
             top = new AffinityTopologyVersion(ver, minorVer);
-        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
index 4e714ed..6e69161 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
@@ -29,7 +30,13 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Request for single partition info.
  */
-abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage {
+public abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage {
+    /** */
+    public static final IgniteProductVersion PART_MAP_COMPRESS_SINCE = IgniteProductVersion.fromString("1.6.11");
+
+    /** */
+    protected static final byte COMPRESSED_FLAG_MASK = 1;
+
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -39,6 +46,9 @@ abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage {
     /** Last used cache version. */
     private GridCacheVersion lastVer;
 
+    /** */
+    private byte flags;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -79,6 +89,20 @@ abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage {
         return lastVer;
     }
 
+    /**
+     * @return {@code True} if message data is compressed.
+     */
+    protected final boolean compressed() {
+        return (flags & COMPRESSED_FLAG_MASK) != 0;
+    }
+
+    /**
+     * @param compressed {@code True} if message data is compressed.
+     */
+    protected final void compressed(boolean compressed) {
+        flags = compressed ? (byte)(flags | COMPRESSED_FLAG_MASK) : (byte)(flags & ~COMPRESSED_FLAG_MASK);
+    }
+
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
@@ -101,6 +125,12 @@ abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage {
                 writer.incrementState();
 
             case 4:
+                if (!writer.writeByte("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
                 if (!writer.writeMessage("lastVer", lastVer))
                     return false;
 
@@ -131,6 +161,14 @@ abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage {
                 reader.incrementState();
 
             case 4:
+                flags = reader.readByte("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
                 lastVer = reader.readMessage("lastVer");
 
                 if (!reader.isLastRead())

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 80b3768..f391265 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -544,7 +546,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     exchId.topologyVersion().equals(cacheCtx.startTopologyVersion());
 
                 if (updateTop && clientTop != null)
-                    cacheCtx.topology().update(exchId, clientTop.partitionMap(true), clientTop.updateCounters());
+                    cacheCtx.topology().update(exchId, clientTop.partitionMap(true), clientTop.updateCounters(false));
             }
 
             top.updateTopologyVersion(exchId, this, updSeq, stopping(cacheCtx.cacheId()));
@@ -668,7 +670,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                             if (top.cacheId() == cacheCtx.cacheId()) {
                                 cacheCtx.topology().update(exchId,
                                     top.partitionMap(true),
-                                    top.updateCounters());
+                                    top.updateCounters(false));
 
                                 break;
                             }
@@ -678,7 +680,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             }
             else {
                 if (!centralizedAff)
-                    sendLocalPartitions(crd, exchId);
+                    sendLocalPartitions(crd);
 
                 initDone();
 
@@ -928,27 +930,14 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
     /**
      * @param node Node.
-     * @param id ID.
      * @throws IgniteCheckedException If failed.
      */
-    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id)
+    private void sendLocalPartitions(ClusterNode node)
         throws IgniteCheckedException {
-        GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
+        GridDhtPartitionsSingleMessage m = cctx.exchange().createPartitionsSingleMessage(node,
+            exchangeId(),
             clientOnlyExchange,
-            cctx.versions().last());
-
-        for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-            if (!cacheCtx.isLocal()) {
-                GridDhtPartitionMap2 locMap = cacheCtx.topology().localPartitionMap();
-
-                if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0)
-                    locMap = new GridDhtPartitionMap(locMap.nodeId(), locMap.updateSequence(), locMap.map());
-
-                m.addLocalPartitionMap(cacheCtx.cacheId(), locMap);
-
-                m.partitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters());
-            }
-        }
+            true);
 
         if (log.isDebugEnabled())
             log.debug("Sending local partitions [nodeId=" + node.id() + ", exchId=" + exchId + ", msg=" + m + ']');
@@ -964,51 +953,16 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
     /**
      * @param nodes Target nodes.
-     * @return Message;
+     * @param compress {@code True} if it is possible to use compression for message.
+     * @return Message.
      */
-    private GridDhtPartitionsFullMessage createPartitionsMessage(Collection<ClusterNode> nodes) {
+    private GridDhtPartitionsFullMessage createPartitionsMessage(Collection<ClusterNode> nodes, boolean compress) {
         GridCacheVersion last = lastVer.get();
 
-        GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(exchangeId(),
+        return cctx.exchange().createPartitionsFullMessage(nodes,
+            exchangeId(),
             last != null ? last : cctx.versions().last(),
-            topologyVersion());
-
-        boolean useOldApi = false;
-
-        if (nodes != null) {
-            for (ClusterNode node : nodes) {
-                if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0)
-                    useOldApi = true;
-            }
-        }
-
-        for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-            if (!cacheCtx.isLocal()) {
-                AffinityTopologyVersion startTopVer = cacheCtx.startTopologyVersion();
-
-                boolean ready = startTopVer == null || startTopVer.compareTo(topologyVersion()) <= 0;
-
-                if (ready) {
-                    GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
-
-                    if (useOldApi)
-                        locMap = new GridDhtPartitionFullMap(locMap.nodeId(), locMap.nodeOrder(), locMap.updateSequence(), locMap);
-
-                    m.addFullPartitionsMap(cacheCtx.cacheId(), locMap);
-
-                    m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters());
-                }
-            }
-        }
-
-        // It is important that client topologies be added after contexts.
-        for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) {
-            m.addFullPartitionsMap(top.cacheId(), top.partitionMap(true));
-
-            m.addPartitionUpdateCounters(top.cacheId(), top.updateCounters());
-        }
-
-        return m;
+            compress);
     }
 
     /**
@@ -1016,7 +970,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @throws IgniteCheckedException If failed.
      */
     private void sendAllPartitions(Collection<ClusterNode> nodes) throws IgniteCheckedException {
-        GridDhtPartitionsFullMessage m = createPartitionsMessage(nodes);
+        GridDhtPartitionsFullMessage m = createPartitionsMessage(nodes, true);
+
+        assert !nodes.contains(cctx.localNode());
 
         if (log.isDebugEnabled())
             log.debug("Sending full partition map [nodeIds=" + F.viewReadOnly(nodes, F.node2id()) +
@@ -1030,7 +986,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      */
     private void sendPartitions(ClusterNode oldestNode) {
         try {
-            sendLocalPartitions(oldestNode, exchId);
+            sendLocalPartitions(oldestNode);
         }
         catch (ClusterTopologyCheckedException ignore) {
             if (log.isDebugEnabled())
@@ -1234,7 +1190,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
             Map<Integer, Map<Integer, List<UUID>>> assignmentChange = fut.get();
 
-            GridDhtPartitionsFullMessage m = createPartitionsMessage(null);
+            GridDhtPartitionsFullMessage m = createPartitionsMessage(null, false);
 
             CacheAffinityChangeMessage msg = new CacheAffinityChangeMessage(exchId, m, assignmentChange);
 


[05/17] ignite git commit: IGNITE-500 CacheLoadingConcurrentGridStartSelfTest fails (DataStreamer data loss at unstable topology in !allowOverwrite mode fixed)

Posted by sb...@apache.org.
IGNITE-500 CacheLoadingConcurrentGridStartSelfTest fails (DataStreamer data loss at unstable topology in !allowOverwrite mode fixed)


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

Branch: refs/heads/ignite-4154-opt2
Commit: b7499828c928e02e8e554f960f3754e4d08bfbe0
Parents: 8b59f4e
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Nov 10 16:10:21 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Nov 10 16:10:21 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteDataStreamer.java   |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   5 +-
 .../processors/cache/GridCacheMvccManager.java  |  77 +++
 .../GridCachePartitionExchangeManager.java      |   5 +
 .../cache/GridCacheSharedContext.java           |   1 +
 .../datastreamer/DataStreamProcessor.java       | 104 +++-
 .../datastreamer/DataStreamerImpl.java          | 603 ++++++++++++++-----
 .../ignite/internal/util/GridLogThrottle.java   |  29 +-
 .../cache/IgniteCacheDynamicStopSelfTest.java   |  48 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 251 +++++++-
 ...ncurrentGridStartSelfTestAllowOverwrite.java |  30 +
 .../DataStreamProcessorSelfTest.java            |   4 +-
 .../datastreamer/DataStreamerImplSelfTest.java  | 170 ++++--
 .../DataStreamerMultiThreadedSelfTest.java      |   2 -
 .../datastreamer/DataStreamerTimeoutTest.java   |  92 ++-
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 16 files changed, 1120 insertions(+), 305 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
index 484fee9..4e00d66 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java
@@ -291,7 +291,7 @@ public interface IgniteDataStreamer<K, V> extends AutoCloseable {
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on streamer.
      */
-    public IgniteFuture<?> removeData(K key)  throws CacheException, IgniteInterruptedException, IllegalStateException;
+    public IgniteFuture<?> removeData(K key) throws CacheException, IgniteInterruptedException, IllegalStateException;
 
     /**
      * Adds data for streaming on remote node. This method can be called from multiple

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 5996672..950153f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -3444,11 +3444,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if (val == null) {
                     skipQryNtf = true;
 
-                    if (cctx.deferredDelete() && !isInternal()) {
-                        assert !deletedUnlocked();
-
+                    if (cctx.deferredDelete() && !deletedUnlocked() && !isInternal())
                         deletedUnlocked(true);
-                    }
                 }
                 else if (deletedUnlocked())
                     deletedUnlocked(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/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 c4db01e..c57e17c 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
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.GridConcurrentFactory;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -108,6 +109,9 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     private final ConcurrentMap<GridCacheVersion, GridCacheAtomicFuture<?>> atomicFuts =
         new ConcurrentHashMap8<>();
 
+    /** Pending data streamer futures. */
+    private final GridConcurrentHashSet<DataStreamerFuture> dataStreamerFuts = new GridConcurrentHashSet<>();
+
     /** */
     private final ConcurrentMap<IgniteUuid, GridCacheFuture<?>> futs = new ConcurrentHashMap8<>();
 
@@ -446,6 +450,13 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @return Collection of pending data streamer futures.
+     */
+    public Collection<DataStreamerFuture> dataStreamerFutures() {
+        return dataStreamerFuts;
+    }
+
+    /**
      * Gets future by given future ID.
      *
      * @param futVer Future ID.
@@ -476,6 +487,21 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @param topVer Topology version.
+     */
+    public GridFutureAdapter addDataStreamerFuture(AffinityTopologyVersion topVer) {
+        final DataStreamerFuture fut = new DataStreamerFuture(topVer);
+
+        boolean add = dataStreamerFuts.add(fut);
+
+        assert add;
+
+        return fut;
+    }
+
+    /**
+
+    /**
      * Adds future.
      *
      * @param fut Future.
@@ -1056,6 +1082,22 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     *
+     * @return Finish update future.
+     */
+    @SuppressWarnings("unchecked")
+    public IgniteInternalFuture<?> finishDataStreamerUpdates() {
+        GridCompoundFuture<Object, Object> res = new GridCompoundFuture<>();
+
+        for (IgniteInternalFuture fut : dataStreamerFuts)
+            res.add(fut);
+
+        res.markInitialized();
+
+        return res;
+    }
+
+    /**
      * @param keys Key for which locks should be released.
      * @param cacheId Cache ID.
      * @param topVer Topology version.
@@ -1294,4 +1336,39 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                 CachePartialUpdateCheckedException.class.isAssignableFrom(cls);
         }
     }
+
+    /**
+     *
+     */
+    private class DataStreamerFuture extends GridFutureAdapter<Void> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Topology version. Instance field for toString method only. */
+        @GridToStringInclude
+        private final AffinityTopologyVersion topVer;
+
+        /**
+         * @param topVer Topology version.
+         */
+        DataStreamerFuture(AffinityTopologyVersion topVer) {
+            this.topVer = topVer;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) {
+            if (super.onDone(res, err)) {
+                dataStreamerFuts.remove(this);
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(DataStreamerFuture.class, this, super.toString());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/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 a901e2a..00d2d16 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
@@ -1309,6 +1309,11 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             for (GridCacheFuture<?> fut : mvcc.atomicFutures())
                 U.warn(log, ">>> " + fut);
 
+            U.warn(log, "Pending data streamer futures:");
+
+            for (IgniteInternalFuture<?> fut : mvcc.dataStreamerFutures())
+                U.warn(log, ">>> " + fut);
+
             if (tm != null) {
                 U.warn(log, "Pending transaction deadlock detection futures:");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 8f39235..117a5c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -636,6 +636,7 @@ public class GridCacheSharedContext<K, V> {
         f.add(mvcc().finishExplicitLocks(topVer));
         f.add(tm().finishTxs(topVer));
         f.add(mvcc().finishAtomicUpdates(topVer));
+        f.add(mvcc().finishDataStreamerUpdates());
 
         f.markInitialized();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
index 7663735..32fda87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.DelayQueue;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
@@ -29,13 +30,18 @@ import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.stream.StreamReceiver;
 import org.apache.ignite.thread.IgniteThread;
@@ -288,32 +294,94 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
                 return;
             }
 
-            Collection<DataStreamerEntry> col = req.entries();
+            localUpdate(nodeId, req, updater, topic);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
 
-            DataStreamerUpdateJob job = new DataStreamerUpdateJob(ctx,
-                log,
-                req.cacheName(),
-                col,
-                req.ignoreDeploymentOwnership(),
-                req.skipStore(),
-                req.keepBinary(),
-                updater);
+    /**
+     * @param nodeId Node id.
+     * @param req Request.
+     * @param updater Updater.
+     * @param topic Topic.
+     */
+    private void localUpdate(final UUID nodeId,
+        final DataStreamerRequest req,
+        final StreamReceiver<K, V> updater,
+        final Object topic) {
+        final boolean allowOverwrite = !(updater instanceof DataStreamerImpl.IsolatedUpdater);
 
-            Exception err = null;
+        try {
+            GridCacheAdapter cache = ctx.cache().internalCache(req.cacheName());
+
+            if (cache == null)
+                throw new IgniteCheckedException("Cache not created or already destroyed.");
+
+            GridCacheContext cctx = cache.context();
+
+            DataStreamerUpdateJob job = null;
+
+            GridFutureAdapter waitFut = null;
+
+            if (!allowOverwrite)
+                cctx.topology().readLock();
 
             try {
-                job.call();
-            }
-            catch (Exception e) {
-                U.error(log, "Failed to finish update job.", e);
+                GridDhtTopologyFuture fut = cctx.topologyVersionFuture();
 
-                err = e;
+                AffinityTopologyVersion topVer = fut.topologyVersion();
+
+                if (!allowOverwrite && !topVer.equals(req.topologyVersion())) {
+                    Exception err = new IgniteCheckedException(
+                        "DataStreamer will retry data transfer at stable topology " +
+                            "[reqTop=" + req.topologyVersion() + ", topVer=" + topVer + ", node=remote]");
+
+                    sendResponse(nodeId, topic, req.requestId(), err, req.forceLocalDeployment());
+                }
+                else if (allowOverwrite || fut.isDone()) {
+                    job = new DataStreamerUpdateJob(ctx,
+                        log,
+                        req.cacheName(),
+                        req.entries(),
+                        req.ignoreDeploymentOwnership(),
+                        req.skipStore(),
+                        req.keepBinary(),
+                        updater);
+
+                    waitFut = allowOverwrite ? null : cctx.mvcc().addDataStreamerFuture(topVer);
+                }
+                else {
+                    fut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
+                            localUpdate(nodeId, req, updater, topic);
+                        }
+                    });
+                }
+            }
+            finally {
+                if (!allowOverwrite)
+                    cctx.topology().readUnlock();
             }
 
-            sendResponse(nodeId, topic, req.requestId(), err, req.forceLocalDeployment());
+            if (job != null) {
+                try {
+                    job.call();
+
+                    sendResponse(nodeId, topic, req.requestId(), null, req.forceLocalDeployment());
+                }
+                finally {
+                    if (waitFut != null)
+                        waitFut.onDone();
+                }
+            }
         }
-        finally {
-            busyLock.leaveBusy();
+        catch (Throwable e) {
+            sendResponse(nodeId, topic, req.requestId(), e, req.forceLocalDeployment());
+
+            if (e instanceof Error)
+                throw (Error)e;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index a6065dd..443783b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.DelayQueue;
 import java.util.concurrent.Delayed;
@@ -39,15 +40,15 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.CacheException;
 import javax.cache.expiry.ExpiryPolicy;
-
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteDataStreamerTimeoutException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.IgniteDataStreamerTimeoutException;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -72,11 +73,13 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
+import org.apache.ignite.internal.processors.cache.GridCacheGateway;
 import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFutureImpl;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.dr.GridDrType;
@@ -92,6 +95,8 @@ import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.GPC;
+import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteClosure;
@@ -102,6 +107,7 @@ import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.stream.StreamReceiver;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
+import org.jsr166.LongAdder8;
 
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -113,12 +119,15 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUB
  */
 @SuppressWarnings("unchecked")
 public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed {
-    /** Default policy reoslver. */
+    /** Default policy resolver. */
     private static final DefaultIoPolicyResolver DFLT_IO_PLC_RSLVR = new DefaultIoPolicyResolver();
 
     /** Isolated receiver. */
     private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater();
 
+    /** Amount of permissions should be available to continue new data processing. */
+    private static final int REMAP_SEMAPHORE_PERMISSIONS_COUNT = Integer.MAX_VALUE;
+
     /** Cache receiver. */
     private StreamReceiver<K, V> rcvr = ISOLATED_UPDATER;
 
@@ -178,6 +187,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     /** {@code True} if data loader has been cancelled. */
     private volatile boolean cancelled;
 
+    /** Fail counter. */
+    private final LongAdder8 failCntr = new LongAdder8();
+
     /** Active futures of this data loader. */
     @GridToStringInclude
     private final Collection<IgniteInternalFuture<?>> activeFuts = new GridConcurrentHashSet<>();
@@ -189,6 +201,16 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             boolean rmv = activeFuts.remove(t);
 
             assert rmv;
+
+            Throwable err = t.error();
+
+            if (err != null && !(err instanceof IgniteClientDisconnectedCheckedException)) {
+                LT.error(log, t.error(), "DataStreamer operation failed.", true);
+
+                failCntr.increment();
+
+                cancelled = true;
+            }
         }
     };
 
@@ -231,6 +253,15 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     /** Whether a warning at {@link DataStreamerImpl#allowOverwrite()} printed */
     private static boolean isWarningPrinted;
 
+    /** Allows to pause new data processing while failed data processing in progress. */
+    private final Semaphore remapSem = new Semaphore(REMAP_SEMAPHORE_PERMISSIONS_COUNT);
+
+    /** */
+    private final ConcurrentLinkedDeque<Runnable> dataToRemap = new ConcurrentLinkedDeque<>();
+
+    /** */
+    private final AtomicBoolean remapOwning = new AtomicBoolean();
+
     /**
      * @param ctx Grid kernal context.
      * @param cacheName Cache name.
@@ -301,7 +332,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 Buffer buf = bufMappings.get(nodeId);
 
                 if (buf != null)
-                    buf.onResponse(res);
+                    buf.onResponse(res, nodeId);
 
                 else if (log.isDebugEnabled())
                     log.debug("Ignoring response since node has left [nodeId=" + nodeId + ", ");
@@ -314,6 +345,17 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         fut = new DataStreamerFuture(this);
 
         publicFut = new IgniteCacheFutureImpl<>(fut);
+
+        GridCacheAdapter cache = ctx.cache().internalCache(cacheName);
+
+        if (cache == null) { // Possible, cache is not configured on node.
+            assert ccfg != null;
+
+            if (ccfg.getCacheMode() == CacheMode.LOCAL)
+                throw new CacheException("Impossible to load Local cache configured remotely.");
+
+            ctx.grid().getOrCreateCache(ccfg);
+        }
     }
 
     /**
@@ -358,6 +400,11 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
             throw new IllegalStateException("Data streamer has been closed.");
         }
+        else if (cancelled) {
+            busyLock.leaveBusy();
+
+            throw new IllegalStateException("Data streamer has been closed.");
+        }
     }
 
     /**
@@ -633,6 +680,37 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     }
 
     /**
+     *
+     */
+    private void acquireRemapSemaphore() throws IgniteInterruptedCheckedException {
+        try {
+            if (remapSem.availablePermits() != REMAP_SEMAPHORE_PERMISSIONS_COUNT) {
+                if (timeout == DFLT_UNLIMIT_TIMEOUT) {
+                    // Wait until failed data being processed.
+                    remapSem.acquire(REMAP_SEMAPHORE_PERMISSIONS_COUNT);
+
+                    remapSem.release(REMAP_SEMAPHORE_PERMISSIONS_COUNT);
+                }
+                else {
+                    // Wait until failed data being processed.
+                    boolean res = remapSem.tryAcquire(REMAP_SEMAPHORE_PERMISSIONS_COUNT, timeout, TimeUnit.MILLISECONDS);
+
+                    if (res)
+                        remapSem.release(REMAP_SEMAPHORE_PERMISSIONS_COUNT);
+                    else
+                        throw new IgniteDataStreamerTimeoutException("Data streamer exceeded timeout " +
+                            "while was waiting for failed data resending finished.");
+                }
+            }
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
+    }
+
+    /**
      * @param entries Entries.
      * @param resFut Result future.
      * @param activeKeys Active keys.
@@ -644,170 +722,266 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         @Nullable final Collection<KeyCacheObjectWrapper> activeKeys,
         final int remaps
     ) {
-        assert entries != null;
+        try {
+            assert entries != null;
 
-        if (!isWarningPrinted) {
-            synchronized (this) {
-                if (!allowOverwrite() && !isWarningPrinted) {
-                    U.warn(log, "Data streamer will not overwrite existing cache entries for better performance " +
-                        "(to change, set allowOverwrite to true)");
-                }
+            final boolean remap = remaps > 0;
 
-                isWarningPrinted = true;
+            if (!remap) { // Failed data should be processed prior to new data.
+                acquireRemapSemaphore();
             }
-        }
 
-        Map<ClusterNode, Collection<DataStreamerEntry>> mappings = new HashMap<>();
+            if (!isWarningPrinted) {
+                synchronized (this) {
+                    if (!allowOverwrite() && !isWarningPrinted) {
+                        U.warn(log, "Data streamer will not overwrite existing cache entries for better performance " +
+                            "(to change, set allowOverwrite to true)");
+                    }
 
-        boolean initPda = ctx.deploy().enabled() && jobPda == null;
+                    isWarningPrinted = true;
+                }
+            }
 
-        AffinityTopologyVersion topVer = ctx.cache().context().exchange().readyAffinityVersion();
+            Map<ClusterNode, Collection<DataStreamerEntry>> mappings = new HashMap<>();
 
-        for (DataStreamerEntry entry : entries) {
-            List<ClusterNode> nodes;
+            boolean initPda = ctx.deploy().enabled() && jobPda == null;
 
-            try {
-                KeyCacheObject key = entry.getKey();
+            GridCacheAdapter cache = ctx.cache().internalCache(cacheName);
 
-                assert key != null;
+            if (cache == null)
+                throw new IgniteCheckedException("Cache not created or already destroyed.");
 
-                if (initPda) {
-                    if (cacheObjCtx.addDeploymentInfo())
-                        jobPda = new DataStreamerPda(key.value(cacheObjCtx, false),
-                            entry.getValue() != null ? entry.getValue().value(cacheObjCtx, false) : null,
-                            rcvr);
-                    else if (rcvr != null)
-                        jobPda = new DataStreamerPda(rcvr);
+            GridCacheContext cctx = cache.context();
 
-                    initPda = false;
-                }
+            GridCacheGateway gate = null;
 
-                nodes = nodes(key, topVer);
-            }
-            catch (IgniteCheckedException e) {
-                resFut.onDone(e);
+            if (!allowOverwrite() && !cctx.isLocal()) { // Cases where cctx required.
+                gate = cctx.gate();
 
-                return;
+                gate.enter();
             }
 
-            if (F.isEmpty(nodes)) {
-                resFut.onDone(new ClusterTopologyException("Failed to map key to node " +
-                    "(no nodes with cache found in topology) [infos=" + entries.size() +
-                    ", cacheName=" + cacheName + ']'));
-
-                return;
-            }
+            try {
+                AffinityTopologyVersion topVer = allowOverwrite() || cctx.isLocal() ?
+                        ctx.cache().context().exchange().readyAffinityVersion() :
+                        cctx.topology().topologyVersion();
 
-            for (ClusterNode node : nodes) {
-                Collection<DataStreamerEntry> col = mappings.get(node);
+                for (DataStreamerEntry entry : entries) {
+                    List<ClusterNode> nodes;
 
-                if (col == null)
-                    mappings.put(node, col = new ArrayList<>());
+                    try {
+                        KeyCacheObject key = entry.getKey();
 
-                col.add(entry);
-            }
-        }
+                        assert key != null;
 
-        for (final Map.Entry<ClusterNode, Collection<DataStreamerEntry>> e : mappings.entrySet()) {
-            final UUID nodeId = e.getKey().id();
+                        if (initPda) {
+                            if (cacheObjCtx.addDeploymentInfo())
+                                jobPda = new DataStreamerPda(key.value(cacheObjCtx, false),
+                                    entry.getValue() != null ? entry.getValue().value(cacheObjCtx, false) : null,
+                                    rcvr);
+                            else if (rcvr != null)
+                                jobPda = new DataStreamerPda(rcvr);
 
-            Buffer buf = bufMappings.get(nodeId);
+                            initPda = false;
+                        }
 
-            if (buf == null) {
-                Buffer old = bufMappings.putIfAbsent(nodeId, buf = new Buffer(e.getKey()));
+                        nodes = nodes(key, topVer, cctx);
+                    }
+                    catch (IgniteCheckedException e) {
+                        resFut.onDone(e);
 
-                if (old != null)
-                    buf = old;
-            }
+                        return;
+                    }
 
-            final Collection<DataStreamerEntry> entriesForNode = e.getValue();
+                    if (F.isEmpty(nodes)) {
+                        resFut.onDone(new ClusterTopologyException("Failed to map key to node " +
+                            "(no nodes with cache found in topology) [infos=" + entries.size() +
+                            ", cacheName=" + cacheName + ']'));
 
-            IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> t) {
-                    try {
-                        t.get();
+                        return;
+                    }
 
-                        if (activeKeys != null) {
-                            for (DataStreamerEntry e : entriesForNode)
-                                activeKeys.remove(new KeyCacheObjectWrapper(e.getKey()));
+                    for (ClusterNode node : nodes) {
+                        Collection<DataStreamerEntry> col = mappings.get(node);
 
-                            if (activeKeys.isEmpty())
-                                resFut.onDone();
-                        }
-                        else {
-                            assert entriesForNode.size() == 1;
+                        if (col == null)
+                            mappings.put(node, col = new ArrayList<>());
 
-                            // That has been a single key,
-                            // so complete result future right away.
-                            resFut.onDone();
-                        }
+                        col.add(entry);
                     }
-                    catch (IgniteClientDisconnectedCheckedException e1) {
-                        if (log.isDebugEnabled())
-                            log.debug("Future finished with disconnect error [nodeId=" + nodeId + ", err=" + e1 + ']');
+                }
 
-                        resFut.onDone(e1);
+                for (final Map.Entry<ClusterNode, Collection<DataStreamerEntry>> e : mappings.entrySet()) {
+                    final UUID nodeId = e.getKey().id();
+
+                    Buffer buf = bufMappings.get(nodeId);
+
+                    if (buf == null) {
+                        Buffer old = bufMappings.putIfAbsent(nodeId, buf = new Buffer(e.getKey()));
+
+                        if (old != null)
+                            buf = old;
                     }
-                    catch (IgniteCheckedException e1) {
-                        if (log.isDebugEnabled())
-                            log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']');
 
-                        if (cancelled) {
-                            resFut.onDone(new IgniteCheckedException("Data streamer has been cancelled: " +
-                                DataStreamerImpl.this, e1));
+                    final Collection<DataStreamerEntry> entriesForNode = e.getValue();
+
+                    IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> t) {
+                            try {
+                                t.get();
+
+                                if (activeKeys != null) {
+                                    for (DataStreamerEntry e : entriesForNode)
+                                        activeKeys.remove(new KeyCacheObjectWrapper(e.getKey()));
+
+                                    if (activeKeys.isEmpty())
+                                        resFut.onDone();
+                                }
+                                else {
+                                    assert entriesForNode.size() == 1;
+
+                                    // That has been a single key,
+                                    // so complete result future right away.
+                                    resFut.onDone();
+                                }
+                            }
+                            catch (IgniteClientDisconnectedCheckedException e1) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Future finished with disconnect error [nodeId=" + nodeId + ", err=" + e1 + ']');
+
+                                resFut.onDone(e1);
+                            }
+                            catch (IgniteCheckedException e1) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']');
+
+                                if (cancelled) {
+                                    resFut.onDone(new IgniteCheckedException("Data streamer has been cancelled: " +
+                                        DataStreamerImpl.this, e1));
+                                }
+                                else if (remaps + 1 > maxRemapCnt) {
+                                    resFut.onDone(new IgniteCheckedException("Failed to finish operation (too many remaps): "
+                                        + remaps, e1));
+                                }
+                                else {
+                                    try {
+                                        remapSem.acquire();
+
+                                        final Runnable r = new Runnable() {
+                                            @Override public void run() {
+                                                try {
+                                                    if (cancelled)
+                                                        throw new IllegalStateException("DataStreamer closed.");
+
+                                                    load0(entriesForNode, resFut, activeKeys, remaps + 1);
+                                                }
+                                                catch (Throwable ex) {
+                                                    resFut.onDone(
+                                                        new IgniteCheckedException("DataStreamer remapping failed. ", ex));
+                                                }
+                                                finally {
+                                                    remapSem.release();
+                                                }
+                                            }
+                                        };
+
+                                        dataToRemap.add(r);
+
+                                        if (!remapOwning.get() && remapOwning.compareAndSet(false, true)) {
+                                            ctx.closure().callLocalSafe(new GPC<Boolean>() {
+                                                @Override public Boolean call() {
+                                                    boolean locked = true;
+
+                                                    while (locked || !dataToRemap.isEmpty()) {
+                                                        if (!locked && !remapOwning.compareAndSet(false, true))
+                                                            return false;
+
+                                                        try {
+                                                            Runnable r = dataToRemap.poll();
+
+                                                            if (r != null)
+                                                                r.run();
+                                                        }
+                                                        finally {
+                                                            if (!dataToRemap.isEmpty())
+                                                                locked = true;
+                                                            else {
+                                                                remapOwning.set(false);
+
+                                                                locked = false;
+                                                            }
+                                                        }
+                                                    }
+
+                                                    return true;
+                                                }
+                                            }, true);
+                                        }
+                                    }
+                                    catch (InterruptedException e2) {
+                                        resFut.onDone(e2);
+                                    }
+                                }
+                            }
                         }
-                        else if (remaps + 1 > maxRemapCnt) {
-                            resFut.onDone(new IgniteCheckedException("Failed to finish operation (too many remaps): "
-                                + remaps), e1);
-                        }
-                        else
-                            load0(entriesForNode, resFut, activeKeys, remaps + 1);
-                    }
-                }
-            };
+                    };
 
-            final GridFutureAdapter<?> f;
+                    final GridFutureAdapter<?> f;
 
-            try {
-                f = buf.update(entriesForNode, topVer, lsnr);
-            }
-            catch (IgniteInterruptedCheckedException e1) {
-                resFut.onDone(e1);
+                    try {
+                        f = buf.update(entriesForNode, topVer, lsnr, remap);
+                    }
+                    catch (IgniteInterruptedCheckedException e1) {
+                        resFut.onDone(e1);
 
-                return;
-            }
+                        return;
+                    }
 
-            if (ctx.discovery().node(nodeId) == null) {
-                if (bufMappings.remove(nodeId, buf)) {
-                    final Buffer buf0 = buf;
+                    if (ctx.discovery().node(nodeId) == null) {
+                        if (bufMappings.remove(nodeId, buf)) {
+                            final Buffer buf0 = buf;
 
-                    waitAffinityAndRun(new Runnable() {
-                        @Override public void run() {
-                            buf0.onNodeLeft();
+                            waitAffinityAndRun(new Runnable() {
+                                @Override public void run() {
+                                    buf0.onNodeLeft();
 
-                            if (f != null)
-                                f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " +
-                                    "(node has left): " + nodeId));
+                                    if (f != null)
+                                        f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " +
+                                            "(node has left): " + nodeId));
+                                }
+                            }, ctx.discovery().topologyVersion(), false);
                         }
-                    }, ctx.discovery().topologyVersion(), false);
+                    }
                 }
             }
+            finally {
+                if (gate != null)
+                    gate.leave();
+            }
+        }
+        catch (Exception ex) {
+            resFut.onDone(new IgniteCheckedException("DataStreamer data loading failed.", ex));
         }
     }
 
     /**
      * @param key Key to map.
      * @param topVer Topology version.
+     * @param cctx Context.
      * @return Nodes to send requests to.
      * @throws IgniteCheckedException If failed.
      */
-    private List<ClusterNode> nodes(KeyCacheObject key, AffinityTopologyVersion topVer) throws IgniteCheckedException {
+    private List<ClusterNode> nodes(KeyCacheObject key,
+        AffinityTopologyVersion topVer,
+        GridCacheContext cctx) throws IgniteCheckedException {
         GridAffinityProcessor aff = ctx.affinity();
 
         List<ClusterNode> res = null;
 
         if (!allowOverwrite())
-            res = aff.mapKeyToPrimaryAndBackups(cacheName, key, topVer);
+            res = cctx.isLocal() ?
+                aff.mapKeyToPrimaryAndBackups(cacheName, key, topVer) :
+                cctx.topology().nodes(cctx.affinity().partition(key), topVer);
         else {
             ClusterNode node = aff.mapKeyToNode(cacheName, key, topVer);
 
@@ -992,7 +1166,10 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
      * @throws IgniteCheckedException If failed.
      */
     public void closeEx(boolean cancel) throws IgniteCheckedException {
-        closeEx(cancel, null);
+        IgniteCheckedException err = closeEx(cancel, null);
+
+        if (err != null)
+            throw err; // Throws at close().
     }
 
     /**
@@ -1000,9 +1177,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
      * @param err Error.
      * @throws IgniteCheckedException If failed.
      */
-    public void closeEx(boolean cancel, IgniteCheckedException err) throws IgniteCheckedException {
+    private IgniteCheckedException closeEx(boolean cancel, IgniteCheckedException err) throws IgniteCheckedException {
         if (!closed.compareAndSet(false, true))
-            return;
+            return null;
 
         busyLock.block();
 
@@ -1029,7 +1206,14 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             throw e;
         }
 
+        long failed = failCntr.longValue();
+
+        if (failed > 0 && err == null)
+            err = new IgniteCheckedException("Some of DataStreamer operations failed [failedCount=" + failed + "]");
+
         fut.onDone(err);
+
+        return err;
     }
 
     /**
@@ -1139,6 +1323,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         /** */
         private final Semaphore sem;
 
+        /** Batch topology. */
+        private AffinityTopologyVersion batchTopVer;
+
         /** Closure to signal on task finish. */
         @GridToStringExclude
         private final IgniteInClosure<IgniteInternalFuture<Object>> signalC = new IgniteInClosure<IgniteInternalFuture<Object>>() {
@@ -1169,37 +1356,64 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         }
 
         /**
+         * @param remap Remapping flag.
+         */
+        private void renewBatch(boolean remap) {
+            entries = newEntries();
+            curFut = new GridFutureAdapter<>();
+
+            batchTopVer = null;
+
+            if (!remap)
+                curFut.listen(signalC);
+        }
+
+        /**
          * @param newEntries Infos.
          * @param topVer Topology version.
          * @param lsnr Listener for the operation future.
+         * @param remap Remapping flag.
          * @return Future for operation.
          * @throws IgniteInterruptedCheckedException If failed.
          */
         @Nullable GridFutureAdapter<?> update(Iterable<DataStreamerEntry> newEntries,
             AffinityTopologyVersion topVer,
-            IgniteInClosure<IgniteInternalFuture<?>> lsnr) throws IgniteInterruptedCheckedException {
+            IgniteInClosure<IgniteInternalFuture<?>> lsnr,
+            boolean remap) throws IgniteInterruptedCheckedException {
             List<DataStreamerEntry> entries0 = null;
+
             GridFutureAdapter<Object> curFut0;
 
+            AffinityTopologyVersion curBatchTopVer;
+
             synchronized (this) {
                 curFut0 = curFut;
 
                 curFut0.listen(lsnr);
 
+                if (batchTopVer == null)
+                    batchTopVer = topVer;
+
+                curBatchTopVer = batchTopVer;
+
                 for (DataStreamerEntry entry : newEntries)
                     entries.add(entry);
 
                 if (entries.size() >= bufSize) {
                     entries0 = entries;
 
-                    entries = newEntries();
-                    curFut = new GridFutureAdapter<>();
-                    curFut.listen(signalC);
+                    renewBatch(remap);
                 }
             }
 
-            if (entries0 != null) {
-                submit(entries0, topVer, curFut0);
+            if (!allowOverwrite() && !topVer.equals(curBatchTopVer)) {
+                renewBatch(remap);
+
+                curFut0.onDone(null, new IgniteCheckedException("Topology changed during batch preparation." +
+                    "[batchTopVer=" + curBatchTopVer + ", topVer=" + topVer + "]"));
+            }
+            else if (entries0 != null) {
+                submit(entries0, curBatchTopVer, curFut0, remap);
 
                 if (cancelled)
                     curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " +
@@ -1227,6 +1441,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             List<DataStreamerEntry> entries0 = null;
             GridFutureAdapter<Object> curFut0 = null;
 
+            acquireRemapSemaphore();
+
             synchronized (this) {
                 if (!entries.isEmpty()) {
                     entries0 = entries;
@@ -1239,7 +1455,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
 
             if (entries0 != null)
-                submit(entries0, null, curFut0);
+                submit(entries0, batchTopVer, curFut0, false);
 
             // Create compound future for this flush.
             GridCompoundFuture<Object, Object> res = null;
@@ -1290,25 +1506,113 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         }
 
         /**
+         * @param entries Entries.
+         * @param reqTopVer Request topology version.
+         * @param curFut Current future.
+         */
+        private void localUpdate(final Collection<DataStreamerEntry> entries,
+            final AffinityTopologyVersion reqTopVer,
+            final GridFutureAdapter<Object> curFut) {
+            try {
+                GridCacheContext cctx = ctx.cache().internalCache(cacheName).context();
+
+                final boolean allowOverwrite = allowOverwrite();
+                final boolean loc = cctx.isLocal();
+
+                if (!loc && !allowOverwrite)
+                    cctx.topology().readLock();
+
+                try {
+                    GridDhtTopologyFuture fut = loc ? null : cctx.topologyVersionFuture();
+
+                    AffinityTopologyVersion topVer = loc ? reqTopVer : fut.topologyVersion();
+
+                    if (!allowOverwrite && !topVer.equals(reqTopVer)) {
+                        curFut.onDone(new IgniteCheckedException(
+                            "DataStreamer will retry data transfer at stable topology. " +
+                                "[reqTop=" + reqTopVer + " ,topVer=" + topVer + ", node=local]"));
+                    }
+                    else if (loc || allowOverwrite || fut.isDone()) {
+                        IgniteInternalFuture<Object> callFut = ctx.closure().callLocalSafe(
+                            new DataStreamerUpdateJob(
+                                ctx,
+                                log,
+                                cacheName,
+                                entries,
+                                false,
+                                skipStore,
+                                keepBinary,
+                                rcvr),
+                            false);
+
+                        locFuts.add(callFut);
+
+                        final GridFutureAdapter waitFut = (loc || allowOverwrite) ?
+                            null :
+                            cctx.mvcc().addDataStreamerFuture(topVer);
+
+                        callFut.listen(new IgniteInClosure<IgniteInternalFuture<Object>>() {
+                            @Override public void apply(IgniteInternalFuture<Object> t) {
+                                try {
+                                    boolean rmv = locFuts.remove(t);
+
+                                    assert rmv;
+
+                                    curFut.onDone(t.get());
+                                }
+                                catch (IgniteCheckedException e) {
+                                    curFut.onDone(e);
+                                }
+                                finally {
+                                    if (waitFut != null)
+                                        waitFut.onDone();
+                                }
+                            }
+                        });
+                    }
+                    else {
+                        fut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                            @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
+                                localUpdate(entries, reqTopVer, curFut);
+                            }
+                        });
+                    }
+                }
+                finally {
+                    if (!loc && !allowOverwrite)
+                        cctx.topology().readUnlock();
+                }
+            }
+            catch (Throwable ex) {
+                curFut.onDone(new IgniteCheckedException("DataStreamer data handling failed.", ex));
+            }
+        }
+
+        /**
          * @param entries Entries to submit.
          * @param topVer Topology version.
          * @param curFut Current future.
+         * @param remap Remapping flag.
          * @throws IgniteInterruptedCheckedException If interrupted.
          */
         private void submit(final Collection<DataStreamerEntry> entries,
             @Nullable AffinityTopologyVersion topVer,
-            final GridFutureAdapter<Object> curFut)
+            final GridFutureAdapter<Object> curFut,
+            boolean remap)
             throws IgniteInterruptedCheckedException {
             assert entries != null;
             assert !entries.isEmpty();
             assert curFut != null;
 
-            try {
-                incrementActiveTasks();
-            }
-            catch (IgniteDataStreamerTimeoutException e) {
-                curFut.onDone(e);
-                throw e;
+            if (!remap) {
+                try {
+                    incrementActiveTasks();
+                }
+                catch (IgniteDataStreamerTimeoutException e) {
+                    curFut.onDone(e);
+
+                    throw e;
+                }
             }
 
             IgniteInternalFuture<Object> fut;
@@ -1318,27 +1622,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             if (plc == null)
                 plc = PUBLIC_POOL;
 
-            if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL) {
-                fut = ctx.closure().callLocalSafe(
-                    new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), false);
-
-                locFuts.add(fut);
-
-                fut.listen(new IgniteInClosure<IgniteInternalFuture<Object>>() {
-                    @Override public void apply(IgniteInternalFuture<Object> t) {
-                        try {
-                            boolean rmv = locFuts.remove(t);
-
-                            assert rmv;
-
-                            curFut.onDone(t.get());
-                        }
-                        catch (IgniteCheckedException e) {
-                            curFut.onDone(e);
-                        }
-                    }
-                });
-            }
+            if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL)
+                localUpdate(entries, topVer, curFut);
             else {
                 try {
                     for (DataStreamerEntry e : entries) {
@@ -1466,8 +1751,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         /**
          * @param res Response.
+         * @param nodeId Node id.
          */
-        void onResponse(DataStreamerResponse res) {
+        void onResponse(DataStreamerResponse res, UUID nodeId) {
             if (log.isDebugEnabled())
                 log.debug("Received data load response: " + res);
 
@@ -1488,9 +1774,10 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 try {
                     GridPeerDeployAware jobPda0 = jobPda;
 
-                    err = U.unmarshal(ctx,
-                        errBytes,
-                        U.resolveClassLoader(jobPda0 != null ? jobPda0.classLoader() : null, ctx.config()));
+                    err = new IgniteCheckedException("DataStreamer request failed [node=" + nodeId + "]",
+                        (Throwable)U.unmarshal(ctx,
+                            errBytes,
+                            U.resolveClassLoader(jobPda0 != null ? jobPda0.classLoader() : null, ctx.config())));
                 }
                 catch (IgniteCheckedException e) {
                     f.onDone(null, new IgniteCheckedException("Failed to unmarshal response.", e));
@@ -1613,7 +1900,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     /**
      * Isolated receiver which only loads entry initial value.
      */
-    private static class IsolatedUpdater implements StreamReceiver<KeyCacheObject, CacheObject>,
+    protected static class IsolatedUpdater implements StreamReceiver<KeyCacheObject, CacheObject>,
         DataStreamerCacheUpdaters.InternalUpdater {
         /** */
         private static final long serialVersionUID = 0L;
@@ -1630,7 +1917,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
             GridCacheContext cctx = internalCache.context();
 
-            AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion();
+            AffinityTopologyVersion topVer = cctx.isLocal() ?
+                cctx.affinity().affinityTopologyVersion() :
+                cctx.topology().topologyVersion();
 
             GridCacheVersion ver = cctx.versions().isolatedStreamerVersion();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
index 745619a..ce6783a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
@@ -72,7 +72,21 @@ public class GridLogThrottle {
     public static void error(@Nullable IgniteLogger log, @Nullable Throwable e, String msg) {
         assert !F.isEmpty(msg);
 
-        log(log, e, msg, null, LogLevel.ERROR, false);
+        log(log, e, msg, null, LogLevel.ERROR, false, false);
+    }
+
+    /**
+     * Logs error if needed.
+     *
+     * @param log Logger.
+     * @param e Error (optional).
+     * @param msg Message.
+     * @param byMessage Errors group by message, not by tuple(error, msg).
+     */
+    public static void error(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean byMessage) {
+        assert !F.isEmpty(msg);
+
+        log(log, e, msg, null, LogLevel.ERROR, false, byMessage);
     }
 
     /**
@@ -85,7 +99,7 @@ public class GridLogThrottle {
     public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg) {
         assert !F.isEmpty(msg);
 
-        log(log, e, msg, null, LogLevel.WARN, false);
+        log(log, e, msg, null, LogLevel.WARN, false, false);
     }
 
     /**
@@ -99,7 +113,7 @@ public class GridLogThrottle {
     public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean quite) {
         assert !F.isEmpty(msg);
 
-        log(log, e, msg, null, LogLevel.WARN, quite);
+        log(log, e, msg, null, LogLevel.WARN, quite, false);
     }
 
     /**
@@ -113,7 +127,7 @@ public class GridLogThrottle {
     public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg) {
         assert !F.isEmpty(longMsg);
 
-        log(log, e, longMsg, shortMsg, LogLevel.WARN, false);
+        log(log, e, longMsg, shortMsg, LogLevel.WARN, false, false);
     }
 
     /**
@@ -126,7 +140,7 @@ public class GridLogThrottle {
     public static void info(@Nullable IgniteLogger log, String msg, boolean quite) {
         assert !F.isEmpty(msg);
 
-        log(log, null, msg, null, LogLevel.INFO, quite);
+        log(log, null, msg, null, LogLevel.INFO, quite, false);
     }
 
     /**
@@ -154,14 +168,15 @@ public class GridLogThrottle {
      * @param longMsg Long message (or just message).
      * @param shortMsg Short message for quite logging.
      * @param level Level where messages should appear.
+     * @param byMessage Errors group by message, not by tuple(error, msg).
      */
     @SuppressWarnings({"RedundantTypeArguments"})
     private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg,
-        LogLevel level, boolean quiet) {
+        LogLevel level, boolean quiet, boolean byMessage) {
         assert !F.isEmpty(longMsg);
 
         IgniteBiTuple<Class<? extends Throwable>, String> tup =
-            e != null ? F.<Class<? extends Throwable>, String>t(e.getClass(), e.getMessage()) :
+            e != null && !byMessage ? F.<Class<? extends Throwable>, String>t(e.getClass(), e.getMessage()) :
                 F.<Class<? extends Throwable>, String>t(null, longMsg);
 
         while (true) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
index 5bd6074..c92ea9e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
@@ -78,27 +78,37 @@ public class IgniteCacheDynamicStopSelfTest extends GridCommonAbstractTest {
         IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
             /** {@inheritDoc} */
             @Override public Object call() throws Exception {
-                try (IgniteDataStreamer<Integer, Integer> str = ignite(0).dataStreamer(null)) {
-                    str.allowOverwrite(allowOverwrite);
-
-                    int i = 0;
-
-                    while (!stop.get()) {
-                        str.addData(i % 10_000, i).listen(new CI1<IgniteFuture<?>>() {
-                            @Override public void apply(IgniteFuture<?> f) {
-                                try {
-                                    f.get();
-                                }
-                                catch (CacheException ignore) {
-                                    // This may be debugged.
-                                }
+                while (!stop.get()) {
+                    try (IgniteDataStreamer<Integer, Integer> str = ignite(0).dataStreamer(null)) {
+                        str.allowOverwrite(allowOverwrite);
+
+                        int i = 0;
+
+                        while (!stop.get()) {
+                            try {
+                                str.addData(i % 10_000, i).listen(new CI1<IgniteFuture<?>>() {
+                                    @Override public void apply(IgniteFuture<?> f) {
+                                        try {
+                                            f.get();
+                                        }
+                                        catch (CacheException ignore) {
+                                            // This may be debugged.
+                                        }
+                                    }
+                                });
+                            }
+                            catch (IllegalStateException ignored) {
+                                break;
                             }
-                        });
 
-                        if (i > 0 && i % 10000 == 0)
-                            info("Added: " + i);
+                            if (i > 0 && i % 10000 == 0)
+                                info("Added: " + i);
 
-                        i++;
+                            i++;
+                        }
+                    }
+                    catch (IllegalStateException | CacheException ignored) {
+                        // This may be debugged.
                     }
                 }
 
@@ -114,6 +124,8 @@ public class IgniteCacheDynamicStopSelfTest extends GridCommonAbstractTest {
             Thread.sleep(500);
 
             ignite(0).createCache(ccfg);
+
+            Thread.sleep(1000);
         }
         finally {
             stop.set(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
index 9da6cf7..0801691 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
@@ -18,6 +18,9 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import java.io.Serializable;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import javax.cache.Cache;
 import javax.cache.configuration.FactoryBuilder;
@@ -28,32 +31,47 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.cluster.ClusterNode;
 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.G;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
 
 /**
  * Tests for cache data loading during simultaneous grids start.
  */
-public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-500");
-    }
-
+public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractTest implements Serializable {
     /** Grids count */
     private static int GRIDS_CNT = 5;
 
     /** Keys count */
     private static int KEYS_CNT = 1_000_000;
 
+    /** Client. */
+    private volatile boolean client;
+
+    /** Config. */
+    private volatile boolean configured;
+
+    /** Allow override. */
+    protected volatile boolean allowOverwrite;
+
+    /** Restarts. */
+    protected volatile boolean restarts;
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -67,7 +85,24 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
 
         ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(new TestCacheStoreAdapter()));
 
-        cfg.setCacheConfiguration(ccfg);
+        if (getTestGridName(0).equals(gridName)) {
+            if (client)
+                cfg.setClientMode(true);
+
+            if (configured)
+                cfg.setCacheConfiguration(ccfg);
+        }
+        else
+            cfg.setCacheConfiguration(ccfg);
+
+        if (!configured)
+            ccfg.setNodeFilter(new P1<ClusterNode>() {
+                @Override public boolean apply(ClusterNode node) {
+                    String name = node.attribute(ATTR_GRID_NAME).toString();
+
+                    return !getTestGridName(0).equals(name);
+                }
+            });
 
         return cfg;
     }
@@ -81,22 +116,35 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
      * @throws Exception if failed
      */
     public void testLoadCacheWithDataStreamer() throws Exception {
-        IgniteInClosure<Ignite> f = new IgniteInClosure<Ignite>() {
-            @Override public void apply(Ignite grid) {
-                try (IgniteDataStreamer<Integer, String> dataStreamer = grid.dataStreamer(null)) {
-                    for (int i = 0; i < KEYS_CNT; i++)
-                        dataStreamer.addData(i, Integer.toString(i));
+        configured = true;
+
+        try {
+            IgniteInClosure<Ignite> f = new IgniteInClosure<Ignite>() {
+                @Override public void apply(Ignite grid) {
+                    try (IgniteDataStreamer<Integer, String> dataStreamer = grid.dataStreamer(null)) {
+                        dataStreamer.allowOverwrite(allowOverwrite);
+
+                        for (int i = 0; i < KEYS_CNT; i++)
+                            dataStreamer.addData(i, Integer.toString(i));
+                    }
+
+                    log.info("Data loaded.");
                 }
-            }
-        };
+            };
 
-        loadCache(f);
+            loadCache(f);
+        }
+        finally {
+            configured = false;
+        }
     }
 
     /**
      * @throws Exception if failed
      */
     public void testLoadCacheFromStore() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-4210");
+
         loadCache(new IgniteInClosure<Ignite>() {
             @Override public void apply(Ignite grid) {
                 grid.cache(null).loadCache(null);
@@ -105,12 +153,177 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
     }
 
     /**
+     * @throws Exception if failed
+     */
+    public void testLoadCacheWithDataStreamerSequentialClient() throws Exception {
+        client = true;
+
+        try {
+            loadCacheWithDataStreamerSequential();
+        }
+        finally {
+            client = false;
+        }
+    }
+
+    /**
+     * @throws Exception if failed
+     */
+    public void testLoadCacheWithDataStreamerSequentialClientWithConfig() throws Exception {
+        client = true;
+        configured = true;
+
+        try {
+            loadCacheWithDataStreamerSequential();
+        }
+        finally {
+            client = false;
+            configured = false;
+        }
+    }
+
+    /**
+     * @throws Exception if failed
+     */
+    public void testLoadCacheWithDataStreamerSequential() throws Exception {
+        loadCacheWithDataStreamerSequential();
+    }
+
+    /**
+     * @throws Exception if failed
+     */
+    public void testLoadCacheWithDataStreamerSequentialWithConfigAndRestarts() throws Exception {
+        restarts = true;
+        configured = true;
+
+        try {
+            loadCacheWithDataStreamerSequential();
+        }
+        finally {
+            restarts = false;
+            configured = false;
+        }
+    }
+
+    /**
+     * @throws Exception if failed
+     */
+    public void testLoadCacheWithDataStreamerSequentialWithConfig() throws Exception {
+        configured = true;
+
+        try {
+            loadCacheWithDataStreamerSequential();
+        }
+        finally {
+            configured = false;
+        }
+    }
+
+    /**
+     * @throws Exception if failed
+     */
+    private void loadCacheWithDataStreamerSequential() throws Exception {
+        startGrid(1);
+
+        Ignite g0 = startGrid(0);
+
+        IgniteInternalFuture<Object> restartFut = runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                while (restarts) {
+                    stopGrid(1);
+
+                    startGrid(1);
+
+                    U.sleep(100);
+                }
+
+                return null;
+            }
+        });
+
+        IgniteInternalFuture<Object> fut = runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 2; i < GRIDS_CNT; i++)
+                    startGrid(i);
+
+                return null;
+            }
+        });
+
+        final HashSet<IgniteFuture> set = new HashSet<>();
+
+        IgniteInClosure<Ignite> f = new IgniteInClosure<Ignite>() {
+            @Override public void apply(Ignite grid) {
+                try (IgniteDataStreamer<Integer, String> dataStreamer = grid.dataStreamer(null)) {
+                    dataStreamer.allowOverwrite(allowOverwrite);
+
+                    for (int i = 0; i < KEYS_CNT; i++) {
+                        set.add(dataStreamer.addData(i, "Data"));
+
+                        if (i % 100000 == 0)
+                            log.info("Streaming " + i + "'th entry.");
+                    }
+                }
+            }
+        };
+
+        f.apply(g0);
+
+        log.info("Data loaded.");
+
+        restarts = false;
+
+        fut.get();
+        restartFut.get();
+
+        for (IgniteFuture res : set)
+            assertNull(res.get());
+
+        IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+        long size = cache.size(CachePeekMode.PRIMARY);
+
+        if (size != KEYS_CNT) {
+            Set<Integer> failedKeys = new LinkedHashSet<>();
+
+            for (int i = 0; i < KEYS_CNT; i++)
+                if (!cache.containsKey(i)) {
+                    log.info("Actual cache size: " + size);
+
+                    for (Ignite ignite : G.allGrids()) {
+                        IgniteEx igniteEx = (IgniteEx)ignite;
+
+                        log.info("Missed key info:" +
+                            igniteEx.localNode().id() +
+                            " primary=" +
+                            ignite.affinity(null).isPrimary(igniteEx.localNode(), i) +
+                            " backup=" +
+                            ignite.affinity(null).isBackup(igniteEx.localNode(), i) +
+                            " local peek=" +
+                            ignite.cache(null).localPeek(i, CachePeekMode.ONHEAP));
+                    }
+
+                    for (int j = i; j < i + 10000; j++) {
+                        if (!cache.containsKey(j))
+                            failedKeys.add(j);
+                    }
+
+                    break;
+                }
+
+            assert failedKeys.isEmpty() : "Some failed keys: " + failedKeys.toString();
+        }
+
+        assertCacheSize();
+    }
+
+    /**
      * Loads cache using closure and asserts cache size.
      *
      * @param f cache loading closure
      * @throws Exception if failed
      */
-    private void loadCache(IgniteInClosure<Ignite> f) throws Exception {
+    protected void loadCache(IgniteInClosure<Ignite> f) throws Exception {
         Ignite g0 = startGrid(0);
 
         IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Ignite>() {
@@ -130,17 +343,17 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
     }
 
     /** Asserts cache size. */
-    private void assertCacheSize() {
+    protected void assertCacheSize() {
         IgniteCache<Integer, String> cache = grid(0).cache(null);
 
-        assertEquals(KEYS_CNT, cache.size(CachePeekMode.PRIMARY));
+        assertEquals("Data lost.", KEYS_CNT, cache.size(CachePeekMode.PRIMARY));
 
         int total = 0;
 
         for (int i = 0; i < GRIDS_CNT; i++)
             total += grid(i).cache(null).localSize(CachePeekMode.PRIMARY);
 
-        assertEquals(KEYS_CNT, total);
+        assertEquals("Data lost.", KEYS_CNT, total);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.java
new file mode 100644
index 0000000..c9cd9fa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class CacheLoadingConcurrentGridStartSelfTestAllowOverwrite extends CacheLoadingConcurrentGridStartSelfTest {
+    /**
+     * Default constructor.
+     */
+    public CacheLoadingConcurrentGridStartSelfTestAllowOverwrite() {
+        allowOverwrite = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
index 9fedc35..0f8ae29 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
@@ -29,9 +29,9 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.Cache;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
@@ -194,7 +194,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (IgniteCheckedException e) {
+        catch (CacheException e) {
             // Cannot load local cache configured remotely.
             info("Caught expected exception: " + e);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7499828/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
index 0c6686f..a6a9f54 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java
@@ -22,13 +22,17 @@ import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CyclicBarrier;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.CacheServerNotFoundException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
 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;
@@ -50,6 +54,16 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest {
     /** Started grid counter. */
     private static int cnt;
 
+    /** No nodes filter. */
+    private static volatile boolean noNodesFilter;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -72,88 +86,149 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testNullPointerExceptionUponDataStreamerClosing() throws Exception {
-        try {
-            startGrids(5);
+        startGrids(5);
 
-            final CyclicBarrier barrier = new CyclicBarrier(2);
+        final CyclicBarrier barrier = new CyclicBarrier(2);
 
-            multithreadedAsync(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    U.awaitQuiet(barrier);
+        multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                U.awaitQuiet(barrier);
 
-                    G.stopAll(true);
+                G.stopAll(true);
 
-                    return null;
-                }
-            }, 1);
+                return null;
+            }
+        }, 1);
 
-            Ignite g4 = grid(4);
+        Ignite g4 = grid(4);
 
-            IgniteDataStreamer<Object, Object> dataLdr = g4.dataStreamer(null);
+        IgniteDataStreamer<Object, Object> dataLdr = g4.dataStreamer(null);
 
-            dataLdr.perNodeBufferSize(32);
+        dataLdr.perNodeBufferSize(32);
 
-            for (int i = 0; i < 100000; i += 2) {
-                dataLdr.addData(i, i);
-                dataLdr.removeData(i + 1);
-            }
+        for (int i = 0; i < 100000; i += 2) {
+            dataLdr.addData(i, i);
+            dataLdr.removeData(i + 1);
+        }
 
-            U.awaitQuiet(barrier);
+        U.awaitQuiet(barrier);
 
-            info("Closing data streamer.");
+        info("Closing data streamer.");
 
-            try {
-                dataLdr.close(true);
-            }
-            catch (IllegalStateException ignore) {
-                // This is ok to ignore this exception as test is racy by it's nature -
-                // grid is stopping in different thread.
-            }
+        try {
+            dataLdr.close(true);
         }
-        finally {
-            G.stopAll(true);
+        catch (CacheException | IllegalStateException ignore) {
+            // This is ok to ignore this exception as test is racy by it's nature -
+            // grid is stopping in different thread.
         }
     }
 
     /**
      * Data streamer should correctly load entries from HashMap in case of grids with more than one node
-     *  and with GridOptimizedMarshaller that requires serializable.
+     * and with GridOptimizedMarshaller that requires serializable.
      *
      * @throws Exception If failed.
      */
     public void testAddDataFromMap() throws Exception {
-        try {
-            cnt = 0;
+        cnt = 0;
 
-            startGrids(2);
+        startGrids(2);
 
-            Ignite g0 = grid(0);
+        Ignite g0 = grid(0);
 
-            IgniteDataStreamer<Integer, String> dataLdr = g0.dataStreamer(null);
+        IgniteDataStreamer<Integer, String> dataLdr = g0.dataStreamer(null);
 
-            Map<Integer, String> map = U.newHashMap(KEYS_COUNT);
+        Map<Integer, String> map = U.newHashMap(KEYS_COUNT);
 
-            for (int i = 0; i < KEYS_COUNT; i ++)
-                map.put(i, String.valueOf(i));
+        for (int i = 0; i < KEYS_COUNT; i++)
+            map.put(i, String.valueOf(i));
 
-            dataLdr.addData(map);
+        dataLdr.addData(map);
 
-            dataLdr.close();
+        dataLdr.close();
 
-            Random rnd = new Random();
+        Random rnd = new Random();
 
-            IgniteCache<Integer, String> c = g0.cache(null);
+        IgniteCache<Integer, String> c = g0.cache(null);
 
-            for (int i = 0; i < KEYS_COUNT; i ++) {
-                Integer k = rnd.nextInt(KEYS_COUNT);
+        for (int i = 0; i < KEYS_COUNT; i++) {
+            Integer k = rnd.nextInt(KEYS_COUNT);
 
-                String v = c.get(k);
+            String v = c.get(k);
+
+            assertEquals(k.toString(), v);
+        }
+    }
+
+    /**
+     * Test logging on {@code DataStreamer.addData()} method when cache have no data nodes
+     *
+     * @throws Exception If fail.
+     */
+    public void testNoDataNodesOnClose() throws Exception {
+        boolean failed = false;
+
+        cnt = 0;
+
+        noNodesFilter = true;
+
+        try {
+            Ignite ignite = startGrid(1);
 
-                assertEquals(k.toString(), v);
+            try (IgniteDataStreamer<Integer, String> streamer = ignite.dataStreamer(null)) {
+                streamer.addData(1, "1");
+            }
+            catch (CacheException ex) {
+                failed = true;
             }
         }
         finally {
-            G.stopAll(true);
+            noNodesFilter = false;
+
+            assertTrue(failed);
+        }
+    }
+
+    /**
+     * Test logging on {@code DataStreamer.addData()} method when cache have no data nodes
+     *
+     * @throws Exception If fail.
+     */
+    public void testNoDataNodesOnFlush() throws Exception {
+        boolean failed = false;
+
+        cnt = 0;
+
+        noNodesFilter = true;
+
+        try {
+            Ignite ignite = startGrid(1);
+
+            IgniteFuture fut = null;
+
+            try (IgniteDataStreamer<Integer, String> streamer = ignite.dataStreamer(null)) {
+                fut = streamer.addData(1, "1");
+
+                streamer.flush();
+            }
+            catch (IllegalStateException ex) {
+                try {
+                    fut.get();
+
+                    fail("DataStreamer ignores failed streaming.");
+                }
+                catch (CacheServerNotFoundException ignored) {
+                    // No-op.
+                }
+
+                failed = true;
+            }
+        }
+        finally {
+            noNodesFilter = false;
+
+            assertTrue(failed);
         }
     }
 
@@ -169,6 +244,9 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest {
         cacheCfg.setBackups(1);
         cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 
+        if (noNodesFilter)
+            cacheCfg.setNodeFilter(F.alwaysFalse());
+
         return cacheCfg;
     }
 


[13/17] ignite git commit: ignite-4154 Optimize amount of data stored in discovery history Discovery history optimizations: - remove discarded message for discovery pending messages - remove duplicated data from TcpDiscoveryNodeAddedMessage.oldNodesDisco

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 0663903..6edfd09 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.CacheConfigurationLeakTest;
 import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRemoveSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest;
 import org.apache.ignite.internal.processors.cache.CacheEnumOperationsTest;
+import org.apache.ignite.internal.processors.cache.CacheExchangeMessageDuplicatedStateTest;
 import org.apache.ignite.internal.processors.cache.CrossCacheTxRandomOperationsTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAtomicMessageCountSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheFinishPartitionsSelfTest;
@@ -264,6 +265,8 @@ public class IgniteCacheTestSuite2 extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteNoCustomEventsOnNodeStart.class));
 
+        suite.addTest(new TestSuite(CacheExchangeMessageDuplicatedStateTest.class));
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
index f21a279..c24fed4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -87,6 +87,13 @@ public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
index 5dbb12c..742da7c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
@@ -159,6 +159,13 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
         return c;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
index 0f60db2..072a081 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
@@ -95,6 +95,13 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr
             g.cache(null).removeAll();
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
     /** */
     public void testRemoteQueryExecutionTimeout() throws Exception {
         testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index d559d2e..070acb1 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -87,13 +87,22 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         return c;
     }
 
-    /** */
+    /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
         ignite = startGrid();
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        ignite = null;
+
+        super.afterTestsStopped();
+    }
+
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/incorrect-store-cache.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/incorrect-store-cache.xml b/modules/spring/src/test/config/incorrect-store-cache.xml
index 2ccda21..b08f84e 100644
--- a/modules/spring/src/test/config/incorrect-store-cache.xml
+++ b/modules/spring/src/test/config/incorrect-store-cache.xml
@@ -23,6 +23,8 @@
         http://www.springframework.org/schema/beans/spring-beans.xsd">
 
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <property name="cacheConfiguration">
             <list>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
index d62e76b..dfaf828 100644
--- a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
+++ b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
@@ -25,7 +25,10 @@
                            http://www.springframework.org/schema/util
                            http://www.springframework.org/schema/util/spring-util.xsd">
     <bean class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <property name="gridName" value="test-cluster"/>
+
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
index 83e0548..9bc9977 100644
--- a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
+++ b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
@@ -25,7 +25,10 @@
                            http://www.springframework.org/schema/util
                            http://www.springframework.org/schema/util/spring-util.xsd">
     <bean class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <property name="gridName" value="test-cluster"/>
+
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/node.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/node.xml b/modules/spring/src/test/config/node.xml
index 6f467eb..e6b0b52 100644
--- a/modules/spring/src/test/config/node.xml
+++ b/modules/spring/src/test/config/node.xml
@@ -25,6 +25,8 @@
     <bean id="simpleDataSource" class="org.h2.jdbcx.JdbcDataSource"/>
 
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/node1.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/node1.xml b/modules/spring/src/test/config/node1.xml
index 76539f7..814736a 100644
--- a/modules/spring/src/test/config/node1.xml
+++ b/modules/spring/src/test/config/node1.xml
@@ -25,6 +25,8 @@
     <bean id="simpleDataSource" class="org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactorySelfTest$DummyDataSource"/>
 
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <property name="gridName" value="ignite1"/>
 
         <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/pojo-incorrect-store-cache.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/pojo-incorrect-store-cache.xml b/modules/spring/src/test/config/pojo-incorrect-store-cache.xml
index 5627bd0..f7f9141 100644
--- a/modules/spring/src/test/config/pojo-incorrect-store-cache.xml
+++ b/modules/spring/src/test/config/pojo-incorrect-store-cache.xml
@@ -23,6 +23,8 @@
         http://www.springframework.org/schema/beans/spring-beans.xsd">
 
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <property name="cacheConfiguration">
             <list>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/store-cache.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/store-cache.xml b/modules/spring/src/test/config/store-cache.xml
index 07a4ab0..d7bc014 100644
--- a/modules/spring/src/test/config/store-cache.xml
+++ b/modules/spring/src/test/config/store-cache.xml
@@ -25,6 +25,8 @@
     <bean id="simpleDataSource" class="org.h2.jdbcx.JdbcDataSource"/>
 
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <property name="cacheConfiguration">
             <list>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/7128a395/modules/spring/src/test/config/store-cache1.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/store-cache1.xml b/modules/spring/src/test/config/store-cache1.xml
index a9e58d6..46eef3f 100644
--- a/modules/spring/src/test/config/store-cache1.xml
+++ b/modules/spring/src/test/config/store-cache1.xml
@@ -26,6 +26,8 @@
     <bean id="simpleDataSource1" class="org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactorySelfTest$DummyDataSource"/>
 
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
         <property name="gridName" value="ignite1"/>
 
         <property name="cacheConfiguration">


[06/17] ignite git commit: IGNITE-4208: Hadoop: Fixed a bug preventing normal secondary file system start. This closes #1228.

Posted by sb...@apache.org.
IGNITE-4208: Hadoop: Fixed a bug preventing normal secondary file system start. This closes #1228.


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

Branch: refs/heads/ignite-4154-opt2
Commit: ef9d6cf9e334c35b03dfa42e4ce0680c85a693a4
Parents: b749982
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Nov 10 18:47:09 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Nov 10 18:48:48 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java   | 2 +-
 .../org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java   | 2 +-
 .../internal/processors/hadoop/delegate/HadoopDelegateUtils.java | 4 ++--
 .../impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java | 3 +--
 .../hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java  | 3 ++-
 .../hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java        | 2 +-
 .../impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java      | 3 +--
 .../impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java    | 4 ++--
 8 files changed, 11 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index 7133c08..866fc1a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -336,7 +336,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
                     HadoopFileSystemFactory factory0 =
                         (HadoopFileSystemFactory)paths.getPayload(getClass().getClassLoader());
 
-                    factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0);
+                    factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0);
                 }
                 catch (IgniteCheckedException e) {
                     throw new IOException("Failed to get secondary file system factory.", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
index 18b8bf9..82ad683 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -345,7 +345,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
                     HadoopFileSystemFactory factory0 =
                         (HadoopFileSystemFactory) paths.getPayload(getClass().getClassLoader());
 
-                    factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0);
+                    factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0);
                 }
                 catch (IgniteCheckedException e) {
                     throw new IOException("Failed to get secondary file system factory.", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java
index 76d9bff..2059c8d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java
@@ -81,13 +81,13 @@ public class HadoopDelegateUtils {
      * @return Delegate.
      */
     @SuppressWarnings("unchecked")
-    public static HadoopFileSystemFactoryDelegate fileSystemFactoryDelegate(Object proxy) {
+    public static HadoopFileSystemFactoryDelegate fileSystemFactoryDelegate(ClassLoader ldr, Object proxy) {
         String clsName = FACTORY_CLS_MAP.get(proxy.getClass().getName());
 
         if (clsName == null)
             clsName = DFLT_FACTORY_CLS;
 
-        return newInstance(clsName, null, proxy);
+        return newInstance(clsName, ldr, proxy);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java
index 203965c..e336fad 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java
@@ -56,7 +56,6 @@ import java.io.OutputStream;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -86,7 +85,7 @@ public class HadoopIgfsSecondaryFileSystemDelegateImpl implements HadoopIgfsSeco
         if (factory0 == null)
             factory0 = new CachingHadoopFileSystemFactory();
 
-        factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0);
+        factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java
index 8c95a0e..f35d644 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java
@@ -70,7 +70,8 @@ public class KerberosHadoopFileSystemFactorySelfTest extends GridCommonAbstractT
 
         GridTestUtils.assertThrows(null, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                HadoopFileSystemFactoryDelegate delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac);
+                HadoopFileSystemFactoryDelegate delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(
+                    getClass().getClassLoader(), fac);
 
                 delegate.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java
index 7cf7f2d..5d955d4 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java
@@ -330,7 +330,7 @@ public class HadoopFIleSystemFactorySelfTest extends IgfsCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void start() throws IgniteException {
-            delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(factory);
+            delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory);
 
             delegate.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java
index adb1330..453d0c7 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java
@@ -30,7 +30,6 @@ import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory;
 import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils;
 import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate;
-import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemTestAdapter;
@@ -50,7 +49,7 @@ public class HadoopIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFi
     public HadoopIgfsSecondaryFileSystemTestAdapter(HadoopFileSystemFactory factory) {
         assert factory != null;
 
-        this.factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory);
+        this.factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory);
 
         this.factory.start();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef9d6cf9/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java
index ae03f14..78f457b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java
@@ -37,7 +37,6 @@ import org.apache.ignite.igfs.IgfsIpcEndpointType;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils;
 import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate;
-import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -185,7 +184,8 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
         fac.setConfigPaths(primaryConfFullPath);
         fac.setUri(primaryFsUriStr);
 
-        HadoopFileSystemFactoryDelegate facDelegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac);
+        HadoopFileSystemFactoryDelegate facDelegate = HadoopDelegateUtils.fileSystemFactoryDelegate(
+            getClass().getClassLoader(), fac);
 
         facDelegate.start();
 


[03/17] ignite git commit: Backport commit of the following:

Posted by sb...@apache.org.
Backport commit of the following:

commit 612eb3daffe608995aac28eed019b3e6ef9d66d3
Author: Aleksei Scherbakov <al...@gmail.com>
Date:   Fri Aug 19 13:28:39 2016 +0300

    ignite-2795 Support 'copyOnRead' for SQL queries


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

Branch: refs/heads/ignite-4154-opt2
Commit: 8b59f4e76138e08e80aa219c1a9cf0c3df6fdb4b
Parents: 26daa57
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Thu Nov 10 14:43:00 2016 +0300
Committer: Andrey V. Mashenkov <an...@gmail.com>
Committed: Thu Nov 10 14:43:00 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      |   8 +
 .../query/h2/GridH2ResultSetIterator.java       |  62 ++++-
 .../processors/query/h2/IgniteH2Indexing.java   |   4 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    |  10 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  34 ++-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../cache/CacheSqlQueryValueCopySelfTest.java   | 226 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite2.java         |   2 +
 8 files changed, 338 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/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 dc9c766..30f1c4b 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
@@ -1699,6 +1699,14 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return {@code True} if the value for the cache object has to be copied because
+     * of {@link CacheConfiguration#isCopyOnRead()}.
+     */
+    public boolean needValueCopy() {
+        return affNode && cacheCfg.isCopyOnRead() && cacheCfg.getMemoryMode() != OFFHEAP_VALUES;
+    }
+
+    /**
      * Converts temporary offheap object to heap-based.
      *
      * @param obj Object.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
index 3603bb5..e0680d3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
@@ -17,24 +17,49 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import java.lang.reflect.Field;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.NoSuchElementException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-
+import org.h2.jdbc.JdbcResultSet;
+import org.h2.result.ResultInterface;
+import org.h2.value.Value;
 
 /**
  * Iterator over result set.
  */
 public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAdapter<T> {
     /** */
+    private static final Field RESULT_FIELD;
+
+    /**
+     * Initialize.
+     */
+    static {
+        try {
+            RESULT_FIELD = JdbcResultSet.class.getDeclaredField("result");
+
+            RESULT_FIELD.setAccessible(true);
+        }
+        catch (NoSuchFieldException e) {
+            throw new IllegalStateException("Check H2 version in classpath.", e);
+        }
+    }
+
+    /** */
     private static final long serialVersionUID = 0L;
 
     /** */
+    private final ResultInterface res;
+
+    /** */
     private final ResultSet data;
 
     /** */
@@ -49,12 +74,20 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
     /**
      * @param data Data array.
      * @param closeStmt If {@code true} closes result set statement when iterator is closed.
+     * @param needCpy {@code True} if need copy cache object's value.
      * @throws IgniteCheckedException If failed.
      */
-    protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt) throws IgniteCheckedException {
+    protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt, boolean needCpy) throws IgniteCheckedException {
         this.data = data;
         this.closeStmt = closeStmt;
 
+        try {
+            res = needCpy ? (ResultInterface)RESULT_FIELD.get(data) : null;
+        }
+        catch (IllegalAccessException e) {
+            throw new IllegalStateException(e); // Must not happen.
+        }
+
         if (data != null) {
             try {
                 row = new Object[data.getMetaData().getColumnCount()];
@@ -78,8 +111,27 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
             if (!data.next())
                 return false;
 
-            for (int c = 0; c < row.length; c++)
-                row[c] = data.getObject(c + 1);
+            if (res != null) {
+                Value[] values = res.currentRow();
+
+                for (int c = 0; c < row.length; c++) {
+                    Value val = values[c];
+
+                    if (val instanceof GridH2ValueCacheObject) {
+                        GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)values[c];
+
+                        GridCacheContext cctx = valCacheObj.getCacheContext();
+
+                        row[c] = valCacheObj.getObject(cctx != null && cctx.needValueCopy());
+                    }
+                    else
+                        row[c] = val.getObject();
+                }
+            }
+            else {
+                for (int c = 0; c < row.length; c++)
+                    row[c] = data.getObject(c + 1);
+            }
 
             return true;
         }
@@ -134,6 +186,6 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString((Class<GridH2ResultSetIterator>)getClass(), this);
+        return S.toString(GridH2ResultSetIterator.class, this);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index ab332c1..5c2fab5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -2230,7 +2230,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @throws IgniteCheckedException If failed.
          */
         protected FieldsIterator(ResultSet data) throws IgniteCheckedException {
-            super(data, false);
+            super(data, false, true);
         }
 
         /** {@inheritDoc} */
@@ -2255,7 +2255,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @throws IgniteCheckedException If failed.
          */
         protected KeyValIterator(ResultSet data) throws IgniteCheckedException {
-            super(data, false);
+            super(data, false, true);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
index fd0e6ed..4d83295 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
@@ -120,7 +120,15 @@ public class GridH2ValueCacheObject extends Value {
 
     /** {@inheritDoc} */
     @Override public Object getObject() {
-        return obj.isPlatformType() ? obj.value(objectContext(), false) : obj;
+        return getObject(false);
+    }
+
+    /**
+     * @param cpy Copy flag.
+     * @return Value.
+     */
+    public Object getObject(boolean cpy) {
+        return obj.isPlatformType() ? obj.value(objectContext(), cpy) : obj;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 1f05bf7..7e4d5b6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -52,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest;
@@ -759,6 +760,9 @@ public class GridMapQueryExecutor {
         private final int rowCount;
 
         /** */
+        private boolean cpNeeded;
+
+        /** */
         private volatile boolean closed;
 
         /**
@@ -767,11 +771,12 @@ public class GridMapQueryExecutor {
          * @param qrySrcNodeId Query source node.
          * @param qry Query.
          */
-        private QueryResult(ResultSet rs, GridCacheContext<?,?> cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) {
+        private QueryResult(ResultSet rs, GridCacheContext<?, ?> cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) {
             this.rs = rs;
             this.cctx = cctx;
             this.qry = qry;
             this.qrySrcNodeId = qrySrcNodeId;
+            this.cpNeeded = cctx.isLocalNode(qrySrcNodeId);
 
             try {
                 res = (ResultInterface)RESULT_FIELD.get(rs);
@@ -803,6 +808,33 @@ public class GridMapQueryExecutor {
 
                 Value[] row = res.currentRow();
 
+                if (cpNeeded) {
+                    boolean copied = false;
+
+                    for (int j = 0; j < row.length; j++) {
+                        Value val = row[j];
+
+                        if (val instanceof GridH2ValueCacheObject) {
+                            GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)val;
+
+                            GridCacheContext cctx = valCacheObj.getCacheContext();
+
+                            if (cctx != null && cctx.needValueCopy()) {
+                                row[j] = new GridH2ValueCacheObject(valCacheObj.getCacheContext(), valCacheObj.getCacheObject()) {
+                                    @Override public Object getObject() {
+                                        return getObject(true);
+                                    }
+                                };
+
+                                copied = true;
+                            }
+                        }
+                    }
+
+                    if (i == 0 && !copied)
+                        cpNeeded = false; // No copy on read caches, skip next checks.
+                }
+
                 assert row != null;
 
                 if (readEvt) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 3fdbf42..29ad94a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -1289,7 +1289,7 @@ public class GridReduceQueryExecutor {
          * @throws IgniteCheckedException If failed.
          */
         protected Iter(ResultSet data) throws IgniteCheckedException {
-            super(data, true);
+            super(data, true, false);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
new file mode 100644
index 0000000..e47e893
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
@@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import java.util.List;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+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.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests modification of values returned by query iterators with enabled copy on read.
+ */
+public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int KEYS = 100;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if ("client".equals(cfg.getGridName()))
+            cfg.setClientMode(true);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        CacheConfiguration<Integer, Value> cc = new CacheConfiguration<>();
+
+        cc.setCopyOnRead(true);
+        cc.setIndexedTypes(Integer.class, Value.class);
+
+        cfg.setCacheConfiguration(cc);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(3);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        for (int i = 0; i < KEYS; i++)
+            cache.put(i, new Value("before"));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        cache.removeAll();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * Tests two step query from dedicated client.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTwoStepSqlClientQuery() throws Exception {
+        try (Ignite client = startGrid("client")) {
+            IgniteCache<Integer, Value> cache = client.cache(null);
+
+            List<Cache.Entry<Integer, Value>> all = cache.query(
+                new SqlQuery<Integer, Value>(Value.class, "select * from Value")).getAll();
+
+            assertEquals(KEYS, all.size());
+
+            for (Cache.Entry<Integer, Value> entry : all)
+                entry.getValue().str = "after";
+
+            check(cache);
+
+            QueryCursor<List<?>> qry = cache.query(new SqlFieldsQuery("select _val from Value"));
+
+            List<List<?>> all0 = qry.getAll();
+
+            assertEquals(KEYS, all0.size());
+
+            for (List<?> entry : all0)
+                ((Value)entry.get(0)).str = "after";
+
+            check(cache);
+        }
+    }
+
+    /**
+     * Test two step query without local reduce phase.
+     */
+    public void testTwoStepSkipReduceSqlQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        List<Cache.Entry<Integer, Value>> all = cache.query(
+            new SqlQuery<Integer, Value>(Value.class, "select * from Value").setPageSize(3)).getAll();
+
+        assertEquals(KEYS, all.size());
+
+        for (Cache.Entry<Integer, Value> entry : all)
+            entry.getValue().str = "after";
+
+        check(cache);
+    }
+
+    /**
+     * Test two step query value copy.
+     */
+    public void testTwoStepReduceSqlQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        QueryCursor<List<?>> qry = cache.query(new SqlFieldsQuery("select _val from Value order by _key"));
+
+        List<List<?>> all = qry.getAll();
+
+        assertEquals(KEYS, all.size());
+
+        for (List<?> entry : all)
+            ((Value)entry.get(0)).str = "after";
+
+        check(cache);
+    }
+
+    /**
+     * Tests local sql query.
+     */
+    public void testLocalSqlQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        SqlQuery<Integer, Value> qry = new SqlQuery<>(Value.class.getSimpleName(), "select * from Value");
+        qry.setLocal(true);
+
+        List<Cache.Entry<Integer, Value>> all = cache.query(qry).getAll();
+
+        assertFalse(all.isEmpty());
+
+        for (Cache.Entry<Integer, Value> entry : all)
+            entry.getValue().str = "after";
+
+        check(cache);
+    }
+
+    /**
+     * Tests local sql query.
+     */
+    public void testLocalSqlFieldsQuery() {
+        IgniteCache<Integer, Value> cache = grid(0).cache(null);
+
+        QueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("select _val from Value").setLocal(true));
+
+        List<List<?>> all = cur.getAll();
+
+        assertFalse(all.isEmpty());
+
+        for (List<?> entry : all)
+            ((Value)entry.get(0)).str = "after";
+
+        check(cache);
+    }
+
+    /** */
+    private static class Value {
+        /** */
+        private String str;
+
+        /**
+         * @param str String.
+         */
+        public Value(String str) {
+            this.str = str;
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     */
+    private void check(IgniteCache<Integer, Value> cache) {
+        int cnt = 0;
+
+        // Value should be not modified by previous assignment.
+        for (Cache.Entry<Integer, Value> entry : cache) {
+            cnt++;
+
+            assertEquals("before", entry.getValue().str);
+        }
+
+        assertEquals(KEYS, cnt);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index 9128f76..4b4a576 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLos
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
@@ -114,6 +115,7 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
         // Other.
         suite.addTestSuite(CacheQueryNewClientSelfTest.class);
         suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class);
+        suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class);
 
         return suite;
     }