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

[01/46] ignite git commit: Ignite-1093 "Rebalancing with default parameters is very slow" fixes.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1753-1282 dbf83b9b4 -> 59fd5cecc


http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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 998c720..c634ff5 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
@@ -17,15 +17,18 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 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.IgniteInternalFuture;
@@ -47,27 +50,42 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalP
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridPlainRunnable;
 import org.apache.ignite.internal.util.typedef.CI1;
+import org.apache.ignite.internal.util.typedef.F;
+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.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
+import org.jsr166.ConcurrentLinkedDeque8;
 
+import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_UNLOADED;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING;
 import static org.apache.ignite.internal.util.GridConcurrentFactory.newMap;
 
 /**
  * DHT cache preloader.
  */
 public class GridDhtPreloader extends GridCachePreloaderAdapter {
+    /**
+     * Rebalancing was refactored at version 1.5.0, but backward compatibility to previous implementation was saved.
+     * Node automatically chose communication protocol depends on remote node's version.
+     * Backward compatibility may be removed at Ignite 2.x.
+     */
+    public static final IgniteProductVersion REBALANCING_VER_2_SINCE = IgniteProductVersion.fromString("1.5.0");
+
     /** Default preload resend timeout. */
     public static final long DFLT_PRELOAD_RESEND_TIMEOUT = 1500;
 
@@ -81,10 +99,10 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     private final ConcurrentMap<IgniteUuid, GridDhtForceKeysFuture<?, ?>> forceKeyFuts = newMap();
 
     /** Partition suppliers. */
-    private GridDhtPartitionSupplyPool supplyPool;
+    private GridDhtPartitionSupplier supplier;
 
     /** Partition demanders. */
-    private GridDhtPartitionDemandPool demandPool;
+    private GridDhtPartitionDemander demander;
 
     /** Start future. */
     private GridFutureAdapter<Object> startFut;
@@ -92,10 +110,19 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     /** Busy lock to prevent activities from accessing exchanger while it's stopping. */
     private final ReadWriteLock busyLock = new ReentrantReadWriteLock();
 
+    /** Demand lock. */
+    private final ReadWriteLock demandLock = new ReentrantReadWriteLock();
+
     /** Pending affinity assignment futures. */
     private ConcurrentMap<AffinityTopologyVersion, GridDhtAssignmentFetchFuture> pendingAssignmentFetchFuts =
         new ConcurrentHashMap8<>();
 
+    /** */
+    private final ConcurrentLinkedDeque8<GridDhtLocalPartition> partsToEvict = new ConcurrentLinkedDeque8<>();
+
+    /** */
+    private final AtomicInteger partsEvictOwning = new AtomicInteger();
+
     /** Discovery listener. */
     private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
         @Override public void onEvent(Event evt) {
@@ -179,8 +206,11 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
                 }
             });
 
-        supplyPool = new GridDhtPartitionSupplyPool(cctx, busyLock);
-        demandPool = new GridDhtPartitionDemandPool(cctx, busyLock);
+        supplier = new GridDhtPartitionSupplier(cctx);
+        demander = new GridDhtPartitionDemander(cctx, busyLock);
+
+        supplier.start();
+        demander.start();
 
         cctx.events().addListener(discoLsnr, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED);
     }
@@ -197,19 +227,16 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
         final long startTopVer = loc.order();
 
         topVer.setIfGreater(startTopVer);
-
-        supplyPool.start();
-        demandPool.start();
     }
 
     /** {@inheritDoc} */
     @Override public void preloadPredicate(IgnitePredicate<GridCacheEntryInfo> preloadPred) {
         super.preloadPredicate(preloadPred);
 
-        assert supplyPool != null && demandPool != null : "preloadPredicate may be called only after start()";
+        assert supplier != null && demander != null : "preloadPredicate may be called only after start()";
 
-        supplyPool.preloadPredicate(preloadPred);
-        demandPool.preloadPredicate(preloadPred);
+        supplier.preloadPredicate(preloadPred);
+        demander.preloadPredicate(preloadPred);
     }
 
     /** {@inheritDoc} */
@@ -223,37 +250,109 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
         // Acquire write busy lock.
         busyLock.writeLock().lock();
 
-        if (supplyPool != null)
-            supplyPool.stop();
+        if (supplier != null)
+            supplier.stop();
 
-        if (demandPool != null)
-            demandPool.stop();
+        if (demander != null)
+            demander.stop();
 
         top = null;
     }
 
     /** {@inheritDoc} */
     @Override public void onInitialExchangeComplete(@Nullable Throwable err) {
-        if (err == null) {
+        if (err == null)
             startFut.onDone();
+        else
+            startFut.onDone(err);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) {
+        demander.updateLastExchangeFuture(lastFut);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onTopologyChanged(AffinityTopologyVersion topVer) {
+        supplier.onTopologyChanged(topVer);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
+        // No assignments for disabled preloader.
+        GridDhtPartitionTopology top = cctx.dht().topology();
+
+        if (!cctx.rebalanceEnabled())
+            return new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
+
+        int partCnt = cctx.affinity().partitions();
 
-            final long start = U.currentTimeMillis();
+        assert exchFut.forcePreload() || exchFut.dummyReassign() ||
+            exchFut.exchangeId().topologyVersion().equals(top.topologyVersion()) :
+            "Topology version mismatch [exchId=" + exchFut.exchangeId() +
+                ", topVer=" + top.topologyVersion() + ']';
 
-            final CacheConfiguration cfg = cctx.config();
+        GridDhtPreloaderAssignments assigns = new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
 
-            if (cfg.getRebalanceDelay() >= 0 && !cctx.kernalContext().clientNode()) {
-                U.log(log, "Starting rebalancing in " + cfg.getRebalanceMode() + " mode: " + cctx.name());
+        AffinityTopologyVersion topVer = assigns.topologyVersion();
 
-                demandPool.syncFuture().listen(new CI1<Object>() {
-                    @Override public void apply(Object t) {
-                        U.log(log, "Completed rebalancing in " + cfg.getRebalanceMode() + " mode " +
-                            "[cache=" + cctx.name() + ", time=" + (U.currentTimeMillis() - start) + " ms]");
+        for (int p = 0; p < partCnt; p++) {
+            if (cctx.shared().exchange().hasPendingExchange()) {
+                if (log.isDebugEnabled())
+                    log.debug("Skipping assignments creation, exchange worker has pending assignments: " +
+                        exchFut.exchangeId());
+
+                break;
+            }
+
+            // If partition belongs to local node.
+            if (cctx.affinity().localNode(p, topVer)) {
+                GridDhtLocalPartition part = top.localPartition(p, topVer, true);
+
+                assert part != null;
+                assert part.id() == p;
+
+                if (part.state() != MOVING) {
+                    if (log.isDebugEnabled())
+                        log.debug("Skipping partition assignment (state is not MOVING): " + part);
+
+                    continue; // For.
+                }
+
+                Collection<ClusterNode> picked = pickedOwners(p, topVer);
+
+                if (picked.isEmpty()) {
+                    top.own(part);
+
+                    if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_DATA_LOST)) {
+                        DiscoveryEvent discoEvt = exchFut.discoveryEvent();
+
+                        cctx.events().addPreloadEvent(p,
+                            EVT_CACHE_REBALANCE_PART_DATA_LOST, discoEvt.eventNode(),
+                            discoEvt.type(), discoEvt.timestamp());
                     }
-                });
+
+                    if (log.isDebugEnabled())
+                        log.debug("Owning partition as there are no other owners: " + part);
+                }
+                else {
+                    ClusterNode n = F.rand(picked);
+
+                    GridDhtPartitionDemandMessage msg = assigns.get(n);
+
+                    if (msg == null) {
+                        assigns.put(n, msg = new GridDhtPartitionDemandMessage(
+                            top.updateSequence(),
+                            exchFut.exchangeId().topologyVersion(),
+                            cctx.cacheId()));
+                    }
+
+                    msg.addPartition(p);
+                }
             }
         }
-        else
-            startFut.onDone(err);
+
+        return assigns;
     }
 
     /** {@inheritDoc} */
@@ -267,24 +366,77 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
         topVer.set(topVer0);
     }
 
-    /** {@inheritDoc} */
-    @Override public void onExchangeFutureAdded() {
-        demandPool.onExchangeFutureAdded();
+    /**
+     * @param p Partition.
+     * @param topVer Topology version.
+     * @return Picked owners.
+     */
+    private Collection<ClusterNode> pickedOwners(int p, AffinityTopologyVersion topVer) {
+        Collection<ClusterNode> affNodes = cctx.affinity().nodes(p, topVer);
+
+        int affCnt = affNodes.size();
+
+        Collection<ClusterNode> rmts = remoteOwners(p, topVer);
+
+        int rmtCnt = rmts.size();
+
+        if (rmtCnt <= affCnt)
+            return rmts;
+
+        List<ClusterNode> sorted = new ArrayList<>(rmts);
+
+        // Sort in descending order, so nodes with higher order will be first.
+        Collections.sort(sorted, CU.nodeComparator(false));
+
+        // Pick newest nodes.
+        return sorted.subList(0, affCnt);
+    }
+
+    /**
+     * @param p Partition.
+     * @param topVer Topology version.
+     * @return Nodes owning this partition.
+     */
+    private Collection<ClusterNode> remoteOwners(int p, AffinityTopologyVersion topVer) {
+        return F.view(cctx.dht().topology().owners(p, topVer), F.remoteNodes(cctx.nodeId()));
     }
 
     /** {@inheritDoc} */
-    @Override public void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) {
-        demandPool.updateLastExchangeFuture(lastFut);
+    public void handleSupplyMessage(int idx, UUID id, final GridDhtPartitionSupplyMessageV2 s) {
+        if (!enterBusy())
+            return;
+
+        try {
+            demandLock.readLock().lock();
+            try {
+                demander.handleSupplyMessage(idx, id, s);
+            }
+            finally {
+                demandLock.readLock().unlock();
+            }
+        }
+        finally {
+            leaveBusy();
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
-        return demandPool.assign(exchFut);
+    public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage d) {
+        if (!enterBusy())
+            return;
+
+        try {
+            supplier.handleDemandMessage(idx, id, d);
+        }
+        finally {
+            leaveBusy();
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload) {
-        demandPool.addAssignments(assignments, forcePreload);
+    @Override public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments,
+        boolean forcePreload, Collection<String> caches, int cnt) {
+        return demander.addAssignments(assignments, forcePreload, caches, cnt);
     }
 
     /**
@@ -296,7 +448,12 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> syncFuture() {
-        return cctx.kernalContext().clientNode() ? startFut : demandPool.syncFuture();
+        return cctx.kernalContext().clientNode() ? startFut : demander.syncFuture();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Boolean> rebalanceFuture() {
+        return cctx.kernalContext().clientNode() ? new GridFinishedFuture<>(true) : demander.rebalanceFuture();
     }
 
     /**
@@ -580,12 +737,19 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
     /** {@inheritDoc} */
     @Override public void forcePreload() {
-        demandPool.forcePreload();
+        demander.forcePreload();
     }
 
     /** {@inheritDoc} */
     @Override public void unwindUndeploys() {
-        demandPool.unwindUndeploys();
+        demandLock.writeLock().lock();
+
+        try {
+            cctx.deploy().unwind(cctx);
+        }
+        finally {
+            demandLock.writeLock().unlock();
+        }
     }
 
     /**
@@ -607,6 +771,44 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     }
 
     /** {@inheritDoc} */
+    @Override public void evictPartitionAsync(GridDhtLocalPartition part) {
+        partsToEvict.add(part);
+
+        if (partsEvictOwning.get() == 0 && partsEvictOwning.compareAndSet(0, 1)) {
+            cctx.closures().callLocalSafe(new GPC<Boolean>() {
+                @Override public Boolean call() {
+                    boolean locked = true;
+
+                    while (locked || !partsToEvict.isEmptyx()) {
+                        if (!locked && !partsEvictOwning.compareAndSet(0, 1))
+                            return false;
+
+                        try {
+                            GridDhtLocalPartition part = partsToEvict.poll();
+
+                            if (part != null)
+                                part.tryEvict();
+                        }
+                        finally {
+                            if (!partsToEvict.isEmptyx())
+                                locked = true;
+                            else {
+                                boolean res = partsEvictOwning.compareAndSet(1, 0);
+
+                                assert res;
+
+                                locked = false;
+                            }
+                        }
+                    }
+
+                    return true;
+                }
+            }, /*system pool*/ true);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void dumpDebugInfo() {
         if (!forceKeyFuts.isEmpty()) {
             U.warn(log, "Pending force key futures [cache=" + cctx.name() +"]:");
@@ -621,6 +823,8 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
             for (GridDhtAssignmentFetchFuture fut : pendingAssignmentFetchFuts.values())
                 U.warn(log, ">>> " + fut);
         }
+
+        supplier.dumpDebugInfo();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 7c5e97c..810bd8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1292,6 +1292,9 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                                         catch (IgniteCheckedException e) {
                                             U.error(log, "Failed to remove count down latch: " + latch0.name(), e);
                                         }
+                                        finally {
+                                            ctx.cache().context().txContextReset();
+                                        }
                                     }
                                 });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index 26a41de..9315d7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -696,7 +696,9 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                         if (log.isDebugEnabled())
                             U.warn(log, "Received response for unknown child job (was job presumed failed?): " + res);
 
-                        return;
+                        selfOccupied = true;
+
+                        continue;
                     }
 
                     // Only process 1st response and ignore following ones. This scenario

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridTuple4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridTuple4.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridTuple4.java
index 835cdcb..c95a859 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridTuple4.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridTuple4.java
@@ -239,7 +239,7 @@ public class GridTuple4<V1, V2, V3, V4> implements Iterable<Object>, Externaliza
 
         GridTuple4<?, ?, ?, ?> t = (GridTuple4<?, ?, ?, ?>)o;
 
-        return F.eq(val1, t.val2) && F.eq(val2, t.val2) && F.eq(val3, t.val3) && F.eq(val4, t.val4);
+        return F.eq(val1, t.val1) && F.eq(val2, t.val2) && F.eq(val3, t.val3) && F.eq(val4, t.val4);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 6254605..854ce95 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1956,7 +1956,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * <p>
      * This method is intended for test purposes only.
      */
-    void simulateNodeFailure() {
+    protected void simulateNodeFailure() {
         impl.simulateNodeFailure();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 1b2b84d..f4423f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -87,7 +87,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     }
 
     /** Test key 1. */
-    public static class TestKey implements Externalizable {
+    protected static class TestKey implements Externalizable {
         /** Field. */
         @QuerySqlField(index = true)
         private String field;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
index cadd03f..fe0b84e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheTxNodeFailureSelfTest.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
@@ -46,12 +51,6 @@ import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionRollbackException;
 
-import javax.cache.CacheException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -192,13 +191,13 @@ public class GridCacheTxNodeFailureSelfTest extends GridCommonAbstractTest {
         boolean backup,
         final boolean commit
     ) throws Exception {
-        startGrids(gridCount());
-        awaitPartitionMapExchange();
+        try {
+            startGrids(gridCount());
+            awaitPartitionMapExchange();
 
-        for (int i = 0; i < gridCount(); i++)
-            info("Grid " + i + ": " + ignite(i).cluster().localNode().id());
+            for (int i = 0; i < gridCount(); i++)
+                info("Grid " + i + ": " + ignite(i).cluster().localNode().id());
 
-        try {
             final Ignite ignite = ignite(0);
 
             final IgniteCache<Object, Object> cache = ignite.cache(null).withNoRetries();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java
new file mode 100644
index 0000000..7759c70
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingAsyncSelfTest.java
@@ -0,0 +1,68 @@
+/*
+ *  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.rebalancing;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+
+/**
+ *
+ */
+public class GridCacheRebalancingAsyncSelfTest extends GridCacheRebalancingSyncSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration iCfg = super.getConfiguration(gridName);
+
+        for (CacheConfiguration cacheCfg : iCfg.getCacheConfiguration()) {
+            cacheCfg.setRebalanceMode(CacheRebalanceMode.ASYNC);
+        }
+
+        return iCfg;
+    }
+
+    /**
+     * @throws Exception Exception.
+     */
+    public void testNodeFailedAtRebalancing() throws Exception {
+        Ignite ignite = startGrid(0);
+
+        generateData(ignite, 0, 0);
+
+        log.info("Preloading started.");
+
+        startGrid(1);
+
+        GridDhtPartitionDemander.RebalanceFuture fut = (GridDhtPartitionDemander.RebalanceFuture)grid(1).context().
+            cache().internalCache(CACHE_NAME_DHT_REPLICATED).preloader().rebalanceFuture();
+
+        fut.get();
+
+        U.sleep(10);
+
+        ((TestTcpDiscoverySpi)grid(1).configuration().getDiscoverySpi()).simulateNodeFailure();
+
+        waitForRebalancing(0, 3);
+
+        checkSupplyContextMapIsEmpty();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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
new file mode 100644
index 0000000..8c5cd40
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
@@ -0,0 +1,506 @@
+/*
+ *  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.rebalancing;
+
+import java.util.Map;
+import java.util.Random;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander;
+import org.apache.ignite.internal.util.typedef.G;
+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;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static int TEST_SIZE = 100_000;
+
+    /** partitioned cache name. */
+    protected static String CACHE_NAME_DHT_PARTITIONED = "cacheP";
+
+    /** partitioned cache 2 name. */
+    protected static String CACHE_NAME_DHT_PARTITIONED_2 = "cacheP2";
+
+    /** replicated cache name. */
+    protected static String CACHE_NAME_DHT_REPLICATED = "cacheR";
+
+    /** replicated cache 2 name. */
+    protected static String CACHE_NAME_DHT_REPLICATED_2 = "cacheR2";
+
+    /** */
+    private volatile boolean concurrentStartFinished;
+
+    /** */
+    private volatile boolean concurrentStartFinished2;
+
+    /** */
+    private volatile boolean concurrentStartFinished3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration iCfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)iCfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)iCfg.getDiscoverySpi()).setForceServerMode(true);
+
+        if (getTestGridName(10).equals(gridName))
+            iCfg.setClientMode(true);
+
+        CacheConfiguration<Integer, Integer> cachePCfg = new CacheConfiguration<>();
+
+        cachePCfg.setName(CACHE_NAME_DHT_PARTITIONED);
+        cachePCfg.setCacheMode(CacheMode.PARTITIONED);
+        cachePCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cachePCfg.setBackups(1);
+        cachePCfg.setRebalanceBatchSize(1);
+        cachePCfg.setRebalanceBatchesPrefetchCount(1);
+        cachePCfg.setRebalanceOrder(2);
+
+        CacheConfiguration<Integer, Integer> cachePCfg2 = new CacheConfiguration<>();
+
+        cachePCfg2.setName(CACHE_NAME_DHT_PARTITIONED_2);
+        cachePCfg2.setCacheMode(CacheMode.PARTITIONED);
+        cachePCfg2.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cachePCfg2.setBackups(1);
+        cachePCfg2.setRebalanceOrder(2);
+        //cachePCfg2.setRebalanceDelay(5000);//Known issue, possible deadlock in case of low priority cache rebalancing delayed.
+
+        CacheConfiguration<Integer, Integer> cacheRCfg = new CacheConfiguration<>();
+
+        cacheRCfg.setName(CACHE_NAME_DHT_REPLICATED);
+        cacheRCfg.setCacheMode(CacheMode.REPLICATED);
+        cacheRCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cacheRCfg.setRebalanceBatchSize(1);
+        cacheRCfg.setRebalanceBatchesPrefetchCount(Integer.MAX_VALUE);
+        ((TcpCommunicationSpi)iCfg.getCommunicationSpi()).setSharedMemoryPort(-1);//Shmem fail fix for Integer.MAX_VALUE.
+
+        CacheConfiguration<Integer, Integer> cacheRCfg2 = new CacheConfiguration<>();
+
+        cacheRCfg2.setName(CACHE_NAME_DHT_REPLICATED_2);
+        cacheRCfg2.setCacheMode(CacheMode.REPLICATED);
+        cacheRCfg2.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cacheRCfg2.setRebalanceOrder(4);
+
+        iCfg.setCacheConfiguration(cachePCfg, cachePCfg2, cacheRCfg, cacheRCfg2);
+
+        iCfg.setRebalanceThreadPoolSize(2);
+
+        return iCfg;
+    }
+
+    /**
+     * @param ignite Ignite.
+     */
+    protected void generateData(Ignite ignite, int from, int iter) {
+        generateData(ignite, CACHE_NAME_DHT_PARTITIONED, from, iter);
+        generateData(ignite, CACHE_NAME_DHT_PARTITIONED_2, from, iter);
+        generateData(ignite, CACHE_NAME_DHT_REPLICATED, from, iter);
+        generateData(ignite, CACHE_NAME_DHT_REPLICATED_2, from, iter);
+    }
+
+    /**
+     * @param ignite Ignite.
+     */
+    protected void generateData(Ignite ignite, String name, int from, int iter) {
+        for (int i = from; i < from + TEST_SIZE; i++) {
+            if (i % (TEST_SIZE / 10) == 0)
+                log.info("Prepared " + i * 100 / (TEST_SIZE) + "% entries (" + TEST_SIZE + ").");
+
+            ignite.cache(name).put(i, i + name.hashCode() + iter);
+        }
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @throws IgniteCheckedException Exception.
+     */
+    protected void checkData(Ignite ignite, int from, int iter) throws IgniteCheckedException {
+        checkData(ignite, CACHE_NAME_DHT_PARTITIONED, from, iter);
+        checkData(ignite, CACHE_NAME_DHT_PARTITIONED_2, from, iter);
+        checkData(ignite, CACHE_NAME_DHT_REPLICATED, from, iter);
+        checkData(ignite, CACHE_NAME_DHT_REPLICATED_2, from, iter);
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @param name Cache name.
+     * @throws IgniteCheckedException Exception.
+     */
+    protected void checkData(Ignite ignite, String name, int from, int iter) throws IgniteCheckedException {
+        for (int i = from; i < from + TEST_SIZE; i++) {
+            if (i % (TEST_SIZE / 10) == 0)
+                log.info("<" + name + "> Checked " + i * 100 / (TEST_SIZE) + "% entries (" + TEST_SIZE + ").");
+
+            assert ignite.cache(name).get(i) != null && ignite.cache(name).get(i).equals(i + name.hashCode() + iter) :
+                i + " value " + (i + name.hashCode() + iter) + " does not match (" + ignite.cache(name).get(i) + ")";
+        }
+    }
+
+    /**
+     * @throws Exception Exception
+     */
+    public void testSimpleRebalancing() throws Exception {
+        Ignite ignite = startGrid(0);
+
+        generateData(ignite, 0, 0);
+
+        log.info("Preloading started.");
+
+        long start = System.currentTimeMillis();
+
+        startGrid(1);
+
+        waitForRebalancing(0, 2);
+        waitForRebalancing(1, 2);
+
+        stopGrid(0);
+
+        waitForRebalancing(1, 3);
+
+        startGrid(2);
+
+        waitForRebalancing(1, 4);
+        waitForRebalancing(2, 4);
+
+        stopGrid(2);
+
+        waitForRebalancing(1, 5);
+
+        long spend = (System.currentTimeMillis() - start) / 1000;
+
+        checkData(grid(1), 0, 0);
+
+        log.info("Spend " + spend + " seconds to rebalance entries.");
+    }
+
+    /**
+     * @throws Exception Exception
+     */
+    public void testLoadRebalancing() throws Exception {
+        final Ignite ignite = startGrid(0);
+
+        startGrid(1);
+
+        generateData(ignite, CACHE_NAME_DHT_PARTITIONED, 0, 0);
+
+        log.info("Preloading started.");
+
+        long start = System.currentTimeMillis();
+
+        concurrentStartFinished = false;
+
+        Thread t1 = new Thread() {
+            @Override public void run() {
+                Random rdm = new Random();
+
+                while (!concurrentStartFinished) {
+                    for (int i = 0; i < TEST_SIZE; i++) {
+                        if (i % (TEST_SIZE / 10) == 0)
+                            log.info("Prepared " + i * 100 / (TEST_SIZE) + "% entries (" + TEST_SIZE + ").");
+
+                        int ii = rdm.nextInt(TEST_SIZE);
+
+                        ignite.cache(CACHE_NAME_DHT_PARTITIONED).put(ii, ii + CACHE_NAME_DHT_PARTITIONED.hashCode());
+                    }
+                }
+            }
+        };
+
+        Thread t2 = new Thread() {
+            @Override public void run() {
+                while (!concurrentStartFinished) {
+                    try {
+                        checkData(ignite, CACHE_NAME_DHT_PARTITIONED, 0, 0);
+                    }
+                    catch (IgniteCheckedException e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        };
+
+        t1.start();
+        t2.start();
+
+        startGrid(2);
+        startGrid(3);
+
+        stopGrid(2);
+
+        startGrid(4);
+
+        waitForRebalancing(3, 6);
+        waitForRebalancing(4, 6);
+
+        concurrentStartFinished = true;
+
+        awaitPartitionMapExchange(true);
+
+        checkSupplyContextMapIsEmpty();
+
+        t1.join();
+        t2.join();
+
+        long spend = (System.currentTimeMillis() - start) / 1000;
+
+        info("Time to rebalance entries: " + spend);
+    }
+
+    /**
+     * @param id Node id.
+     * @param major Major ver.
+     * @param minor Minor ver.
+     * @throws IgniteCheckedException Exception.
+     */
+    protected void waitForRebalancing(int id, int major, int minor) throws IgniteCheckedException {
+        waitForRebalancing(id, new AffinityTopologyVersion(major, minor));
+    }
+
+    /**
+     * @param id Node id.
+     * @param major Major ver.
+     * @throws IgniteCheckedException Exception.
+     */
+    protected void waitForRebalancing(int id, int major) throws IgniteCheckedException {
+        waitForRebalancing(id, new AffinityTopologyVersion(major));
+    }
+
+    /**
+     * @param id Node id.
+     * @param top Topology version.
+     * @throws IgniteCheckedException
+     */
+    protected void waitForRebalancing(int id, AffinityTopologyVersion top) throws IgniteCheckedException {
+        boolean finished = false;
+
+        while (!finished) {
+            finished = true;
+
+            for (GridCacheAdapter c : grid(id).context().cache().internalCaches()) {
+                GridDhtPartitionDemander.RebalanceFuture fut = (GridDhtPartitionDemander.RebalanceFuture)c.preloader().rebalanceFuture();
+                if (fut.topologyVersion() == null || !fut.topologyVersion().equals(top)) {
+                    finished = false;
+
+                    break;
+                }
+                else if (!fut.get()) {
+                    finished = false;
+
+                    log.warning("Rebalancing finished with missed partitions.");
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    protected void checkSupplyContextMapIsEmpty() {
+        for (Ignite g : G.allGrids()) {
+            for (GridCacheAdapter c : ((IgniteEx)g).context().cache().internalCaches()) {
+
+                Object supplier = U.field(c.preloader(), "supplier");
+
+                Map map = U.field(supplier, "scMap");
+
+                synchronized (map) {
+                    assert map.isEmpty();
+                }
+            }
+        }
+    }
+
+    @Override protected long getTestTimeout() {
+        return 5 * 60_000;
+    }
+
+    /**
+     * @throws Exception
+     */
+    public void testComplexRebalancing() throws Exception {
+        final Ignite ignite = startGrid(0);
+
+        generateData(ignite, 0, 0);
+
+        log.info("Preloading started.");
+
+        long start = System.currentTimeMillis();
+
+        concurrentStartFinished = false;
+        concurrentStartFinished2 = false;
+        concurrentStartFinished3 = false;
+
+        Thread t1 = new Thread() {
+            @Override public void run() {
+                try {
+                    startGrid(1);
+                    startGrid(2);
+
+                    while (!concurrentStartFinished2) {
+                        U.sleep(10);
+                    }
+
+                    waitForRebalancing(0, 5, 0);
+                    waitForRebalancing(1, 5, 0);
+                    waitForRebalancing(2, 5, 0);
+                    waitForRebalancing(3, 5, 0);
+                    waitForRebalancing(4, 5, 0);
+
+                    //New cache should start rebalancing.
+                    CacheConfiguration<Integer, Integer> cacheRCfg = new CacheConfiguration<>();
+
+                    cacheRCfg.setName(CACHE_NAME_DHT_PARTITIONED + "_NEW");
+                    cacheRCfg.setCacheMode(CacheMode.PARTITIONED);
+                    cacheRCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+
+                    grid(0).getOrCreateCache(cacheRCfg);
+
+                    while (!concurrentStartFinished3) {
+                        U.sleep(10);
+                    }
+
+                    concurrentStartFinished = true;
+                }
+                catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+
+        Thread t2 = new Thread() {
+            @Override public void run() {
+                try {
+                    startGrid(3);
+                    startGrid(4);
+
+                    concurrentStartFinished2 = true;
+                }
+                catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+
+        Thread t3 = new Thread() {
+            @Override public void run() {
+                generateData(ignite, 0, 1);
+
+                concurrentStartFinished3 = true;
+            }
+        };
+
+        t1.start();
+        t2.start();// Should cancel t1 rebalancing.
+        t3.start();
+
+        t1.join();
+        t2.join();
+        t3.join();
+
+        waitForRebalancing(0, 5, 1);
+        waitForRebalancing(1, 5, 1);
+        waitForRebalancing(2, 5, 1);
+        waitForRebalancing(3, 5, 1);
+        waitForRebalancing(4, 5, 1);
+
+        awaitPartitionMapExchange(true);
+
+        checkSupplyContextMapIsEmpty();
+
+        checkData(grid(4), 0, 1);
+
+        final Ignite ignite3 = grid(3);
+
+        Thread t4 = new Thread() {
+            @Override public void run() {
+                generateData(ignite3, 0, 2);
+
+            }
+        };
+
+        t4.start();
+
+        stopGrid(1);
+
+        waitForRebalancing(0, 6);
+        waitForRebalancing(2, 6);
+        waitForRebalancing(3, 6);
+        waitForRebalancing(4, 6);
+
+        awaitPartitionMapExchange(true);
+
+        checkSupplyContextMapIsEmpty();
+
+        stopGrid(0);
+
+        waitForRebalancing(2, 7);
+        waitForRebalancing(3, 7);
+        waitForRebalancing(4, 7);
+
+        awaitPartitionMapExchange(true);
+
+        checkSupplyContextMapIsEmpty();
+
+        stopGrid(2);
+
+        waitForRebalancing(3, 8);
+        waitForRebalancing(4, 8);
+
+        awaitPartitionMapExchange(true);
+
+        checkSupplyContextMapIsEmpty();
+
+        t4.join();
+
+        stopGrid(3);
+
+        waitForRebalancing(4, 9);
+
+        checkSupplyContextMapIsEmpty();
+
+        long spend = (System.currentTimeMillis() - start) / 1000;
+
+        checkData(grid(4), 0, 2);
+
+        log.info("Spend " + spend + " seconds to rebalance entries.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java
new file mode 100644
index 0000000..831e82d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingUnmarshallingFailedSelfTest.java
@@ -0,0 +1,147 @@
+/*
+ *  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.rebalancing;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+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;
+
+/**
+ *
+ */
+public class GridCacheRebalancingUnmarshallingFailedSelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** partitioned cache name. */
+    protected static String CACHE = "cache";
+
+    /** Allows to change behavior of readExternal method. */
+    protected static AtomicInteger readCnt = new AtomicInteger();
+
+    /** Test key 1. */
+    private static class TestKey implements Externalizable {
+        /** Field. */
+        @QuerySqlField(index = true)
+        private String field;
+
+        /**
+         * @param field Test key 1.
+         */
+        public TestKey(String field) {
+            this.field = field;
+        }
+
+        /** Test key 1. */
+        public TestKey() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestKey key = (TestKey)o;
+
+            return !(field != null ? !field.equals(key.field) : key.field != null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return field != null ? field.hashCode() : 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeObject(field);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            field = (String)in.readObject();
+
+            if (readCnt.decrementAndGet() <= 0)
+                throw new IOException("Class can not be unmarshalled.");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration iCfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<TestKey, Integer> cfg = new CacheConfiguration<>();
+
+        cfg.setName(CACHE);
+        cfg.setCacheMode(CacheMode.PARTITIONED);
+        cfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cfg.setBackups(0);
+
+        iCfg.setCacheConfiguration(cfg);
+
+        return iCfg;
+    }
+
+    /**
+     * @throws Exception e.
+     */
+    public void test() throws Exception {
+        readCnt.set(Integer.MAX_VALUE);
+
+        startGrid(0);
+
+        for (int i = 0; i < 100; i++) {
+            grid(0).cache(CACHE).put(new TestKey(String.valueOf(i)), i);
+        }
+
+        readCnt.set(1);
+
+        startGrid(1);
+
+        readCnt.set(Integer.MAX_VALUE);
+
+        for (int i = 0; i < 50; i++) {
+            assert grid(1).cache(CACHE).get(new TestKey(String.valueOf(i))) != null;
+        }
+
+        stopGrid(0);
+
+        for (int i = 50; i < 100; i++) {
+            assert grid(1).cache(CACHE).get(new TestKey(String.valueOf(i))) == null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
index c4ad169..64f1495 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
@@ -142,26 +142,6 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception If failed.
-     */
-    public void testSingleZeroPoolSize() throws Exception {
-        preloadMode = SYNC;
-        poolSize = 0;
-
-        try {
-            startGrid(1);
-
-            assert false : "Grid should have been failed to start.";
-        }
-        catch (IgniteCheckedException e) {
-            info("Caught expected exception: " + e);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
      * @throws Exception If test failed.
      */
     public void testIntegrity() throws Exception {
@@ -602,4 +582,4 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             // No-op.
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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 0280e9c..51d8a2d 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
@@ -511,23 +511,6 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
-     */
-    private static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
-        /** */
-        private boolean ignorePingResponse;
-
-        /** {@inheritDoc} */
-        protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, long timeout) throws IOException,
-            IgniteCheckedException {
-            if (msg instanceof TcpDiscoveryPingResponse && ignorePingResponse)
-                return;
-            else
-                super.writeToSocket(sock, msg, timeout);
-        }
-    }
-
-    /**
      * @throws Exception If any error occurs.
      */
     public void testNodeAdded() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java
new file mode 100644
index 0000000..dbc54bc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java
@@ -0,0 +1,46 @@
+/*
+ *  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.spi.discovery.tcp;
+
+import java.io.IOException;
+import java.net.Socket;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse;
+
+/**
+ *
+ */
+public class TestTcpDiscoverySpi extends TcpDiscoverySpi {
+    /** */
+    public boolean ignorePingResponse;
+
+    /** {@inheritDoc} */
+    protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, long timeout) throws IOException,
+        IgniteCheckedException {
+        if (msg instanceof TcpDiscoveryPingResponse && ignorePingResponse)
+            return;
+        else
+            super.writeToSocket(sock, msg, timeout);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void simulateNodeFailure() {
+        super.simulateNodeFailure();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index d133a84..41d4b4a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -77,6 +77,7 @@ import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -1228,7 +1229,7 @@ public abstract class GridAbstractTest extends TestCase {
 
         cfg.setCommunicationSpi(commSpi);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        TcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi();
 
         if (isDebug()) {
             discoSpi.setMaxMissedHeartbeats(Integer.MAX_VALUE);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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 28d5c73..71f3ee3 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
@@ -63,6 +63,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
+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.colocated.GridDhtColocatedCache;
@@ -414,6 +416,15 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      */
     @SuppressWarnings("BusyWait")
     protected void awaitPartitionMapExchange() throws InterruptedException {
+        awaitPartitionMapExchange(false);
+    }
+
+    /**
+     * @param waitEvicts If {@code true} will wait for evictions finished.
+     * @throws InterruptedException If interrupted.
+     */
+    @SuppressWarnings("BusyWait")
+    protected void awaitPartitionMapExchange(boolean waitEvicts) throws InterruptedException {
         for (Ignite g : G.allGrids()) {
             IgniteKernal g0 = (IgniteKernal)g;
 
@@ -451,7 +462,10 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
                                 int actual = owners.size();
 
-                                if (affNodes.size() != owners.size() || !affNodes.containsAll(owners)) {
+                                GridDhtLocalPartition loc = top.localPartition(p, readyVer, false);
+
+                                if (affNodes.size() != owners.size() || !affNodes.containsAll(owners) ||
+                                    (waitEvicts && loc != null && loc.state() == GridDhtPartitionState.RENTING)) {
                                     LT.warn(log(), null, "Waiting for topology map update [" +
                                         "grid=" + g.name() +
                                         ", cache=" + cfg.getName() +
@@ -484,7 +498,9 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                 if (i == 0)
                                     start = System.currentTimeMillis();
 
-                                if (System.currentTimeMillis() - start > 30_000)
+                                if (System.currentTimeMillis() - start > 30_000) {
+                                    U.dumpThreads(log);
+
                                     throw new IgniteException("Timeout of waiting for topology map update [" +
                                         "grid=" + g.name() +
                                         ", cache=" + cfg.getName() +
@@ -493,6 +509,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                         ", p=" + p +
                                         ", readVer=" + readyVer +
                                         ", locNode=" + g.cluster().localNode() + ']');
+                                }
 
                                 Thread.sleep(200); // Busy wait.
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index 796c531..c3c3659 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -49,6 +49,8 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNea
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearPartitionedP2PEnabledByteArrayValuesSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePutArrayValueSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteTxReentryNearSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingAsyncSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalancingSyncSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheDaemonNodeReplicatedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicGetAndTransformStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedBasicApiTest;
@@ -135,6 +137,8 @@ public class IgniteCacheTestSuite3 extends TestSuite {
         suite.addTestSuite(IgniteTxReentryColocatedSelfTest.class);
 
         suite.addTestSuite(GridCacheOrderedPreloadingSelfTest.class);
+        suite.addTestSuite(GridCacheRebalancingSyncSelfTest.class);
+        suite.addTestSuite(GridCacheRebalancingAsyncSelfTest.class);
 
         // Test for byte array value special case.
         suite.addTestSuite(GridCacheLocalByteArrayValuesSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
index 0226046..582bfe3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
@@ -107,25 +107,33 @@ public class GridOrderedMessageCancelSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testTask() throws Exception {
+        Map map = U.field(((IgniteKernal)grid(0)).context().io(), "msgSetMap");
+
+        int initSize =  map.size();
+
         ComputeTaskFuture<?> fut = executeAsync(compute(grid(0).cluster().forRemotes()), Task.class, null);
 
-        testMessageSet(fut);
+        testMessageSet(fut, initSize, map);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testTaskException() throws Exception {
+        Map map = U.field(((IgniteKernal)grid(0)).context().io(), "msgSetMap");
+
+        int initSize =  map.size();
+
         ComputeTaskFuture<?> fut = executeAsync(compute(grid(0).cluster().forRemotes()), FailTask.class, null);
 
-        testMessageSet(fut);
+        testMessageSet(fut, initSize, map);
     }
 
     /**
      * @param fut Future to cancel.
      * @throws Exception If failed.
      */
-    private void testMessageSet(IgniteFuture<?> fut) throws Exception {
+    private void testMessageSet(IgniteFuture<?> fut, int initSize, Map map) throws Exception {
         cancelLatch.await();
 
         assertTrue(fut.cancel());
@@ -134,11 +142,9 @@ public class GridOrderedMessageCancelSelfTest extends GridCommonAbstractTest {
 
         assertTrue(U.await(finishLatch, 5000, MILLISECONDS));
 
-        Map map = U.field(((IgniteKernal)grid(0)).context().io(), "msgSetMap");
-
         info("Map: " + map);
 
-        assertTrue(map.isEmpty());
+        assertEquals(map.size(), initSize);
     }
 
     /**


[29/46] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1753-1282
Commit: 6b158c9ddf61159ce8f622832713e0bc508d1fb8
Parents: d567d259 388a892
Author: sboikov <sb...@gridgain.com>
Authored: Wed Nov 11 10:20:22 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Nov 11 10:20:22 2015 +0300

----------------------------------------------------------------------
 .../cache/store/CacheStoreSessionListener.java  |   9 +-
 .../jdbc/CacheJdbcStoreSessionListener.java     |   8 +-
 .../GridCachePartitionExchangeManager.java      |  34 ++--
 .../dht/preloader/GridDhtPartitionDemander.java |  25 ++-
 .../store/GridCacheStoreManagerAdapter.java     |  26 ++-
 .../GridCacheAtomicReferenceImpl.java           |   2 +-
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../org/apache/ignite/stream/StreamAdapter.java |   2 +-
 .../store/GridCacheBalancingStoreSelfTest.java  | 181 ++++++++++++++++++-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |  20 +-
 modules/rest-http/pom.xml                       |  11 ++
 .../yardstick/config/ignite-store-config.xml    |  31 ++--
 modules/zookeeper/pom.xml                       |   2 +-
 13 files changed, 293 insertions(+), 60 deletions(-)
----------------------------------------------------------------------



[30/46] ignite git commit: IGNITE-1888 Start org.apache.ignite.yardstick.IgniteNode with Spring Application context.

Posted by ak...@apache.org.
 IGNITE-1888 Start org.apache.ignite.yardstick.IgniteNode with Spring Application context.


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

Branch: refs/heads/ignite-1753-1282
Commit: 92881e07a561f25db5edd29ca5b9adc2c060d4b5
Parents: 6b158c9
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Nov 11 14:29:22 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Nov 11 14:29:22 2015 +0700

----------------------------------------------------------------------
 .../org/apache/ignite/yardstick/IgniteNode.java | 23 ++++++++++++++------
 1 file changed, 16 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/92881e07/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
index ea3bd07..93c80d3 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -22,6 +22,7 @@ import java.net.URL;
 import java.util.Map;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteSpring;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -30,9 +31,11 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.springframework.beans.BeansException;
 import org.springframework.beans.factory.xml.XmlBeanDefinitionReader;
+import org.springframework.context.ApplicationContext;
 import org.springframework.context.support.GenericApplicationContext;
 import org.springframework.core.io.UrlResource;
 import org.yardstickframework.BenchmarkConfiguration;
@@ -73,15 +76,20 @@ public class IgniteNode implements BenchmarkServer {
 
         BenchmarkUtils.jcommander(cfg.commandLineArguments(), args, "<ignite-node>");
 
-        IgniteConfiguration c = loadConfiguration(args.configuration());
+        IgniteBiTuple<IgniteConfiguration, ? extends ApplicationContext> tup = loadConfiguration(args.configuration());
+
+        IgniteConfiguration c = tup.get1();
 
         assert c != null;
 
+        ApplicationContext appCtx = tup.get2();
+
+        assert appCtx != null;
+
         for (CacheConfiguration cc : c.getCacheConfiguration()) {
             // IgniteNode can not run in CLIENT_ONLY mode,
             // except the case when it's used inside IgniteAbstractBenchmark.
-            boolean cl = args.isClientOnly() && !args.isNearCache() && !clientMode ?
-                false : args.isClientOnly();
+            boolean cl = args.isClientOnly() && (args.isNearCache() || clientMode);
 
             if (cl)
                 c.setClientMode(true);
@@ -141,15 +149,16 @@ public class IgniteNode implements BenchmarkServer {
 
         c.setCommunicationSpi(commSpi);
 
-        ignite = Ignition.start(c);
+        ignite = IgniteSpring.start(c, appCtx);
     }
 
     /**
      * @param springCfgPath Spring configuration file path.
-     * @return Grid configuration.
+     * @return Tuple with grid configuration and Spring application context.
      * @throws Exception If failed.
      */
-    private static IgniteConfiguration loadConfiguration(String springCfgPath) throws Exception {
+    private static IgniteBiTuple<IgniteConfiguration, ? extends ApplicationContext> loadConfiguration(String springCfgPath)
+        throws Exception {
         URL url;
 
         try {
@@ -191,7 +200,7 @@ public class IgniteNode implements BenchmarkServer {
         if (cfgMap == null || cfgMap.isEmpty())
             throw new Exception("Failed to find ignite configuration in: " + url);
 
-        return cfgMap.values().iterator().next();
+        return new IgniteBiTuple<>(cfgMap.values().iterator().next(), springCtx);
     }
 
     /** {@inheritDoc} */


[03/46] ignite git commit: Ignite-1093 "Rebalancing with default parameters is very slow" fixes.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
new file mode 100644
index 0000000..35cedf9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -0,0 +1,1389 @@
+/*
+ * 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.dht.preloader;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReadWriteLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
+import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
+import org.apache.ignite.internal.util.GridLeanSet;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+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.CI2;
+import org.apache.ignite.internal.util.typedef.T2;
+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.IgnitePredicate;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_LOADED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STOPPED;
+import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING;
+import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE;
+import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRELOAD;
+
+/**
+ * Thread pool for requesting partitions from other nodes and populating local cache.
+ */
+@SuppressWarnings("NonConstantFieldWithUpperCaseName")
+public class GridDhtPartitionDemander {
+    /** */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** Preload predicate. */
+    private IgnitePredicate<GridCacheEntryInfo> preloadPred;
+
+    /** Future for preload mode {@link CacheRebalanceMode#SYNC}. */
+    @GridToStringInclude
+    private final GridFutureAdapter syncFut = new GridFutureAdapter();
+
+    /** Rebalance future. */
+    @GridToStringInclude
+    private volatile RebalanceFuture rebalanceFut;
+
+    /** Last timeout object. */
+    private AtomicReference<GridTimeoutObject> lastTimeoutObj = new AtomicReference<>();
+
+    /** Last exchange future. */
+    private volatile GridDhtPartitionsExchangeFuture lastExchangeFut;
+
+    /** Demand lock. */
+    @Deprecated//Backward compatibility. To be removed in future.
+    private final ReadWriteLock demandLock;
+
+    /** DemandWorker index. */
+    @Deprecated//Backward compatibility. To be removed in future.
+    private final AtomicInteger dmIdx = new AtomicInteger();
+
+    /** Cached rebalance topics. */
+    private final Map<Integer, Object> rebalanceTopics;
+
+    /**
+     * @param cctx Cctx.
+     * @param demandLock Demand lock.
+     */
+    public GridDhtPartitionDemander(GridCacheContext<?, ?> cctx, ReadWriteLock demandLock) {
+        assert cctx != null;
+
+        this.cctx = cctx;
+        this.demandLock = demandLock;
+
+        log = cctx.logger(getClass());
+
+        boolean enabled = cctx.rebalanceEnabled() && !cctx.kernalContext().clientNode();
+
+        rebalanceFut = new RebalanceFuture();//Dummy.
+
+        if (!enabled) {
+            // Calling onDone() immediately since preloading is disabled.
+            rebalanceFut.onDone(true);
+            syncFut.onDone();
+        }
+
+        Map<Integer, Object> tops = new HashMap<>();
+
+        for (int idx = 0; idx < cctx.gridConfig().getRebalanceThreadPoolSize(); idx++) {
+            tops.put(idx, GridCachePartitionExchangeManager.rebalanceTopic(idx));
+        }
+
+        rebalanceTopics = tops;
+    }
+
+    /**
+     * Start.
+     */
+    void start() {
+        // No-op.
+    }
+
+    /**
+     * Stop.
+     */
+    void stop() {
+        try {
+            rebalanceFut.cancel();
+        }
+        catch (Exception ex) {
+            rebalanceFut.onDone(false);
+        }
+
+        lastExchangeFut = null;
+
+        lastTimeoutObj.set(null);
+    }
+
+    /**
+     * @return Future for {@link CacheRebalanceMode#SYNC} mode.
+     */
+    IgniteInternalFuture<?> syncFuture() {
+        return syncFut;
+    }
+
+    /**
+     * @return Rebalance future.
+     */
+    IgniteInternalFuture<Boolean> rebalanceFuture() {
+        return rebalanceFut;
+    }
+
+    /**
+     * Sets preload predicate for demand pool.
+     *
+     * @param preloadPred Preload predicate.
+     */
+    void preloadPredicate(IgnitePredicate<GridCacheEntryInfo> preloadPred) {
+        this.preloadPred = preloadPred;
+    }
+
+    /**
+     * Force preload.
+     */
+    void forcePreload() {
+        GridTimeoutObject obj = lastTimeoutObj.getAndSet(null);
+
+        if (obj != null)
+            cctx.time().removeTimeoutObject(obj);
+
+        final GridDhtPartitionsExchangeFuture exchFut = lastExchangeFut;
+
+        if (exchFut != null) {
+            if (log.isDebugEnabled())
+                log.debug("Forcing rebalance event for future: " + exchFut);
+
+            exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                    cctx.shared().exchange().forcePreloadExchange(exchFut);
+                }
+            });
+        }
+        else if (log.isDebugEnabled())
+            log.debug("Ignoring force rebalance request (no topology event happened yet).");
+    }
+
+    /**
+     * @param fut Future.
+     * @return {@code True} if topology changed.
+     */
+    private boolean topologyChanged(RebalanceFuture fut) {
+        return
+            !cctx.affinity().affinityTopologyVersion().equals(fut.topologyVersion()) || // Topology already changed.
+                fut != rebalanceFut; // Same topology, but dummy exchange forced because of missing partitions.
+    }
+
+    /**
+     * @param part Partition.
+     * @param type Type.
+     * @param discoEvt Discovery event.
+     */
+    private void preloadEvent(int part, int type, DiscoveryEvent discoEvt) {
+        assert discoEvt != null;
+
+        cctx.events().addPreloadEvent(part, type, discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp());
+    }
+
+    /**
+     * @param name Cache name.
+     * @param fut Future.
+     */
+    private boolean waitForCacheRebalancing(String name, RebalanceFuture fut) throws IgniteCheckedException {
+        if (log.isDebugEnabled())
+            log.debug("Waiting for another cache to start rebalancing [cacheName=" + cctx.name() +
+                ", waitCache=" + name + ']');
+
+        RebalanceFuture wFut = (RebalanceFuture)cctx.kernalContext().cache().internalCache(name)
+            .preloader().rebalanceFuture();
+
+        if (!topologyChanged(fut) && wFut.updateSeq == fut.updateSeq) {
+            if (!wFut.get()) {
+                U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() +
+                    "] (cache rebalanced with missed partitions)");
+
+                return false;
+            }
+
+            return true;
+        }
+        else {
+            U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() +
+                "] (topology already changed)");
+
+            return false;
+        }
+    }
+
+    /**
+     * @param assigns Assignments.
+     * @param force {@code True} if dummy reassign.
+     * @param caches Rebalancing of these caches will be finished before this started.
+     * @param cnt Counter.
+     * @throws IgniteCheckedException Exception
+     */
+    Callable<Boolean> addAssignments(final GridDhtPreloaderAssignments assigns, boolean force,
+        final Collection<String> caches, int cnt) {
+        if (log.isDebugEnabled())
+            log.debug("Adding partition assignments: " + assigns);
+
+        long delay = cctx.config().getRebalanceDelay();
+
+        if (delay == 0 || force) {
+            assert assigns != null;
+
+            final RebalanceFuture oldFut = rebalanceFut;
+
+            final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, oldFut.isInitial(), cnt);
+
+            if (!oldFut.isInitial())
+                oldFut.cancel();
+            else
+                fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+                    @Override public void apply(IgniteInternalFuture<Boolean> future) {
+                        oldFut.onDone(fut.result());
+                    }
+                });
+
+            rebalanceFut = fut;
+
+            if (assigns.isEmpty()) {
+                fut.doneIfEmpty();
+
+                return null;
+            }
+
+            return new Callable<Boolean>() {
+                @Override public Boolean call() throws Exception {
+                    for (String c : caches) {
+                        if (!waitForCacheRebalancing(c, fut))
+                            return false;
+                    }
+
+                    return requestPartitions(fut, assigns);
+                }
+            };
+        }
+        else if (delay > 0) {
+            GridTimeoutObject obj = lastTimeoutObj.get();
+
+            if (obj != null)
+                cctx.time().removeTimeoutObject(obj);
+
+            final GridDhtPartitionsExchangeFuture exchFut = lastExchangeFut;
+
+            assert exchFut != null : "Delaying rebalance process without topology event.";
+
+            obj = new GridTimeoutObjectAdapter(delay) {
+                @Override public void onTimeout() {
+                    exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> f) {
+                            cctx.shared().exchange().forcePreloadExchange(exchFut);
+                        }
+                    });
+                }
+            };
+
+            lastTimeoutObj.set(obj);
+
+            cctx.time().addTimeoutObject(obj);
+        }
+
+        return null;
+    }
+
+    /**
+     * @param fut Future.
+     */
+    private boolean requestPartitions(
+        RebalanceFuture fut,
+        GridDhtPreloaderAssignments assigns
+    ) throws IgniteCheckedException {
+        for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
+            if (topologyChanged(fut))
+                return false;
+
+            final ClusterNode node = e.getKey();
+
+            GridDhtPartitionDemandMessage d = e.getValue();
+
+            fut.appendPartitions(node.id(), d.partitions());//Future preparation.
+        }
+
+        for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
+            final ClusterNode node = e.getKey();
+
+            final CacheConfiguration cfg = cctx.config();
+
+            final Collection<Integer> parts = fut.remaining.get(node.id()).get2();
+
+            GridDhtPartitionDemandMessage d = e.getValue();
+
+            //Check remote node rebalancing API version.
+            if (node.version().compareTo(GridDhtPreloader.REBALANCING_VER_2_SINCE) >= 0) {
+                U.log(log, "Starting rebalancing [cache=" + cctx.name() + ", mode=" + cfg.getRebalanceMode() +
+                    ", fromNode=" + node.id() + ", partitionsCount=" + parts.size() +
+                    ", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]");
+
+                int lsnrCnt = cctx.gridConfig().getRebalanceThreadPoolSize();
+
+                List<Set<Integer>> sParts = new ArrayList<>(lsnrCnt);
+
+                for (int cnt = 0; cnt < lsnrCnt; cnt++)
+                    sParts.add(new HashSet<Integer>());
+
+                Iterator<Integer> it = parts.iterator();
+
+                int cnt = 0;
+
+                while (it.hasNext())
+                    sParts.get(cnt++ % lsnrCnt).add(it.next());
+
+                for (cnt = 0; cnt < lsnrCnt; cnt++) {
+                    if (!sParts.get(cnt).isEmpty()) {
+
+                        // Create copy.
+                        GridDhtPartitionDemandMessage initD = new GridDhtPartitionDemandMessage(d, sParts.get(cnt));
+
+                        initD.topic(rebalanceTopics.get(cnt));
+                        initD.updateSequence(fut.updateSeq);
+                        initD.timeout(cctx.config().getRebalanceTimeout());
+
+                        synchronized (fut) {
+                            if (!fut.isDone())// Future can be already cancelled at this moment and all failovers happened.
+                                // New requests will not be covered by failovers.
+                                cctx.io().sendOrderedMessage(node,
+                                    rebalanceTopics.get(cnt), initD, cctx.ioPolicy(), initD.timeout());
+                        }
+
+                        if (log.isDebugEnabled())
+                            log.debug("Requested rebalancing [from node=" + node.id() + ", listener index=" +
+                                cnt + ", partitions count=" + sParts.get(cnt).size() +
+                                " (" + partitionsList(sParts.get(cnt)) + ")]");
+                    }
+                }
+            }
+            else {
+                U.log(log, "Starting rebalancing (old api) [cache=" + cctx.name() + ", mode=" + cfg.getRebalanceMode() +
+                    ", fromNode=" + node.id() + ", partitionsCount=" + parts.size() +
+                    ", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]");
+
+                d.timeout(cctx.config().getRebalanceTimeout());
+                d.workerId(0);//old api support.
+
+                DemandWorker dw = new DemandWorker(dmIdx.incrementAndGet(), fut);
+
+                dw.run(node, d);
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * @param c Partitions.
+     * @return String representation of partitions list.
+     */
+    private String partitionsList(Collection<Integer> c) {
+        List<Integer> s = new ArrayList<>(c);
+
+        Collections.sort(s);
+
+        StringBuilder sb = new StringBuilder();
+
+        int start = -1;
+
+        int prev = -1;
+
+        Iterator<Integer> sit = s.iterator();
+
+        while (sit.hasNext()) {
+            int p = sit.next();
+
+            if (start == -1) {
+                start = p;
+                prev = p;
+            }
+
+            if (prev < p - 1) {
+                sb.append(start);
+
+                if (start != prev)
+                    sb.append("-").append(prev);
+
+                sb.append(", ");
+
+                start = p;
+            }
+
+            if (!sit.hasNext()) {
+                sb.append(start);
+
+                if (start != p)
+                    sb.append("-").append(p);
+            }
+
+            prev = p;
+        }
+
+        return sb.toString();
+    }
+
+    /**
+     * @param idx Index.
+     * @param id Node id.
+     * @param supply Supply.
+     */
+    public void handleSupplyMessage(
+        int idx,
+        final UUID id,
+        final GridDhtPartitionSupplyMessageV2 supply
+    ) {
+        AffinityTopologyVersion topVer = supply.topologyVersion();
+
+        final RebalanceFuture fut = rebalanceFut;
+
+        ClusterNode node = cctx.node(id);
+
+        if (node == null)
+            return;
+
+        if (!fut.isActual(supply.updateSequence())) // Current future have another update sequence.
+            return; // Supple message based on another future.
+
+        if (topologyChanged(fut)) // Topology already changed (for the future that supply message based on).
+            return;
+
+        if (log.isDebugEnabled())
+            log.debug("Received supply message: " + supply);
+
+        // Check whether there were class loading errors on unmarshal
+        if (supply.classError() != null) {
+            U.warn(log, "Rebalancing from node cancelled [node=" + id +
+                "]. Class got undeployed during preloading: " + supply.classError());
+
+            fut.cancel(id);
+
+            return;
+        }
+
+        final GridDhtPartitionTopology top = cctx.dht().topology();
+
+        try {
+            // Preload.
+            for (Map.Entry<Integer, CacheEntryInfoCollection> e : supply.infos().entrySet()) {
+                int p = e.getKey();
+
+                if (cctx.affinity().localNode(p, topVer)) {
+                    GridDhtLocalPartition part = top.localPartition(p, topVer, true);
+
+                    assert part != null;
+
+                    if (part.state() == MOVING) {
+                        boolean reserved = part.reserve();
+
+                        assert reserved : "Failed to reserve partition [gridName=" +
+                            cctx.gridName() + ", cacheName=" + cctx.namex() + ", part=" + part + ']';
+
+                        part.lock();
+
+                        try {
+                            // Loop through all received entries and try to preload them.
+                            for (GridCacheEntryInfo entry : e.getValue().infos()) {
+                                if (!part.preloadingPermitted(entry.key(), entry.version())) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Preloading is not permitted for entry due to " +
+                                            "evictions [key=" + entry.key() +
+                                            ", ver=" + entry.version() + ']');
+
+                                    continue;
+                                }
+
+                                if (!preloadEntry(node, p, entry, topVer)) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Got entries for invalid partition during " +
+                                            "preloading (will skip) [p=" + p + ", entry=" + entry + ']');
+
+                                    break;
+                                }
+                            }
+
+                            boolean last = supply.last().contains(p);
+
+                            // If message was last for this partition,
+                            // then we take ownership.
+                            if (last) {
+                                top.own(part);
+
+                                fut.partitionDone(id, p);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Finished rebalancing partition: " + part);
+                            }
+                        }
+                        finally {
+                            part.unlock();
+                            part.release();
+                        }
+                    }
+                    else {
+                        fut.partitionDone(id, p);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping rebalancing partition (state is not MOVING): " + part);
+                    }
+                }
+                else {
+                    fut.partitionDone(id, p);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Skipping rebalancing partition (it does not belong on current node): " + p);
+                }
+            }
+
+            // Only request partitions based on latest topology version.
+            for (Integer miss : supply.missed()) {
+                if (cctx.affinity().localNode(miss, topVer))
+                    fut.partitionMissed(id, miss);
+            }
+
+            for (Integer miss : supply.missed())
+                fut.partitionDone(id, miss);
+
+            GridDhtPartitionDemandMessage d = new GridDhtPartitionDemandMessage(
+                supply.updateSequence(), supply.topologyVersion(), cctx.cacheId());
+
+            d.timeout(cctx.config().getRebalanceTimeout());
+
+            d.topic(rebalanceTopics.get(idx));
+
+            if (!topologyChanged(fut) && !fut.isDone()) {
+                // Send demand message.
+                cctx.io().sendOrderedMessage(node, rebalanceTopics.get(idx),
+                    d, cctx.ioPolicy(), cctx.config().getRebalanceTimeout());
+            }
+        }
+        catch (IgniteCheckedException e) {
+            if (log.isDebugEnabled())
+                log.debug("Node left during rebalancing [node=" + node.id() +
+                    ", msg=" + e.getMessage() + ']');
+        }
+        catch (IgniteSpiException e) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to send message to node (current node is stopping?) [node=" + node.id() +
+                    ", msg=" + e.getMessage() + ']');
+        }
+    }
+
+    /**
+     * @param pick Node picked for preloading.
+     * @param p Partition.
+     * @param entry Preloaded entry.
+     * @param topVer Topology version.
+     * @return {@code False} if partition has become invalid during preloading.
+     * @throws IgniteInterruptedCheckedException If interrupted.
+     */
+    private boolean preloadEntry(
+        ClusterNode pick,
+        int p,
+        GridCacheEntryInfo entry,
+        AffinityTopologyVersion topVer
+    ) throws IgniteCheckedException {
+        try {
+            GridCacheEntryEx cached = null;
+
+            try {
+                cached = cctx.dht().entryEx(entry.key());
+
+                if (log.isDebugEnabled())
+                    log.debug("Rebalancing key [key=" + entry.key() + ", part=" + p + ", node=" + pick.id() + ']');
+
+                if (cctx.dht().isIgfsDataCache() &&
+                    cctx.dht().igfsDataSpaceUsed() > cctx.dht().igfsDataSpaceMax()) {
+                    LT.error(log, null, "Failed to rebalance IGFS data cache (IGFS space size exceeded maximum " +
+                        "value, will ignore rebalance entries)");
+
+                    if (cached.markObsoleteIfEmpty(null))
+                        cached.context().cache().removeIfObsolete(cached.key());
+
+                    return true;
+                }
+
+                if (preloadPred == null || preloadPred.apply(entry)) {
+                    if (cached.initialValue(
+                        entry.value(),
+                        entry.version(),
+                        entry.ttl(),
+                        entry.expireTime(),
+                        true,
+                        topVer,
+                        cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE
+                    )) {
+                        cctx.evicts().touch(cached, topVer); // Start tracking.
+
+                        if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_OBJECT_LOADED) && !cached.isInternal())
+                            cctx.events().addEvent(cached.partition(), cached.key(), cctx.localNodeId(),
+                                (IgniteUuid)null, null, EVT_CACHE_REBALANCE_OBJECT_LOADED, entry.value(), true, null,
+                                false, null, null, null);
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Rebalancing entry is already in cache (will ignore) [key=" + cached.key() +
+                            ", part=" + p + ']');
+                }
+                else if (log.isDebugEnabled())
+                    log.debug("Rebalance predicate evaluated to false for entry (will ignore): " + entry);
+            }
+            catch (GridCacheEntryRemovedException ignored) {
+                if (log.isDebugEnabled())
+                    log.debug("Entry has been concurrently removed while rebalancing (will ignore) [key=" +
+                        cached.key() + ", part=" + p + ']');
+            }
+            catch (GridDhtInvalidPartitionException ignored) {
+                if (log.isDebugEnabled())
+                    log.debug("Partition became invalid during rebalancing (will ignore): " + p);
+
+                return false;
+            }
+        }
+        catch (IgniteInterruptedCheckedException e) {
+            throw e;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Failed to cache rebalanced entry (will stop rebalancing) [local=" +
+                cctx.nodeId() + ", node=" + pick.id() + ", key=" + entry.key() + ", part=" + p + ']', e);
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridDhtPartitionDemander.class, this);
+    }
+
+    /**
+     * Sets last exchange future.
+     *
+     * @param lastFut Last future to set.
+     */
+    void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) {
+        lastExchangeFut = lastFut;
+    }
+
+    /**
+     *
+     */
+    public static class RebalanceFuture extends GridFutureAdapter<Boolean> {
+        /** */
+        private static final long serialVersionUID = 1L;
+
+        /** Should EVT_CACHE_REBALANCE_STOPPED event be sent of not. */
+        private final boolean sendStoppedEvnt;
+
+        /** */
+        private final GridCacheContext<?, ?> cctx;
+
+        /** */
+        private final IgniteLogger log;
+
+        /** Remaining. T2: startTime, partitions */
+        private final Map<UUID, T2<Long, Collection<Integer>>> remaining = new HashMap<>();
+
+        /** Missed. */
+        private final Map<UUID, Collection<Integer>> missed = new HashMap<>();
+
+        /** Exchange future. */
+        @GridToStringExclude
+        private final GridDhtPartitionsExchangeFuture exchFut;
+
+        /** Topology version. */
+        private final AffinityTopologyVersion topVer;
+
+        /** Unique (per demander) sequence id. */
+        private final long updateSeq;
+
+        /**
+         * @param assigns Assigns.
+         * @param cctx Context.
+         * @param log Logger.
+         * @param sentStopEvnt Stop event flag.
+         */
+        RebalanceFuture(GridDhtPreloaderAssignments assigns,
+            GridCacheContext<?, ?> cctx,
+            IgniteLogger log,
+            boolean sentStopEvnt,
+            long updateSeq) {
+            assert assigns != null;
+
+            this.exchFut = assigns.exchangeFuture();
+            this.topVer = assigns.topologyVersion();
+            this.cctx = cctx;
+            this.log = log;
+            this.sendStoppedEvnt = sentStopEvnt;
+            this.updateSeq = updateSeq;
+        }
+
+        /**
+         * Dummy future. Will be done by real one.
+         */
+        public RebalanceFuture() {
+            this.exchFut = null;
+            this.topVer = null;
+            this.cctx = null;
+            this.log = null;
+            this.sendStoppedEvnt = false;
+            this.updateSeq = -1;
+        }
+
+        /**
+         * @return Topology version.
+         */
+        public AffinityTopologyVersion topologyVersion() {
+            return topVer;
+        }
+
+        /**
+         * @param updateSeq Update sequence.
+         * @return true in case future created for specified updateSeq, false in other case.
+         */
+        private boolean isActual(long updateSeq) {
+            return this.updateSeq == updateSeq;
+        }
+
+        /**
+         * @return Is initial (created at demander creation).
+         */
+        private boolean isInitial() {
+            return topVer == null;
+        }
+
+        /**
+         * @param nodeId Node id.
+         * @param parts Parts.
+         */
+        private void appendPartitions(UUID nodeId, Collection<Integer> parts) {
+            synchronized (this) {
+                remaining.put(nodeId, new T2<>(U.currentTimeMillis(), parts));
+            }
+        }
+
+        /**
+         *
+         */
+        private void doneIfEmpty() {
+            synchronized (this) {
+                if (isDone())
+                    return;
+
+                assert remaining.isEmpty();
+
+                if (log.isDebugEnabled())
+                    log.debug("Rebalancing is not required [cache=" + cctx.name() +
+                        ", topology=" + topVer + "]");
+
+                checkIsDone();
+            }
+        }
+
+        /**
+         * Cancels this future.
+         *
+         * @return {@code true}.
+         */
+        @Override public boolean cancel() {
+            synchronized (this) {
+                if (isDone())
+                    return true;
+
+                U.log(log, "Cancelled rebalancing from all nodes [cache=" + cctx.name()
+                    + ", topology=" + topologyVersion());
+
+                for (UUID nodeId : remaining.keySet()) {
+                    cleanupRemoteContexts(nodeId);
+                }
+
+                remaining.clear();
+
+                checkIsDone(true /* cancelled */);
+            }
+
+            return true;
+        }
+
+        /**
+         * @param nodeId Node id.
+         */
+        private void cancel(UUID nodeId) {
+            synchronized (this) {
+                if (isDone())
+                    return;
+
+                U.log(log, ("Cancelled rebalancing [cache=" + cctx.name() +
+                    ", fromNode=" + nodeId + ", topology=" + topologyVersion() +
+                    ", time=" + (U.currentTimeMillis() - remaining.get(nodeId).get1()) + " ms]"));
+
+                cleanupRemoteContexts(nodeId);
+
+                remaining.remove(nodeId);
+
+                checkIsDone();
+            }
+
+        }
+
+        /**
+         * @param nodeId Node id.
+         * @param p P.
+         */
+        private void partitionMissed(UUID nodeId, int p) {
+            synchronized (this) {
+                if (isDone())
+                    return;
+
+                if (missed.get(nodeId) == null)
+                    missed.put(nodeId, new HashSet<Integer>());
+
+                missed.get(nodeId).add(p);
+            }
+        }
+
+        /**
+         * @param nodeId Node id.
+         */
+        private void cleanupRemoteContexts(UUID nodeId) {
+            ClusterNode node = cctx.discovery().node(nodeId);
+
+            if (node == null)
+                return;
+
+            //Check remote node rebalancing API version.
+            if (node.version().compareTo(GridDhtPreloader.REBALANCING_VER_2_SINCE) >= 0) {
+
+                GridDhtPartitionDemandMessage d = new GridDhtPartitionDemandMessage(
+                    -1/* remove supply context signal */, this.topologyVersion(), cctx.cacheId());
+
+                d.timeout(cctx.config().getRebalanceTimeout());
+
+                try {
+                    for (int idx = 0; idx < cctx.gridConfig().getRebalanceThreadPoolSize(); idx++) {
+                        d.topic(GridCachePartitionExchangeManager.rebalanceTopic(idx));
+
+                        cctx.io().sendOrderedMessage(node, GridCachePartitionExchangeManager.rebalanceTopic(idx),
+                            d, cctx.ioPolicy(), cctx.config().getRebalanceTimeout());
+                    }
+                }
+                catch (IgniteCheckedException e) {
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to send failover context cleanup request to node");
+                }
+            }
+        }
+
+        /**
+         * @param nodeId Node id.
+         * @param p P.
+         */
+        private void partitionDone(UUID nodeId, int p) {
+            synchronized (this) {
+                if (isDone())
+                    return;
+
+                if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_LOADED))
+                    preloadEvent(p, EVT_CACHE_REBALANCE_PART_LOADED,
+                        exchFut.discoveryEvent());
+
+                Collection<Integer> parts = remaining.get(nodeId).get2();
+
+                if (parts != null) {
+                    boolean removed = parts.remove(p);
+
+                    assert removed;
+
+                    if (parts.isEmpty()) {
+                        U.log(log, "Completed " + ((remaining.size() == 1 ? "(final) " : "") +
+                            "rebalancing [cache=" + cctx.name() +
+                            ", fromNode=" + nodeId + ", topology=" + topologyVersion() +
+                            ", time=" + (U.currentTimeMillis() - remaining.get(nodeId).get1()) + " ms]"));
+
+                        remaining.remove(nodeId);
+                    }
+                }
+
+                checkIsDone();
+            }
+        }
+
+        /**
+         * @param part Partition.
+         * @param type Type.
+         * @param discoEvt Discovery event.
+         */
+        private void preloadEvent(int part, int type, DiscoveryEvent discoEvt) {
+            assert discoEvt != null;
+
+            cctx.events().addPreloadEvent(part, type, discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp());
+        }
+
+        /**
+         * @param type Type.
+         * @param discoEvt Discovery event.
+         */
+        private void preloadEvent(int type, DiscoveryEvent discoEvt) {
+            preloadEvent(-1, type, discoEvt);
+        }
+
+        /**
+         *
+         */
+        private void checkIsDone() {
+            checkIsDone(false);
+        }
+
+        /**
+         * @param cancelled Is cancelled.
+         */
+        private void checkIsDone(boolean cancelled) {
+            if (remaining.isEmpty()) {
+                if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED) && (!cctx.isReplicated() || sendStoppedEvnt))
+                    preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
+
+                if (log.isDebugEnabled())
+                    log.debug("Completed rebalance future.");
+
+                cctx.shared().exchange().scheduleResendPartitions();
+
+                Collection<Integer> m = new HashSet<>();
+
+                for (Map.Entry<UUID, Collection<Integer>> e : missed.entrySet()) {
+                    if (e.getValue() != null && !e.getValue().isEmpty())
+                        m.addAll(e.getValue());
+                }
+
+                if (!m.isEmpty()) {
+                    U.log(log, ("Reassigning partitions that were missed: " + m));
+
+                    onDone(false); //Finished but has missed partitions, will force dummy exchange
+
+                    cctx.shared().exchange().forceDummyExchange(true, exchFut);
+
+                    return;
+                }
+
+                if (!cancelled && !cctx.preloader().syncFuture().isDone())
+                    ((GridFutureAdapter)cctx.preloader().syncFuture()).onDone();
+
+                onDone(!cancelled);
+            }
+        }
+    }
+
+    /**
+     * Supply message wrapper.
+     */
+    @Deprecated//Backward compatibility. To be removed in future.
+    private static class SupplyMessage {
+        /** Sender ID. */
+        private UUID sndId;
+
+        /** Supply message. */
+        private GridDhtPartitionSupplyMessage supply;
+
+        /**
+         * Dummy constructor.
+         */
+        private SupplyMessage() {
+            // No-op.
+        }
+
+        /**
+         * @param sndId Sender ID.
+         * @param supply Supply message.
+         */
+        SupplyMessage(UUID sndId, GridDhtPartitionSupplyMessage supply) {
+            this.sndId = sndId;
+            this.supply = supply;
+        }
+
+        /**
+         * @return Sender ID.
+         */
+        UUID senderId() {
+            return sndId;
+        }
+
+        /**
+         * @return Message.
+         */
+        GridDhtPartitionSupplyMessage supply() {
+            return supply;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(SupplyMessage.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    @Deprecated//Backward compatibility. To be removed in future.
+    private class DemandWorker {
+        /** Worker ID. */
+        private int id;
+
+        /** Partition-to-node assignments. */
+        private final LinkedBlockingDeque<GridDhtPreloaderAssignments> assignQ = new LinkedBlockingDeque<>();
+
+        /** Message queue. */
+        private final LinkedBlockingDeque<SupplyMessage> msgQ =
+            new LinkedBlockingDeque<>();
+
+        /** Counter. */
+        private long cntr;
+
+        /** Hide worker logger and use cache logger instead. */
+        private IgniteLogger log = GridDhtPartitionDemander.this.log;
+
+        private volatile RebalanceFuture fut;
+
+        /**
+         * @param id Worker ID.
+         */
+        private DemandWorker(int id, RebalanceFuture fut) {
+            assert id >= 0;
+
+            this.id = id;
+            this.fut = fut;
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void addMessage(SupplyMessage msg) {
+            msgQ.offer(msg);
+        }
+
+        /**
+         * @param deque Deque to poll from.
+         * @param time Time to wait.
+         * @return Polled item.
+         * @throws InterruptedException If interrupted.
+         */
+        @Nullable private <T> T poll(BlockingQueue<T> deque, long time) throws InterruptedException {
+            return deque.poll(time, MILLISECONDS);
+        }
+
+        /**
+         * @param idx Unique index for this topic.
+         * @return Topic for partition.
+         */
+        public Object topic(long idx) {
+            return TOPIC_CACHE.topic(cctx.namexx(), cctx.nodeId(), id, idx);
+        }
+
+        /**
+         * @param node Node to demand from.
+         * @param topVer Topology version.
+         * @param d Demand message.
+         * @param exchFut Exchange future.
+         * @throws InterruptedException If interrupted.
+         * @throws ClusterTopologyCheckedException If node left.
+         * @throws IgniteCheckedException If failed to send message.
+         */
+        private void demandFromNode(
+            ClusterNode node,
+            final AffinityTopologyVersion topVer,
+            GridDhtPartitionDemandMessage d,
+            GridDhtPartitionsExchangeFuture exchFut
+        ) throws InterruptedException, IgniteCheckedException {
+            GridDhtPartitionTopology top = cctx.dht().topology();
+
+            cntr++;
+
+            d.topic(topic(cntr));
+            d.workerId(id);
+
+            if (topologyChanged(fut))
+                return;
+
+            cctx.io().addOrderedHandler(d.topic(), new CI2<UUID, GridDhtPartitionSupplyMessage>() {
+                @Override public void apply(UUID nodeId, GridDhtPartitionSupplyMessage msg) {
+                    addMessage(new SupplyMessage(nodeId, msg));
+                }
+            });
+
+            try {
+                boolean retry;
+
+                // DoWhile.
+                // =======
+                do {
+                    retry = false;
+
+                    // Create copy.
+                    d = new GridDhtPartitionDemandMessage(d, fut.remaining.get(node.id()).get2());
+
+                    long timeout = cctx.config().getRebalanceTimeout();
+
+                    d.timeout(timeout);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Sending demand message [node=" + node.id() + ", demand=" + d + ']');
+
+                    // Send demand message.
+                    cctx.io().send(node, d, cctx.ioPolicy());
+
+                    // While.
+                    // =====
+                    while (!topologyChanged(fut)) {
+                        SupplyMessage s = poll(msgQ, timeout);
+
+                        // If timed out.
+                        if (s == null) {
+                            if (msgQ.isEmpty()) { // Safety check.
+                                U.warn(log, "Timed out waiting for partitions to load, will retry in " + timeout +
+                                    " ms (you may need to increase 'networkTimeout' or 'rebalanceBatchSize'" +
+                                    " configuration properties).");
+
+                                // Ordered listener was removed if timeout expired.
+                                cctx.io().removeOrderedHandler(d.topic());
+
+                                // Must create copy to be able to work with IO manager thread local caches.
+                                d = new GridDhtPartitionDemandMessage(d, fut.remaining.get(node.id()).get2());
+
+                                // Create new topic.
+                                d.topic(topic(++cntr));
+
+                                // Create new ordered listener.
+                                cctx.io().addOrderedHandler(d.topic(),
+                                    new CI2<UUID, GridDhtPartitionSupplyMessage>() {
+                                        @Override public void apply(UUID nodeId,
+                                            GridDhtPartitionSupplyMessage msg) {
+                                            addMessage(new SupplyMessage(nodeId, msg));
+                                        }
+                                    });
+
+                                // Resend message with larger timeout.
+                                retry = true;
+
+                                break; // While.
+                            }
+                            else
+                                continue; // While.
+                        }
+
+                        // Check that message was received from expected node.
+                        if (!s.senderId().equals(node.id())) {
+                            U.warn(log, "Received supply message from unexpected node [expectedId=" + node.id() +
+                                ", rcvdId=" + s.senderId() + ", msg=" + s + ']');
+
+                            continue; // While.
+                        }
+
+                        if (log.isDebugEnabled())
+                            log.debug("Received supply message: " + s);
+
+                        GridDhtPartitionSupplyMessage supply = s.supply();
+
+                        // Check whether there were class loading errors on unmarshal
+                        if (supply.classError() != null) {
+                            if (log.isDebugEnabled())
+                                log.debug("Class got undeployed during preloading: " + supply.classError());
+
+                            retry = true;
+
+                            // Quit preloading.
+                            break;
+                        }
+
+                        // Preload.
+                        for (Map.Entry<Integer, CacheEntryInfoCollection> e : supply.infos().entrySet()) {
+                            int p = e.getKey();
+
+                            if (cctx.affinity().localNode(p, topVer)) {
+                                GridDhtLocalPartition part = top.localPartition(p, topVer, true);
+
+                                assert part != null;
+
+                                if (part.state() == MOVING) {
+                                    boolean reserved = part.reserve();
+
+                                    assert reserved : "Failed to reserve partition [gridName=" +
+                                        cctx.gridName() + ", cacheName=" + cctx.namex() + ", part=" + part + ']';
+
+                                    part.lock();
+
+                                    try {
+                                        Collection<Integer> invalidParts = new GridLeanSet<>();
+
+                                        // Loop through all received entries and try to preload them.
+                                        for (GridCacheEntryInfo entry : e.getValue().infos()) {
+                                            if (!invalidParts.contains(p)) {
+                                                if (!part.preloadingPermitted(entry.key(), entry.version())) {
+                                                    if (log.isDebugEnabled())
+                                                        log.debug("Preloading is not permitted for entry due to " +
+                                                            "evictions [key=" + entry.key() +
+                                                            ", ver=" + entry.version() + ']');
+
+                                                    continue;
+                                                }
+
+                                                if (!preloadEntry(node, p, entry, topVer)) {
+                                                    invalidParts.add(p);
+
+                                                    if (log.isDebugEnabled())
+                                                        log.debug("Got entries for invalid partition during " +
+                                                            "preloading (will skip) [p=" + p + ", entry=" + entry + ']');
+                                                }
+                                            }
+                                        }
+
+                                        boolean last = supply.last().contains(p);
+
+                                        // If message was last for this partition,
+                                        // then we take ownership.
+                                        if (last) {
+                                            fut.partitionDone(node.id(), p);
+
+                                            top.own(part);
+
+                                            if (log.isDebugEnabled())
+                                                log.debug("Finished rebalancing partition: " + part);
+
+                                            if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_LOADED))
+                                                preloadEvent(p, EVT_CACHE_REBALANCE_PART_LOADED,
+                                                    exchFut.discoveryEvent());
+                                        }
+                                    }
+                                    finally {
+                                        part.unlock();
+                                        part.release();
+                                    }
+                                }
+                                else {
+                                    fut.partitionDone(node.id(), p);
+
+                                    if (log.isDebugEnabled())
+                                        log.debug("Skipping rebalancing partition (state is not MOVING): " + part);
+                                }
+                            }
+                            else {
+                                fut.partitionDone(node.id(), p);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Skipping rebalancing partition (it does not belong on current node): " + p);
+                            }
+                        }
+
+                        // Only request partitions based on latest topology version.
+                        for (Integer miss : s.supply().missed()) {
+                            if (cctx.affinity().localNode(miss, topVer))
+                                fut.partitionMissed(node.id(), miss);
+                        }
+
+                        for (Integer miss : s.supply().missed())
+                            fut.partitionDone(node.id(), miss);
+
+                        if (fut.remaining.get(node.id()) == null)
+                            break; // While.
+
+                        if (s.supply().ack()) {
+                            retry = true;
+
+                            break;
+                        }
+                    }
+                }
+                while (retry && !topologyChanged(fut));
+            }
+            finally {
+                cctx.io().removeOrderedHandler(d.topic());
+            }
+        }
+
+        /**
+         * @param node Node.
+         * @param d D.
+         */
+        public void run(ClusterNode node, GridDhtPartitionDemandMessage d) throws IgniteCheckedException {
+            demandLock.readLock().lock();
+
+            try {
+                GridDhtPartitionsExchangeFuture exchFut = fut.exchFut;
+
+                AffinityTopologyVersion topVer = fut.topVer;
+
+                try {
+                    demandFromNode(node, topVer, d, exchFut);
+                }
+                catch (InterruptedException e) {
+                    throw new IgniteCheckedException(e);
+                }
+            }
+            finally {
+                demandLock.readLock().unlock();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(DemandWorker.class, this, "assignQ", assignQ, "msgQ", msgQ, "super", super.toString());
+        }
+    }
+}


[14/46] ignite git commit: Ignite-1093 Backward compatibility fix.

Posted by ak...@apache.org.
Ignite-1093 Backward compatibility fix.


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

Branch: refs/heads/ignite-1753-1282
Commit: 5e36b267ef65b5ee804d78d73f874c685692e3d9
Parents: 6ea3b5c
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Nov 9 18:11:43 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Nov 9 19:05:06 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/preloader/GridDhtPreloader.java          | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5e36b267/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 c634ff5..8e56c2d 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
@@ -207,7 +207,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
             });
 
         supplier = new GridDhtPartitionSupplier(cctx);
-        demander = new GridDhtPartitionDemander(cctx, busyLock);
+        demander = new GridDhtPartitionDemander(cctx, demandLock);
 
         supplier.start();
         demander.start();


[33/46] ignite git commit: Added tx-getAll PutAll benchmarks (fix in properties)

Posted by ak...@apache.org.
Added tx-getAll PutAll benchmarks (fix in properties)


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

Branch: refs/heads/ignite-1753-1282
Commit: 8cc9fa4ff51926190d4d75b5b41f07b20e910144
Parents: 847b616
Author: ashutak <as...@gridgain.com>
Authored: Wed Nov 11 18:55:25 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Nov 11 18:55:25 2015 +0300

----------------------------------------------------------------------
 modules/yardstick/config/benchmark-multicast.properties | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8cc9fa4f/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index 577d611..c508471 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -111,7 +111,7 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllBenchmark -sn IgniteNode -ds ${ver}atomic-putAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-optimistic-getAllPutAll-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-pessimistic-getAllPutAll-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC -txi SERIALIZABLE -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-opt-serializable-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-optimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-pessimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC -txi SERIALIZABLE -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-opt-serializable-getAllPutAll-1-backup,\
 "


[22/46] ignite git commit: Ignite-1093 Logging fixes.

Posted by ak...@apache.org.
Ignite-1093 Logging fixes.


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

Branch: refs/heads/ignite-1753-1282
Commit: 2b6fc4942f126ed42bb6c89ed20f5a9b4b37f655
Parents: 67f8858
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Nov 10 16:54:48 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Nov 10 16:54:48 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCachePartitionExchangeManager.java    | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2b6fc494/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 5b4fee3..81ff028 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
@@ -1357,7 +1357,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             if (futQ.isEmpty()) {
                                 U.log(log, "Rebalancing required" +
                                     "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
-                                    ", node=" + exchFut.discoveryEvent().node().id() + ']');
+                                    ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
 
                                 if (marshR != null)
                                     try {
@@ -1402,12 +1402,12 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             else
                                 U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
                                     "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
-                                    ", node=" + exchFut.discoveryEvent().node().id() + ']');
+                                    ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
                         }
                         else
                             U.log(log, "Skipping rebalancing (nothing scheduled) " +
                                 "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
-                                ", node=" + exchFut.discoveryEvent().node().id() + ']');
+                                ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
                     }
                 }
                 catch (IgniteInterruptedCheckedException e) {


[24/46] ignite git commit: IGNITE-1885 ignite-zookeeper: Upgrade Curator dependency to 2.9.1.

Posted by ak...@apache.org.
IGNITE-1885 ignite-zookeeper: Upgrade Curator dependency to 2.9.1.


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

Branch: refs/heads/ignite-1753-1282
Commit: cb28819fcc004c209cc822eaddcbed374332921c
Parents: 77ee065
Author: Raul Kripalani <ra...@apache.org>
Authored: Tue Nov 10 21:09:36 2015 +0000
Committer: Raul Kripalani <ra...@apache.org>
Committed: Tue Nov 10 21:10:18 2015 +0000

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/cb28819f/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index 9f5bc42..303b941 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -35,7 +35,7 @@
     <url>http://ignite.apache.org</url>
 
     <properties>
-        <curator.version>2.8.0</curator.version>
+        <curator.version>2.9.1</curator.version>
     </properties>
 
     <dependencies>


[42/46] ignite git commit: Merge branch ignite-1.5 into ignite-1282

Posted by ak...@apache.org.
Merge branch ignite-1.5 into ignite-1282


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

Branch: refs/heads/ignite-1753-1282
Commit: 0ccba79f9109d1e2042581b45e72d9dedeef9d9a
Parents: 25292c7 d54fcbe
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 17 13:35:12 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 17 13:35:12 2015 +0300

----------------------------------------------------------------------
 .../rest/AbstractRestProcessorSelfTest.java     |    5 +-
 .../JettyRestProcessorAbstractSelfTest.java     |  142 +-
 .../java/org/apache/ignite/IgniteCache.java     |    3 +-
 .../eviction/random/RandomEvictionPolicy.java   |    4 +
 .../cache/store/CacheStoreSessionListener.java  |    9 +-
 .../store/jdbc/CacheAbstractJdbcStore.java      |   34 +-
 .../jdbc/CacheJdbcStoreSessionListener.java     |    8 +-
 .../configuration/CacheConfiguration.java       |   95 +-
 .../configuration/ConnectorConfiguration.java   |   61 +-
 .../configuration/IgniteConfiguration.java      |   32 +-
 .../apache/ignite/internal/IgniteKernal.java    |   21 +
 .../org/apache/ignite/internal/IgnitionEx.java  |    3 +
 .../managers/communication/GridIoMessage.java   |    4 +-
 .../communication/GridIoMessageFactory.java     |   10 +-
 .../discovery/GridDiscoveryManager.java         |    5 +-
 .../cache/CacheStoreBalancingWrapper.java       |    5 +-
 .../processors/cache/GridCacheAdapter.java      |   22 +-
 .../processors/cache/GridCacheContext.java      |   29 +-
 .../processors/cache/GridCacheEntryEx.java      |    4 +-
 .../processors/cache/GridCacheIoManager.java    |   19 +-
 .../processors/cache/GridCacheMapEntry.java     |   91 +-
 .../processors/cache/GridCacheMvccManager.java  |  145 +-
 .../GridCachePartitionExchangeManager.java      |  184 ++-
 .../processors/cache/GridCachePreloader.java    |   55 +-
 .../cache/GridCachePreloaderAdapter.java        |   40 +-
 .../processors/cache/GridCacheProcessor.java    |   55 +-
 .../processors/cache/IgniteCacheProxy.java      |    6 +
 .../distributed/GridDistributedCacheEntry.java  |    2 +-
 .../distributed/GridDistributedTxMapping.java   |    8 +-
 .../GridDistributedTxRemoteAdapter.java         |    5 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   11 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   63 +-
 .../distributed/dht/GridDhtLockFuture.java      |    7 +-
 .../dht/GridDhtPartitionsReservation.java       |    2 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   12 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   39 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   78 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   33 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    7 +-
 .../GridDhtPartitionDemandMessage.java          |   14 +-
 .../preloader/GridDhtPartitionDemandPool.java   | 1192 ---------------
 .../dht/preloader/GridDhtPartitionDemander.java | 1408 ++++++++++++++++++
 .../dht/preloader/GridDhtPartitionSupplier.java | 1037 +++++++++++++
 .../GridDhtPartitionSupplyMessageV2.java        |  380 +++++
 .../preloader/GridDhtPartitionSupplyPool.java   |  555 -------
 .../GridDhtPartitionsExchangeFuture.java        |    2 +
 .../dht/preloader/GridDhtPreloader.java         |  282 +++-
 .../near/GridNearOptimisticTxPrepareFuture.java |   11 +-
 .../near/GridNearTransactionalCache.java        |    7 +-
 .../near/GridNearTxFinishFuture.java            |  157 +-
 .../cache/distributed/near/GridNearTxLocal.java |   21 +-
 .../store/GridCacheStoreManagerAdapter.java     |   29 +-
 .../cache/transactions/IgniteInternalTx.java    |    3 +-
 .../cache/transactions/IgniteTxAdapter.java     |  251 ++--
 .../cache/transactions/IgniteTxHandler.java     |   37 +-
 .../transactions/IgniteTxLocalAdapter.java      |   22 +-
 .../cache/transactions/IgniteTxManager.java     |  171 ++-
 .../datastructures/DataStructuresProcessor.java |    3 +
 .../GridCacheAtomicReferenceImpl.java           |    2 +-
 .../processors/igfs/IgfsMetaManager.java        |    2 +-
 .../handlers/cache/GridCacheCommandHandler.java |  128 +-
 .../handlers/query/QueryCommandHandler.java     |  254 +++-
 .../top/GridTopologyCommandHandler.java         |    4 -
 .../processors/task/GridTaskWorker.java         |    4 +-
 .../GridBoundedConcurrentLinkedHashMap.java     |    7 +-
 .../GridBoundedConcurrentLinkedHashSet.java     |    7 +-
 .../util/GridBoundedConcurrentOrderedMap.java   |   39 +-
 .../internal/util/GridConcurrentFactory.java    |   11 +-
 .../util/GridConcurrentLinkedHashSet.java       |    9 +-
 .../ignite/internal/util/IgniteUuidCache.java   |    6 +-
 .../util/future/GridCompoundFuture.java         |  155 +-
 .../ignite/internal/util/lang/GridTuple4.java   |    2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  360 +++--
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   95 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   37 +
 .../TcpDiscoveryStatusCheckMessage.java         |   11 +
 .../java/org/jsr166/ConcurrentHashMap8.java     |    2 +-
 .../java/org/jsr166/ConcurrentLinkedDeque8.java |    2 +-
 .../org/jsr166/ConcurrentLinkedHashMap.java     |  195 ++-
 .../GridCacheAffinityBackupsSelfTest.java       |    8 +
 .../store/GridCacheBalancingStoreSelfTest.java  |  220 ++-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |  145 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    2 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   40 +-
 .../processors/cache/GridCacheTestEntryEx.java  |    3 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |    2 +-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |   20 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   21 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   10 +
 ...achePartitionedMultiNodeFullApiSelfTest.java |    2 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |   68 +
 .../GridCacheRebalancingSyncSelfTest.java       |  506 +++++++
 ...eRebalancingUnmarshallingFailedSelfTest.java |  147 ++
 .../GridCacheReplicatedPreloadSelfTest.java     |   22 +-
 .../continuous/GridEventConsumeSelfTest.java    |    2 +-
 ...dBoundedConcurrentLinkedHashMapSelfTest.java |    2 +-
 .../GridConcurrentLinkedHashMapSelfTest.java    |   62 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  158 +-
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |   10 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  476 +++++-
 .../spi/discovery/tcp/TestTcpDiscoverySpi.java  |   46 +
 .../TcpDiscoveryMulticastIpFinderSelfTest.java  |   28 +-
 .../testframework/junits/GridAbstractTest.java  |   32 +-
 .../junits/common/GridCommonAbstractTest.java   |   25 +-
 .../testsuites/IgniteCacheTestSuite3.java       |    4 +
 ...rrentLinkedHashMapMultiThreadedSelfTest.java |  104 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   18 +-
 modules/rest-http/pom.xml                       |   11 +
 .../visor/commands/open/VisorOpenCommand.scala  |   14 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    3 +-
 modules/yardstick/README.txt                    |   16 +-
 .../config/benchmark-multicast.properties       |    9 +-
 .../yardstick/config/benchmark-store.properties |   86 +-
 .../yardstick/config/ignite-store-config.xml    |   31 +-
 .../yardstick/IgniteBenchmarkArguments.java     |    2 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |   75 +
 .../org/apache/ignite/yardstick/IgniteNode.java |   23 +-
 .../IgniteAccountSerializableTxBenchmark.java   |   11 +-
 .../cache/IgniteAccountTxBenchmark.java         |   11 +-
 .../cache/IgniteGetAllPutAllTxBenchmark.java    |   73 +
 .../cache/IgnitePutGetTxBenchmark.java          |   41 +-
 .../yardstick/cache/IgnitePutTxBenchmark.java   |   10 +
 .../cache/IgnitePutTxPrimaryOnlyBenchmark.java  |   65 +
 .../IgnitePutTxSkipLocalBackupBenchmark.java    |   65 +
 .../cache/WaitMapExchangeFinishCallable.java    |   95 ++
 ...IgniteTransactionalWriteInvokeBenchmark.java |   41 +-
 .../IgniteTransactionalWriteReadBenchmark.java  |   41 +-
 modules/zookeeper/pom.xml                       |    2 +-
 130 files changed, 7772 insertions(+), 3414 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index af0122f,1b8d41c..9cde3a2
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@@ -202,9 -174,8 +206,12 @@@ public class CacheConfiguration<K, V> e
      /** Default size for onheap SQL row cache size. */
      public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024;
  
 +    /** Default value for keep portable in store behavior .*/
 +    @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"})
-     public static final Boolean DFLT_KEEP_PORTABLE_IN_STORE  = new Boolean(true);
++    public static final Boolean DFLT_KEEP_PORTABLE_IN_STORE = new Boolean(true);
++
+     /** Default threshold for concurrent loading of keys from {@link CacheStore}. */
+     public static final int DFLT_CONCURRENT_LOAD_ALL_THRESHOLD = 5;
  
      /** Cache name. */
      private String name;
@@@ -438,16 -408,15 +452,17 @@@
          name = cc.getName();
          nearCfg = cc.getNearConfiguration();
          nodeFilter = cc.getNodeFilter();
 -        rebalanceBatchesPrefetchCount = cc.getRebalanceBatchesPrefetchCount();
 +        pluginCfgs = cc.getPluginConfigurations();
 +        qryEntities = cc.getQueryEntities();
-         rebalanceMode = cc.getRebalanceMode();
++        readFromBackup = cc.isReadFromBackup();
          rebalanceBatchSize = cc.getRebalanceBatchSize();
++        rebalanceBatchesPrefetchCount = cc.getRebalanceBatchesPrefetchCount();
          rebalanceDelay = cc.getRebalanceDelay();
+         rebalanceMode = cc.getRebalanceMode();
          rebalanceOrder = cc.getRebalanceOrder();
          rebalancePoolSize = cc.getRebalanceThreadPoolSize();
          rebalanceTimeout = cc.getRebalanceTimeout();
          rebalanceThrottle = cc.getRebalanceThrottle();
--        readFromBackup = cc.isReadFromBackup();
          sqlEscapeAll = cc.isSqlEscapeAll();
          sqlFuncCls = cc.getSqlFunctionClasses();
          sqlOnheapRowCacheSize = cc.getSqlOnheapRowCacheSize();
@@@ -870,38 -840,37 +885,69 @@@
      }
  
      /**
 +     * Flag indicating that {@link CacheStore} implementation
 +     * is working with portable objects instead of Java objects.
 +     * Default value of this flag is {@link #DFLT_KEEP_PORTABLE_IN_STORE},
 +     * because this is recommended behavior from performance standpoint.
 +     * <p>
 +     * If set to {@code false}, Ignite will deserialize keys and
 +     * values stored in portable format before they are passed
 +     * to cache store.
 +     * <p>
 +     * Note that setting this flag to {@code false} can simplify
 +     * store implementation in some cases, but it can cause performance
 +     * degradation due to additional serializations and deserializations
 +     * of portable objects. You will also need to have key and value
 +     * classes on all nodes since binary will be deserialized when
 +     * store is called.
 +     *
 +     * @return Keep binary in store flag.
 +     */
 +    public Boolean isKeepPortableInStore() {
 +        return keepPortableInStore;
 +    }
 +
 +    /**
 +     * Sets keep binary in store flag.
 +     *
 +     * @param keepPortableInStore Keep binary in store flag.
 +     */
 +    public void setKeepPortableInStore(boolean keepPortableInStore) {
 +        this.keepPortableInStore = keepPortableInStore;
 +    }
 +
 +    /**
+      * Gets the threshold used in cases when values for multiple keys are being loaded from an underlying
+      * {@link CacheStore} in parallel. In the situation when several threads load the same or intersecting set of keys
+      * and the total number of keys to load is less or equal to this threshold then there will be no a second call to
+      * the storage in order to load a key from thread A if the same key is already being loaded by thread B.
+      *
+      * The threshold should be controlled wisely. On the one hand if it's set to a big value then the interaction with
+      * a storage during the load of missing keys will be minimal. On the other hand the big value may result in
+      * significant performance degradation because it is needed to check for every key whether it's being loaded or not.
+      *
+      * When not set, default value is {@link #DFLT_CONCURRENT_LOAD_ALL_THRESHOLD}.
+      *
+      * @return The concurrent load-all threshold.
+      */
+     public int getStoreConcurrentLoadAllThreshold() {
+         return storeConcurrentLoadAllThreshold;
+     }
+ 
+     /**
+      * Sets the concurrent load-all threshold used for cases when keys' values are being loaded from {@link CacheStore}
+      * in parallel.
+      *
+      * @param storeConcurrentLoadAllThreshold The concurrent load-all threshold.
+      * @return {@code this} for chaining.
+      */
+     public CacheConfiguration<K, V> setStoreConcurrentLoadAllThreshold(int storeConcurrentLoadAllThreshold) {
+         this.storeConcurrentLoadAllThreshold = storeConcurrentLoadAllThreshold;
+ 
+         return this;
+     }
+ 
+     /**
       * Gets key topology resolver to provide mapping from keys to nodes.
       *
       * @return Key topology resolver to provide mapping from keys to nodes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index a91aa7e,9298c6d..2069a90
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@@ -506,9 -504,9 +512,10 @@@ public class IgniteConfiguration 
          p2pLocClsPathExcl = cfg.getPeerClassLoadingLocalClassPathExclude();
          p2pMissedCacheSize = cfg.getPeerClassLoadingMissedResourcesCacheSize();
          p2pPoolSize = cfg.getPeerClassLoadingThreadPoolSize();
 +        platformCfg = cfg.getPlatformConfiguration();
          pluginCfgs = cfg.getPluginConfigurations();
          pubPoolSize = cfg.getPublicThreadPoolSize();
+         rebalanceThreadPoolSize = cfg.getRebalanceThreadPoolSize();
          segChkFreq = cfg.getSegmentCheckFrequency();
          segPlc = cfg.getSegmentationPolicy();
          segResolveAttempts = cfg.getSegmentationResolveAttempts();

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index a5762ad,df9f5c4..5ced545
--- 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
@@@ -1135,9 -1106,10 +1135,10 @@@ public abstract class GridCacheMapEntr
              if (intercept) {
                  val0 = CU.value(val, cctx, false);
  
 -                CacheLazyEntry e = new CacheLazyEntry(cctx, key, old);
 +                CacheLazyEntry e = new CacheLazyEntry(cctx, key, old, keepPortable);
  
-                 Object interceptorVal = cctx.config().getInterceptor().onBeforePut(new CacheLazyEntry(cctx, key, old, keepPortable),
+                 Object interceptorVal = cctx.config().getInterceptor().onBeforePut(
 -                    new CacheLazyEntry(cctx, key, old),
++                    new CacheLazyEntry(cctx, key, old, keepPortable),
                      val0);
  
                  key0 = e.key();
@@@ -1380,41 -1352,34 +1384,34 @@@
              if (cctx.isLocal() || cctx.isReplicated() || (tx != null && tx.local() && !isNear()))
                  cctx.continuousQueries().onEntryUpdated(this, key, null, old, false);
  
 -            cctx.dataStructures().onEntryUpdated(key, true);
 +            cctx.dataStructures().onEntryUpdated(key, true, keepPortable);
-         }
- 
-         // Persist outside of synchronization. The correctness of the
-         // value will be handled by current transaction.
-         if (writeThrough)
-             cctx.store().remove(tx, keyValue(false));
  
-         if (cctx.deferredDelete() && !detached() && !isInternal())
-             cctx.onDeferredDelete(this, newVer);
-         else {
-             boolean marked = false;
+             deferred = cctx.deferredDelete() && !detached() && !isInternal();
  
-             synchronized (this) {
+             if (!deferred) {
                  // If entry is still removed.
-                 if (newVer == ver) {
-                     if (obsoleteVer == null || !(marked = markObsolete0(obsoleteVer, true))) {
-                         if (log.isDebugEnabled())
-                             log.debug("Entry could not be marked obsolete (it is still used): " + this);
-                     }
-                     else {
-                         recordNodeId(affNodeId, topVer);
+                 assert newVer == ver;
  
-                         // If entry was not marked obsolete, then removed lock
-                         // will be registered whenever removeLock is called.
-                         cctx.mvcc().addRemoved(cctx, obsoleteVer);
+                 if (obsoleteVer == null || !(marked = markObsolete0(obsoleteVer, true, null))) {
+                     if (log.isDebugEnabled())
+                         log.debug("Entry could not be marked obsolete (it is still used): " + this);
+                 }
+                 else {
+                     recordNodeId(affNodeId, topVer);
  
-                         if (log.isDebugEnabled())
-                             log.debug("Entry was marked obsolete: " + this);
-                     }
+                     if (log.isDebugEnabled())
+                         log.debug("Entry was marked obsolete: " + this);
                  }
              }
+         }
  
-             if (marked)
-                 onMarkedObsolete();
+         if (deferred)
+             cctx.onDeferredDelete(this, newVer);
+ 
+         if (marked) {
+             assert !deferred;
+ 
+             onMarkedObsolete();
          }
  
          if (intercept)

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 3aa7e1c,93303c8..29f67ce
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@@ -583,10 -583,8 +583,9 @@@ public class GridDistributedTxRemoteAda
                                                  eventNodeId(),
                                                  nodeId,
                                                  false,
-                                                 false,
                                                  true,
                                                  true,
 +                                                txEntry.keepBinary(),
                                                  topVer,
                                                  null,
                                                  replicate ? DR_BACKUP : DR_NONE,
@@@ -631,10 -629,8 +631,9 @@@
                                              eventNodeId(),
                                              nodeId,
                                              false,
-                                             false,
                                              true,
                                              true,
 +                                            txEntry.keepBinary(),
                                              topVer,
                                              null,
                                              replicate ? DR_BACKUP : DR_NONE,

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
index ad8a402,e268a88..ca677b2
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
@@@ -183,19 -184,19 +184,19 @@@ public class GridDhtTxRemote extends Gr
          int taskNameHash
      ) {
          super(
-             ctx, 
-             nodeId, 
-             rmtThreadId, 
-             xidVer, 
-             commitVer, 
+             ctx,
+             nodeId,
+             rmtThreadId,
+             xidVer,
+             commitVer,
              sys,
              plc,
 -            concurrency,
 -            isolation,
 -            invalidate,
 +            concurrency, 
 +            isolation, 
 +            invalidate, 
              timeout,
              txSize,
-             subjId, 
+             subjId,
              taskNameHash
          );
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 214ba45,2c7bf8a..9f52699
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@@ -1062,10 -1054,8 +1062,9 @@@ public abstract class IgniteTxLocalAdap
                                              eventNodeId(),
                                              txEntry.nodeId(),
                                              false,
-                                             false,
                                              evt,
                                              metrics,
 +                                            txEntry.keepBinary(),
                                              topVer,
                                              null,
                                              cached.detached()  ? DR_NONE : drType,
@@@ -1080,10 -1070,8 +1079,9 @@@
                                                  eventNodeId(),
                                                  nodeId,
                                                  false,
-                                                 false,
                                                  false,
                                                  metrics,
 +                                                txEntry.keepBinary(),
                                                  topVer,
                                                  CU.empty0(),
                                                  DR_NONE,

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccba79f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceApiSelfAbstractTest.java
----------------------------------------------------------------------


[12/46] ignite git commit: IGNITE-1161 Fixed after merge.

Posted by ak...@apache.org.
IGNITE-1161 Fixed after merge.


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

Branch: refs/heads/ignite-1753-1282
Commit: 4ab2bd32535d04a2f70286dd318dbbb32279e36c
Parents: a4848a7
Author: Andrey <an...@gridgain.com>
Authored: Mon Nov 9 17:40:45 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Mon Nov 9 17:40:45 2015 +0700

----------------------------------------------------------------------
 .../processors/rest/handlers/query/QueryCommandHandler.java      | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4ab2bd32/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
index 54cdd29..2961a27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
@@ -337,10 +337,6 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
                             break;
                     }
 
-                    List<GridQueryFieldMetadata> fieldsMeta = ((QueryCursorImpl<?>)qryCur).fieldsMeta();
-
-                    res.setFieldsMetadata(convertMetadata(fieldsMeta));
-
                     return new GridRestResponse(res);
                 }
                 finally {


[17/46] ignite git commit: ignite-1395: Additional fix for unecessary messages that are printed out by REST module

Posted by ak...@apache.org.
ignite-1395: Additional fix for unecessary messages that are printed out by REST module


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

Branch: refs/heads/ignite-1753-1282
Commit: 7ba2efb7cb31ffefa6868f0d7dad3d61993a52c6
Parents: e02b68c
Author: Roman Shtykh <ap...@gmail.com>
Authored: Tue Nov 10 13:17:51 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Nov 10 13:17:51 2015 +0300

----------------------------------------------------------------------
 modules/rest-http/pom.xml | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7ba2efb7/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 730e28a..9aa79f2 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -116,8 +116,19 @@
 
         <dependency>
             <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>1.7.7</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
             <version>1.7.7</version>
         </dependency>
+
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
     </dependencies>
 </project>


[21/46] ignite git commit: Ignite-1093 Logging & Backward compatibility failover fixes.

Posted by ak...@apache.org.
Ignite-1093 Logging & Backward compatibility failover fixes.


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

Branch: refs/heads/ignite-1753-1282
Commit: 67f88584a4ab330bbda956b3d0d830468d28920f
Parents: 37cafb6
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Nov 10 16:14:15 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Nov 10 16:14:15 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      | 34 +++++++-------------
 .../dht/preloader/GridDhtPartitionDemander.java | 25 ++++++++++++--
 2 files changed, 34 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67f88584/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 479a0b6..5b4fee3 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
@@ -617,13 +617,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * @return {@code True} if topology has changed.
-     */
-    public boolean topologyChanged() {
-        return exchWorker.topologyChanged();
-    }
-
-    /**
      * @param exchFut Exchange future.
      * @param reassign Dummy reassign flag.
      */
@@ -673,7 +666,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (log.isDebugEnabled())
             log.debug("Refreshing partitions [oldest=" + oldest.id() + ", loc=" + cctx.localNodeId() + ']');
 
-        Collection<ClusterNode> rmts = null;
+        Collection<ClusterNode> rmts;
 
         // If this is the oldest node.
         if (oldest.id().equals(cctx.localNodeId())) {
@@ -1362,7 +1355,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                         if (marshR != null || !rebalanceQ.isEmpty()) {
                             if (futQ.isEmpty()) {
-                                U.log(log, "Starting caches rebalancing [top=" + exchFut.topologyVersion() + "]");
+                                U.log(log, "Rebalancing required" +
+                                    "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
+                                    ", node=" + exchFut.discoveryEvent().node().id() + ']');
 
                                 if (marshR != null)
                                     try {
@@ -1404,13 +1399,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                     }
                                 }, /*system pool*/ true);
                             }
-                            else {
-                                U.log(log, "Obsolete exchange, skipping rebalancing [top=" + exchFut.topologyVersion() + "]");
-                            }
-                        }
-                        else {
-                            U.log(log, "Nothing scheduled, skipping rebalancing [top=" + exchFut.topologyVersion() + "]");
+                            else
+                                U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
+                                    "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
+                                    ", node=" + exchFut.discoveryEvent().node().id() + ']');
                         }
+                        else
+                            U.log(log, "Skipping rebalancing (nothing scheduled) " +
+                                "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
+                                ", node=" + exchFut.discoveryEvent().node().id() + ']');
                     }
                 }
                 catch (IgniteInterruptedCheckedException e) {
@@ -1425,13 +1422,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 }
             }
         }
-
-        /**
-         * @return {@code True} if another exchange future has been queued up.
-         */
-        boolean topologyChanged() {
-            return !futQ.isEmpty() || busy;
-        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/67f88584/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 29ca5f4..40d3dc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -114,6 +114,10 @@ public class GridDhtPartitionDemander {
     @Deprecated//Backward compatibility. To be removed in future.
     private final AtomicInteger dmIdx = new AtomicInteger();
 
+    /** DemandWorker. */
+    @Deprecated//Backward compatibility. To be removed in future.
+    private volatile DemandWorker worker;
+
     /** Cached rebalance topics. */
     private final Map<Integer, Object> rebalanceTopics;
 
@@ -166,6 +170,11 @@ public class GridDhtPartitionDemander {
             rebalanceFut.onDone(false);
         }
 
+        DemandWorker dw = worker;
+
+        if (dw != null)
+            dw.cancel();
+
         lastExchangeFut = null;
 
         lastTimeoutObj.set(null);
@@ -426,9 +435,9 @@ public class GridDhtPartitionDemander {
                 d.timeout(cctx.config().getRebalanceTimeout());
                 d.workerId(0);//old api support.
 
-                DemandWorker dw = new DemandWorker(dmIdx.incrementAndGet(), fut);
+                worker = new DemandWorker(dmIdx.incrementAndGet(), fut);
 
-                dw.run(node, d);
+                worker.run(node, d);
             }
         }
 
@@ -1137,6 +1146,13 @@ public class GridDhtPartitionDemander {
             return TOPIC_CACHE.topic(cctx.namexx(), cctx.nodeId(), id, idx);
         }
 
+        /** */
+        public void cancel() {
+            msgQ.clear();
+
+            msgQ.offer(new SupplyMessage(null, null));
+        }
+
         /**
          * @param node Node to demand from.
          * @param topVer Topology version.
@@ -1159,7 +1175,7 @@ public class GridDhtPartitionDemander {
             d.topic(topic(cntr));
             d.workerId(id);
 
-            if (topologyChanged(fut))
+            if (fut.isDone() || topologyChanged(fut))
                 return;
 
             cctx.io().addOrderedHandler(d.topic(), new CI2<UUID, GridDhtPartitionSupplyMessage>() {
@@ -1228,6 +1244,9 @@ public class GridDhtPartitionDemander {
                                 continue; // While.
                         }
 
+                        if (s.senderId() == null)
+                            return; // Stopping now.
+
                         // Check that message was received from expected node.
                         if (!s.senderId().equals(node.id())) {
                             U.warn(log, "Received supply message from unexpected node [expectedId=" + node.id() +


[02/46] ignite git commit: Ignite-1093 "Rebalancing with default parameters is very slow" fixes.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
new file mode 100644
index 0000000..865bad8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
@@ -0,0 +1,1034 @@
+/*
+ * 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.dht.preloader;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryInfoCollectSwapListener;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.util.lang.GridCloseableIterator;
+import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.IgniteSpiException;
+
+import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING;
+
+/**
+ * Thread pool for supplying partitions to demanding nodes.
+ */
+class GridDhtPartitionSupplier {
+    /** */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private GridDhtPartitionTopology top;
+
+    /** */
+    private final boolean depEnabled;
+
+    /** Preload predicate. */
+    private IgnitePredicate<GridCacheEntryInfo> preloadPred;
+
+    /** Supply context map. T2: nodeId, idx, topVer. */
+    private final Map<T3<UUID, Integer, AffinityTopologyVersion>, SupplyContext> scMap = new HashMap<>();
+
+    /**
+     * @param cctx Cache context.
+     */
+    GridDhtPartitionSupplier(GridCacheContext<?, ?> cctx) {
+        assert cctx != null;
+
+        this.cctx = cctx;
+
+        log = cctx.logger(getClass());
+
+        top = cctx.dht().topology();
+
+        depEnabled = cctx.gridDeploy().enabled();
+    }
+
+    /**
+     *
+     */
+    void start() {
+        startOldListeners();
+    }
+
+    /**
+     *
+     */
+    void stop() {
+        synchronized (scMap) {
+            Iterator<T3<UUID, Integer, AffinityTopologyVersion>> it = scMap.keySet().iterator();
+
+            while (it.hasNext()) {
+                T3<UUID, Integer, AffinityTopologyVersion> t = it.next();
+
+                clearContext(scMap.get(t), log);
+
+                it.remove();
+            }
+        }
+
+        stopOldListeners();
+    }
+
+    /**
+     * Clear context.
+     *
+     * @param sc Supply context.
+     * @param log Logger.
+     * @return true in case context was removed.
+     */
+    private static void clearContext(
+        final SupplyContext sc,
+        final IgniteLogger log) {
+        if (sc != null) {
+            final Iterator it = sc.entryIt;
+
+            if (it != null && it instanceof GridCloseableIterator && !((GridCloseableIterator)it).isClosed()) {
+                try {
+                    ((GridCloseableIterator)it).close();
+                }
+                catch (IgniteCheckedException e) {
+                    log.error("Iterator close failed.", e);
+                }
+            }
+
+            final GridDhtLocalPartition loc = sc.loc;
+
+            if (loc != null) {
+                assert loc.reservations() > 0;
+
+                loc.release();
+            }
+        }
+    }
+
+    /**
+     * Handles new topology.
+     *
+     * @param topVer Topology version.
+     */
+    public void onTopologyChanged(AffinityTopologyVersion topVer) {
+        synchronized (scMap) {
+            Iterator<T3<UUID, Integer, AffinityTopologyVersion>> it = scMap.keySet().iterator();
+
+            while (it.hasNext()) {
+                T3<UUID, Integer, AffinityTopologyVersion> t = it.next();
+
+                if (topVer.compareTo(t.get3()) > 0) {// Clear all obsolete contexts.
+                    clearContext(scMap.get(t), log);
+
+                    it.remove();
+
+                    if (log.isDebugEnabled())
+                        log.debug("Supply context removed [node=" + t.get1() + "]");
+                }
+            }
+        }
+    }
+
+    /**
+     * Sets preload predicate for supply pool.
+     *
+     * @param preloadPred Preload predicate.
+     */
+    void preloadPredicate(IgnitePredicate<GridCacheEntryInfo> preloadPred) {
+        this.preloadPred = preloadPred;
+    }
+
+    /**
+     * @param d Demand message.
+     * @param idx Index.
+     * @param id Node uuid.
+     */
+    public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage d) {
+        assert d != null;
+        assert id != null;
+
+        AffinityTopologyVersion cutTop = cctx.affinity().affinityTopologyVersion();
+        AffinityTopologyVersion demTop = d.topologyVersion();
+
+        T3<UUID, Integer, AffinityTopologyVersion> scId = new T3<>(id, idx, demTop);
+
+        if (d.updateSequence() == -1) {//Demand node requested context cleanup.
+            synchronized (scMap) {
+                clearContext(scMap.remove(scId), log);
+
+                return;
+            }
+        }
+
+        if (cutTop.compareTo(demTop) > 0) {
+            if (log.isDebugEnabled())
+                log.debug("Demand request cancelled [current=" + cutTop + ", demanded=" + demTop +
+                    ", from=" + id + ", idx=" + idx + "]");
+
+            return;
+        }
+
+        if (log.isDebugEnabled())
+            log.debug("Demand request accepted [current=" + cutTop + ", demanded=" + demTop +
+                ", from=" + id + ", idx=" + idx + "]");
+
+        GridDhtPartitionSupplyMessageV2 s = new GridDhtPartitionSupplyMessageV2(
+            d.updateSequence(), cctx.cacheId(), d.topologyVersion(), cctx.deploymentEnabled());
+
+        ClusterNode node = cctx.discovery().node(id);
+
+        if (node == null)
+            return; //Context will be cleaned at topology change.
+
+        try {
+            SupplyContext sctx;
+
+            synchronized (scMap) {
+                sctx = scMap.remove(scId);
+
+                assert sctx == null || d.updateSequence() == sctx.updateSeq;
+            }
+
+            // Initial demand request should contain partitions list.
+            if (sctx == null && d.partitions() == null)
+                return;
+
+            assert !(sctx != null && d.partitions() != null);
+
+            long bCnt = 0;
+
+            SupplyContextPhase phase = SupplyContextPhase.NEW;
+
+            boolean newReq = true;
+
+            long maxBatchesCnt = cctx.config().getRebalanceBatchesPrefetchCount();
+
+            if (sctx != null) {
+                phase = sctx.phase;
+
+                maxBatchesCnt = 1;
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Starting supplying rebalancing [cache=" + cctx.name() +
+                        ", fromNode=" + node.id() + ", partitionsCount=" + d.partitions().size() +
+                        ", topology=" + d.topologyVersion() + ", updateSeq=" + d.updateSequence() +
+                        ", idx=" + idx + "]");
+            }
+
+            Iterator<Integer> partIt = sctx != null ? sctx.partIt : d.partitions().iterator();
+
+            while ((sctx != null && newReq) || partIt.hasNext()) {
+                int part = sctx != null && newReq ? sctx.part : partIt.next();
+
+                newReq = false;
+
+                GridDhtLocalPartition loc;
+
+                if (sctx != null && sctx.loc != null) {
+                    loc = sctx.loc;
+
+                    assert loc.reservations() > 0;
+                }
+                else {
+                    loc = top.localPartition(part, d.topologyVersion(), false);
+
+                    if (loc == null || loc.state() != OWNING || !loc.reserve()) {
+                        // Reply with partition of "-1" to let sender know that
+                        // this node is no longer an owner.
+                        s.missed(part);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Requested partition is not owned by local node [part=" + part +
+                                ", demander=" + id + ']');
+
+                        continue;
+                    }
+                }
+
+                GridCacheEntryInfoCollectSwapListener swapLsnr = null;
+
+                try {
+                    if (phase == SupplyContextPhase.NEW && cctx.isSwapOrOffheapEnabled()) {
+                        swapLsnr = new GridCacheEntryInfoCollectSwapListener(log);
+
+                        cctx.swap().addOffHeapListener(part, swapLsnr);
+                        cctx.swap().addSwapListener(part, swapLsnr);
+                    }
+
+                    boolean partMissing = false;
+
+                    if (phase == SupplyContextPhase.NEW)
+                        phase = SupplyContextPhase.ONHEAP;
+
+                    if (phase == SupplyContextPhase.ONHEAP) {
+                        Iterator<GridDhtCacheEntry> entIt = sctx != null ?
+                            (Iterator<GridDhtCacheEntry>)sctx.entryIt : loc.entries().iterator();
+
+                        while (entIt.hasNext()) {
+                            if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
+                                // Demander no longer needs this partition, so we send '-1' partition and move on.
+                                s.missed(part);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Demanding node does not need requested partition [part=" + part +
+                                        ", nodeId=" + id + ']');
+
+                                partMissing = true;
+
+                                break;
+                            }
+
+                            if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
+                                if (++bCnt >= maxBatchesCnt) {
+                                    saveSupplyContext(scId,
+                                        phase,
+                                        partIt,
+                                        part,
+                                        entIt,
+                                        swapLsnr,
+                                        loc,
+                                        d.topologyVersion(),
+                                        d.updateSequence());
+
+                                    swapLsnr = null;
+                                    loc = null;
+
+                                    reply(node, d, s, scId);
+
+                                    return;
+                                }
+                                else {
+                                    if (!reply(node, d, s, scId))
+                                        return;
+
+                                    s = new GridDhtPartitionSupplyMessageV2(d.updateSequence(),
+                                        cctx.cacheId(), d.topologyVersion(), cctx.deploymentEnabled());
+                                }
+                            }
+
+                            GridCacheEntryEx e = entIt.next();
+
+                            GridCacheEntryInfo info = e.info();
+
+                            if (info != null && !info.isNew()) {
+                                if (preloadPred == null || preloadPred.apply(info))
+                                    s.addEntry(part, info, cctx);
+                                else if (log.isDebugEnabled())
+                                    log.debug("Rebalance predicate evaluated to false (will not sender cache entry): " +
+                                        info);
+                            }
+                        }
+
+                        if (partMissing)
+                            continue;
+
+                    }
+
+                    if (phase == SupplyContextPhase.ONHEAP) {
+                        phase = SupplyContextPhase.SWAP;
+
+                        if (sctx != null) {
+                            sctx = new SupplyContext(
+                                phase,
+                                partIt,
+                                null,
+                                swapLsnr,
+                                part,
+                                loc,
+                                d.updateSequence());
+                        }
+                    }
+
+                    if (phase == SupplyContextPhase.SWAP && cctx.isSwapOrOffheapEnabled()) {
+                        GridCloseableIterator<Map.Entry<byte[], GridCacheSwapEntry>> iter =
+                            sctx != null && sctx.entryIt != null ?
+                                (GridCloseableIterator<Map.Entry<byte[], GridCacheSwapEntry>>)sctx.entryIt :
+                                cctx.swap().iterator(part);
+
+                        // Iterator may be null if space does not exist.
+                        if (iter != null) {
+                            boolean prepared = false;
+
+                            while (iter.hasNext()) {
+                                if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
+                                    // Demander no longer needs this partition,
+                                    // so we send '-1' partition and move on.
+                                    s.missed(part);
+
+                                    if (log.isDebugEnabled())
+                                        log.debug("Demanding node does not need requested partition " +
+                                            "[part=" + part + ", nodeId=" + id + ']');
+
+                                    partMissing = true;
+
+                                    break; // For.
+                                }
+
+                                if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
+                                    if (++bCnt >= maxBatchesCnt) {
+                                        saveSupplyContext(scId,
+                                            phase,
+                                            partIt,
+                                            part,
+                                            iter,
+                                            swapLsnr,
+                                            loc,
+                                            d.topologyVersion(),
+                                            d.updateSequence());
+
+                                        swapLsnr = null;
+                                        loc = null;
+
+                                        reply(node, d, s, scId);
+
+                                        return;
+                                    }
+                                    else {
+                                        if (!reply(node, d, s, scId))
+                                            return;
+
+                                        s = new GridDhtPartitionSupplyMessageV2(d.updateSequence(),
+                                            cctx.cacheId(), d.topologyVersion(), cctx.deploymentEnabled());
+                                    }
+                                }
+
+                                Map.Entry<byte[], GridCacheSwapEntry> e = iter.next();
+
+                                GridCacheSwapEntry swapEntry = e.getValue();
+
+                                GridCacheEntryInfo info = new GridCacheEntryInfo();
+
+                                info.keyBytes(e.getKey());
+                                info.ttl(swapEntry.ttl());
+                                info.expireTime(swapEntry.expireTime());
+                                info.version(swapEntry.version());
+                                info.value(swapEntry.value());
+
+                                if (preloadPred == null || preloadPred.apply(info))
+                                    s.addEntry0(part, info, cctx);
+                                else {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Rebalance predicate evaluated to false (will not send " +
+                                            "cache entry): " + info);
+
+                                    continue;
+                                }
+
+                                // Need to manually prepare cache message.
+                                if (depEnabled && !prepared) {
+                                    ClassLoader ldr = swapEntry.keyClassLoaderId() != null ?
+                                        cctx.deploy().getClassLoader(swapEntry.keyClassLoaderId()) :
+                                        swapEntry.valueClassLoaderId() != null ?
+                                            cctx.deploy().getClassLoader(swapEntry.valueClassLoaderId()) :
+                                            null;
+
+                                    if (ldr == null)
+                                        continue;
+
+                                    if (ldr instanceof GridDeploymentInfo) {
+                                        s.prepare((GridDeploymentInfo)ldr);
+
+                                        prepared = true;
+                                    }
+                                }
+                            }
+
+                            iter.close();
+
+                            if (partMissing)
+                                continue;
+                        }
+                    }
+
+                    if (swapLsnr == null && sctx != null)
+                        swapLsnr = sctx.swapLsnr;
+
+                    // Stop receiving promote notifications.
+                    if (swapLsnr != null) {
+                        cctx.swap().removeOffHeapListener(part, swapLsnr);
+                        cctx.swap().removeSwapListener(part, swapLsnr);
+                    }
+
+                    if (phase == SupplyContextPhase.SWAP) {
+                        phase = SupplyContextPhase.EVICTED;
+
+                        if (sctx != null) {
+                            sctx = new SupplyContext(
+                                phase,
+                                partIt,
+                                null,
+                                null,
+                                part,
+                                loc,
+                                d.updateSequence());
+                        }
+                    }
+
+                    if (phase == SupplyContextPhase.EVICTED && swapLsnr != null) {
+                        Collection<GridCacheEntryInfo> entries = swapLsnr.entries();
+
+                        swapLsnr = null;
+
+                        Iterator<GridCacheEntryInfo> lsnrIt = sctx != null && sctx.entryIt != null ?
+                            (Iterator<GridCacheEntryInfo>)sctx.entryIt : entries.iterator();
+
+                        while (lsnrIt.hasNext()) {
+                            if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
+                                // Demander no longer needs this partition,
+                                // so we send '-1' partition and move on.
+                                s.missed(part);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Demanding node does not need requested partition " +
+                                        "[part=" + part + ", nodeId=" + id + ']');
+
+                                // No need to continue iteration over swap entries.
+                                break;
+                            }
+
+                            if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
+                                if (++bCnt >= maxBatchesCnt) {
+                                    saveSupplyContext(scId,
+                                        phase,
+                                        partIt,
+                                        part,
+                                        lsnrIt,
+                                        swapLsnr,
+                                        loc,
+                                        d.topologyVersion(),
+                                        d.updateSequence());
+
+                                    loc = null;
+
+                                    reply(node, d, s, scId);
+
+                                    return;
+                                }
+                                else {
+                                    if (!reply(node, d, s, scId))
+                                        return;
+
+                                    s = new GridDhtPartitionSupplyMessageV2(d.updateSequence(),
+                                        cctx.cacheId(), d.topologyVersion(), cctx.deploymentEnabled());
+                                }
+                            }
+
+                            GridCacheEntryInfo info = lsnrIt.next();
+
+                            if (preloadPred == null || preloadPred.apply(info))
+                                s.addEntry(part, info, cctx);
+                            else if (log.isDebugEnabled())
+                                log.debug("Rebalance predicate evaluated to false (will not sender cache entry): " +
+                                    info);
+                        }
+                    }
+
+                    // Mark as last supply message.
+                    s.last(part);
+
+                    phase = SupplyContextPhase.NEW;
+
+                    sctx = null;
+                }
+                finally {
+                    if (loc != null)
+                        loc.release();
+
+                    if (swapLsnr != null) {
+                        cctx.swap().removeOffHeapListener(part, swapLsnr);
+                        cctx.swap().removeSwapListener(part, swapLsnr);
+                    }
+                }
+            }
+
+            reply(node, d, s, scId);
+
+            if (log.isDebugEnabled())
+                log.debug("Finished supplying rebalancing [cache=" + cctx.name() +
+                    ", fromNode=" + node.id() +
+                    ", topology=" + d.topologyVersion() + ", updateSeq=" + d.updateSequence() +
+                    ", idx=" + idx + "]");
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send partition supply message to node: " + id, e);
+        }
+        catch (IgniteSpiException e) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to send message to node (current node is stopping?) [node=" + node.id() +
+                    ", msg=" + e.getMessage() + ']');
+        }
+    }
+
+    /**
+     * @param n Node.
+     * @param d DemandMessage
+     * @param s Supply message.
+     * @return {@code True} if message was sent, {@code false} if recipient left grid.
+     * @throws IgniteCheckedException If failed.
+     */
+    private boolean reply(ClusterNode n,
+        GridDhtPartitionDemandMessage d,
+        GridDhtPartitionSupplyMessageV2 s,
+        T3<UUID, Integer, AffinityTopologyVersion> scId)
+        throws IgniteCheckedException {
+
+        try {
+            if (log.isDebugEnabled())
+                log.debug("Replying to partition demand [node=" + n.id() + ", demand=" + d + ", supply=" + s + ']');
+
+            cctx.io().sendOrderedMessage(n, d.topic(), s, cctx.ioPolicy(), d.timeout());
+
+            // Throttle preloading.
+            if (cctx.config().getRebalanceThrottle() > 0)
+                U.sleep(cctx.config().getRebalanceThrottle());
+
+            return true;
+        }
+        catch (ClusterTopologyCheckedException ignore) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to send partition supply message because node left grid: " + n.id());
+
+            synchronized (scMap) {
+                clearContext(scMap.remove(scId), log);
+            }
+
+            return false;
+        }
+    }
+
+    /**
+     * @param t Tuple.
+     * @param phase Phase.
+     * @param partIt Partition it.
+     * @param part Partition.
+     * @param entryIt Entry it.
+     * @param swapLsnr Swap listener.
+     */
+    private void saveSupplyContext(
+        T3<UUID, Integer, AffinityTopologyVersion> t,
+        SupplyContextPhase phase,
+        Iterator<Integer> partIt,
+        int part,
+        Iterator<?> entryIt, GridCacheEntryInfoCollectSwapListener swapLsnr,
+        GridDhtLocalPartition loc,
+        AffinityTopologyVersion topVer,
+        long updateSeq) {
+        synchronized (scMap) {
+            if (cctx.affinity().affinityTopologyVersion().equals(topVer)) {
+                assert scMap.get(t) == null;
+
+                scMap.put(t,
+                    new SupplyContext(phase,
+                        partIt,
+                        entryIt,
+                        swapLsnr,
+                        part,
+                        loc,
+                        updateSeq));
+            }
+            else if (loc != null) {
+                assert loc.reservations() > 0;
+
+                loc.release();
+            }
+        }
+    }
+
+    /**
+     * Supply context phase.
+     */
+    private enum SupplyContextPhase {
+        NEW,
+        ONHEAP,
+        SWAP,
+        EVICTED
+    }
+
+    /**
+     * Supply context.
+     */
+    private static class SupplyContext {
+        /** Phase. */
+        private final SupplyContextPhase phase;
+
+        /** Partition iterator. */
+        private final Iterator<Integer> partIt;
+
+        /** Entry iterator. */
+        private final Iterator<?> entryIt;
+
+        /** Swap listener. */
+        private final GridCacheEntryInfoCollectSwapListener swapLsnr;
+
+        /** Partition. */
+        private final int part;
+
+        /** Local partition. */
+        private final GridDhtLocalPartition loc;
+
+        /** Update seq. */
+        private final long updateSeq;
+
+        /**
+         * @param phase Phase.
+         * @param partIt Partition iterator.
+         * @param entryIt Entry iterator.
+         * @param swapLsnr Swap listener.
+         * @param part Partition.
+         */
+        public SupplyContext(SupplyContextPhase phase,
+            Iterator<Integer> partIt,
+            Iterator<?> entryIt,
+            GridCacheEntryInfoCollectSwapListener swapLsnr,
+            int part,
+            GridDhtLocalPartition loc,
+            long updateSeq) {
+            this.phase = phase;
+            this.partIt = partIt;
+            this.entryIt = entryIt;
+            this.swapLsnr = swapLsnr;
+            this.part = part;
+            this.loc = loc;
+            this.updateSeq = updateSeq;
+        }
+    }
+
+    @Deprecated//Backward compatibility. To be removed in future.
+    public void startOldListeners() {
+        if (!cctx.kernalContext().clientNode() && cctx.rebalanceEnabled()) {
+            cctx.io().addHandler(cctx.cacheId(), GridDhtPartitionDemandMessage.class, new CI2<UUID, GridDhtPartitionDemandMessage>() {
+                @Override public void apply(UUID id, GridDhtPartitionDemandMessage m) {
+                    processOldDemandMessage(m, id);
+                }
+            });
+        }
+    }
+
+    @Deprecated//Backward compatibility. To be removed in future.
+    public void stopOldListeners() {
+        if (!cctx.kernalContext().clientNode() && cctx.rebalanceEnabled()) {
+
+            cctx.io().removeHandler(cctx.cacheId(), GridDhtPartitionDemandMessage.class);
+        }
+    }
+
+    /**
+     * @param d D.
+     * @param id Id.
+     */
+    @Deprecated//Backward compatibility. To be removed in future.
+    private void processOldDemandMessage(GridDhtPartitionDemandMessage d, UUID id) {
+        GridDhtPartitionSupplyMessage s = new GridDhtPartitionSupplyMessage(d.workerId(),
+            d.updateSequence(), cctx.cacheId(), cctx.deploymentEnabled());
+
+        ClusterNode node = cctx.node(id);
+
+        long preloadThrottle = cctx.config().getRebalanceThrottle();
+
+        boolean ack = false;
+
+        try {
+            for (int part : d.partitions()) {
+                GridDhtLocalPartition loc = top.localPartition(part, d.topologyVersion(), false);
+
+                if (loc == null || loc.state() != OWNING || !loc.reserve()) {
+                    // Reply with partition of "-1" to let sender know that
+                    // this node is no longer an owner.
+                    s.missed(part);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Requested partition is not owned by local node [part=" + part +
+                            ", demander=" + id + ']');
+
+                    continue;
+                }
+
+                GridCacheEntryInfoCollectSwapListener swapLsnr = null;
+
+                try {
+                    if (cctx.isSwapOrOffheapEnabled()) {
+                        swapLsnr = new GridCacheEntryInfoCollectSwapListener(log);
+
+                        cctx.swap().addOffHeapListener(part, swapLsnr);
+                        cctx.swap().addSwapListener(part, swapLsnr);
+                    }
+
+                    boolean partMissing = false;
+
+                    for (GridCacheEntryEx e : loc.entries()) {
+                        if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
+                            // Demander no longer needs this partition, so we send '-1' partition and move on.
+                            s.missed(part);
+
+                            if (log.isDebugEnabled())
+                                log.debug("Demanding node does not need requested partition [part=" + part +
+                                    ", nodeId=" + id + ']');
+
+                            partMissing = true;
+
+                            break;
+                        }
+
+                        if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
+                            ack = true;
+
+                            if (!replyOld(node, d, s))
+                                return;
+
+                            // Throttle preloading.
+                            if (preloadThrottle > 0)
+                                U.sleep(preloadThrottle);
+
+                            s = new GridDhtPartitionSupplyMessage(d.workerId(), d.updateSequence(),
+                                cctx.cacheId(), cctx.deploymentEnabled());
+                        }
+
+                        GridCacheEntryInfo info = e.info();
+
+                        if (info != null && !info.isNew()) {
+                            if (preloadPred == null || preloadPred.apply(info))
+                                s.addEntry(part, info, cctx);
+                            else if (log.isDebugEnabled())
+                                log.debug("Rebalance predicate evaluated to false (will not sender cache entry): " +
+                                    info);
+                        }
+                    }
+
+                    if (partMissing)
+                        continue;
+
+                    if (cctx.isSwapOrOffheapEnabled()) {
+                        GridCloseableIterator<Map.Entry<byte[], GridCacheSwapEntry>> iter =
+                            cctx.swap().iterator(part);
+
+                        // Iterator may be null if space does not exist.
+                        if (iter != null) {
+                            try {
+                                boolean prepared = false;
+
+                                for (Map.Entry<byte[], GridCacheSwapEntry> e : iter) {
+                                    if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
+                                        // Demander no longer needs this partition,
+                                        // so we send '-1' partition and move on.
+                                        s.missed(part);
+
+                                        if (log.isDebugEnabled())
+                                            log.debug("Demanding node does not need requested partition " +
+                                                "[part=" + part + ", nodeId=" + id + ']');
+
+                                        partMissing = true;
+
+                                        break; // For.
+                                    }
+
+                                    if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
+                                        ack = true;
+
+                                        if (!replyOld(node, d, s))
+                                            return;
+
+                                        // Throttle preloading.
+                                        if (preloadThrottle > 0)
+                                            U.sleep(preloadThrottle);
+
+                                        s = new GridDhtPartitionSupplyMessage(d.workerId(),
+                                            d.updateSequence(), cctx.cacheId(), cctx.deploymentEnabled());
+                                    }
+
+                                    GridCacheSwapEntry swapEntry = e.getValue();
+
+                                    GridCacheEntryInfo info = new GridCacheEntryInfo();
+
+                                    info.keyBytes(e.getKey());
+                                    info.ttl(swapEntry.ttl());
+                                    info.expireTime(swapEntry.expireTime());
+                                    info.version(swapEntry.version());
+                                    info.value(swapEntry.value());
+
+                                    if (preloadPred == null || preloadPred.apply(info))
+                                        s.addEntry0(part, info, cctx);
+                                    else {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Rebalance predicate evaluated to false (will not send " +
+                                                "cache entry): " + info);
+
+                                        continue;
+                                    }
+
+                                    // Need to manually prepare cache message.
+                                    if (depEnabled && !prepared) {
+                                        ClassLoader ldr = swapEntry.keyClassLoaderId() != null ?
+                                            cctx.deploy().getClassLoader(swapEntry.keyClassLoaderId()) :
+                                            swapEntry.valueClassLoaderId() != null ?
+                                                cctx.deploy().getClassLoader(swapEntry.valueClassLoaderId()) :
+                                                null;
+
+                                        if (ldr == null)
+                                            continue;
+
+                                        if (ldr instanceof GridDeploymentInfo) {
+                                            s.prepare((GridDeploymentInfo)ldr);
+
+                                            prepared = true;
+                                        }
+                                    }
+                                }
+
+                                if (partMissing)
+                                    continue;
+                            }
+                            finally {
+                                iter.close();
+                            }
+                        }
+                    }
+
+                    // Stop receiving promote notifications.
+                    if (swapLsnr != null) {
+                        cctx.swap().removeOffHeapListener(part, swapLsnr);
+                        cctx.swap().removeSwapListener(part, swapLsnr);
+                    }
+
+                    if (swapLsnr != null) {
+                        Collection<GridCacheEntryInfo> entries = swapLsnr.entries();
+
+                        swapLsnr = null;
+
+                        for (GridCacheEntryInfo info : entries) {
+                            if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
+                                // Demander no longer needs this partition,
+                                // so we send '-1' partition and move on.
+                                s.missed(part);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Demanding node does not need requested partition " +
+                                        "[part=" + part + ", nodeId=" + id + ']');
+
+                                // No need to continue iteration over swap entries.
+                                break;
+                            }
+
+                            if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
+                                ack = true;
+
+                                if (!replyOld(node, d, s))
+                                    return;
+
+                                s = new GridDhtPartitionSupplyMessage(d.workerId(),
+                                    d.updateSequence(),
+                                    cctx.cacheId(),
+                                    cctx.deploymentEnabled());
+                            }
+
+                            if (preloadPred == null || preloadPred.apply(info))
+                                s.addEntry(part, info, cctx);
+                            else if (log.isDebugEnabled())
+                                log.debug("Rebalance predicate evaluated to false (will not sender cache entry): " +
+                                    info);
+                        }
+                    }
+
+                    // Mark as last supply message.
+                    s.last(part);
+
+                    if (ack) {
+                        s.markAck();
+
+                        break; // Partition for loop.
+                    }
+                }
+                finally {
+                    loc.release();
+
+                    if (swapLsnr != null) {
+                        cctx.swap().removeOffHeapListener(part, swapLsnr);
+                        cctx.swap().removeSwapListener(part, swapLsnr);
+                    }
+                }
+            }
+
+            replyOld(node, d, s);
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send partition supply message to node: " + node.id(), e);
+        }
+    }
+
+    /**
+     * @param n Node.
+     * @param d Demand message.
+     * @param s Supply message.
+     * @return {@code True} if message was sent, {@code false} if recipient left grid.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Deprecated//Backward compatibility. To be removed in future.
+    private boolean replyOld(ClusterNode n, GridDhtPartitionDemandMessage d, GridDhtPartitionSupplyMessage s)
+        throws IgniteCheckedException {
+        try {
+            if (log.isDebugEnabled())
+                log.debug("Replying to partition demand [node=" + n.id() + ", demand=" + d + ", supply=" + s + ']');
+
+            cctx.io().sendOrderedMessage(n, d.topic(), s, cctx.ioPolicy(), d.timeout());
+
+            return true;
+        }
+        catch (ClusterTopologyCheckedException ignore) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to send partition supply message because node left grid: " + n.id());
+
+            return false;
+        }
+    }
+
+    /**
+     * Dumps debug information.
+     */
+    public void dumpDebugInfo() {
+        synchronized (scMap) {
+            if (!scMap.isEmpty()) {
+                U.warn(log, "Rebalancing supplier reserved following partitions:");
+
+                for (SupplyContext sc : scMap.values()) {
+                    if (sc.loc != null)
+                        U.warn(log, ">>> " + sc.loc);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java
new file mode 100644
index 0000000..41454f9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessageV2.java
@@ -0,0 +1,380 @@
+/*
+ *  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.dht.preloader;
+
+import java.io.Externalizable;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectCollection;
+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.CacheEntryInfoCollection;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ * Partition supply message.
+ */
+public class GridDhtPartitionSupplyMessageV2 extends GridCacheMessage implements GridCacheDeployable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Update sequence. */
+    private long updateSeq;
+
+    /** Topology version. */
+    private AffinityTopologyVersion topVer;
+
+    /** Partitions that have been fully sent. */
+    @GridDirectCollection(int.class)
+    private Collection<Integer> last;
+
+    /** Partitions which were not found. */
+    @GridToStringInclude
+    @GridDirectCollection(int.class)
+    private Collection<Integer> missed;
+
+    /** Entries. */
+    @GridDirectMap(keyType = int.class, valueType = CacheEntryInfoCollection.class)
+    private Map<Integer, CacheEntryInfoCollection> infos;
+
+    /** Message size. */
+    @GridDirectTransient
+    private int msgSize;
+
+    /**
+     * @param updateSeq Update sequence for this node.
+     * @param cacheId Cache ID.
+     * @param addDepInfo Deployment info flag.
+     */
+    GridDhtPartitionSupplyMessageV2(long updateSeq, int cacheId, AffinityTopologyVersion topVer, boolean addDepInfo) {
+        this.cacheId = cacheId;
+        this.updateSeq = updateSeq;
+        this.topVer = topVer;
+        this.addDepInfo = addDepInfo;    }
+
+    /**
+     * Empty constructor required for {@link Externalizable}.
+     */
+    public GridDhtPartitionSupplyMessageV2() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean ignoreClassErrors() {
+        return true;
+    }
+
+    /**
+     * @return Update sequence.
+     */
+    long updateSequence() {
+        return updateSeq;
+    }
+
+    /**
+     * @return Topology version for which demand message is sent.
+     */
+    @Override public AffinityTopologyVersion topologyVersion() {
+        return topVer;
+    }
+
+    /**
+     * @return Flag to indicate last message for partition.
+     */
+    Collection<Integer> last() {
+        return last == null ? Collections.<Integer>emptySet() : last;
+    }
+
+    /**
+     * @param p Partition which was fully sent.
+     */
+    void last(int p) {
+        if (last == null)
+            last = new HashSet<>();
+
+        if (last.add(p)) {
+            msgSize += 4;
+
+            // If partition is empty, we need to add it.
+            if (!infos().containsKey(p)) {
+                CacheEntryInfoCollection infoCol = new CacheEntryInfoCollection();
+
+                infoCol.init();
+
+                infos().put(p, infoCol);
+            }
+        }
+    }
+
+    /**
+     * @param p Missed partition.
+     */
+    void missed(int p) {
+        if (missed == null)
+            missed = new HashSet<>();
+
+        if (missed.add(p))
+            msgSize += 4;
+    }
+
+    /**
+     * @return Missed partitions.
+     */
+    Collection<Integer> missed() {
+        return missed == null ? Collections.<Integer>emptySet() : missed;
+    }
+
+    /**
+     * @return Entries.
+     */
+    Map<Integer, CacheEntryInfoCollection> infos() {
+        if (infos == null)
+            infos = new HashMap<>();
+
+        return infos;
+    }
+
+    /**
+     * @return Message size.
+     */
+    int messageSize() {
+        return msgSize;
+    }
+
+    /**
+     * @param p Partition.
+     * @param info Entry to add.
+     * @param ctx Cache context.
+     * @throws IgniteCheckedException If failed.
+     */
+    void addEntry(int p, GridCacheEntryInfo info, GridCacheContext ctx) throws IgniteCheckedException {
+        assert info != null;
+
+        marshalInfo(info, ctx);
+
+        msgSize += info.marshalledSize(ctx);
+
+        CacheEntryInfoCollection infoCol = infos().get(p);
+
+        if (infoCol == null) {
+            msgSize += 4;
+
+            infos().put(p, infoCol = new CacheEntryInfoCollection());
+
+            infoCol.init();
+        }
+
+        infoCol.add(info);
+    }
+
+    /**
+     * @param p Partition.
+     * @param info Entry to add.
+     * @param ctx Cache context.
+     * @throws IgniteCheckedException If failed.
+     */
+    void addEntry0(int p, GridCacheEntryInfo info, GridCacheContext ctx) throws IgniteCheckedException {
+        assert info != null;
+        assert (info.key() != null || info.keyBytes() != null);
+        assert info.value() != null;
+
+        // Need to call this method to initialize info properly.
+        marshalInfo(info, ctx);
+
+        msgSize += info.marshalledSize(ctx);
+
+        CacheEntryInfoCollection infoCol = infos().get(p);
+
+        if (infoCol == null) {
+            msgSize += 4;
+
+            infos().put(p, infoCol = new CacheEntryInfoCollection());
+
+            infoCol.init();
+        }
+
+        infoCol.add(info);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
+    @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException {
+        super.finishUnmarshal(ctx, ldr);
+
+        GridCacheContext cacheCtx = ctx.cacheContext(cacheId);
+
+        for (CacheEntryInfoCollection col : infos().values()) {
+            List<GridCacheEntryInfo> entries = col.infos();
+
+            for (int i = 0; i < entries.size(); i++)
+                entries.get(i).unmarshal(cacheCtx, ldr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean addDeploymentInfo() {
+        return addDepInfo;
+    }
+
+    /**
+     * @return Number of entries in message.
+     */
+    public int size() {
+        return infos().size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 3:
+                if (!writer.writeMap("infos", infos, MessageCollectionItemType.INT, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeCollection("last", last, MessageCollectionItemType.INT))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
+                if (!writer.writeCollection("missed", missed, MessageCollectionItemType.INT))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
+                if (!writer.writeMessage("topVer", topVer))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
+                if (!writer.writeLong("updateSeq", updateSeq))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 3:
+                infos = reader.readMap("infos", MessageCollectionItemType.INT, MessageCollectionItemType.MSG, false);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                last = reader.readCollection("last", MessageCollectionItemType.INT);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
+                missed = reader.readCollection("missed", MessageCollectionItemType.INT);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 6:
+                topVer = reader.readMessage("topVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
+                updateSeq = reader.readLong("updateSeq");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GridDhtPartitionSupplyMessageV2.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return 114;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridDhtPartitionSupplyMessageV2.class, this,
+            "size", size(),
+            "parts", infos().keySet(),
+            "super", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
deleted file mode 100644
index 28a73b1..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
+++ /dev/null
@@ -1,555 +0,0 @@
-/*
- * 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.dht.preloader;
-
-import java.io.Externalizable;
-import java.util.Collection;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.locks.ReadWriteLock;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
-import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryInfoCollectSwapListener;
-import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
-import org.apache.ignite.internal.util.lang.GridCloseableIterator;
-import org.apache.ignite.internal.util.typedef.CI2;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.util.worker.GridWorker;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.thread.IgniteThread;
-import org.jetbrains.annotations.Nullable;
-
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING;
-
-/**
- * Thread pool for supplying partitions to demanding nodes.
- */
-class GridDhtPartitionSupplyPool {
-    /** */
-    private final GridCacheContext<?, ?> cctx;
-
-    /** */
-    private final IgniteLogger log;
-
-    /** */
-    private final ReadWriteLock busyLock;
-
-    /** */
-    private GridDhtPartitionTopology top;
-
-    /** */
-    private final Collection<SupplyWorker> workers = new LinkedList<>();
-
-    /** */
-    private final BlockingQueue<DemandMessage> queue = new LinkedBlockingDeque<>();
-
-    /** */
-    private final boolean depEnabled;
-
-    /** Preload predicate. */
-    private IgnitePredicate<GridCacheEntryInfo> preloadPred;
-
-    /**
-     * @param cctx Cache context.
-     * @param busyLock Shutdown lock.
-     */
-    GridDhtPartitionSupplyPool(GridCacheContext<?, ?> cctx, ReadWriteLock busyLock) {
-        assert cctx != null;
-        assert busyLock != null;
-
-        this.cctx = cctx;
-        this.busyLock = busyLock;
-
-        log = cctx.logger(getClass());
-
-        top = cctx.dht().topology();
-
-        if (!cctx.kernalContext().clientNode()) {
-            int poolSize = cctx.rebalanceEnabled() ? cctx.config().getRebalanceThreadPoolSize() : 0;
-
-            for (int i = 0; i < poolSize; i++)
-                workers.add(new SupplyWorker());
-
-            cctx.io().addHandler(cctx.cacheId(), GridDhtPartitionDemandMessage.class, new CI2<UUID, GridDhtPartitionDemandMessage>() {
-                @Override public void apply(UUID id, GridDhtPartitionDemandMessage m) {
-                    processDemandMessage(id, m);
-                }
-            });
-        }
-
-        depEnabled = cctx.gridDeploy().enabled();
-    }
-
-    /**
-     *
-     */
-    void start() {
-        for (SupplyWorker w : workers)
-            new IgniteThread(cctx.gridName(), "preloader-supply-worker", w).start();
-    }
-
-    /**
-     *
-     */
-    void stop() {
-        U.cancel(workers);
-        U.join(workers, log);
-
-        top = null;
-    }
-
-    /**
-     * Sets preload predicate for supply pool.
-     *
-     * @param preloadPred Preload predicate.
-     */
-    void preloadPredicate(IgnitePredicate<GridCacheEntryInfo> preloadPred) {
-        this.preloadPred = preloadPred;
-    }
-
-    /**
-     * @return Size of this thread pool.
-     */
-    int poolSize() {
-        return cctx.config().getRebalanceThreadPoolSize();
-    }
-
-    /**
-     * @return {@code true} if entered to busy state.
-     */
-    private boolean enterBusy() {
-        if (busyLock.readLock().tryLock())
-            return true;
-
-        if (log.isDebugEnabled())
-            log.debug("Failed to enter to busy state (supplier is stopping): " + cctx.nodeId());
-
-        return false;
-    }
-
-    /**
-     * @param nodeId Sender node ID.
-     * @param d Message.
-     */
-    private void processDemandMessage(UUID nodeId, GridDhtPartitionDemandMessage d) {
-        if (!enterBusy())
-            return;
-
-        try {
-            if (cctx.rebalanceEnabled()) {
-                if (log.isDebugEnabled())
-                    log.debug("Received partition demand [node=" + nodeId + ", demand=" + d + ']');
-
-                queue.offer(new DemandMessage(nodeId, d));
-            }
-            else
-                U.warn(log, "Received partition demand message when rebalancing is disabled (will ignore): " + d);
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /**
-     *
-     */
-    private void leaveBusy() {
-        busyLock.readLock().unlock();
-    }
-
-    /**
-     * @param deque Deque to poll from.
-     * @param w Worker.
-     * @return Polled item.
-     * @throws InterruptedException If interrupted.
-     */
-    @Nullable private <T> T poll(BlockingQueue<T> deque, GridWorker w) throws InterruptedException {
-        assert w != null;
-
-        // There is currently a case where {@code interrupted}
-        // flag on a thread gets flipped during stop which causes the pool to hang.  This check
-        // will always make sure that interrupted flag gets reset before going into wait conditions.
-        // The true fix should actually make sure that interrupted flag does not get reset or that
-        // interrupted exception gets propagated. Until we find a real fix, this method should
-        // always work to make sure that there is no hanging during stop.
-        if (w.isCancelled())
-            Thread.currentThread().interrupt();
-
-        return deque.poll(2000, MILLISECONDS);
-    }
-
-    /**
-     * Supply work.
-     */
-    private class SupplyWorker extends GridWorker {
-        /** Hide worker logger and use cache logger. */
-        private IgniteLogger log = GridDhtPartitionSupplyPool.this.log;
-
-        /**
-         * Default constructor.
-         */
-        private SupplyWorker() {
-            super(cctx.gridName(), "preloader-supply-worker", GridDhtPartitionSupplyPool.this.log);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
-            while (!isCancelled()) {
-                DemandMessage msg = poll(queue, this);
-
-                if (msg == null)
-                    continue;
-
-                ClusterNode node = cctx.discovery().node(msg.senderId());
-
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Received message from non-existing node (will ignore): " + msg);
-
-                    continue;
-                }
-
-                processMessage(msg, node);
-            }
-        }
-
-        /**
-         * @param msg Message.
-         * @param node Demander.
-         */
-        private void processMessage(DemandMessage msg, ClusterNode node) {
-            assert msg != null;
-            assert node != null;
-
-            GridDhtPartitionDemandMessage d = msg.message();
-
-            GridDhtPartitionSupplyMessage s = new GridDhtPartitionSupplyMessage(d.workerId(),
-                d.updateSequence(), cctx.cacheId(), cctx.deploymentEnabled());
-
-            long preloadThrottle = cctx.config().getRebalanceThrottle();
-
-            boolean ack = false;
-
-            try {
-                for (int part : d.partitions()) {
-                    GridDhtLocalPartition loc = top.localPartition(part, d.topologyVersion(), false);
-
-                    if (loc == null || loc.state() != OWNING || !loc.reserve()) {
-                        // Reply with partition of "-1" to let sender know that
-                        // this node is no longer an owner.
-                        s.missed(part);
-
-                        if (log.isDebugEnabled())
-                            log.debug("Requested partition is not owned by local node [part=" + part +
-                                ", demander=" + msg.senderId() + ']');
-
-                        continue;
-                    }
-
-                    GridCacheEntryInfoCollectSwapListener swapLsnr = null;
-
-                    try {
-                        if (cctx.isSwapOrOffheapEnabled()) {
-                            swapLsnr = new GridCacheEntryInfoCollectSwapListener(log);
-
-                            cctx.swap().addOffHeapListener(part, swapLsnr);
-                            cctx.swap().addSwapListener(part, swapLsnr);
-                        }
-
-                        boolean partMissing = false;
-
-                        for (GridCacheEntryEx e : loc.entries()) {
-                            if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
-                                // Demander no longer needs this partition, so we send '-1' partition and move on.
-                                s.missed(part);
-
-                                if (log.isDebugEnabled())
-                                    log.debug("Demanding node does not need requested partition [part=" + part +
-                                        ", nodeId=" + msg.senderId() + ']');
-
-                                partMissing = true;
-
-                                break;
-                            }
-
-                            if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
-                                ack = true;
-
-                                if (!reply(node, d, s))
-                                    return;
-
-                                // Throttle preloading.
-                                if (preloadThrottle > 0)
-                                    U.sleep(preloadThrottle);
-
-                                s = new GridDhtPartitionSupplyMessage(d.workerId(), d.updateSequence(),
-                                    cctx.cacheId(), cctx.deploymentEnabled());
-                            }
-
-                            GridCacheEntryInfo info = e.info();
-
-                            if (info != null && !info.isNew()) {
-                                if (preloadPred == null || preloadPred.apply(info))
-                                    s.addEntry(part, info, cctx);
-                                else if (log.isDebugEnabled())
-                                    log.debug("Rebalance predicate evaluated to false (will not sender cache entry): " +
-                                        info);
-                            }
-                        }
-
-                        if (partMissing)
-                            continue;
-
-                        if (cctx.isSwapOrOffheapEnabled()) {
-                            GridCloseableIterator<Map.Entry<byte[], GridCacheSwapEntry>> iter =
-                                cctx.swap().iterator(part);
-
-                            // Iterator may be null if space does not exist.
-                            if (iter != null) {
-                                try {
-                                    boolean prepared = false;
-
-                                    for (Map.Entry<byte[], GridCacheSwapEntry> e : iter) {
-                                        if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
-                                            // Demander no longer needs this partition,
-                                            // so we send '-1' partition and move on.
-                                            s.missed(part);
-
-                                            if (log.isDebugEnabled())
-                                                log.debug("Demanding node does not need requested partition " +
-                                                    "[part=" + part + ", nodeId=" + msg.senderId() + ']');
-
-                                            partMissing = true;
-
-                                            break; // For.
-                                        }
-
-                                        if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
-                                            ack = true;
-
-                                            if (!reply(node, d, s))
-                                                return;
-
-                                            // Throttle preloading.
-                                            if (preloadThrottle > 0)
-                                                U.sleep(preloadThrottle);
-
-                                            s = new GridDhtPartitionSupplyMessage(d.workerId(),
-                                                d.updateSequence(), cctx.cacheId(), cctx.deploymentEnabled());
-                                        }
-
-                                        GridCacheSwapEntry swapEntry = e.getValue();
-
-                                        GridCacheEntryInfo info = new GridCacheEntryInfo();
-
-                                        info.keyBytes(e.getKey());
-                                        info.ttl(swapEntry.ttl());
-                                        info.expireTime(swapEntry.expireTime());
-                                        info.version(swapEntry.version());
-                                        info.value(swapEntry.value());
-
-                                        if (preloadPred == null || preloadPred.apply(info))
-                                            s.addEntry0(part, info, cctx);
-                                        else {
-                                            if (log.isDebugEnabled())
-                                                log.debug("Rebalance predicate evaluated to false (will not send " +
-                                                    "cache entry): " + info);
-
-                                            continue;
-                                        }
-
-                                        // Need to manually prepare cache message.
-                                        if (depEnabled && !prepared) {
-                                            ClassLoader ldr = swapEntry.keyClassLoaderId() != null ?
-                                                cctx.deploy().getClassLoader(swapEntry.keyClassLoaderId()) :
-                                                swapEntry.valueClassLoaderId() != null ?
-                                                    cctx.deploy().getClassLoader(swapEntry.valueClassLoaderId()) :
-                                                    null;
-
-                                            if (ldr == null)
-                                                continue;
-
-                                            if (ldr instanceof GridDeploymentInfo) {
-                                                s.prepare((GridDeploymentInfo)ldr);
-
-                                                prepared = true;
-                                            }
-                                        }
-                                    }
-
-                                    if (partMissing)
-                                        continue;
-                                }
-                                finally {
-                                    iter.close();
-                                }
-                            }
-                        }
-
-                        // Stop receiving promote notifications.
-                        if (swapLsnr != null) {
-                            cctx.swap().removeOffHeapListener(part, swapLsnr);
-                            cctx.swap().removeSwapListener(part, swapLsnr);
-                        }
-
-                        if (swapLsnr != null) {
-                            Collection<GridCacheEntryInfo> entries = swapLsnr.entries();
-
-                            swapLsnr = null;
-
-                            for (GridCacheEntryInfo info : entries) {
-                                if (!cctx.affinity().belongs(node, part, d.topologyVersion())) {
-                                    // Demander no longer needs this partition,
-                                    // so we send '-1' partition and move on.
-                                    s.missed(part);
-
-                                    if (log.isDebugEnabled())
-                                        log.debug("Demanding node does not need requested partition " +
-                                            "[part=" + part + ", nodeId=" + msg.senderId() + ']');
-
-                                    // No need to continue iteration over swap entries.
-                                    break;
-                                }
-
-                                if (s.messageSize() >= cctx.config().getRebalanceBatchSize()) {
-                                    ack = true;
-
-                                    if (!reply(node, d, s))
-                                        return;
-
-                                    s = new GridDhtPartitionSupplyMessage(d.workerId(),
-                                        d.updateSequence(),
-                                        cctx.cacheId(), cctx.deploymentEnabled());
-                                }
-
-                                if (preloadPred == null || preloadPred.apply(info))
-                                    s.addEntry(part, info, cctx);
-                                else if (log.isDebugEnabled())
-                                    log.debug("Rebalance predicate evaluated to false (will not sender cache entry): " +
-                                        info);
-                            }
-                        }
-
-                        // Mark as last supply message.
-                        s.last(part);
-
-                        if (ack) {
-                            s.markAck();
-
-                            break; // Partition for loop.
-                        }
-                    }
-                    finally {
-                        loc.release();
-
-                        if (swapLsnr != null) {
-                            cctx.swap().removeOffHeapListener(part, swapLsnr);
-                            cctx.swap().removeSwapListener(part, swapLsnr);
-                        }
-                    }
-                }
-
-                reply(node, d, s);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send partition supply message to node: " + node.id(), e);
-            }
-        }
-
-        /**
-         * @param n Node.
-         * @param d Demand message.
-         * @param s Supply message.
-         * @return {@code True} if message was sent, {@code false} if recipient left grid.
-         * @throws IgniteCheckedException If failed.
-         */
-        private boolean reply(ClusterNode n, GridDhtPartitionDemandMessage d, GridDhtPartitionSupplyMessage s)
-            throws IgniteCheckedException {
-            try {
-                if (log.isDebugEnabled())
-                    log.debug("Replying to partition demand [node=" + n.id() + ", demand=" + d + ", supply=" + s + ']');
-
-                cctx.io().sendOrderedMessage(n, d.topic(), s, cctx.ioPolicy(), d.timeout());
-
-                return true;
-            }
-            catch (ClusterTopologyCheckedException ignore) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to send partition supply message because node left grid: " + n.id());
-
-                return false;
-            }
-        }
-    }
-
-    /**
-     * Demand message wrapper.
-     */
-    private static class DemandMessage extends IgniteBiTuple<UUID, GridDhtPartitionDemandMessage> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * @param sndId Sender ID.
-         * @param msg Message.
-         */
-        DemandMessage(UUID sndId, GridDhtPartitionDemandMessage msg) {
-            super(sndId, msg);
-        }
-
-        /**
-         * Empty constructor required for {@link Externalizable}.
-         */
-        public DemandMessage() {
-            // No-op.
-        }
-
-        /**
-         * @return Sender ID.
-         */
-        UUID senderId() {
-            return get1();
-        }
-
-        /**
-         * @return Message.
-         */
-        public GridDhtPartitionDemandMessage message() {
-            return get2();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "DemandMessage [senderId=" + senderId() + ", msg=" + message() + ']';
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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 cef38e8..2f2944d 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
@@ -742,6 +742,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     // Must initialize topology after we get discovery event.
                     initTopology(cacheCtx);
 
+                    cacheCtx.preloader().onTopologyChanged(exchId.topologyVersion());
+
                     cacheCtx.preloader().updateLastExchangeFuture(this);
                 }
 


[38/46] ignite git commit: Ignite-1093 Backward compatibility fix.

Posted by ak...@apache.org.
Ignite-1093 Backward compatibility fix.


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

Branch: refs/heads/ignite-1753-1282
Commit: c9cea76656fb00c03811350260bd158ffdf8739a
Parents: 7a40364
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Nov 13 14:19:13 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Nov 13 14:19:13 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/preloader/GridDhtPartitionSupplier.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c9cea766/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
index 865bad8..4e33d8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
@@ -759,6 +759,9 @@ class GridDhtPartitionSupplier {
 
         ClusterNode node = cctx.node(id);
 
+        if (node == null)
+            return;
+
         long preloadThrottle = cctx.config().getRebalanceThrottle();
 
         boolean ack = false;


[11/46] ignite git commit: Performance optimizations.

Posted by ak...@apache.org.
Performance optimizations.


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

Branch: refs/heads/ignite-1753-1282
Commit: a4848a702fea1573af7f36af91d02f7df3ab64f4
Parents: 7393227
Author: sboikov <sb...@gridgain.com>
Authored: Mon Nov 9 12:16:16 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Nov 9 12:16:16 2015 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoMessage.java   |   4 +-
 .../processors/cache/GridCacheContext.java      |  29 +--
 .../processors/cache/GridCacheEntryEx.java      |   4 +-
 .../processors/cache/GridCacheMapEntry.java     |  55 ++--
 .../processors/cache/GridCacheMvccManager.java  | 145 +++++------
 .../distributed/GridDistributedCacheEntry.java  |   2 +-
 .../distributed/GridDistributedTxMapping.java   |   8 +-
 .../GridDistributedTxRemoteAdapter.java         |   5 +-
 .../distributed/dht/GridDhtLockFuture.java      |   7 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  13 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  43 ++--
 .../distributed/dht/GridDhtTxPrepareFuture.java |  78 +++---
 .../cache/distributed/dht/GridDhtTxRemote.java  |  45 ++--
 .../dht/atomic/GridDhtAtomicCache.java          |   1 -
 .../dht/colocated/GridDhtColocatedCache.java    |   7 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  11 +-
 .../near/GridNearTransactionalCache.java        |   7 +-
 .../near/GridNearTxFinishFuture.java            | 157 ++++++------
 .../cache/distributed/near/GridNearTxLocal.java |  21 +-
 .../cache/transactions/IgniteInternalTx.java    |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     | 251 ++++++++++---------
 .../cache/transactions/IgniteTxHandler.java     |  37 +--
 .../transactions/IgniteTxLocalAdapter.java      |  26 +-
 .../cache/transactions/IgniteTxManager.java     | 171 ++++++-------
 .../GridBoundedConcurrentLinkedHashMap.java     |   7 +-
 .../GridBoundedConcurrentLinkedHashSet.java     |   7 +-
 .../util/GridBoundedConcurrentOrderedMap.java   |  39 +--
 .../internal/util/GridConcurrentFactory.java    |  11 +-
 .../util/GridConcurrentLinkedHashSet.java       |   9 +-
 .../ignite/internal/util/IgniteUuidCache.java   |   6 +-
 .../util/future/GridCompoundFuture.java         | 155 ++++++++----
 .../java/org/jsr166/ConcurrentHashMap8.java     |   2 +-
 .../java/org/jsr166/ConcurrentLinkedDeque8.java |   2 +-
 .../org/jsr166/ConcurrentLinkedHashMap.java     | 195 +++++++++++---
 .../GridCacheAffinityBackupsSelfTest.java       |   8 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |   2 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |  40 +--
 .../processors/cache/GridCacheTestEntryEx.java  |   3 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   2 +-
 .../continuous/GridEventConsumeSelfTest.java    |   2 +-
 ...dBoundedConcurrentLinkedHashMapSelfTest.java |   2 +-
 .../GridConcurrentLinkedHashMapSelfTest.java    |  62 ++++-
 .../junits/common/GridCommonAbstractTest.java   |   4 +-
 ...rrentLinkedHashMapMultiThreadedSelfTest.java | 104 ++++----
 .../yardstick/cache/IgnitePutTxBenchmark.java   |  10 +
 .../cache/IgnitePutTxPrimaryOnlyBenchmark.java  |  65 +++++
 .../IgnitePutTxSkipLocalBackupBenchmark.java    |  65 +++++
 .../cache/WaitMapExchangeFinishCallable.java    |  95 +++++++
 48 files changed, 1220 insertions(+), 807 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
index c83a281..cb19ba0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
@@ -143,7 +143,7 @@ public class GridIoMessage implements Message {
     /**
      * @return Message.
      */
-    public Object message() {
+    public Message message() {
         return msg;
     }
 
@@ -320,4 +320,4 @@ public class GridIoMessage implements Message {
     @Override public String toString() {
         return S.toString(GridIoMessage.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/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 1f4852c..ee4da46 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
@@ -1488,10 +1488,9 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param log Log.
      * @param dhtMap Dht mappings.
      * @param nearMap Near mappings.
-     * @return {@code True} if mapped.
      * @throws GridCacheEntryRemovedException If reader for entry is removed.
      */
-    public boolean dhtMap(
+    public void dhtMap(
         UUID nearNodeId,
         AffinityTopologyVersion topVer,
         GridDhtCacheEntry entry,
@@ -1509,7 +1508,7 @@ public class GridCacheContext<K, V> implements Externalizable {
 
         Collection<ClusterNode> dhtRemoteNodes = F.view(dhtNodes, F.remoteNodes(nodeId())); // Exclude local node.
 
-        boolean ret = map(entry, dhtRemoteNodes, dhtMap);
+        map(entry, dhtRemoteNodes, dhtMap);
 
         Collection<ClusterNode> nearRemoteNodes = null;
 
@@ -1530,7 +1529,7 @@ public class GridCacheContext<K, V> implements Externalizable {
             if (nearNodes != null && !nearNodes.isEmpty()) {
                 nearRemoteNodes = F.view(nearNodes, F.notIn(dhtNodes));
 
-                ret |= map(entry, nearRemoteNodes, nearMap);
+                map(entry, nearRemoteNodes, nearMap);
             }
         }
 
@@ -1540,8 +1539,6 @@ public class GridCacheContext<K, V> implements Externalizable {
 
             entry.mappings(explicitLockVer, dhtNodeIds, nearNodeIds);
         }
-
-        return ret;
     }
 
     /**
@@ -1549,10 +1546,9 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param log Log.
      * @param dhtMap Dht mappings.
      * @param nearMap Near mappings.
-     * @return {@code True} if mapped.
      * @throws GridCacheEntryRemovedException If reader for entry is removed.
      */
-    public boolean dhtMap(
+    public void dhtMap(
         GridDhtCacheEntry entry,
         GridCacheVersion explicitLockVer,
         IgniteLogger log,
@@ -1571,27 +1567,20 @@ public class GridCacheContext<K, V> implements Externalizable {
 
             Collection<ClusterNode> nearNodes = cand.mappedNearNodes();
 
-            boolean ret = map(entry, dhtNodes, dhtMap);
+            map(entry, dhtNodes, dhtMap);
 
             if (nearNodes != null && !nearNodes.isEmpty())
-                ret |= map(entry, nearNodes, nearMap);
-
-            return ret;
+                map(entry, nearNodes, nearMap);
         }
-
-        return false;
     }
 
     /**
      * @param entry Entry.
      * @param nodes Nodes.
      * @param map Map.
-     * @return {@code True} if mapped.
      */
-    private boolean map(GridDhtCacheEntry entry, Iterable<ClusterNode> nodes,
+    private void map(GridDhtCacheEntry entry, Iterable<ClusterNode> nodes,
         Map<ClusterNode, List<GridDhtCacheEntry>> map) {
-        boolean ret = false;
-
         if (nodes != null) {
             for (ClusterNode n : nodes) {
                 List<GridDhtCacheEntry> entries = map.get(n);
@@ -1600,12 +1589,8 @@ public class GridCacheContext<K, V> implements Externalizable {
                     map.put(n, entries = new LinkedList<>());
 
                 entries.add(entry);
-
-                ret = true;
             }
         }
-
-        return ret;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 50b01c8..af62e39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -389,7 +389,6 @@ public interface GridCacheEntryEx {
      * @param tx Cache transaction.
      * @param evtNodeId ID of node responsible for this change.
      * @param affNodeId Partitioned node iD.
-     * @param writeThrough If {@code true}, persist to the storage.
      * @param retval {@code True} if value should be returned (and unmarshalled if needed).
      * @param evt Flag to signal event notification.
      * @param metrics Flag to signal metrics notification.
@@ -409,7 +408,6 @@ public interface GridCacheEntryEx {
         @Nullable IgniteInternalTx tx,
         UUID evtNodeId,
         UUID affNodeId,
-        boolean writeThrough,
         boolean retval,
         boolean evt,
         boolean metrics,
@@ -1014,4 +1012,4 @@ public interface GridCacheEntryEx {
      * Calls {@link GridDhtLocalPartition#onUnlock()} for this entry's partition.
      */
     public void onUnlock();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/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 ca0995a..df9f5c4 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
@@ -1108,7 +1108,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 CacheLazyEntry e = new CacheLazyEntry(cctx, key, old);
 
-                Object interceptorVal = cctx.config().getInterceptor().onBeforePut(new CacheLazyEntry(cctx, key, old),
+                Object interceptorVal = cctx.config().getInterceptor().onBeforePut(
+                    new CacheLazyEntry(cctx, key, old),
                     val0);
 
                 key0 = e.key();
@@ -1212,7 +1213,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         @Nullable IgniteInternalTx tx,
         UUID evtNodeId,
         UUID affNodeId,
-        boolean writeThrough,
         boolean retval,
         boolean evt,
         boolean metrics,
@@ -1244,6 +1244,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         Cache.Entry entry0 = null;
 
+        boolean deferred;
+
+        boolean marked = false;
+
         synchronized (this) {
             checkObsolete();
 
@@ -1349,40 +1353,33 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 cctx.continuousQueries().onEntryUpdated(this, key, null, old, false);
 
             cctx.dataStructures().onEntryUpdated(key, true);
-        }
-
-        // Persist outside of synchronization. The correctness of the
-        // value will be handled by current transaction.
-        if (writeThrough)
-            cctx.store().remove(tx, keyValue(false));
 
-        if (cctx.deferredDelete() && !detached() && !isInternal())
-            cctx.onDeferredDelete(this, newVer);
-        else {
-            boolean marked = false;
+            deferred = cctx.deferredDelete() && !detached() && !isInternal();
 
-            synchronized (this) {
+            if (!deferred) {
                 // If entry is still removed.
-                if (newVer == ver) {
-                    if (obsoleteVer == null || !(marked = markObsolete0(obsoleteVer, true, null))) {
-                        if (log.isDebugEnabled())
-                            log.debug("Entry could not be marked obsolete (it is still used): " + this);
-                    }
-                    else {
-                        recordNodeId(affNodeId, topVer);
+                assert newVer == ver;
 
-                        // If entry was not marked obsolete, then removed lock
-                        // will be registered whenever removeLock is called.
-                        cctx.mvcc().addRemoved(cctx, obsoleteVer);
+                if (obsoleteVer == null || !(marked = markObsolete0(obsoleteVer, true, null))) {
+                    if (log.isDebugEnabled())
+                        log.debug("Entry could not be marked obsolete (it is still used): " + this);
+                }
+                else {
+                    recordNodeId(affNodeId, topVer);
 
-                        if (log.isDebugEnabled())
-                            log.debug("Entry was marked obsolete: " + this);
-                    }
+                    if (log.isDebugEnabled())
+                        log.debug("Entry was marked obsolete: " + this);
                 }
             }
+        }
 
-            if (marked)
-                onMarkedObsolete();
+        if (deferred)
+            cctx.onDeferredDelete(this, newVer);
+
+        if (marked) {
+            assert !deferred;
+
+            onMarkedObsolete();
         }
 
         if (intercept)
@@ -4247,4 +4244,4 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             return "IteratorEntry [key=" + key + ']';
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/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 0960c9d..2c14209 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -24,7 +25,6 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.Queue;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
@@ -88,7 +89,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     private ConcurrentMap<Long, GridCacheExplicitLockSpan> pendingExplicit;
 
     /** Set of removed lock versions. */
-    private Collection<GridCacheVersion> rmvLocks =
+    private GridBoundedConcurrentLinkedHashSet<GridCacheVersion> rmvLocks =
         new GridBoundedConcurrentLinkedHashSet<>(MAX_REMOVED_LOCKS, MAX_REMOVED_LOCKS, 0.75f, 16, PER_SEGMENT_Q);
 
     /** Current local candidates. */
@@ -114,7 +115,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     private final ConcurrentMap<GridCacheVersion, GridCacheVersion> near2dht = newMap();
 
     /** Finish futures. */
-    private final Queue<FinishLockFuture> finishFuts = new ConcurrentLinkedDeque8<>();
+    private final ConcurrentLinkedDeque8<FinishLockFuture> finishFuts = new ConcurrentLinkedDeque8<>();
 
     /** Logger. */
     @SuppressWarnings( {"FieldAccessedSynchronizedAndUnsynchronized"})
@@ -143,17 +144,18 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                 Collection<? extends GridCacheFuture> futCol = futs.get(owner.version());
 
                 if (futCol != null) {
-                    for (GridCacheFuture fut : futCol) {
-                        if (fut instanceof GridCacheMvccFuture && !fut.isDone()) {
-                            GridCacheMvccFuture<Boolean> mvccFut =
-                                (GridCacheMvccFuture<Boolean>)fut;
-
-                            // Since this method is called outside of entry synchronization,
-                            // we can safely invoke any method on the future.
-                            // Also note that we don't remove future here if it is done.
-                            // The removal is initiated from within future itself.
-                            if (mvccFut.onOwnerChanged(entry, owner))
-                                return;
+                    synchronized (futCol) {
+                        for (GridCacheFuture fut : futCol) {
+                            if (fut instanceof GridCacheMvccFuture && !fut.isDone()) {
+                                GridCacheMvccFuture<Boolean> mvccFut = (GridCacheMvccFuture<Boolean>)fut;
+
+                                // Since this method is called outside of entry synchronization,
+                                // we can safely invoke any method on the future.
+                                // Also note that we don't remove future here if it is done.
+                                // The removal is initiated from within future itself.
+                                if (mvccFut.onOwnerChanged(entry, owner))
+                                    return;
+                            }
                         }
                     }
                 }
@@ -171,8 +173,10 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
             else if (log.isDebugEnabled())
                 log.debug("Failed to find transaction for changed owner: " + owner);
 
-            for (FinishLockFuture f : finishFuts)
-                f.recheck(entry);
+            if (!finishFuts.isEmptyx()) {
+                for (FinishLockFuture f : finishFuts)
+                    f.recheck(entry);
+            }
         }
 
         /** {@inheritDoc} */
@@ -203,21 +207,8 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
             if (log.isDebugEnabled())
                 log.debug("Processing node left [nodeId=" + discoEvt.eventNode().id() + "]");
 
-            for (Collection<GridCacheFuture<?>> futsCol : futs.values()) {
-                for (GridCacheFuture<?> fut : futsCol) {
-                    if (!fut.trackable()) {
-                        if (log.isDebugEnabled())
-                            log.debug("Skipping non-trackable future: " + fut);
-
-                        continue;
-                    }
-
-                    fut.onNodeLeft(discoEvt.eventNode().id());
-
-                    if (fut.isCancelled() || fut.isDone())
-                        removeFuture(fut);
-                }
-            }
+            for (GridCacheFuture<?> fut : activeFutures())
+                fut.onNodeLeft(discoEvt.eventNode().id());
 
             for (IgniteInternalFuture<?> fut : atomicFuts.values()) {
                 if (fut instanceof GridCacheFuture) {
@@ -272,7 +263,15 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @return Collection of active futures.
      */
     public Collection<GridCacheFuture<?>> activeFutures() {
-        return F.flatCollections(futs.values());
+        ArrayList<GridCacheFuture<?>> col = new ArrayList<>();
+
+        for (Collection<GridCacheFuture<?>> verFuts : futs.values()) {
+            synchronized (verFuts) {
+                col.addAll(verFuts);
+            }
+        }
+
+        return col;
     }
 
     /**
@@ -345,10 +344,8 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @param err Error.
      */
     private void cancelClientFutures(IgniteCheckedException err) {
-        for (Collection<GridCacheFuture<?>> futures : futs.values()) {
-            for (GridCacheFuture<?> future : futures)
-                ((GridFutureAdapter)future).onDone(err);
-        }
+        for (GridCacheFuture<?> fut : activeFutures())
+            ((GridFutureAdapter)fut).onDone(err);
 
         for (GridCacheAtomicFuture<?> future : atomicFuts.values())
             ((GridFutureAdapter)future).onDone(err);
@@ -444,11 +441,10 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
             return true;
 
         while (true) {
-            Collection<GridCacheFuture<?>> old = futs.putIfAbsent(fut.version(),
-                new ConcurrentLinkedDeque8<GridCacheFuture<?>>() {
-                    /** */
-                    private int hash;
+            Collection<GridCacheFuture<?>> old = futs.get(fut.version());
 
+            if (old == null) {
+                Collection<GridCacheFuture<?>> col = new HashSet<GridCacheFuture<?>>(U.capacity(4), 0.75f) {
                     {
                         // Make sure that we add future to queue before
                         // adding queue to the map of futures.
@@ -456,16 +452,16 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                     }
 
                     @Override public int hashCode() {
-                        if (hash == 0)
-                            hash = System.identityHashCode(this);
-
-                        return hash;
+                        return System.identityHashCode(this);
                     }
 
                     @Override public boolean equals(Object obj) {
                         return obj == this;
                     }
-                });
+                };
+
+                old = futs.putIfAbsent(fut.version(), col);
+            }
 
             if (old != null) {
                 boolean empty, dup = false;
@@ -474,10 +470,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                     empty = old.isEmpty();
 
                     if (!empty)
-                        dup = old.contains(fut);
-
-                    if (!empty && !dup)
-                        old.add(fut);
+                        dup = !old.add(fut);
                 }
 
                 // Future is being removed, so we force-remove here and try again.
@@ -594,14 +587,18 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     @Nullable public GridCacheFuture future(GridCacheVersion ver, IgniteUuid futId) {
         Collection<? extends GridCacheFuture> futs = this.futs.get(ver);
 
-        if (futs != null)
-            for (GridCacheFuture<?> fut : futs)
-                if (fut.futureId().equals(futId)) {
-                    if (log.isDebugEnabled())
-                        log.debug("Found future in futures map: " + fut);
+        if (futs != null) {
+            synchronized (futs) {
+                for (GridCacheFuture<?> fut : futs) {
+                    if (fut.futureId().equals(futId)) {
+                        if (log.isDebugEnabled())
+                            log.debug("Found future in futures map: " + fut);
 
-                    return fut;
+                        return fut;
+                    }
                 }
+            }
+        }
 
         if (log.isDebugEnabled())
             log.debug("Failed to find future in futures map [ver=" + ver + ", futId=" + futId + ']');
@@ -619,7 +616,13 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     public <T> Collection<? extends IgniteInternalFuture<T>> futures(GridCacheVersion ver) {
         Collection c = futs.get(ver);
 
-        return c == null ? Collections.<IgniteInternalFuture<T>>emptyList() : (Collection<IgniteInternalFuture<T>>)c;
+        if (c == null)
+            return Collections.<IgniteInternalFuture<T>>emptyList();
+        else {
+            synchronized (c) {
+                return new ArrayList<>((Collection<IgniteInternalFuture<T>>)c);
+            }
+        }
     }
 
     /**
@@ -949,12 +952,12 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     @Override public void printMemoryStats() {
         X.println(">>> ");
         X.println(">>> Mvcc manager memory stats [grid=" + cctx.gridName() + ']');
-        X.println(">>>   rmvLocksSize: " + rmvLocks.size());
+        X.println(">>>   rmvLocksSize: " + rmvLocks.sizex());
         X.println(">>>   dhtLocCandsSize: " + dhtLocCands.size());
         X.println(">>>   lockedSize: " + locked.size());
         X.println(">>>   futsSize: " + futs.size());
         X.println(">>>   near2dhtSize: " + near2dht.size());
-        X.println(">>>   finishFutsSize: " + finishFuts.size());
+        X.println(">>>   finishFutsSize: " + finishFuts.sizex());
     }
 
     /**
@@ -974,9 +977,11 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     public Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> unfinishedLocks(AffinityTopologyVersion topVer) {
         Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> cands = new HashMap<>();
 
-        for (FinishLockFuture fut : finishFuts) {
-            if (fut.topologyVersion().equals(topVer))
-                cands.putAll(fut.pendingLocks());
+        if (!finishFuts.isEmptyx()) {
+            for (FinishLockFuture fut : finishFuts) {
+                if (fut.topologyVersion().equals(topVer))
+                    cands.putAll(fut.pendingLocks());
+            }
         }
 
         return cands;
@@ -1054,8 +1059,10 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @param topVer Topology version.
      * @return Future that signals when all locks for given partitions will be released.
      */
-    private IgniteInternalFuture<?> finishLocks(@Nullable final IgnitePredicate<GridDistributedCacheEntry> filter,
-        AffinityTopologyVersion topVer) {
+    private IgniteInternalFuture<?> finishLocks(
+        @Nullable final IgnitePredicate<GridDistributedCacheEntry> filter,
+        AffinityTopologyVersion topVer
+    ) {
         assert topVer.topologyVersion() != 0;
 
         if (topVer.equals(AffinityTopologyVersion.NONE))
@@ -1069,10 +1076,6 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         finishFut.listen(new CI1<IgniteInternalFuture<?>>() {
             @Override public void apply(IgniteInternalFuture<?> e) {
                 finishFuts.remove(finishFut);
-
-                // This call is required to make sure that the concurrent queue
-                // clears memory occupied by internal nodes.
-                finishFuts.peek();
             }
         });
 
@@ -1088,8 +1091,10 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         if (exchLog.isDebugEnabled())
             exchLog.debug("Rechecking pending locks for completion.");
 
-        for (FinishLockFuture fut : finishFuts)
-            fut.recheck();
+        if (!finishFuts.isEmptyx()) {
+            for (FinishLockFuture fut : finishFuts)
+                fut.recheck();
+        }
     }
 
     /**
@@ -1250,4 +1255,4 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                 return S.toString(FinishLockFuture.class, this, super.toString());
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index a138d30..a3eb723 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -403,7 +403,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
             doomed = mvcc == null ? null : mvcc.candidate(ver);
 
-            if (doomed == null || doomed.dhtLocal() || (!doomed.local() && !doomed.nearLocal()))
+            if (doomed == null)
                 addRemoved(ver);
 
             GridCacheVersion obsoleteVer = obsoleteVersionExtras();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
index 1e78ba2..2d2d935 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
@@ -23,13 +23,13 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.LinkedHashSet;
 import java.util.UUID;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.GridConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -87,7 +87,7 @@ public class GridDistributedTxMapping implements Externalizable {
     public GridDistributedTxMapping(ClusterNode node) {
         this.node = node;
 
-        entries = new GridConcurrentLinkedHashSet<>();
+        entries = new LinkedHashSet<>();
     }
 
     /**
@@ -297,7 +297,7 @@ public class GridDistributedTxMapping implements Externalizable {
      */
     private void ensureModifiable() {
         if (readOnly) {
-            entries = new GridConcurrentLinkedHashSet<>(entries);
+            entries = new LinkedHashSet<>(entries);
 
             readOnly = false;
         }
@@ -330,4 +330,4 @@ public class GridDistributedTxMapping implements Externalizable {
     @Override public String toString() {
         return S.toString(GridDistributedTxMapping.class, this, "node", node.id());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index fcbf58d..93303c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -583,13 +583,13 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                                 eventNodeId(),
                                                 nodeId,
                                                 false,
-                                                false,
                                                 true,
                                                 true,
                                                 topVer,
                                                 null,
                                                 replicate ? DR_BACKUP : DR_NONE,
-                                                near() ? null : explicitVer, CU.subjectId(this, cctx),
+                                                near() ? null : explicitVer,
+                                                CU.subjectId(this, cctx),
                                                 resolveTaskName(),
                                                 dhtVer);
                                         else {
@@ -629,7 +629,6 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
                                             eventNodeId(),
                                             nodeId,
                                             false,
-                                            false,
                                             true,
                                             true,
                                             topVer,

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index c175b0b..579d701 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -782,14 +782,12 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             if (log.isDebugEnabled())
                 log.debug("Mapping entry for DHT lock future: " + this);
 
-            boolean hasRmtNodes = false;
-
             // Assign keys to primary nodes.
             for (GridDhtCacheEntry entry : entries) {
                 try {
                     while (true) {
                         try {
-                            hasRmtNodes = cctx.dhtMap(
+                            cctx.dhtMap(
                                 nearNodeId,
                                 topVer,
                                 entry,
@@ -823,9 +821,6 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                 }
             }
 
-            if (tx != null)
-                tx.needsCompletedVersions(hasRmtNodes);
-
             if (isDone()) {
                 if (log.isDebugEnabled())
                     log.debug("Mapping won't proceed because future is done: " + this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 4ce4759..3069afd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -62,9 +62,9 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridLeanSet;
-import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.lang.GridClosureException;
+import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.C2;
 import org.apache.ignite.internal.util.typedef.CI1;
@@ -1090,8 +1090,9 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
                 // We have to add completed versions for cases when nearLocal and remote transactions
                 // execute concurrently.
-                res.completedVersions(ctx.tm().committedVersions(req.version()),
-                    ctx.tm().rolledbackVersions(req.version()));
+                IgnitePair<Collection<GridCacheVersion>> versPair = ctx.tm().versions(req.version());
+
+                res.completedVersions(versPair.get1(), versPair.get2());
 
                 int i = 0;
 
@@ -1510,8 +1511,10 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             }
         }
 
-        Collection<GridCacheVersion> committed = ctx.tm().committedVersions(ver);
-        Collection<GridCacheVersion> rolledback = ctx.tm().rolledbackVersions(ver);
+        IgnitePair<Collection<GridCacheVersion>> versPair = ctx.tm().versions(ver);
+
+        Collection<GridCacheVersion> committed = versPair.get1();
+        Collection<GridCacheVersion> rolledback = versPair.get2();
 
         // Backups.
         for (Map.Entry<ClusterNode, List<KeyCacheObject>> entry : dhtMap.entrySet()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 8c7d985..6de8795 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -84,7 +83,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
     protected Map<UUID, GridDistributedTxMapping> dhtMap = new ConcurrentHashMap8<>();
 
     /** Mapped flag. */
-    protected AtomicBoolean mapped = new AtomicBoolean();
+    protected volatile boolean mapped;
 
     /** */
     private long dhtThreadId;
@@ -92,9 +91,6 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
     /** */
     protected boolean explicitLock;
 
-    /** */
-    private boolean needsCompletedVers;
-
     /** Versions of pending locks for entries of this tx. */
     private Collection<GridCacheVersion> pendingVers;
 
@@ -141,20 +137,20 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         int taskNameHash
     ) {
         super(
-            cctx, 
-            xidVer, 
-            implicit, 
-            implicitSingle, 
-            sys, 
-            plc, 
-            concurrency, 
-            isolation, 
-            timeout, 
+            cctx,
+            xidVer,
+            implicit,
+            implicitSingle,
+            sys,
+            plc,
+            concurrency,
+            isolation,
+            timeout,
             invalidate,
             storeEnabled,
             onePhaseCommit,
-            txSize, 
-            subjId, 
+            txSize,
+            subjId,
             taskNameHash
         );
 
@@ -244,16 +240,9 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
      */
     protected abstract void sendFinishReply(boolean commit, @Nullable Throwable err);
 
-    /**
-     * @param needsCompletedVers {@code True} if needs completed versions.
-     */
-    public void needsCompletedVersions(boolean needsCompletedVers) {
-        this.needsCompletedVers |= needsCompletedVers;
-    }
-
     /** {@inheritDoc} */
     @Override public boolean needsCompletedVersions() {
-        return needsCompletedVers;
+        return nearOnOriginatingNode;
     }
 
     /**
@@ -281,10 +270,10 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
      * Map explicit locks.
      */
     protected void mapExplicitLocks() {
-        if (!mapped.get()) {
+        if (!mapped) {
             // Explicit locks may participate in implicit transactions only.
             if (!implicit()) {
-                mapped.set(true);
+                mapped = true;
 
                 return;
             }
@@ -343,7 +332,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
             if (!F.isEmpty(nearEntryMap))
                 addNearNodeEntryMapping(nearEntryMap);
 
-            mapped.set(true);
+            mapped = true;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 61975d7..1d6f633 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -20,9 +20,11 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
@@ -58,10 +60,10 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.util.F0;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.C1;
@@ -177,7 +179,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
     /** Keys that should be locked. */
     @GridToStringInclude
-    private GridConcurrentHashSet<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
+    private final Set<IgniteTxKey> lockKeys = new HashSet<>();
 
     /** Force keys future for correct transforms. */
     private IgniteInternalFuture<?> forceKeysFut;
@@ -267,7 +269,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         if (log.isDebugEnabled())
             log.debug("Transaction future received owner changed callback: " + entry);
 
-        boolean rmv = lockKeys.remove(entry.txKey());
+        boolean rmv;
+
+        synchronized (lockKeys) {
+            rmv = lockKeys.remove(entry.txKey());
+        }
 
         return rmv && mapIfLocked();
     }
@@ -293,7 +299,12 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      * @return {@code True} if all locks are owned.
      */
     private boolean checkLocks() {
-        return locksReady && lockKeys.isEmpty();
+        if (!locksReady)
+            return false;
+
+        synchronized (lockKeys) {
+            return lockKeys.isEmpty();
+        }
     }
 
     /** {@inheritDoc} */
@@ -495,8 +506,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 txEntry.cached(entry);
             }
 
-            if (tx.optimistic() && txEntry.explicitVersion() == null)
-                lockKeys.add(txEntry.txKey());
+            if (tx.optimistic() && txEntry.explicitVersion() == null) {
+                synchronized (lockKeys) {
+                    lockKeys.add(txEntry.txKey());
+                }
+            }
 
             while (true) {
                 try {
@@ -689,7 +703,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
             GridCacheVersion min = tx.minVersion();
 
-            res.completedVersions(cctx.tm().committedVersions(min), cctx.tm().rolledbackVersions(min));
+            if (tx.needsCompletedVersions()) {
+                IgnitePair<Collection<GridCacheVersion>> versPair = cctx.tm().versions(min);
+
+                res.completedVersions(versPair.get1(), versPair.get2());
+            }
 
             res.pending(localDhtPendingVersions(tx.writeEntries(), min));
 
@@ -987,21 +1005,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 if (err0 != null) {
                     err.compareAndSet(null, err0);
 
+                    tx.rollbackAsync();
+
                     final GridNearTxPrepareResponse res = createPrepareResponse(err.get());
 
-                    tx.rollbackAsync().listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
-                        @Override public void apply(IgniteInternalFuture<IgniteInternalTx> fut) {
-                            if (GridDhtTxPrepareFuture.super.onDone(res, res.error())) {
-                                try {
-                                    if (replied.compareAndSet(false, true))
-                                        sendPrepareResponse(res);
-                                }
-                                catch (IgniteCheckedException e) {
-                                    U.error(log, "Failed to send prepare response for transaction: " + tx, e);
-                                }
-                            }
-                        }
-                    });
+                    onDone(res, res.error());
 
                     return;
                 }
@@ -1017,20 +1025,16 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 Map<UUID, GridDistributedTxMapping> futDhtMap = new HashMap<>();
                 Map<UUID, GridDistributedTxMapping> futNearMap = new HashMap<>();
 
-                boolean hasRemoteNodes = false;
-
                 // Assign keys to primary nodes.
                 if (!F.isEmpty(writes)) {
                     for (IgniteTxEntry write : writes)
-                        hasRemoteNodes |= map(tx.entry(write.txKey()), futDhtMap, futNearMap);
+                        map(tx.entry(write.txKey()), futDhtMap, futNearMap);
                 }
 
                 if (!F.isEmpty(reads)) {
                     for (IgniteTxEntry read : reads)
-                        hasRemoteNodes |= map(tx.entry(read.txKey()), futDhtMap, futNearMap);
+                        map(tx.entry(read.txKey()), futDhtMap, futNearMap);
                 }
-
-                tx.needsCompletedVersions(hasRemoteNodes);
             }
 
             if (isDone())
@@ -1223,15 +1227,14 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      * @param entry Transaction entry.
      * @param futDhtMap DHT mapping.
      * @param futNearMap Near mapping.
-     * @return {@code True} if mapped.
      */
-    private boolean map(
+    private void map(
         IgniteTxEntry entry,
         Map<UUID, GridDistributedTxMapping> futDhtMap,
         Map<UUID, GridDistributedTxMapping> futNearMap
     ) {
         if (entry.cached().isLocal())
-            return false;
+            return;
 
         GridDhtCacheEntry cached = (GridDhtCacheEntry)entry.cached();
 
@@ -1247,8 +1250,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
             entry.ttl(CU.toTtl(expiry.getExpiryForAccess()));
         }
 
-        boolean ret;
-
         while (true) {
             try {
                 Collection<ClusterNode> dhtNodes = dht.topology().nodes(cached.partition(), tx.topologyVersion());
@@ -1272,10 +1273,10 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                     log.debug("Entry has no near readers: " + entry);
 
                 // Exclude local node.
-                ret = map(entry, F.view(dhtNodes, F.remoteNodes(cctx.localNodeId())), dhtMap, futDhtMap);
+                map(entry, F.view(dhtNodes, F.remoteNodes(cctx.localNodeId())), dhtMap, futDhtMap);
 
                 // Exclude DHT nodes.
-                ret |= map(entry, F.view(nearNodes, F0.notIn(dhtNodes)), nearMap, futNearMap);
+                map(entry, F.view(nearNodes, F0.notIn(dhtNodes)), nearMap, futNearMap);
 
                 break;
             }
@@ -1285,8 +1286,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 entry.cached(cached);
             }
         }
-
-        return ret;
     }
 
     /**
@@ -1294,16 +1293,13 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      * @param nodes Nodes.
      * @param globalMap Map.
      * @param locMap Exclude map.
-     * @return {@code True} if mapped.
      */
-    private boolean map(
+    private void map(
         IgniteTxEntry entry,
         Iterable<ClusterNode> nodes,
         Map<UUID, GridDistributedTxMapping> globalMap,
         Map<UUID, GridDistributedTxMapping> locMap
     ) {
-        boolean ret = false;
-
         if (nodes != null) {
             for (ClusterNode n : nodes) {
                 GridDistributedTxMapping global = globalMap.get(n.id());
@@ -1332,12 +1328,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                     locMap.put(n.id(), loc = new GridDistributedTxMapping(n));
 
                 loc.add(entry);
-
-                ret = true;
             }
         }
-
-        return ret;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
index f8be2a7..e268a88 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java
@@ -39,6 +39,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.tostring.GridToStringBuilder;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -112,19 +113,19 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
         int taskNameHash
     ) {
         super(
-            ctx, 
-            nodeId, 
-            rmtThreadId, 
-            xidVer, 
-            commitVer, 
+            ctx,
+            nodeId,
+            rmtThreadId,
+            xidVer,
+            commitVer,
             sys,
             plc,
-            concurrency, 
-            isolation, 
-            invalidate, 
-            timeout, 
+            concurrency,
+            isolation,
+            invalidate,
+            timeout,
             txSize,
-            subjId, 
+            subjId,
             taskNameHash
         );
 
@@ -138,7 +139,7 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
 
         readMap = Collections.emptyMap();
 
-        writeMap = new ConcurrentLinkedHashMap<>(txSize, 1.0f);
+        writeMap = new ConcurrentLinkedHashMap<>(U.capacity(txSize), 0.75f, 1);
 
         topologyVersion(topVer);
     }
@@ -183,19 +184,19 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
         int taskNameHash
     ) {
         super(
-            ctx, 
-            nodeId, 
-            rmtThreadId, 
-            xidVer, 
-            commitVer, 
+            ctx,
+            nodeId,
+            rmtThreadId,
+            xidVer,
+            commitVer,
             sys,
             plc,
-            concurrency, 
-            isolation, 
-            invalidate, 
-            timeout, 
+            concurrency,
+            isolation,
+            invalidate,
+            timeout,
             txSize,
-            subjId, 
+            subjId,
             taskNameHash
         );
 
@@ -207,7 +208,7 @@ public class GridDhtTxRemote extends GridDistributedTxRemoteAdapter {
         this.rmtFutId = rmtFutId;
 
         readMap = Collections.emptyMap();
-        writeMap = new ConcurrentLinkedHashMap<>(txSize, 1.0f);
+        writeMap = new ConcurrentLinkedHashMap<>(U.capacity(txSize), 0.75f, 1);
 
         topologyVersion(topVer);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 4cd9e84..7f9edb2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -53,7 +53,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
 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.GridCacheFilterFailedException;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index f03b461..83c220d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -64,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.typedef.C2;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
@@ -688,8 +689,10 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             if (map == null || map.isEmpty())
                 return;
 
-            Collection<GridCacheVersion> committed = ctx.tm().committedVersions(ver);
-            Collection<GridCacheVersion> rolledback = ctx.tm().rolledbackVersions(ver);
+            IgnitePair<Collection<GridCacheVersion>> versPair = ctx.tm().versions(ver);
+
+            Collection<GridCacheVersion> committed = versPair.get1();
+            Collection<GridCacheVersion> rolledback = versPair.get2();
 
             for (Map.Entry<ClusterNode, GridNearUnlockRequest> mapping : map.entrySet()) {
                 ClusterNode n = mapping.getKey();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index af43113..0002180 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import java.util.ArrayDeque;
 import java.util.Collection;
 import java.util.List;
+import java.util.Queue;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCheckedException;
@@ -55,7 +57,6 @@ import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionTimeoutException;
 import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentLinkedDeque8;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.transactions.TransactionState.PREPARED;
@@ -293,7 +294,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
 
         txMapping = new GridDhtTxMapping();
 
-        ConcurrentLinkedDeque8<GridDistributedTxMapping> mappings = new ConcurrentLinkedDeque8<>();
+        Queue<GridDistributedTxMapping> mappings = new ArrayDeque<>();
 
         if (!F.isEmpty(writes)) {
             for (int cacheId : tx.activeCacheIds()) {
@@ -353,7 +354,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
      *
      * @param mappings Queue of mappings.
      */
-    private void proceedPrepare(final ConcurrentLinkedDeque8<GridDistributedTxMapping> mappings) {
+    private void proceedPrepare(final Queue<GridDistributedTxMapping> mappings) {
         if (isDone())
             return;
 
@@ -556,7 +557,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         private AtomicBoolean rcvRes = new AtomicBoolean(false);
 
         /** Mappings to proceed prepare. */
-        private ConcurrentLinkedDeque8<GridDistributedTxMapping> mappings;
+        private Queue<GridDistributedTxMapping> mappings;
 
         /**
          * @param m Mapping.
@@ -564,7 +565,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
          */
         MiniFuture(
             GridDistributedTxMapping m,
-            ConcurrentLinkedDeque8<GridDistributedTxMapping> mappings
+            Queue<GridDistributedTxMapping> mappings
         ) {
             this.m = m;
             this.mappings = mappings;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index 0e8aa0d..5ab85b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -712,8 +713,10 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
             if (map == null || map.isEmpty())
                 return;
 
-            Collection<GridCacheVersion> committed = ctx.tm().committedVersions(ver);
-            Collection<GridCacheVersion> rolledback = ctx.tm().rolledbackVersions(ver);
+            IgnitePair<Collection<GridCacheVersion>> versPair = ctx.tm().versions(ver);
+
+            Collection<GridCacheVersion> committed = versPair.get1();
+            Collection<GridCacheVersion> rolledback = versPair.get2();
 
             for (Map.Entry<ClusterNode, GridNearUnlockRequest> mapping : map.entrySet()) {
                 ClusterNode n = mapping.getKey();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 46c9f3e..a9dbda2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -21,7 +21,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
@@ -88,15 +87,15 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
     /** Commit flag. */
     private boolean commit;
 
-    /** Error. */
-    private AtomicReference<Throwable> err = new AtomicReference<>(null);
-
     /** Node mappings. */
-    private ConcurrentMap<UUID, GridDistributedTxMapping> mappings;
+    private Map<UUID, GridDistributedTxMapping> mappings;
 
     /** Trackable flag. */
     private boolean trackable = true;
 
+    /** */
+    private boolean finishOnePhaseCalled;
+
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -176,38 +175,6 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
     }
 
     /**
-     * @param e Error.
-     */
-    void onError(Throwable e) {
-        tx.commitError(e);
-
-        if (err.compareAndSet(null, e)) {
-            boolean marked = tx.setRollbackOnly();
-
-            if (e instanceof IgniteTxRollbackCheckedException) {
-                if (marked) {
-                    try {
-                        tx.rollback();
-                    }
-                    catch (IgniteCheckedException ex) {
-                        U.error(log, "Failed to automatically rollback transaction: " + tx, ex);
-                    }
-                }
-            }
-            else if (tx.implicit() && tx.isSystemInvalidate()) { // Finish implicit transaction on heuristic error.
-                try {
-                    tx.close();
-                }
-                catch (IgniteCheckedException ex) {
-                    U.error(log, "Failed to invalidate transaction: " + tx, ex);
-                }
-            }
-
-            onComplete();
-        }
-    }
-
-    /**
      * @param nodeId Sender.
      * @param res Result.
      */
@@ -247,24 +214,56 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx tx0, Throwable err) {
-        if ((initialized() || err != null)) {
-            if (tx.needCheckBackup()) {
-                assert tx.onePhaseCommit();
+        if (isDone())
+            return false;
 
-                if (err != null)
-                    err = new TransactionRollbackException("Failed to commit transaction.", err);
+        synchronized (this) {
+            if (isDone())
+                return false;
 
-                try {
-                    tx.finish(err == null);
+            if (err != null) {
+                tx.commitError(err);
+
+                boolean marked = tx.setRollbackOnly();
+
+                if (err instanceof IgniteTxRollbackCheckedException) {
+                    if (marked) {
+                        try {
+                            tx.rollback();
+                        }
+                        catch (IgniteCheckedException ex) {
+                            U.error(log, "Failed to automatically rollback transaction: " + tx, ex);
+                        }
+                    }
                 }
-                catch (IgniteCheckedException e) {
-                    if (err != null)
-                        err.addSuppressed(e);
-                    else
-                        err = e;
+                else if (tx.implicit() && tx.isSystemInvalidate()) { // Finish implicit transaction on heuristic error.
+                    try {
+                        tx.close();
+                    }
+                    catch (IgniteCheckedException ex) {
+                        U.error(log, "Failed to invalidate transaction: " + tx, ex);
+                    }
                 }
             }
 
+            if (initialized() || err != null) {
+                if (tx.needCheckBackup()) {
+                    assert tx.onePhaseCommit();
+
+                    if (err != null)
+                        err = new TransactionRollbackException("Failed to commit transaction.", err);
+
+                    try {
+                        tx.finish(err == null);
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (err != null)
+                            err.addSuppressed(e);
+                        else
+                            err = e;
+                    }
+                }
+
             if (tx.onePhaseCommit()) {
                 boolean commit = this.commit && err == null;
 
@@ -273,36 +272,35 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                 tx.tmFinish(commit);
             }
 
-            Throwable th = this.err.get();
-
-            if (super.onDone(tx0, th != null ? th : err)) {
-                if (error() instanceof IgniteTxHeuristicCheckedException) {
-                    AffinityTopologyVersion topVer = tx.topologyVersion();
+                if (super.onDone(tx0, err)) {
+                    if (error() instanceof IgniteTxHeuristicCheckedException) {
+                        AffinityTopologyVersion topVer = tx.topologyVersion();
 
-                    for (IgniteTxEntry e : tx.writeMap().values()) {
-                        GridCacheContext cacheCtx = e.context();
+                        for (IgniteTxEntry e : tx.writeMap().values()) {
+                            GridCacheContext cacheCtx = e.context();
 
-                        try {
-                            if (e.op() != NOOP && !cacheCtx.affinity().localNode(e.key(), topVer)) {
-                                GridCacheEntryEx entry = cacheCtx.cache().peekEx(e.key());
+                            try {
+                                if (e.op() != NOOP && !cacheCtx.affinity().localNode(e.key(), topVer)) {
+                                    GridCacheEntryEx entry = cacheCtx.cache().peekEx(e.key());
 
-                                if (entry != null)
-                                    entry.invalidate(null, tx.xidVersion());
+                                    if (entry != null)
+                                        entry.invalidate(null, tx.xidVersion());
+                                }
                             }
-                        }
-                        catch (Throwable t) {
-                            U.error(log, "Failed to invalidate entry.", t);
+                            catch (Throwable t) {
+                                U.error(log, "Failed to invalidate entry.", t);
 
-                            if (t instanceof Error)
-                                throw (Error)t;
+                                if (t instanceof Error)
+                                    throw (Error)t;
+                            }
                         }
                     }
-                }
 
-                // Don't forget to clean up.
-                cctx.mvcc().removeFuture(this);
+                    // Don't forget to clean up.
+                    cctx.mvcc().removeFuture(this);
 
-                return true;
+                    return true;
+                }
             }
         }
 
@@ -321,7 +319,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      * Completeness callback.
      */
     private void onComplete() {
-        onDone(tx, err.get());
+        onDone(tx);
     }
 
     /**
@@ -354,7 +352,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
                 markInitialized();
 
-                if (!isSync()) {
+                if (!isSync() && !isDone()) {
                     boolean complete = true;
 
                     for (IgniteInternalFuture<?> f : pending())
@@ -367,15 +365,15 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                 }
             }
             else
-                onError(new IgniteCheckedException("Failed to commit transaction: " + CU.txString(tx)));
+                onDone(new IgniteCheckedException("Failed to commit transaction: " + CU.txString(tx)));
         }
         catch (Error | RuntimeException e) {
-            onError(e);
+            onDone(e);
 
             throw e;
         }
         catch (IgniteCheckedException e) {
-            onError(e);
+            onDone(e);
         }
     }
 
@@ -415,7 +413,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                         "(backup has left grid): " + tx.xidVersion(), cause));
                 }
                 else if (backup.isLocal()) {
-                    boolean committed = cctx.tm().txHandler().checkDhtRemoteTxCommitted(tx.xidVersion());
+                    boolean committed = !cctx.tm().addRolledbackTx(tx);
 
                     readyNearMappingFromBackup(mapping);
 
@@ -515,6 +513,13 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      * @param commit Commit flag.
      */
     private void finishOnePhase(boolean commit) {
+        assert Thread.holdsLock(this);
+
+        if (finishOnePhaseCalled)
+            return;
+
+        finishOnePhaseCalled = true;
+
         // No need to send messages as transaction was already committed on remote node.
         // Finish local mapping only as we need send commit message to backups.
         for (GridDistributedTxMapping m : mappings.values()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 883c285..db4a4b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -23,7 +23,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;
@@ -88,7 +87,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     private static final long serialVersionUID = 0L;
 
     /** DHT mappings. */
-    private ConcurrentMap<UUID, GridDistributedTxMapping> mappings = new ConcurrentHashMap8<>();
+    private Map<UUID, GridDistributedTxMapping> mappings = new ConcurrentHashMap8<>();
 
     /** Future. */
     @GridToStringExclude
@@ -217,7 +216,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
 
     /** {@inheritDoc} */
     @Override protected IgniteInternalFuture<Boolean> addReader(
-        long msgId, 
+        long msgId,
         GridDhtCacheEntry cached,
         IgniteTxEntry entry,
         AffinityTopologyVersion topVer
@@ -472,7 +471,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     /**
      * @return DHT map.
      */
-    ConcurrentMap<UUID, GridDistributedTxMapping> mappings() {
+    Map<UUID, GridDistributedTxMapping> mappings() {
         return mappings;
     }
 
@@ -798,14 +797,14 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                 catch (Error | RuntimeException e) {
                     commitErr.compareAndSet(null, e);
 
-                    fut0.onError(e);
+                    fut0.onDone(e);
 
                     throw e;
                 }
                 catch (IgniteCheckedException e) {
                     commitErr.compareAndSet(null, e);
 
-                    fut0.onError(e);
+                    fut0.onDone(e);
                 }
             }
         });
@@ -1152,8 +1151,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
 
     /** {@inheritDoc} */
     @Override protected GridCacheEntryEx entryEx(
-        GridCacheContext cacheCtx, 
-        IgniteTxKey key, 
+        GridCacheContext cacheCtx,
+        IgniteTxKey key,
         AffinityTopologyVersion topVer
     ) {
         if (cacheCtx.isColocated()) {
@@ -1245,7 +1244,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     @Override public void onRemap(AffinityTopologyVersion topVer) {
         assert cctx.kernalContext().clientNode();
 
-        mapped.set(false);
+        mapped = false;
         nearLocallyMapped = false;
         colocatedLocallyMapped = false;
         txNodes = null;
@@ -1254,7 +1253,9 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         dhtMap.clear();
         mappings.clear();
 
-        this.topVer.set(topVer);
+        synchronized (this) {
+            this.topVer = topVer;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
index 20fb8c2..94af6bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
@@ -536,9 +536,8 @@ public interface IgniteInternalTx extends AutoCloseable, GridTimeoutObject {
 
     /**
      * @param commitVer Commit version.
-     * @return {@code True} if version was set.
      */
-    public boolean commitVersion(GridCacheVersion commitVer);
+    public void commitVersion(GridCacheVersion commitVer);
 
     /**
      * @return End version (a.k.a. <tt>'tnc'</tt> or <tt>'transaction number counter'</tt>)


[15/46] ignite git commit: ignite-1817 Deprecate RandomEvictionPolicy and IgniteCache.randomEntry method - Fixes #191.

Posted by ak...@apache.org.
ignite-1817 Deprecate RandomEvictionPolicy and IgniteCache.randomEntry method - Fixes #191.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-1753-1282
Commit: 3de9d47a88d2e13c4807e44e2c391549929d3c9d
Parents: 5e36b26
Author: agura <ag...@gridgain.com>
Authored: Mon Nov 9 20:36:53 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 9 20:36:53 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  1 +
 .../eviction/random/RandomEvictionPolicy.java   |  4 ++++
 .../processors/cache/GridCacheAdapter.java      | 22 ++++++++++++++++----
 .../processors/cache/IgniteCacheProxy.java      |  6 ++++++
 4 files changed, 29 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3de9d47a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index 5558a26..6c4b507 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -104,6 +104,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      *
      * @return Random entry, or {@code null} if cache is empty.
      */
+    @Deprecated
     public Entry<K, V> randomEntry();
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3de9d47a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
index bf04d92..f77551d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
@@ -39,7 +39,11 @@ import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE
  * <p>
  * Random eviction will provide the best performance over any key queue in which every
  * key has the same probability of being accessed.
+ *
+ * @deprecated This eviction policy implementation doesn't support near cache
+ * and doesn't work on client nodes. Also it seems that random eviction doesn't make sense.
  */
+@Deprecated
 public class RandomEvictionPolicy<K, V> implements EvictionPolicy<K, V>, RandomEvictionPolicyMBean, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3de9d47a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 74951b5..419ccec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -99,7 +99,6 @@ import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.F0;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridLeanMap;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -122,7 +121,6 @@ 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;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
@@ -3526,10 +3524,26 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * @return Random cache entry.
      */
+    @Deprecated
     @Nullable public Cache.Entry<K, V> randomEntry() {
-        GridCacheMapEntry e = map.randomEntry();
+        GridCacheMapEntry entry;
 
-        return e == null || e.obsolete() ? null : e.<K, V>wrapLazyValue();
+        if (ctx.offheapTiered()) {
+            Iterator<Cache.Entry<K, V>> it;
+
+            try {
+                it = ctx.swap().offheapIterator(true, true, ctx.affinity().affinityTopologyVersion());
+            }
+            catch (IgniteCheckedException e) {
+                throw CU.convertToCacheException(e);
+            }
+
+            return it.hasNext() ? it.next() : null;
+        }
+        else
+            entry = map.randomEntry();
+
+        return entry == null || entry.obsolete() ? null : entry.<K, V>wrapLazyValue();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3de9d47a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index c563e59..4b03bb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -61,6 +61,7 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.AsyncSupportAdapter;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
@@ -276,6 +277,11 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
     /** {@inheritDoc} */
     @Nullable @Override public Cache.Entry<K, V> randomEntry() {
+        GridKernalContext kctx = ctx.kernalContext();
+
+        if (kctx.isDaemon() || kctx.clientNode())
+            throw new UnsupportedOperationException("Not applicable for daemon or client node.");
+
         GridCacheGateway<K, V> gate = this.gate;
 
         CacheOperationContext prev = onEnter(gate, opCtx);


[18/46] ignite git commit: ignite-1.5 Fixed benchmark config.

Posted by ak...@apache.org.
ignite-1.5 Fixed benchmark config.


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

Branch: refs/heads/ignite-1753-1282
Commit: 9396ccb74f5136a60c68e2de451340281458c0ed
Parents: 7ba2efb
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Nov 10 17:52:36 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 10 17:53:46 2015 +0700

----------------------------------------------------------------------
 .../yardstick/config/ignite-store-config.xml    | 31 +++++++++++---------
 1 file changed, 17 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9396ccb7/modules/yardstick/config/ignite-store-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-store-config.xml b/modules/yardstick/config/ignite-store-config.xml
index d233455..35b8e19 100644
--- a/modules/yardstick/config/ignite-store-config.xml
+++ b/modules/yardstick/config/ignite-store-config.xml
@@ -30,14 +30,9 @@
     <!--
         Store data source.
     -->
-    <bean id="storeDataSource" class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore">
-        <property name="dataSource">
-            <bean class="org.h2.jdbcx.JdbcConnectionPool" factory-method="create">
-                <constructor-arg value="jdbc:h2:tcp://localhost/store-benchmark/h2-benchmark"/>
-                <constructor-arg value="sa"/>
-                <constructor-arg value=""/>
-            </bean>
-        </property>
+    <bean id="storeDataSource" class="org.h2.jdbcx.JdbcDataSource">
+        <property name="url" value="jdbc:h2:tcp://localhost/store-benchmark/h2-benchmark"/>
+        <property name="user" value="sa"/>
     </bean>
 
     <bean id="sampleTypeMetadata" class="org.apache.ignite.cache.CacheTypeMetadata">
@@ -70,7 +65,7 @@
         </property>
     </bean>
 
-    <bean class="org.apache.ignite.configuration.IgniteConfiguration" >
+    <bean class="org.apache.ignite.configuration.IgniteConfiguration">
         <property name="peerClassLoadingEnabled" value="false"/>
 
         <property name="cacheConfiguration">
@@ -91,8 +86,11 @@
                     </property>
 
                     <property name="cacheStoreFactory">
-                        <bean class="javax.cache.configuration.FactoryBuilder$SingletonFactory">
-                            <constructor-arg ref="storeDataSource"/>
+                        <bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory">
+                            <property name="dataSourceBean" value="storeDataSource"/>
+                            <property name="dialect">
+                                <bean class="org.apache.ignite.cache.store.jdbc.dialect.H2Dialect"/>
+                            </property>
                         </bean>
                     </property>
                 </bean>
@@ -113,15 +111,20 @@
                     </property>
 
                     <property name="cacheStoreFactory">
-                        <bean class="javax.cache.configuration.FactoryBuilder$SingletonFactory">
-                            <constructor-arg ref="storeDataSource"/>
+                        <bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory">
+                            <property name="dataSourceBean" value="storeDataSource"/>
+                            <property name="dialect">
+                                <bean class="org.apache.ignite.cache.store.jdbc.dialect.H2Dialect"/>
+                            </property>
                         </bean>
                     </property>
                 </bean>
             </list>
         </property>
 
-        <property name="connectorConfiguration"><null/></property>
+        <property name="connectorConfiguration">
+            <null/>
+        </property>
 
         <property name="includeEventTypes">
             <list/>


[04/46] ignite git commit: Ignite-1093 "Rebalancing with default parameters is very slow" fixes.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
deleted file mode 100644
index e993a88..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ /dev/null
@@ -1,1192 +0,0 @@
-/*
- * 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.dht.preloader;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.BrokenBarrierException;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.CacheRebalanceMode;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.events.DiscoveryEvent;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
-import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
-import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
-import org.apache.ignite.internal.util.GridLeanSet;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
-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.internal.CU;
-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.internal.util.worker.GridWorker;
-import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.thread.IgniteThread;
-import org.jetbrains.annotations.Nullable;
-
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_LOADED;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STOPPED;
-import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
-import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING;
-import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE;
-import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRELOAD;
-
-/**
- * Thread pool for requesting partitions from other nodes
- * and populating local cache.
- */
-@SuppressWarnings("NonConstantFieldWithUpperCaseName")
-public class GridDhtPartitionDemandPool {
-    /** Dummy message to wake up a blocking queue if a node leaves. */
-    private final SupplyMessage DUMMY_TOP = new SupplyMessage();
-
-    /** */
-    private final GridCacheContext<?, ?> cctx;
-
-    /** */
-    private final IgniteLogger log;
-
-    /** */
-    private final ReadWriteLock busyLock;
-
-    /** */
-    @GridToStringInclude
-    private final Collection<DemandWorker> dmdWorkers;
-
-    /** Preload predicate. */
-    private IgnitePredicate<GridCacheEntryInfo> preloadPred;
-
-    /** Future for preload mode {@link CacheRebalanceMode#SYNC}. */
-    @GridToStringInclude
-    private SyncFuture syncFut;
-
-    /** Preload timeout. */
-    private final AtomicLong timeout;
-
-    /** Allows demand threads to synchronize their step. */
-    private CyclicBarrier barrier;
-
-    /** Demand lock. */
-    private final ReadWriteLock demandLock = new ReentrantReadWriteLock();
-
-    /** */
-    private int poolSize;
-
-    /** Last timeout object. */
-    private AtomicReference<GridTimeoutObject> lastTimeoutObj = new AtomicReference<>();
-
-    /** Last exchange future. */
-    private volatile GridDhtPartitionsExchangeFuture lastExchangeFut;
-
-    /**
-     * @param cctx Cache context.
-     * @param busyLock Shutdown lock.
-     */
-    public GridDhtPartitionDemandPool(GridCacheContext<?, ?> cctx, ReadWriteLock busyLock) {
-        assert cctx != null;
-        assert busyLock != null;
-
-        this.cctx = cctx;
-        this.busyLock = busyLock;
-
-        log = cctx.logger(getClass());
-
-        boolean enabled = cctx.rebalanceEnabled() && !cctx.kernalContext().clientNode();
-
-        poolSize = enabled ? cctx.config().getRebalanceThreadPoolSize() : 0;
-
-        if (enabled) {
-            barrier = new CyclicBarrier(poolSize);
-
-            dmdWorkers = new ArrayList<>(poolSize);
-
-            for (int i = 0; i < poolSize; i++)
-                dmdWorkers.add(new DemandWorker(i));
-
-            syncFut = new SyncFuture(dmdWorkers);
-        }
-        else {
-            dmdWorkers = Collections.emptyList();
-
-            syncFut = new SyncFuture(dmdWorkers);
-
-            // Calling onDone() immediately since preloading is disabled.
-            syncFut.onDone();
-        }
-
-        timeout = new AtomicLong(cctx.config().getRebalanceTimeout());
-    }
-
-    /**
-     *
-     */
-    void start() {
-        if (poolSize > 0) {
-            for (DemandWorker w : dmdWorkers)
-                new IgniteThread(cctx.gridName(), "preloader-demand-worker", w).start();
-        }
-    }
-
-    /**
-     *
-     */
-    void stop() {
-        U.cancel(dmdWorkers);
-
-        if (log.isDebugEnabled())
-            log.debug("Before joining on demand workers: " + dmdWorkers);
-
-        U.join(dmdWorkers, log);
-
-        if (log.isDebugEnabled())
-            log.debug("After joining on demand workers: " + dmdWorkers);
-
-        lastExchangeFut = null;
-
-        lastTimeoutObj.set(null);
-    }
-
-    /**
-     * @return Future for {@link CacheRebalanceMode#SYNC} mode.
-     */
-    IgniteInternalFuture<?> syncFuture() {
-        return syncFut;
-    }
-
-    /**
-     * Sets preload predicate for demand pool.
-     *
-     * @param preloadPred Preload predicate.
-     */
-    void preloadPredicate(IgnitePredicate<GridCacheEntryInfo> preloadPred) {
-        this.preloadPred = preloadPred;
-    }
-
-    /**
-     * @return Size of this thread pool.
-     */
-    int poolSize() {
-        return poolSize;
-    }
-
-    /**
-     * Wakes up demand workers when new exchange future was added.
-     */
-    void onExchangeFutureAdded() {
-        synchronized (dmdWorkers) {
-            for (DemandWorker w : dmdWorkers)
-                w.addMessage(DUMMY_TOP);
-        }
-    }
-
-    /**
-     * Force preload.
-     */
-    void forcePreload() {
-        GridTimeoutObject obj = lastTimeoutObj.getAndSet(null);
-
-        if (obj != null)
-            cctx.time().removeTimeoutObject(obj);
-
-        final GridDhtPartitionsExchangeFuture exchFut = lastExchangeFut;
-
-        if (exchFut != null) {
-            if (log.isDebugEnabled())
-                log.debug("Forcing rebalance event for future: " + exchFut);
-
-            exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                    cctx.shared().exchange().forcePreloadExchange(exchFut);
-                }
-            });
-        }
-        else if (log.isDebugEnabled())
-            log.debug("Ignoring force rebalance request (no topology event happened yet).");
-    }
-
-    /**
-     * @return {@code true} if entered to busy state.
-     */
-    private boolean enterBusy() {
-        if (busyLock.readLock().tryLock())
-            return true;
-
-        if (log.isDebugEnabled())
-            log.debug("Failed to enter to busy state (demander is stopping): " + cctx.nodeId());
-
-        return false;
-    }
-
-    /**
-     *
-     */
-    private void leaveBusy() {
-        busyLock.readLock().unlock();
-    }
-
-    /**
-     * @param type Type.
-     * @param discoEvt Discovery event.
-     */
-    private void preloadEvent(int type, DiscoveryEvent discoEvt) {
-        preloadEvent(-1, type, discoEvt);
-    }
-
-    /**
-     * @param part Partition.
-     * @param type Type.
-     * @param discoEvt Discovery event.
-     */
-    private void preloadEvent(int part, int type, DiscoveryEvent discoEvt) {
-        assert discoEvt != null;
-
-        cctx.events().addPreloadEvent(part, type, discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp());
-    }
-
-    /**
-     * @param msg Message to check.
-     * @return {@code True} if dummy message.
-     */
-    private boolean dummyTopology(SupplyMessage msg) {
-        return msg == DUMMY_TOP;
-    }
-
-    /**
-     * @param deque Deque to poll from.
-     * @param time Time to wait.
-     * @param w Worker.
-     * @return Polled item.
-     * @throws InterruptedException If interrupted.
-     */
-    @Nullable private <T> T poll(BlockingQueue<T> deque, long time, GridWorker w) throws InterruptedException {
-        assert w != null;
-
-        // There is currently a case where {@code interrupted}
-        // flag on a thread gets flipped during stop which causes the pool to hang.  This check
-        // will always make sure that interrupted flag gets reset before going into wait conditions.
-        // The true fix should actually make sure that interrupted flag does not get reset or that
-        // interrupted exception gets propagated. Until we find a real fix, this method should
-        // always work to make sure that there is no hanging during stop.
-        if (w.isCancelled())
-            Thread.currentThread().interrupt();
-
-        return deque.poll(time, MILLISECONDS);
-    }
-
-    /**
-     * @param p Partition.
-     * @param topVer Topology version.
-     * @return Picked owners.
-     */
-    private Collection<ClusterNode> pickedOwners(int p, AffinityTopologyVersion topVer) {
-        Collection<ClusterNode> affNodes = cctx.affinity().nodes(p, topVer);
-
-        int affCnt = affNodes.size();
-
-        Collection<ClusterNode> rmts = remoteOwners(p, topVer);
-
-        int rmtCnt = rmts.size();
-
-        if (rmtCnt <= affCnt)
-            return rmts;
-
-        List<ClusterNode> sorted = new ArrayList<>(rmts);
-
-        // Sort in descending order, so nodes with higher order will be first.
-        Collections.sort(sorted, CU.nodeComparator(false));
-
-        // Pick newest nodes.
-        return sorted.subList(0, affCnt);
-    }
-
-    /**
-     * @param p Partition.
-     * @param topVer Topology version.
-     * @return Nodes owning this partition.
-     */
-    private Collection<ClusterNode> remoteOwners(int p, AffinityTopologyVersion topVer) {
-        return F.view(cctx.dht().topology().owners(p, topVer), F.remoteNodes(cctx.nodeId()));
-    }
-
-    /**
-     * @param assigns Assignments.
-     * @param force {@code True} if dummy reassign.
-     */
-    void addAssignments(final GridDhtPreloaderAssignments assigns, boolean force) {
-        if (log.isDebugEnabled())
-            log.debug("Adding partition assignments: " + assigns);
-
-        long delay = cctx.config().getRebalanceDelay();
-
-        if (delay == 0 || force) {
-            assert assigns != null;
-
-            synchronized (dmdWorkers) {
-                for (DemandWorker w : dmdWorkers) {
-                    w.addAssignments(assigns);
-
-                    w.addMessage(DUMMY_TOP);
-                }
-            }
-        }
-        else if (delay > 0) {
-            assert !force;
-
-            GridTimeoutObject obj = lastTimeoutObj.get();
-
-            if (obj != null)
-                cctx.time().removeTimeoutObject(obj);
-
-            final GridDhtPartitionsExchangeFuture exchFut = lastExchangeFut;
-
-            assert exchFut != null : "Delaying rebalance process without topology event.";
-
-            obj = new GridTimeoutObjectAdapter(delay) {
-                @Override public void onTimeout() {
-                    exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> f) {
-                            cctx.shared().exchange().forcePreloadExchange(exchFut);
-                        }
-                    });
-                }
-            };
-
-            lastTimeoutObj.set(obj);
-
-            cctx.time().addTimeoutObject(obj);
-        }
-    }
-
-    /**
-     *
-     */
-    void unwindUndeploys() {
-        demandLock.writeLock().lock();
-
-        try {
-            cctx.deploy().unwind(cctx);
-        }
-        finally {
-            demandLock.writeLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridDhtPartitionDemandPool.class, this);
-    }
-
-    /**
-     *
-     */
-    private class DemandWorker extends GridWorker {
-        /** Worker ID. */
-        private int id;
-
-        /** Partition-to-node assignments. */
-        private final LinkedBlockingDeque<GridDhtPreloaderAssignments> assignQ = new LinkedBlockingDeque<>();
-
-        /** Message queue. */
-        private final LinkedBlockingDeque<SupplyMessage> msgQ =
-            new LinkedBlockingDeque<>();
-
-        /** Counter. */
-        private long cntr;
-
-        /** Hide worker logger and use cache logger instead. */
-        private IgniteLogger log = GridDhtPartitionDemandPool.this.log;
-
-        /**
-         * @param id Worker ID.
-         */
-        private DemandWorker(int id) {
-            super(cctx.gridName(), "preloader-demand-worker", GridDhtPartitionDemandPool.this.log);
-
-            assert id >= 0;
-
-            this.id = id;
-        }
-
-        /**
-         * @param assigns Assignments.
-         */
-        void addAssignments(GridDhtPreloaderAssignments assigns) {
-            assert assigns != null;
-
-            assignQ.offer(assigns);
-
-            if (log.isDebugEnabled())
-                log.debug("Added assignments to worker: " + this);
-        }
-
-        /**
-         * @return {@code True} if topology changed.
-         */
-        private boolean topologyChanged() {
-            return !assignQ.isEmpty() || cctx.shared().exchange().topologyChanged();
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void addMessage(SupplyMessage msg) {
-            if (!enterBusy())
-                return;
-
-            try {
-                assert dummyTopology(msg) || msg.supply().workerId() == id;
-
-                msgQ.offer(msg);
-            }
-            finally {
-                leaveBusy();
-            }
-        }
-
-        /**
-         * @param timeout Timed out value.
-         */
-        private void growTimeout(long timeout) {
-            long newTimeout = (long)(timeout * 1.5D);
-
-            // Account for overflow.
-            if (newTimeout < 0)
-                newTimeout = Long.MAX_VALUE;
-
-            // Grow by 50% only if another thread didn't do it already.
-            if (GridDhtPartitionDemandPool.this.timeout.compareAndSet(timeout, newTimeout))
-                U.warn(log, "Increased rebalancing message timeout from " + timeout + "ms to " +
-                    newTimeout + "ms.");
-        }
-
-        /**
-         * @param pick Node picked for preloading.
-         * @param p Partition.
-         * @param entry Preloaded entry.
-         * @param topVer Topology version.
-         * @return {@code False} if partition has become invalid during preloading.
-         * @throws IgniteInterruptedCheckedException If interrupted.
-         */
-        private boolean preloadEntry(
-            ClusterNode pick,
-            int p,
-            GridCacheEntryInfo entry,
-            AffinityTopologyVersion topVer
-        ) throws IgniteCheckedException {
-            try {
-                GridCacheEntryEx cached = null;
-
-                try {
-                    cached = cctx.dht().entryEx(entry.key());
-
-                    if (log.isDebugEnabled())
-                        log.debug("Rebalancing key [key=" + entry.key() + ", part=" + p + ", node=" + pick.id() + ']');
-
-                    if (cctx.dht().isIgfsDataCache() &&
-                        cctx.dht().igfsDataSpaceUsed() > cctx.dht().igfsDataSpaceMax()) {
-                        LT.error(log, null, "Failed to rebalance IGFS data cache (IGFS space size exceeded maximum " +
-                            "value, will ignore rebalance entries): " + name());
-
-                        if (cached.markObsoleteIfEmpty(null))
-                            cached.context().cache().removeIfObsolete(cached.key());
-
-                        return true;
-                    }
-
-                    if (preloadPred == null || preloadPred.apply(entry)) {
-                        if (cached.initialValue(
-                            entry.value(),
-                            entry.version(),
-                            entry.ttl(),
-                            entry.expireTime(),
-                            true,
-                            topVer,
-                            cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE
-                        )) {
-                            cctx.evicts().touch(cached, topVer); // Start tracking.
-
-                            if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_OBJECT_LOADED) && !cached.isInternal())
-                                cctx.events().addEvent(cached.partition(), cached.key(), cctx.localNodeId(),
-                                    (IgniteUuid)null, null, EVT_CACHE_REBALANCE_OBJECT_LOADED, entry.value(), true, null,
-                                    false, null, null, null);
-                        }
-                        else if (log.isDebugEnabled())
-                            log.debug("Rebalancing entry is already in cache (will ignore) [key=" + cached.key() +
-                                ", part=" + p + ']');
-                    }
-                    else if (log.isDebugEnabled())
-                        log.debug("Rebalance predicate evaluated to false for entry (will ignore): " + entry);
-                }
-                catch (GridCacheEntryRemovedException ignored) {
-                    if (log.isDebugEnabled())
-                        log.debug("Entry has been concurrently removed while rebalancing (will ignore) [key=" +
-                            cached.key() + ", part=" + p + ']');
-                }
-                catch (GridDhtInvalidPartitionException ignored) {
-                    if (log.isDebugEnabled())
-                        log.debug("Partition became invalid during rebalancing (will ignore): " + p);
-
-                    return false;
-                }
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                throw e;
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteCheckedException("Failed to cache rebalanced entry (will stop rebalancing) [local=" +
-                    cctx.nodeId() + ", node=" + pick.id() + ", key=" + entry.key() + ", part=" + p + ']', e);
-            }
-
-            return true;
-        }
-
-        /**
-         * @param idx Unique index for this topic.
-         * @return Topic for partition.
-         */
-        public Object topic(long idx) {
-            return TOPIC_CACHE.topic(cctx.namexx(), cctx.nodeId(), id, idx);
-        }
-
-        /**
-         * @param node Node to demand from.
-         * @param topVer Topology version.
-         * @param d Demand message.
-         * @param exchFut Exchange future.
-         * @return Missed partitions.
-         * @throws InterruptedException If interrupted.
-         * @throws ClusterTopologyCheckedException If node left.
-         * @throws IgniteCheckedException If failed to send message.
-         */
-        private Set<Integer> demandFromNode(
-            ClusterNode node,
-            final AffinityTopologyVersion topVer,
-            GridDhtPartitionDemandMessage d,
-            GridDhtPartitionsExchangeFuture exchFut
-        ) throws InterruptedException, IgniteCheckedException {
-            GridDhtPartitionTopology top = cctx.dht().topology();
-
-            cntr++;
-
-            d.topic(topic(cntr));
-            d.workerId(id);
-
-            Set<Integer> missed = new HashSet<>();
-
-            // Get the same collection that will be sent in the message.
-            Collection<Integer> remaining = d.partitions();
-
-            // Drain queue before processing a new node.
-            drainQueue();
-
-            if (isCancelled() || topologyChanged())
-                return missed;
-
-            cctx.io().addOrderedHandler(d.topic(), new CI2<UUID, GridDhtPartitionSupplyMessage>() {
-                @Override public void apply(UUID nodeId, GridDhtPartitionSupplyMessage msg) {
-                    addMessage(new SupplyMessage(nodeId, msg));
-                }
-            });
-
-            try {
-                boolean retry;
-
-                // DoWhile.
-                // =======
-                do {
-                    retry = false;
-
-                    // Create copy.
-                    d = new GridDhtPartitionDemandMessage(d, remaining);
-
-                    long timeout = GridDhtPartitionDemandPool.this.timeout.get();
-
-                    d.timeout(timeout);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Sending demand message [node=" + node.id() + ", demand=" + d + ']');
-
-                    // Send demand message.
-                    cctx.io().send(node, d, cctx.ioPolicy());
-
-                    // While.
-                    // =====
-                    while (!isCancelled() && !topologyChanged()) {
-                        SupplyMessage s = poll(msgQ, timeout, this);
-
-                        // If timed out.
-                        if (s == null) {
-                            if (msgQ.isEmpty()) { // Safety check.
-                                U.warn(log, "Timed out waiting for partitions to load, will retry in " + timeout +
-                                    " ms (you may need to increase 'networkTimeout' or 'rebalanceBatchSize'" +
-                                    " configuration properties).");
-
-                                growTimeout(timeout);
-
-                                // Ordered listener was removed if timeout expired.
-                                cctx.io().removeOrderedHandler(d.topic());
-
-                                // Must create copy to be able to work with IO manager thread local caches.
-                                d = new GridDhtPartitionDemandMessage(d, remaining);
-
-                                // Create new topic.
-                                d.topic(topic(++cntr));
-
-                                // Create new ordered listener.
-                                cctx.io().addOrderedHandler(d.topic(),
-                                    new CI2<UUID, GridDhtPartitionSupplyMessage>() {
-                                        @Override public void apply(UUID nodeId,
-                                            GridDhtPartitionSupplyMessage msg) {
-                                            addMessage(new SupplyMessage(nodeId, msg));
-                                        }
-                                    });
-
-                                // Resend message with larger timeout.
-                                retry = true;
-
-                                break; // While.
-                            }
-                            else
-                                continue; // While.
-                        }
-
-                        // If topology changed.
-                        if (dummyTopology(s)) {
-                            if (topologyChanged())
-                                break; // While.
-                            else
-                                continue; // While.
-                        }
-
-                        // Check that message was received from expected node.
-                        if (!s.senderId().equals(node.id())) {
-                            U.warn(log, "Received supply message from unexpected node [expectedId=" + node.id() +
-                                ", rcvdId=" + s.senderId() + ", msg=" + s + ']');
-
-                            continue; // While.
-                        }
-
-                        if (log.isDebugEnabled())
-                            log.debug("Received supply message: " + s);
-
-                        GridDhtPartitionSupplyMessage supply = s.supply();
-
-                        // Check whether there were class loading errors on unmarshal
-                        if (supply.classError() != null) {
-                            if (log.isDebugEnabled())
-                                log.debug("Class got undeployed during preloading: " + supply.classError());
-
-                            retry = true;
-
-                            // Quit preloading.
-                            break;
-                        }
-
-                        // Preload.
-                        for (Map.Entry<Integer, CacheEntryInfoCollection> e : supply.infos().entrySet()) {
-                            int p = e.getKey();
-
-                            if (cctx.affinity().localNode(p, topVer)) {
-                                GridDhtLocalPartition part = top.localPartition(p, topVer, true);
-
-                                assert part != null;
-
-                                if (part.state() == MOVING) {
-                                    boolean reserved = part.reserve();
-
-                                    assert reserved : "Failed to reserve partition [gridName=" +
-                                        cctx.gridName() + ", cacheName=" + cctx.namex() + ", part=" + part + ']';
-
-                                    part.lock();
-
-                                    try {
-                                        Collection<Integer> invalidParts = new GridLeanSet<>();
-
-                                        // Loop through all received entries and try to preload them.
-                                        for (GridCacheEntryInfo entry : e.getValue().infos()) {
-                                            if (!invalidParts.contains(p)) {
-                                                if (!part.preloadingPermitted(entry.key(), entry.version())) {
-                                                    if (log.isDebugEnabled())
-                                                        log.debug("Preloading is not permitted for entry due to " +
-                                                            "evictions [key=" + entry.key() +
-                                                            ", ver=" + entry.version() + ']');
-
-                                                    continue;
-                                                }
-
-                                                if (!preloadEntry(node, p, entry, topVer)) {
-                                                    invalidParts.add(p);
-
-                                                    if (log.isDebugEnabled())
-                                                        log.debug("Got entries for invalid partition during " +
-                                                            "preloading (will skip) [p=" + p + ", entry=" + entry + ']');
-                                                }
-                                            }
-                                        }
-
-                                        boolean last = supply.last().contains(p);
-
-                                        // If message was last for this partition,
-                                        // then we take ownership.
-                                        if (last) {
-                                            remaining.remove(p);
-
-                                            top.own(part);
-
-                                            if (log.isDebugEnabled())
-                                                log.debug("Finished rebalancing partition: " + part);
-
-                                            if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_LOADED))
-                                                preloadEvent(p, EVT_CACHE_REBALANCE_PART_LOADED,
-                                                    exchFut.discoveryEvent());
-                                        }
-                                    }
-                                    finally {
-                                        part.unlock();
-                                        part.release();
-                                    }
-                                }
-                                else {
-                                    remaining.remove(p);
-
-                                    if (log.isDebugEnabled())
-                                        log.debug("Skipping rebalancing partition (state is not MOVING): " + part);
-                                }
-                            }
-                            else {
-                                remaining.remove(p);
-
-                                if (log.isDebugEnabled())
-                                    log.debug("Skipping rebalancing partition (it does not belong on current node): " + p);
-                            }
-                        }
-
-                        remaining.removeAll(s.supply().missed());
-
-                        // Only request partitions based on latest topology version.
-                        for (Integer miss : s.supply().missed())
-                            if (cctx.affinity().localNode(miss, topVer))
-                                missed.add(miss);
-
-                        if (remaining.isEmpty())
-                            break; // While.
-
-                        if (s.supply().ack()) {
-                            retry = true;
-
-                            break;
-                        }
-                    }
-                }
-                while (retry && !isCancelled() && !topologyChanged());
-
-                return missed;
-            }
-            finally {
-                cctx.io().removeOrderedHandler(d.topic());
-            }
-        }
-
-        /**
-         * @throws InterruptedException If interrupted.
-         */
-        private void drainQueue() throws InterruptedException {
-            while (!msgQ.isEmpty()) {
-                SupplyMessage msg = msgQ.take();
-
-                if (log.isDebugEnabled())
-                    log.debug("Drained supply message: " + msg);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
-            try {
-                int rebalanceOrder = cctx.config().getRebalanceOrder();
-
-                if (!CU.isMarshallerCache(cctx.name())) {
-                    if (log.isDebugEnabled())
-                        log.debug("Waiting for marshaller cache preload [cacheName=" + cctx.name() + ']');
-
-                    try {
-                        cctx.kernalContext().cache().marshallerCache().preloader().syncFuture().get();
-                    }
-                    catch (IgniteInterruptedCheckedException ignored) {
-                        if (log.isDebugEnabled())
-                            log.debug("Failed to wait for marshaller cache preload future (grid is stopping): " +
-                                "[cacheName=" + cctx.name() + ']');
-
-                        return;
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new Error("Ordered preload future should never fail: " + e.getMessage(), e);
-                    }
-                }
-
-                if (rebalanceOrder > 0) {
-                    IgniteInternalFuture<?> fut = cctx.kernalContext().cache().orderedPreloadFuture(rebalanceOrder);
-
-                    try {
-                        if (fut != null) {
-                            if (log.isDebugEnabled())
-                                log.debug("Waiting for dependant caches rebalance [cacheName=" + cctx.name() +
-                                    ", rebalanceOrder=" + rebalanceOrder + ']');
-
-                            fut.get();
-                        }
-                    }
-                    catch (IgniteInterruptedCheckedException ignored) {
-                        if (log.isDebugEnabled())
-                            log.debug("Failed to wait for ordered rebalance future (grid is stopping): " +
-                                "[cacheName=" + cctx.name() + ", rebalanceOrder=" + rebalanceOrder + ']');
-
-                        return;
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new Error("Ordered rebalance future should never fail: " + e.getMessage(), e);
-                    }
-                }
-
-                GridDhtPartitionsExchangeFuture exchFut = null;
-
-                boolean stopEvtFired = false;
-
-                while (!isCancelled()) {
-                    try {
-                        barrier.await();
-
-                        if (id == 0 && exchFut != null && !exchFut.dummy() &&
-                            cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED)) {
-
-                            if (!cctx.isReplicated() || !stopEvtFired) {
-                                preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
-
-                                stopEvtFired = true;
-                            }
-                        }
-                    }
-                    catch (BrokenBarrierException ignore) {
-                        throw new InterruptedException("Demand worker stopped.");
-                    }
-
-                    // Sync up all demand threads at this step.
-                    GridDhtPreloaderAssignments assigns = null;
-
-                    while (assigns == null)
-                        assigns = poll(assignQ, cctx.gridConfig().getNetworkTimeout(), this);
-
-                    demandLock.readLock().lock();
-
-                    try {
-                        exchFut = assigns.exchangeFuture();
-
-                        // Assignments are empty if preloading is disabled.
-                        if (assigns.isEmpty())
-                            continue;
-
-                        boolean resync = false;
-
-                        // While.
-                        // =====
-                        while (!isCancelled() && !topologyChanged() && !resync) {
-                            Collection<Integer> missed = new HashSet<>();
-
-                            // For.
-                            // ===
-                            for (ClusterNode node : assigns.keySet()) {
-                                if (topologyChanged() || isCancelled())
-                                    break; // For.
-
-                                GridDhtPartitionDemandMessage d = assigns.remove(node);
-
-                                // If another thread is already processing this message,
-                                // move to the next node.
-                                if (d == null)
-                                    continue; // For.
-
-                                try {
-                                    Set<Integer> set = demandFromNode(node, assigns.topologyVersion(), d, exchFut);
-
-                                    if (!set.isEmpty()) {
-                                        if (log.isDebugEnabled())
-                                            log.debug("Missed partitions from node [nodeId=" + node.id() + ", missed=" +
-                                                set + ']');
-
-                                        missed.addAll(set);
-                                    }
-                                }
-                                catch (IgniteInterruptedCheckedException e) {
-                                    throw e;
-                                }
-                                catch (ClusterTopologyCheckedException e) {
-                                    if (log.isDebugEnabled())
-                                        log.debug("Node left during rebalancing (will retry) [node=" + node.id() +
-                                            ", msg=" + e.getMessage() + ']');
-
-                                    resync = true;
-
-                                    break; // For.
-                                }
-                                catch (IgniteCheckedException e) {
-                                    U.error(log, "Failed to receive partitions from node (rebalancing will not " +
-                                        "fully finish) [node=" + node.id() + ", msg=" + d + ']', e);
-                                }
-                            }
-
-                            // Processed missed entries.
-                            if (!missed.isEmpty()) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Reassigning partitions that were missed: " + missed);
-
-                                assert exchFut.exchangeId() != null;
-
-                                cctx.shared().exchange().forceDummyExchange(true, exchFut);
-                            }
-                            else
-                                break; // While.
-                        }
-                    }
-                    finally {
-                        demandLock.readLock().unlock();
-
-                        syncFut.onWorkerDone(this);
-                    }
-
-                    cctx.shared().exchange().scheduleResendPartitions();
-                }
-            }
-            finally {
-                // Safety.
-                syncFut.onWorkerDone(this);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(DemandWorker.class, this, "assignQ", assignQ, "msgQ", msgQ, "super", super.toString());
-        }
-    }
-
-    /**
-     * Sets last exchange future.
-     *
-     * @param lastFut Last future to set.
-     */
-    void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) {
-        lastExchangeFut = lastFut;
-    }
-
-    /**
-     * @param exchFut Exchange future.
-     * @return Assignments of partitions to nodes.
-     */
-    GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
-        // No assignments for disabled preloader.
-        GridDhtPartitionTopology top = cctx.dht().topology();
-
-        if (!cctx.rebalanceEnabled())
-            return new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
-
-        int partCnt = cctx.affinity().partitions();
-
-        assert exchFut.forcePreload() || exchFut.dummyReassign() ||
-            exchFut.exchangeId().topologyVersion().equals(top.topologyVersion()) :
-            "Topology version mismatch [exchId=" + exchFut.exchangeId() +
-                ", topVer=" + top.topologyVersion() + ']';
-
-        GridDhtPreloaderAssignments assigns = new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
-
-        AffinityTopologyVersion topVer = assigns.topologyVersion();
-
-        for (int p = 0; p < partCnt; p++) {
-            if (cctx.shared().exchange().hasPendingExchange()) {
-                if (log.isDebugEnabled())
-                    log.debug("Skipping assignments creation, exchange worker has pending assignments: " +
-                        exchFut.exchangeId());
-
-                break;
-            }
-
-            // If partition belongs to local node.
-            if (cctx.affinity().localNode(p, topVer)) {
-                GridDhtLocalPartition part = top.localPartition(p, topVer, true);
-
-                assert part != null;
-                assert part.id() == p;
-
-                if (part.state() != MOVING) {
-                    if (log.isDebugEnabled())
-                        log.debug("Skipping partition assignment (state is not MOVING): " + part);
-
-                    continue; // For.
-                }
-
-                Collection<ClusterNode> picked = pickedOwners(p, topVer);
-
-                if (picked.isEmpty()) {
-                    top.own(part);
-
-                    if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_DATA_LOST)) {
-                        DiscoveryEvent discoEvt = exchFut.discoveryEvent();
-
-                        cctx.events().addPreloadEvent(p,
-                            EVT_CACHE_REBALANCE_PART_DATA_LOST, discoEvt.eventNode(),
-                            discoEvt.type(), discoEvt.timestamp());
-                    }
-
-                    if (log.isDebugEnabled())
-                        log.debug("Owning partition as there are no other owners: " + part);
-                }
-                else {
-                    ClusterNode n = F.first(picked);
-
-                    GridDhtPartitionDemandMessage msg = assigns.get(n);
-
-                    if (msg == null) {
-                        assigns.put(n, msg = new GridDhtPartitionDemandMessage(
-                            top.updateSequence(),
-                            exchFut.exchangeId().topologyVersion(),
-                            cctx.cacheId()));
-                    }
-
-                    msg.addPartition(p);
-                }
-            }
-        }
-
-        return assigns;
-    }
-
-    /**
-     *
-     */
-    private class SyncFuture extends GridFutureAdapter<Object> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Remaining workers. */
-        private Collection<DemandWorker> remaining;
-
-        /**
-         * @param workers List of workers.
-         */
-        private SyncFuture(Collection<DemandWorker> workers) {
-            assert workers.size() == poolSize();
-
-            remaining = Collections.synchronizedList(new LinkedList<>(workers));
-        }
-
-        /**
-         * @param w Worker who iterated through all partitions.
-         */
-        void onWorkerDone(DemandWorker w) {
-            if (isDone())
-                return;
-
-            if (remaining.remove(w))
-                if (log.isDebugEnabled())
-                    log.debug("Completed full partition iteration for worker [worker=" + w + ']');
-
-            if (remaining.isEmpty()) {
-                if (log.isDebugEnabled())
-                    log.debug("Completed sync future.");
-
-                onDone();
-            }
-        }
-    }
-
-    /**
-     * Supply message wrapper.
-     */
-    private static class SupplyMessage {
-        /** Sender ID. */
-        private UUID sndId;
-
-        /** Supply message. */
-        private GridDhtPartitionSupplyMessage supply;
-
-        /**
-         * Dummy constructor.
-         */
-        private SupplyMessage() {
-            // No-op.
-        }
-
-        /**
-         * @param sndId Sender ID.
-         * @param supply Supply message.
-         */
-        SupplyMessage(UUID sndId, GridDhtPartitionSupplyMessage supply) {
-            this.sndId = sndId;
-            this.supply = supply;
-        }
-
-        /**
-         * @return Sender ID.
-         */
-        UUID senderId() {
-            return sndId;
-        }
-
-        /**
-         * @return Message.
-         */
-        GridDhtPartitionSupplyMessage supply() {
-            return supply;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(SupplyMessage.class, this);
-        }
-    }
-}
\ No newline at end of file


[20/46] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5

Posted by ak...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5


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

Branch: refs/heads/ignite-1753-1282
Commit: 37cafb6ef750e8dd094803cc02f1cc4d36e9dc87
Parents: d70f7ed 9396ccb
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Nov 10 14:21:07 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Nov 10 14:21:07 2015 +0300

----------------------------------------------------------------------
 .../yardstick/config/ignite-store-config.xml    | 31 +++++++++++---------
 1 file changed, 17 insertions(+), 14 deletions(-)
----------------------------------------------------------------------



[23/46] ignite git commit: IGNITE-1868

Posted by ak...@apache.org.
IGNITE-1868


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

Branch: refs/heads/ignite-1753-1282
Commit: 77ee0654b0d3b15368a94782339ec9db4cad959d
Parents: 2b6fc49
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Nov 10 17:50:35 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Nov 10 17:50:35 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/igfs/IgfsMetaManager.java    | 2 +-
 .../core/src/main/java/org/apache/ignite/stream/StreamAdapter.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/77ee0654/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index e75fe51..ab73e7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -990,7 +990,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @param expIds Expected IDs for this path. Might contain additional elements, e.g. because they were created
      *     on a child path.
      * @param infos Locked infos.
-     * @return
+     * @return verification result.
      */
     private static boolean verifyPathIntegrity(IgfsPath path, List<IgniteUuid> expIds,
         Map<IgniteUuid, IgfsFileInfo> infos) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/77ee0654/modules/core/src/main/java/org/apache/ignite/stream/StreamAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/stream/StreamAdapter.java b/modules/core/src/main/java/org/apache/ignite/stream/StreamAdapter.java
index e7d224c..2cb7db7 100644
--- a/modules/core/src/main/java/org/apache/ignite/stream/StreamAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/stream/StreamAdapter.java
@@ -33,7 +33,7 @@ import org.apache.ignite.IgniteDataStreamer;
  *     <li>A single tuple extractor, which extracts either no or 1 tuple out of a message. See
  *     see {@link #setTupleExtractor(StreamTupleExtractor)}.</li>
  *     <li>A multiple tuple extractor, which is capable of extracting multiple tuples out of a single message, in the
- *     form of a {@link Map<K, V>}. See {@link #setMultipleTupleExtractor(StreamMultipleTupleExtractor)}.</li>
+ *     form of a {@link Map}. See {@link #setMultipleTupleExtractor(StreamMultipleTupleExtractor)}.</li>
  * </ol>
  */
 public abstract class StreamAdapter<T, K, V> {


[45/46] ignite git commit: Fixing ignite-1377

Posted by ak...@apache.org.
Fixing ignite-1377


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

Branch: refs/heads/ignite-1753-1282
Commit: c00e4ac48d1d9e0629f8f589612c16c726548152
Parents: 6fa22e2
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 17 15:53:24 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 17 15:53:24 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      |   4 +-
 .../CacheObjectBinaryProcessorImpl.java         |   2 +-
 .../resources/META-INF/classnames.properties    | 196 +++++++++++++------
 3 files changed, 138 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c00e4ac4/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index f779706..928be63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -570,9 +570,7 @@ public class PortableContext implements Externalizable {
 
         mappers.putIfAbsent(typeId, idMapper);
 
-        // TODO uncomment for https://issues.apache.org/jira/browse/IGNITE-1377
-//        if (registerMetadata && isMetaDataEnabled(typeId))
-//            metaHnd.addMeta(typeId, new PortableMetaDataImpl(typeName, desc.fieldsMeta(), null));
+        metaHnd.addMeta(typeId, new BinaryMetaDataImpl(typeName, desc.fieldsMeta(), null));
 
         return desc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c00e4ac4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 2b3aa09..98b619e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -337,7 +337,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
-        metaDataCache = ctx.cache().jcache(CU.UTILITY_CACHE_NAME);
+        metaDataCache = (IgniteCacheProxy)ctx.cache().jcache(CU.UTILITY_CACHE_NAME).withNoRetries();
 
         if (clientNode) {
             assert !metaDataCache.context().affinityNode();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c00e4ac4/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 9376613..4b2c8ee 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -23,6 +23,9 @@ org.apache.ignite.IgniteException
 org.apache.ignite.IgniteIllegalStateException
 org.apache.ignite.IgniteInterruptedException
 org.apache.ignite.IgniteState
+org.apache.ignite.binary.BinaryInvalidTypeException
+org.apache.ignite.binary.BinaryObject
+org.apache.ignite.binary.BinaryObjectException
 org.apache.ignite.cache.CacheAtomicUpdateTimeoutException
 org.apache.ignite.cache.CacheAtomicWriteOrderMode
 org.apache.ignite.cache.CacheAtomicityMode
@@ -31,6 +34,7 @@ org.apache.ignite.cache.CacheEntryProcessor
 org.apache.ignite.cache.CacheExistsException
 org.apache.ignite.cache.CacheInterceptor
 org.apache.ignite.cache.CacheInterceptorAdapter
+org.apache.ignite.cache.CacheKeyConfiguration
 org.apache.ignite.cache.CacheMemoryMode
 org.apache.ignite.cache.CacheMode
 org.apache.ignite.cache.CachePartialUpdateException
@@ -40,6 +44,9 @@ org.apache.ignite.cache.CacheServerNotFoundException
 org.apache.ignite.cache.CacheTypeFieldMetadata
 org.apache.ignite.cache.CacheTypeMetadata
 org.apache.ignite.cache.CacheWriteSynchronizationMode
+org.apache.ignite.cache.QueryEntity
+org.apache.ignite.cache.QueryIndex
+org.apache.ignite.cache.QueryIndexType
 org.apache.ignite.cache.affinity.AffinityFunction
 org.apache.ignite.cache.affinity.AffinityKey
 org.apache.ignite.cache.affinity.AffinityKeyMapper
@@ -48,6 +55,9 @@ org.apache.ignite.cache.affinity.AffinityNodeHashResolver
 org.apache.ignite.cache.affinity.AffinityNodeIdHashResolver
 org.apache.ignite.cache.affinity.AffinityUuid
 org.apache.ignite.cache.affinity.fair.FairAffinityFunction
+org.apache.ignite.cache.affinity.fair.FairAffinityFunction$1
+org.apache.ignite.cache.affinity.fair.FairAffinityFunction$FullAssignmentMap$1
+org.apache.ignite.cache.affinity.fair.FairAffinityFunction$FullAssignmentMap$2
 org.apache.ignite.cache.affinity.fair.FairAffinityFunction$PartitionSetComparator
 org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction
 org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction$HashComparator
@@ -205,7 +215,7 @@ org.apache.ignite.internal.IgniteFutureTimeoutCheckedException
 org.apache.ignite.internal.IgniteInterruptedCheckedException
 org.apache.ignite.internal.IgniteKernal
 org.apache.ignite.internal.IgniteKernal$1
-org.apache.ignite.internal.IgniteKernal$5
+org.apache.ignite.internal.IgniteKernal$4
 org.apache.ignite.internal.IgniteMessagingImpl
 org.apache.ignite.internal.IgniteSchedulerImpl
 org.apache.ignite.internal.IgniteServicesImpl
@@ -254,10 +264,6 @@ org.apache.ignite.internal.executor.GridExecutorService
 org.apache.ignite.internal.executor.GridExecutorService$1
 org.apache.ignite.internal.executor.GridExecutorService$TaskTerminateListener
 org.apache.ignite.internal.igfs.common.IgfsIpcCommand
-org.apache.ignite.internal.interop.InteropAwareEventFilter
-org.apache.ignite.internal.interop.InteropBootstrapFactory
-org.apache.ignite.internal.interop.InteropException
-org.apache.ignite.internal.interop.InteropNoCallbackException
 org.apache.ignite.internal.jdbc2.JdbcConnection$JdbcConnectionValidationTask
 org.apache.ignite.internal.jdbc2.JdbcDatabaseMetadata$UpdateMetadataTask
 org.apache.ignite.internal.jdbc2.JdbcQueryTask
@@ -269,7 +275,6 @@ org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest
 org.apache.ignite.internal.managers.communication.GridIoManager$ConcurrentHashMap0
 org.apache.ignite.internal.managers.communication.GridIoMessage
 org.apache.ignite.internal.managers.communication.GridIoUserMessage
-org.apache.ignite.internal.managers.communication.GridLifecycleAwareMessageFilter
 org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean
 org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2
 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest
@@ -287,14 +292,14 @@ org.apache.ignite.internal.managers.eventstorage.GridEventStorageMessage
 org.apache.ignite.internal.managers.indexing.GridIndexingManager$1
 org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerAdapter
 org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager$1
-org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter
+org.apache.ignite.internal.portable.BinaryMetaDataImpl
+org.apache.ignite.internal.portable.BinaryObjectEx
+org.apache.ignite.internal.portable.BinaryObjectImpl
+org.apache.ignite.internal.portable.BinaryObjectOffheapImpl
+org.apache.ignite.internal.portable.BinaryReaderExImpl$Flag
 org.apache.ignite.internal.portable.PortableClassDescriptor$Mode
 org.apache.ignite.internal.portable.PortableContext
 org.apache.ignite.internal.portable.builder.PortableLazyMap$1$1$1
-org.apache.ignite.internal.portable.PortableMetaDataImpl
-org.apache.ignite.internal.portable.PortableObjectEx
-org.apache.ignite.internal.portable.PortableObjectImpl
-org.apache.ignite.internal.portable.PortableObjectOffheapImpl
 org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion
 org.apache.ignite.internal.processors.affinity.GridAffinityAssignment
 org.apache.ignite.internal.processors.affinity.GridAffinityMessage
@@ -326,38 +331,39 @@ org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQu
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest
 org.apache.ignite.internal.processors.cache.GridCacheAdapter
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$10
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$11
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$12
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$13
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$13$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$14
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$15$1
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$16
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$17
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$15
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$2
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$25$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$23$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$25
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$26$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$27
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$28$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$29
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$3
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$31
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$4
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$6
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$62
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$63
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$64
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$65
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$66
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$67
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$67$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$68
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$69
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$69$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$7
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$70
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$71
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$72
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$9
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOpRetryFuture$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOpRetryFuture$1$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$ClearTask
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalClearAllJob
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalClearAllNearJob
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalClearKeySetJob
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalClearKeySetNearJob
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$LoadCacheClosure
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$LoadKeysCallable
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$SizeJob
@@ -407,7 +413,6 @@ org.apache.ignite.internal.processors.cache.GridCacheGateway$State
 org.apache.ignite.internal.processors.cache.GridCacheIndexUpdateException
 org.apache.ignite.internal.processors.cache.GridCacheIoManager$1$1
 org.apache.ignite.internal.processors.cache.GridCacheIoManager$2
-org.apache.ignite.internal.processors.cache.GridCacheIoManager$3
 org.apache.ignite.internal.processors.cache.GridCacheIterator
 org.apache.ignite.internal.processors.cache.GridCacheKeySet
 org.apache.ignite.internal.processors.cache.GridCacheLoaderWriterStore
@@ -421,8 +426,6 @@ org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate$Mask
 org.apache.ignite.internal.processors.cache.GridCacheMvccManager$4
 org.apache.ignite.internal.processors.cache.GridCacheMvccManager$5
 org.apache.ignite.internal.processors.cache.GridCacheMvccManager$6
-org.apache.ignite.internal.processors.cache.GridCacheMvccManager$7
-org.apache.ignite.internal.processors.cache.GridCacheMvccManager$8
 org.apache.ignite.internal.processors.cache.GridCacheMvccManager$FinishLockFuture$1
 org.apache.ignite.internal.processors.cache.GridCacheOperation
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$1$1
@@ -436,24 +439,27 @@ org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$Me
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$2
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$3
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$4
+org.apache.ignite.internal.processors.cache.GridCacheProcessor$5
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$6
+org.apache.ignite.internal.processors.cache.GridCacheProcessor$8
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$LocalAffinityFunction
 org.apache.ignite.internal.processors.cache.GridCacheProxyImpl
 org.apache.ignite.internal.processors.cache.GridCacheReturn
-org.apache.ignite.internal.processors.cache.GridCacheSwapManager$10
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$12
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$14
-org.apache.ignite.internal.processors.cache.GridCacheSwapManager$15
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$16
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$17
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$18
-org.apache.ignite.internal.processors.cache.GridCacheSwapManager$2
-org.apache.ignite.internal.processors.cache.GridCacheSwapManager$21
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$19
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$20
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$23
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$3
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$4
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$5
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$6
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$7
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$8
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$9
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$CloseablePartitionsIterator
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$IteratorWrapper
 org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException
@@ -484,13 +490,15 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$8
 org.apache.ignite.internal.processors.cache.GridCacheUtils$9
 org.apache.ignite.internal.processors.cache.GridCacheValueCollection
 org.apache.ignite.internal.processors.cache.GridCacheValueCollection$1
-org.apache.ignite.internal.processors.cache.GridLoadCacheCloseablePredicate
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$1
+org.apache.ignite.internal.processors.cache.IgniteCacheProxy$10
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$2
+org.apache.ignite.internal.processors.cache.IgniteCacheProxy$3
+org.apache.ignite.internal.processors.cache.IgniteCacheProxy$4
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$5
-org.apache.ignite.internal.processors.cache.IgniteCacheProxy$6
-org.apache.ignite.internal.processors.cache.IgniteCacheProxy$7
+org.apache.ignite.internal.processors.cache.IgniteCacheProxy$8
+org.apache.ignite.internal.processors.cache.IgniteCacheProxy$9
 org.apache.ignite.internal.processors.cache.KeyCacheObject
 org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl
 org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy
@@ -501,6 +509,7 @@ org.apache.ignite.internal.processors.cache.distributed.GridCacheCommittedTxInfo
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture$1
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture$2
+org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture$3
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse
 org.apache.ignite.internal.processors.cache.distributed.GridDistributedBaseMessage
@@ -543,6 +552,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionStat
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsReservation$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$1
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$10
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$3
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$4
@@ -552,8 +562,6 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactional
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$8
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$9
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$9$1
-org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$9$1$1
-org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$9$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest
@@ -568,6 +576,8 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFutu
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$3
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$4
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$5
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$6
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemote
@@ -575,9 +585,11 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$2
+org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$3
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$MiniFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$MiniFuture$2
-org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$MiniFuture$2$1
+org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$MiniFuture$3
+org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$MiniFuture$3$1
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$10
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$11
@@ -605,7 +617,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomic
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$3
-org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$4
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$UpdateState$1
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache
@@ -618,7 +630,9 @@ org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtCol
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$3
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$4
+org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$5
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$MiniFuture$1
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage
@@ -664,9 +678,11 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapte
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter$EntrySet$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$2
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$3
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$MiniFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$MiniFuture$2
-org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$MiniFuture$2$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$MiniFuture$3
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture$MiniFuture$3$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$1
@@ -676,11 +692,18 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$MiniFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticSerializableTxPrepareFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticSerializableTxPrepareFuture$2
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticSerializableTxPrepareFuture$3
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticSerializableTxPrepareFuture$4
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticSerializableTxPrepareFuture$ClientRemapFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticSerializableTxPrepareFuture$MiniFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticSerializableTxPrepareFuture$MiniFuture$1$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$3
-org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$4
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$MiniFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFutureAdapter$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$3
@@ -714,13 +737,12 @@ org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$5
 org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$6
 org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$9
 org.apache.ignite.internal.processors.cache.portable.CacheDefaultPortableAffinityKeyMapper
-org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$1
-org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$4
-org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$MetaDataEntryFilter
-org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$MetaDataPredicate
-org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$MetaDataProcessor
+org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl$1
+org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl$4
+org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl$MetaDataEntryFilter
+org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl$MetaDataPredicate
+org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl$MetaDataProcessor
 org.apache.ignite.internal.processors.cache.portable.PortableMetaDataKey
-org.apache.ignite.internal.processors.cache.query.CacheQueryCloseableScanBiPredicate
 org.apache.ignite.internal.processors.cache.query.CacheQueryType
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryFuture$1
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryFuture$3
@@ -730,9 +752,6 @@ org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManag
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$5
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$6
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$1
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$2
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$CacheQueryFallbackFuture$1
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$CacheQueryFallbackFuture$1$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$1
@@ -767,7 +786,6 @@ org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata
 org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEvent
-org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterEx
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler$2
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler$DeployableObject
@@ -782,7 +800,6 @@ org.apache.ignite.internal.processors.cache.query.jdbc.GridCacheQueryJdbcValidat
 org.apache.ignite.internal.processors.cache.store.GridCacheStoreManagerAdapter$1
 org.apache.ignite.internal.processors.cache.store.GridCacheStoreManagerAdapter$2
 org.apache.ignite.internal.processors.cache.store.GridCacheStoreManagerAdapter$3
-org.apache.ignite.internal.processors.cache.store.GridCacheStoreManagerAdapter$4
 org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore$StatefulValue
 org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore$StoreOperation
 org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore$ValueStatus
@@ -795,7 +812,6 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$11
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$12
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$13
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$14
-org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$15
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$2
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$3
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$4
@@ -806,20 +822,17 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$8
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$9
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$1
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$10
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$11
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$12
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$13
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$14
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$15
-org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$16
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$2
-org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$3
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$4
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$5
-org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$6
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$7
-org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$8
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$9
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$FinishClosure
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PLC1
@@ -1008,9 +1021,10 @@ org.apache.ignite.internal.processors.igfs.IgfsInvalidRangeException
 org.apache.ignite.internal.processors.igfs.IgfsIpcHandler$2
 org.apache.ignite.internal.processors.igfs.IgfsJobImpl
 org.apache.ignite.internal.processors.igfs.IgfsListingEntry
-org.apache.ignite.internal.processors.igfs.IgfsMetaManager$1
 org.apache.ignite.internal.processors.igfs.IgfsMetaManager$2
-org.apache.ignite.internal.processors.igfs.IgfsMetaManager$3$1
+org.apache.ignite.internal.processors.igfs.IgfsMetaManager$3
+org.apache.ignite.internal.processors.igfs.IgfsMetaManager$4
+org.apache.ignite.internal.processors.igfs.IgfsMetaManager$LockFileProcessor
 org.apache.ignite.internal.processors.igfs.IgfsMetaManager$UpdateListing
 org.apache.ignite.internal.processors.igfs.IgfsMetaManager$UpdateListingEntry
 org.apache.ignite.internal.processors.igfs.IgfsMetaManager$UpdatePath
@@ -1030,6 +1044,65 @@ org.apache.ignite.internal.processors.job.GridJobProcessor$6
 org.apache.ignite.internal.processors.job.GridJobProcessor$7
 org.apache.ignite.internal.processors.job.GridJobWorker$3
 org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor$SnapshotReducer
+org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure
+org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate
+org.apache.ignite.internal.processors.platform.PlatformEventFilterListener
+org.apache.ignite.internal.processors.platform.PlatformException
+org.apache.ignite.internal.processors.platform.PlatformExtendedException
+org.apache.ignite.internal.processors.platform.PlatformNativeException
+org.apache.ignite.internal.processors.platform.PlatformNoCallbackException
+org.apache.ignite.internal.processors.platform.cache.PlatformCache$1
+org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter
+org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl
+org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor
+org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessorImpl
+org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException
+org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity$1
+org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery
+org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter
+org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl
+org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryRemoteFilter
+org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilter
+org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilterImpl
+org.apache.ignite.internal.processors.platform.compute.PlatformAbstractJob
+org.apache.ignite.internal.processors.platform.compute.PlatformAbstractTask
+org.apache.ignite.internal.processors.platform.compute.PlatformBalancingMultiClosureTask
+org.apache.ignite.internal.processors.platform.compute.PlatformBalancingSingleClosureAffinityTask
+org.apache.ignite.internal.processors.platform.compute.PlatformBalancingSingleClosureTask
+org.apache.ignite.internal.processors.platform.compute.PlatformBroadcastingMultiClosureTask
+org.apache.ignite.internal.processors.platform.compute.PlatformBroadcastingSingleClosureTask
+org.apache.ignite.internal.processors.platform.compute.PlatformClosureJob
+org.apache.ignite.internal.processors.platform.compute.PlatformCompute$1
+org.apache.ignite.internal.processors.platform.compute.PlatformCompute$2
+org.apache.ignite.internal.processors.platform.compute.PlatformFullJob
+org.apache.ignite.internal.processors.platform.compute.PlatformFullTask
+org.apache.ignite.internal.processors.platform.compute.PlatformJob
+org.apache.ignite.internal.processors.platform.cpp.PlatformCppConfigurationClosure
+org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver
+org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiverImpl
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$1
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$2
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$3
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$4
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$5
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$5$1$1$1
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$6
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$7
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$8
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl
+org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl
+org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter
+org.apache.ignite.internal.processors.platform.messaging.PlatformMessageFilterImpl
+org.apache.ignite.internal.processors.platform.messaging.PlatformMessageLocalFilter
+org.apache.ignite.internal.processors.platform.services.PlatformAbstractService
+org.apache.ignite.internal.processors.platform.services.PlatformService
+org.apache.ignite.internal.processors.platform.services.PlatformServices$2
+org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions$1
+org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$1
+org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$FutureListenable$1
+org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$InternalFutureListenable$1
 org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
 org.apache.ignite.internal.processors.query.GridQueryIndexType
 org.apache.ignite.internal.processors.query.GridQueryProcessor$2
@@ -1040,6 +1113,7 @@ org.apache.ignite.internal.processors.query.GridQueryProcessor$6
 org.apache.ignite.internal.processors.query.GridQueryProcessor$7
 org.apache.ignite.internal.processors.query.GridQueryProcessor$8
 org.apache.ignite.internal.processors.query.GridQueryProcessor$9
+org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest
@@ -1065,8 +1139,8 @@ org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBe
 org.apache.ignite.internal.processors.rest.client.message.GridClientTopologyRequest
 org.apache.ignite.internal.processors.rest.client.message.GridRouterRequest
 org.apache.ignite.internal.processors.rest.client.message.GridRouterResponse
-org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$1
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$2
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$3
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$4
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$AddCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$AppendCommand
@@ -1084,6 +1158,7 @@ org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandle
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetAndRemoveCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetAndReplaceCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$MetadataCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$MetricsCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$PrependCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$PutAllCommand
@@ -1109,6 +1184,7 @@ org.apache.ignite.internal.processors.rest.protocols.tcp.GridMemcachedMessage
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioListener$1
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioListener$2
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1
+org.apache.ignite.internal.processors.rest.request.RestQueryRequest$QueryType
 org.apache.ignite.internal.processors.service.GridServiceAssignments
 org.apache.ignite.internal.processors.service.GridServiceAssignmentsKey
 org.apache.ignite.internal.processors.service.GridServiceDeployment
@@ -1219,6 +1295,8 @@ org.apache.ignite.internal.util.future.GridEmbeddedFuture$2
 org.apache.ignite.internal.util.future.GridEmbeddedFuture$2$1
 org.apache.ignite.internal.util.future.GridEmbeddedFuture$3
 org.apache.ignite.internal.util.future.GridEmbeddedFuture$3$1
+org.apache.ignite.internal.util.future.GridEmbeddedFuture$4
+org.apache.ignite.internal.util.future.GridEmbeddedFuture$4$1
 org.apache.ignite.internal.util.future.GridEmbeddedFuture$AL1
 org.apache.ignite.internal.util.future.GridEmbeddedFuture$AL2
 org.apache.ignite.internal.util.future.GridEmbeddedFuture$AsyncListener1
@@ -1559,6 +1637,7 @@ org.apache.ignite.lifecycle.LifecycleEventType
 org.apache.ignite.marshaller.jdk.JdkMarshallerDummySerializable
 org.apache.ignite.marshaller.optimized.OptimizedFieldType
 org.apache.ignite.messaging.MessagingListenActor
+org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory
 org.apache.ignite.plugin.CachePluginConfiguration
 org.apache.ignite.plugin.PluginNotFoundException
 org.apache.ignite.plugin.PluginValidationException
@@ -1572,9 +1651,6 @@ org.apache.ignite.plugin.security.SecuritySubject
 org.apache.ignite.plugin.security.SecuritySubjectType
 org.apache.ignite.plugin.segmentation.SegmentationPolicy
 org.apache.ignite.plugin.segmentation.SegmentationResolver
-org.apache.ignite.portable.PortableException
-org.apache.ignite.portable.PortableInvalidClassException
-org.apache.ignite.portable.PortableObject
 org.apache.ignite.services.Service
 org.apache.ignite.services.ServiceConfiguration
 org.apache.ignite.services.ServiceContext


[06/46] ignite git commit: IGNITE-1161 Close rest sql cursor after delay. - Fixes #197.

Posted by ak...@apache.org.
IGNITE-1161 Close rest sql cursor after delay. - Fixes #197.

Signed-off-by: Andrey <an...@gridgain.com>


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

Branch: refs/heads/ignite-1753-1282
Commit: 621ecac317f55ba467bdb16321bfe550f5d3319b
Parents: 7dfaa3b
Author: Andrey <an...@gridgain.com>
Authored: Mon Nov 9 10:30:56 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Mon Nov 9 10:30:56 2015 +0700

----------------------------------------------------------------------
 .../rest/AbstractRestProcessorSelfTest.java     |   5 +-
 .../JettyRestProcessorAbstractSelfTest.java     |  37 +++
 .../configuration/ConnectorConfiguration.java   |  61 ++++-
 .../handlers/query/QueryCommandHandler.java     | 258 +++++++++++++++----
 4 files changed, 302 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/621ecac3/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/AbstractRestProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/AbstractRestProcessorSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/AbstractRestProcessorSelfTest.java
index bde9180..9a030a7 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/AbstractRestProcessorSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/AbstractRestProcessorSelfTest.java
@@ -75,6 +75,9 @@ abstract class AbstractRestProcessorSelfTest extends GridCommonAbstractTest {
 
         clientCfg.setJettyPath("modules/clients/src/test/resources/jetty/rest-jetty.xml");
 
+        clientCfg.setIdleQueryCursorTimeout(5000);
+        clientCfg.setIdleQueryCursorCheckFrequency(5000);
+
         cfg.setConnectorConfiguration(clientCfg);
 
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
@@ -99,4 +102,4 @@ abstract class AbstractRestProcessorSelfTest extends GridCommonAbstractTest {
     @Override protected <K, V> IgniteCache<K, V> jcache() {
         return grid(0).cache(null);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/621ecac3/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index c413bbd..c9c4ced 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata;
 import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandler;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -1406,6 +1407,42 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
         assertFalse(queryCursorFound());
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryDelay() throws Exception {
+        String qry = "salary > ? and salary <= ?";
+
+        Map<String, String> params = new HashMap<>();
+        params.put("cmd", GridRestCommand.EXECUTE_SQL_QUERY.key());
+        params.put("type", "Person");
+        params.put("pageSize", "1");
+        params.put("cacheName", "person");
+        params.put("qry", URLEncoder.encode(qry));
+        params.put("arg1", "1000");
+        params.put("arg2", "2000");
+
+        String ret = null;
+
+        for (int i = 0; i < 10; ++i)
+            ret = content(params);
+
+        assertNotNull(ret);
+        assertTrue(!ret.isEmpty());
+
+        JSONObject json = JSONObject.fromObject(ret);
+
+        List items = (List)((Map)json.get("response")).get("items");
+
+        assertEquals(1, items.size());
+
+        assertTrue(queryCursorFound());
+
+        U.sleep(10000);
+
+        assertFalse(queryCursorFound());
+    }
+
     protected abstract String signature() throws Exception;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/621ecac3/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
index 88d015c..1bfcbe4 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
@@ -59,6 +59,12 @@ public class ConnectorConfiguration {
     /** Default socket send and receive buffer size. */
     public static final int DFLT_SOCK_BUF_SIZE = 32 * 1024;
 
+    /** Default REST idle timeout for query cursor. */
+    private static final long DFLT_IDLE_QRY_CUR_TIMEOUT = 10 * 60 * 1000;
+
+    /** Default REST check frequency for idle query cursor. */
+    private static final long DFLT_IDLE_QRY_CUR_CHECK_FRQ = 60 * 1000;
+
     /** Jetty XML configuration path. */
     private String jettyPath;
 
@@ -83,6 +89,12 @@ public class ConnectorConfiguration {
     /** REST TCP receive buffer size. */
     private int rcvBufSize = DFLT_SOCK_BUF_SIZE;
 
+    /** REST idle timeout for query cursor. */
+    private long idleQryCurTimeout = DFLT_IDLE_QRY_CUR_TIMEOUT;
+
+    /** REST idle check frequency for query cursor. */
+    private long idleQryCurCheckFreq = DFLT_IDLE_QRY_CUR_CHECK_FRQ;
+
     /** REST TCP send queue limit. */
     private int sndQueueLimit;
 
@@ -146,6 +158,8 @@ public class ConnectorConfiguration {
         sslClientAuth = cfg.isSslClientAuth();
         sslCtxFactory = cfg.getSslContextFactory();
         sslEnabled = cfg.isSslEnabled();
+        idleQryCurTimeout = cfg.getIdleQueryCursorTimeout();
+        idleQryCurCheckFreq = cfg.getIdleQueryCursorCheckFrequency();
     }
 
     /**
@@ -545,4 +559,49 @@ public class ConnectorConfiguration {
     public void setMessageInterceptor(ConnectorMessageInterceptor interceptor) {
         msgInterceptor = interceptor;
     }
-}
\ No newline at end of file
+
+    /**
+     * Sets idle query cursors timeout.
+     *
+     * @param idleQryCurTimeout Idle query cursors timeout in milliseconds.
+     * @see #getIdleQueryCursorTimeout()
+     */
+    public void setIdleQueryCursorTimeout(long idleQryCurTimeout) {
+        this.idleQryCurTimeout = idleQryCurTimeout;
+    }
+
+    /**
+     * Gets idle query cursors timeout in milliseconds.
+     * <p>
+     * This setting is used to reject open query cursors that is not used. If no fetch query request
+     * come within idle timeout, it will be removed on next check for old query cursors
+     * (see {@link #getIdleQueryCursorCheckFrequency()}).
+     *
+     * @return Idle query cursors timeout in milliseconds
+     */
+    public long getIdleQueryCursorTimeout() {
+        return idleQryCurTimeout;
+    }
+
+    /**
+     * Sets idle query cursor check frequency.
+     *
+     * @param idleQryCurCheckFreq Idle query check frequency in milliseconds.
+     * @see #getIdleQueryCursorCheckFrequency()
+     */
+    public void setIdleQueryCursorCheckFrequency(long idleQryCurCheckFreq) {
+        this.idleQryCurCheckFreq = idleQryCurCheckFreq;
+    }
+
+    /**
+     * Gets idle query cursors check frequency.
+     * This setting is used to reject open query cursors that is not used.
+     * <p>
+     * Scheduler tries with specified period to close queries' cursors that are overtime.
+     *
+     * @return Idle query cursor check frequency in milliseconds.
+     */
+    public long getIdleQueryCursorCheckFrequency() {
+        return idleQryCurCheckFreq;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/621ecac3/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
index f4ddd59..54cdd29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/query/QueryCommandHandler.java
@@ -23,9 +23,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.query.Query;
@@ -45,7 +47,6 @@ import org.apache.ignite.internal.processors.rest.request.RestQueryRequest;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
-import org.apache.ignite.lang.IgniteBiTuple;
 
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLOSE_SQL_QUERY;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.EXECUTE_SCAN_QUERY;
@@ -68,25 +69,53 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
     private static final AtomicLong qryIdGen = new AtomicLong();
 
     /** Current queries cursors. */
-    private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<Long, QueryCursorIterator> qryCurs = new ConcurrentHashMap<>();
 
     /**
      * @param ctx Context.
      */
     public QueryCommandHandler(GridKernalContext ctx) {
         super(ctx);
+
+        final long idleQryCurTimeout = ctx.config().getConnectorConfiguration().getIdleQueryCursorTimeout();
+
+        long idleQryCurCheckFreq = ctx.config().getConnectorConfiguration().getIdleQueryCursorCheckFrequency();
+
+        ctx.timeout().schedule(new Runnable() {
+            @Override public void run() {
+                long time = U.currentTimeMillis();
+
+                for (Map.Entry<Long, QueryCursorIterator> e : qryCurs.entrySet()) {
+                    QueryCursorIterator qryCurIt = e.getValue();
+
+                    long createTime = qryCurIt.timestamp();
+
+                    if (createTime + idleQryCurTimeout > time && qryCurIt.tryLock()) {
+                        try {
+                            qryCurIt.timestamp(-1);
+
+                            qryCurs.remove(e.getKey(), qryCurIt);
+
+                            qryCurIt.close();
+                        }
+                        finally {
+                            qryCurIt.unlock();
+                        }
+                    }
+                }
+            }
+        }, idleQryCurCheckFreq, idleQryCurCheckFreq);
     }
 
     /**
-     * @param qryCurs Query cursors.
      * @param cur Current cursor.
      * @param req Sql request.
      * @param qryId Query id.
+     * @param qryCurs Query cursors.
      * @return Query result with items.
      */
     private static CacheQueryResult createQueryResult(
-        ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs,
-        Iterator cur, RestQueryRequest req, Long qryId) {
+        Iterator cur, RestQueryRequest req, Long qryId, ConcurrentHashMap<Long, QueryCursorIterator> qryCurs) {
         CacheQueryResult res = new CacheQueryResult();
 
         List<Object> items = new ArrayList<>();
@@ -101,12 +130,39 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         res.setQueryId(qryId);
 
         if (!cur.hasNext())
-            qryCurs.remove(qryId);
+            removeQueryCursor(qryId, qryCurs);
 
         return res;
     }
 
     /**
+     * Removes query cursor.
+     *
+     * @param qryId Query id.
+     * @param qryCurs Query cursors.
+     */
+    private static void removeQueryCursor(Long qryId, ConcurrentHashMap<Long, QueryCursorIterator> qryCurs) {
+        QueryCursorIterator qryCurIt = qryCurs.get(qryId);
+
+        if (qryCurIt == null)
+            return;
+
+        qryCurIt.lock();
+
+        try {
+            if (qryCurIt.timestamp() == -1)
+                return;
+
+            qryCurIt.close();
+
+            qryCurs.remove(qryId);
+        }
+        finally {
+            qryCurIt.unlock();
+        }
+    }
+
+    /**
      * Creates class instance.
      *
      * @param cls Target class.
@@ -169,7 +225,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
             case FETCH_SQL_QUERY: {
                 return ctx.closure().callLocalSafe(
-                    new FetchQueryCallable(ctx, (RestQueryRequest)req, qryCurs), false);
+                    new FetchQueryCallable((RestQueryRequest)req, qryCurs), false);
             }
 
             case CLOSE_SQL_QUERY: {
@@ -191,16 +247,16 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         /** Execute query request. */
         private RestQueryRequest req;
 
-        /** Queries cursors. */
-        private ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs;
+        /** Current queries cursors. */
+        private final ConcurrentHashMap<Long, QueryCursorIterator> qryCurs;
 
         /**
          * @param ctx Kernal context.
          * @param req Execute query request.
-         * @param qryCurs Queries cursors.
+         * @param qryCurs Query cursors.
          */
         public ExecuteQueryCallable(GridKernalContext ctx, RestQueryRequest req,
-            ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs) {
+            ConcurrentHashMap<Long, QueryCursorIterator> qryCurs) {
             this.ctx = ctx;
             this.req = req;
             this.qryCurs = qryCurs;
@@ -208,7 +264,7 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
 
         /** {@inheritDoc} */
         @Override public GridRestResponse call() throws Exception {
-            long qryId = qryIdGen.getAndIncrement();
+            final long qryId = qryIdGen.getAndIncrement();
 
             try {
                 Query qry;
@@ -248,38 +304,51 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,
                         "Failed to find cache with name: " + req.cacheName());
 
-                QueryCursor qryCur = cache.query(qry);
+                final QueryCursor qryCur = cache.query(qry);
 
                 Iterator cur = qryCur.iterator();
 
-                qryCurs.put(qryId, new IgniteBiTuple<>(qryCur, cur));
+                QueryCursorIterator qryCurIt = new QueryCursorIterator(qryCur, cur);
 
-                CacheQueryResult res = createQueryResult(qryCurs, cur, req, qryId);
+                qryCurIt.lock();
 
-                switch (req.queryType()) {
-                    case SQL:
-                    case SQL_FIELDS:
-                        List<GridQueryFieldMetadata> fieldsMeta = ((QueryCursorImpl) qryCur).fieldsMeta();
+                try {
+                    qryCurs.put(qryId, qryCurIt);
 
-                        res.setFieldsMetadata(convertMetadata(fieldsMeta));
+                    CacheQueryResult res = createQueryResult(cur, req, qryId, qryCurs);
 
-                        break;
-                    case SCAN:
-                        CacheQueryFieldsMetaResult keyField = new CacheQueryFieldsMetaResult();
-                        keyField.setFieldName("key");
+                    switch (req.queryType()) {
+                        case SQL:
+                        case SQL_FIELDS:
+                            List<GridQueryFieldMetadata> fieldsMeta = ((QueryCursorImpl)qryCur).fieldsMeta();
 
-                        CacheQueryFieldsMetaResult valField = new CacheQueryFieldsMetaResult();
-                        valField.setFieldName("value");
+                            res.setFieldsMetadata(convertMetadata(fieldsMeta));
 
-                        res.setFieldsMetadata(U.sealList(keyField, valField));
+                            break;
+                        case SCAN:
+                            CacheQueryFieldsMetaResult keyField = new CacheQueryFieldsMetaResult();
+                            keyField.setFieldName("key");
 
-                        break;
-                }
+                            CacheQueryFieldsMetaResult valField = new CacheQueryFieldsMetaResult();
+                            valField.setFieldName("value");
+
+                            res.setFieldsMetadata(U.sealList(keyField, valField));
 
-                return new GridRestResponse(res);
+                            break;
+                    }
+
+                    List<GridQueryFieldMetadata> fieldsMeta = ((QueryCursorImpl<?>)qryCur).fieldsMeta();
+
+                    res.setFieldsMetadata(convertMetadata(fieldsMeta));
+
+                    return new GridRestResponse(res);
+                }
+                finally {
+                    qryCurIt.unlock();
+                }
             }
             catch (Exception e) {
-                qryCurs.remove(qryId);
+                removeQueryCursor(qryId, qryCurs);
 
                 return new GridRestResponse(GridRestResponse.STATUS_FAILED, e.getMessage());
             }
@@ -305,17 +374,17 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
      * Close query callable.
      */
     private static class CloseQueryCallable implements Callable<GridRestResponse> {
-        /** Queries cursors. */
-        private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs;
+        /** Current queries cursors. */
+        private final ConcurrentHashMap<Long, QueryCursorIterator> qryCurs;
+
         /** Execute query request. */
         private RestQueryRequest req;
 
         /**
          * @param req Execute query request.
-         * @param qryCurs Queries cursors.
+         * @param qryCurs Query cursors.
          */
-        public CloseQueryCallable(RestQueryRequest req,
-            ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs) {
+        public CloseQueryCallable(RestQueryRequest req, ConcurrentHashMap<Long, QueryCursorIterator> qryCurs) {
             this.req = req;
             this.qryCurs = qryCurs;
         }
@@ -323,20 +392,29 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         /** {@inheritDoc} */
         @Override public GridRestResponse call() throws Exception {
             try {
-                QueryCursor cur = qryCurs.get(req.queryId()).get1();
+                QueryCursorIterator qryCurIt = qryCurs.get(req.queryId());
 
-                if (cur == null)
-                    return new GridRestResponse(GridRestResponse.STATUS_FAILED,
-                        "Failed to find query with ID: " + req.queryId());
+                if (qryCurIt == null)
+                    return new GridRestResponse(true);
 
-                cur.close();
+                qryCurIt.lock();
 
-                qryCurs.remove(req.queryId());
+                try {
+                    if (qryCurIt.timestamp() == -1)
+                        return new GridRestResponse(true);
+
+                    qryCurIt.close();
+
+                    qryCurs.remove(req.queryId());
+                }
+                finally {
+                    qryCurIt.unlock();
+                }
 
                 return new GridRestResponse(true);
             }
             catch (Exception e) {
-                qryCurs.remove(req.queryId());
+                removeQueryCursor(req.queryId(), qryCurs);
 
                 return new GridRestResponse(GridRestResponse.STATUS_FAILED, e.getMessage());
             }
@@ -347,21 +425,17 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
      * Fetch query callable.
      */
     private static class FetchQueryCallable implements Callable<GridRestResponse> {
-        /** Queries cursors. */
-        private final ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs;
-        /** Grid kernal context. */
-        private final GridKernalContext ctx;
+        /** Current queries cursors. */
+        private final ConcurrentHashMap<Long, QueryCursorIterator> qryCurs;
+
         /** Execute query request. */
         private RestQueryRequest req;
 
         /**
-         * @param ctx Grid kernal context.
          * @param req Execute query request.
-         * @param qryCurs Queries cursors.
+         * @param qryCurs Query cursors.
          */
-        public FetchQueryCallable(GridKernalContext ctx, RestQueryRequest req,
-            ConcurrentHashMap<Long, IgniteBiTuple<QueryCursor, Iterator>> qryCurs) {
-            this.ctx = ctx;
+        public FetchQueryCallable(RestQueryRequest req, ConcurrentHashMap<Long, QueryCursorIterator> qryCurs) {
             this.req = req;
             this.qryCurs = qryCurs;
         }
@@ -369,21 +443,91 @@ public class QueryCommandHandler extends GridRestCommandHandlerAdapter {
         /** {@inheritDoc} */
         @Override public GridRestResponse call() throws Exception {
             try {
-                Iterator cur = qryCurs.get(req.queryId()).get2();
+                QueryCursorIterator qryCurIt = qryCurs.get(req.queryId());
 
-                if (cur == null)
+                if (qryCurIt == null)
                     return new GridRestResponse(GridRestResponse.STATUS_FAILED,
                         "Failed to find query with ID: " + req.queryId());
 
-                CacheQueryResult res = createQueryResult(qryCurs, cur, req, req.queryId());
+                qryCurIt.lock();
+
+                try {
+                    if (qryCurIt.timestamp() == -1)
+                        return new GridRestResponse(GridRestResponse.STATUS_FAILED,
+                            "Query is closed by timeout. Restart query with ID: " + req.queryId());
+
+                    qryCurIt.timestamp(U.currentTimeMillis());
 
-                return new GridRestResponse(res);
+                    Iterator cur = qryCurIt.iterator();
+
+                    CacheQueryResult res = createQueryResult(cur, req, req.queryId(), qryCurs);
+
+                    return new GridRestResponse(res);
+                }
+                finally {
+                    qryCurIt.unlock();
+                }
             }
             catch (Exception e) {
-                qryCurs.remove(req.queryId());
+                removeQueryCursor(req.queryId(), qryCurs);
 
                 return new GridRestResponse(GridRestResponse.STATUS_FAILED, e.getMessage());
             }
         }
     }
+
+    /**
+     * Query cursor iterator.
+     */
+    private static class QueryCursorIterator extends ReentrantLock {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Query cursor. */
+        private QueryCursor cur;
+
+        /** Query iterator. */
+        private Iterator it;
+
+        /** Last timestamp. */
+        private volatile long ts;
+
+        /**
+         * @param cur Query cursor.
+         * @param it Query iterator.
+         */
+        public QueryCursorIterator(QueryCursor cur, Iterator it) {
+            this.cur = cur;
+            this.it = it;
+            ts = U.currentTimeMillis();
+        }
+
+        /**
+         * @return Query iterator.
+         */
+        public Iterator iterator() {
+            return it;
+        }
+
+        /**
+         * @return Timestamp.
+         */
+        public long timestamp() {
+            return ts;
+        }
+
+        /**
+         * @param time Current time or -1 if cursor is closed.
+         */
+        public void timestamp(long time) {
+            ts = time;
+        }
+
+        /**
+         * Close query cursor.
+         */
+        public void close() {
+            cur.close();
+        }
+    }
 }


[34/46] ignite git commit: ignite-1.5 Update store benchmark config.

Posted by ak...@apache.org.
ignite-1.5 Update store benchmark config.


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

Branch: refs/heads/ignite-1753-1282
Commit: 55c227a5920223c30be73b304e98dfc34ea8f998
Parents: 8cc9fa4
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Nov 12 09:45:39 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Nov 12 09:45:39 2015 +0700

----------------------------------------------------------------------
 .../yardstick/config/benchmark-store.properties | 86 ++++++++++++--------
 1 file changed, 50 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/55c227a5/modules/yardstick/config/benchmark-store.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-store.properties b/modules/yardstick/config/benchmark-store.properties
index bda136e..746fa11 100644
--- a/modules/yardstick/config/benchmark-store.properties
+++ b/modules/yardstick/config/benchmark-store.properties
@@ -1,4 +1,3 @@
-#
 # 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.
@@ -13,7 +12,6 @@
 # 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.
-#
 
 #
 # Contains all benchmarks for:
@@ -28,36 +26,40 @@ JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
 
 # Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
 JVM_OPTS=${JVM_OPTS}" \
-  -Xloggc:./gc${now0}.log \
-  -XX:+PrintGCDetails \
-  -verbose:gc \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
+-XX:+UseTLAB \
+-XX:NewSize=128m \
+-XX:MaxNewSize=128m \
+-XX:MaxTenuringThreshold=0 \
+-XX:SurvivorRatio=1024 \
+-XX:+UseCMSInitiatingOccupancyOnly \
+-XX:CMSInitiatingOccupancyFraction=60 \
 "
 
+#Ignite version
+ver="RELEASE-"
+
 # List of default probes.
 # Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
-BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe
 
 # Packages where the specified benchmark is searched by reflection mechanism.
 BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
+# Restart servers for each benchmark.
 RESTART_SERVERS=true
 
 # Probe point writer class name.
 # BENCHMARK_WRITER=
 
-# Comma-separated list of the hosts to run BenchmarkServers on. 2 nodes on local host are enabled by default.
+# Comma-separated list of the hosts to run BenchmarkServers on.
 SERVER_HOSTS=localhost
 
-# Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+# Comma-separated list of the hosts to run BenchmarkDrivers on.
 DRIVER_HOSTS=localhost
 
 # Remote username.
@@ -66,27 +68,39 @@ DRIVER_HOSTS=localhost
 # Number of nodes, used to wait for the specified number of nodes to start.
 nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
 
-# Each benchmark warm_up set to 60 seconds (1 minute).
-warm_up=60
+# Backups count.
+b=1
+
+# Warmup.
+w=60
+
+# Duration.
+d=300
+
+# Threads count.
+t=64
+
+# Sync mode.
+sm=PRIMARY_SYNC
 
-# Each benchmark run set to 300 seconds (5 mins).
-run=300
+# Jobs.
+j=10
 
 # Run configuration which contains all benchmarks.
 CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStoreGetBenchmark -sn IgniteNode -ds atomic-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStoreGetBenchmark -sn IgniteNode -cs -ds atomic-store-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStoreGetBenchmark -sn IgniteNode -cs -wb -ds atomic-writeBehind-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStoreGetTxBenchmark -sn IgniteNode -ds tx-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStoreGetTxBenchmark -sn IgniteNode -cs -ds tx-store-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutBenchmark -sn IgniteNode -ds atomic-put,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutBenchmark -sn IgniteNode -cs -ds atomic-store-put,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutBenchmark -sn IgniteNode -cs -wb -ds atomic-writeBehind-put,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutTxBenchmark -sn IgniteNode -ds tx-put,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutTxBenchmark -sn IgniteNode -cs -ds tx-store-put,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutGetBenchmark -sn IgniteNode -ds atomic-put-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutGetBenchmark -sn IgniteNode -cs -ds atomic-store-put-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutGetBenchmark -sn IgniteNode -cs -wb -ds atomic-writeBehind-put-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutGetTxBenchmark -sn IgniteNode -ds tx-put-get,\
--cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b 1 -w ${warm_up} -d ${run} -t 64 -sm PRIMARY_SYNC -dn IgniteJdbcStorePutGetTxBenchmark -sn IgniteNode -cs -ds tx-store-put-get\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStoreGetBenchmark -sn IgniteNode -ds atomic-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStoreGetBenchmark -sn IgniteNode -cs -ds atomic-store-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStoreGetBenchmark -sn IgniteNode -cs -wb -ds atomic-writeBehind-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStoreGetTxBenchmark -sn IgniteNode -ds tx-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStoreGetTxBenchmark -sn IgniteNode -cs -ds tx-store-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutBenchmark -sn IgniteNode -ds atomic-put,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutBenchmark -sn IgniteNode -cs -ds atomic-store-put,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutBenchmark -sn IgniteNode -cs -wb -ds atomic-writeBehind-put,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutTxBenchmark -sn IgniteNode -ds tx-put,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutTxBenchmark -sn IgniteNode -cs -ds tx-store-put,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutGetBenchmark -sn IgniteNode -ds atomic-put-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutGetBenchmark -sn IgniteNode -cs -ds atomic-store-put-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutGetBenchmark -sn IgniteNode -cs -wb -ds atomic-writeBehind-put-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutGetTxBenchmark -sn IgniteNode -ds tx-put-get,\
+-cfg ${SCRIPT_DIR}/../config/ignite-store-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteJdbcStorePutGetTxBenchmark -sn IgniteNode -cs -ds tx-store-put-get\
 "


[08/46] ignite git commit: IGNITE-1807 Visor cmd write logs to file in quite mode. - Fixes #192.

Posted by ak...@apache.org.
IGNITE-1807 Visor cmd write logs to file in quite mode. - Fixes #192.

Signed-off-by: Andrey <an...@gridgain.com>


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

Branch: refs/heads/ignite-1753-1282
Commit: 7393227477d2dfeb7ce4925fee9945ed6bb80127
Parents: 354358c
Author: Andrey <an...@gridgain.com>
Authored: Mon Nov 9 10:52:31 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Mon Nov 9 10:52:31 2015 +0700

----------------------------------------------------------------------
 .../ignite/visor/commands/open/VisorOpenCommand.scala | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/73932274/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
index 7d2ee3c..1cfbde4 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
@@ -19,6 +19,8 @@
 
 package org.apache.ignite.visor.commands.open
 
+import java.util.logging.{ConsoleHandler, Logger}
+
 import org.apache.ignite.IgniteSystemProperties._
 import org.apache.ignite.configuration.IgniteConfiguration
 import org.apache.ignite.internal.IgniteComponentType._
@@ -26,7 +28,6 @@ import org.apache.ignite.internal.IgniteEx
 import org.apache.ignite.internal.util.scala.impl
 import org.apache.ignite.internal.util.spring.IgniteSpringHelper
 import org.apache.ignite.internal.util.{IgniteUtils => U}
-import org.apache.ignite.logger.NullLogger
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi
 import org.apache.ignite.visor.commands.common.{VisorConsoleCommand, VisorTextTable}
 import org.apache.ignite.visor.visor._
@@ -158,6 +159,13 @@ class VisorOpenCommand extends VisorConsoleCommand {
 
                 val cfg = cfgs.iterator().next()
 
+                if (log4jTup != null)
+                    System.setProperty(IgniteSystemProperties.IGNITE_CONSOLE_APPENDER, "false")
+                else
+                    Logger.getGlobal.getHandlers.foreach({
+                        case handler: ConsoleHandler => Logger.getGlobal.removeHandler(handler)
+                    })
+
                 // Setting up 'Config URL' for properly print in console.
                 System.setProperty(IgniteSystemProperties.IGNITE_CONFIG_URL, url.getPath)
 
@@ -236,10 +244,6 @@ class VisorOpenCommand extends VisorConsoleCommand {
         // Make sure visor starts without shutdown hook.
         System.setProperty(IGNITE_NO_SHUTDOWN_HOOK, "true")
 
-        // Set NullLoger in quite mode.
-        if ("true".equalsIgnoreCase(sys.props.getOrElse(IGNITE_QUIET, "true")))
-            cfg.setGridLogger(new NullLogger)
-
         ignite = try {
             // We need to stop previous daemon node before to start new one.
             prevIgnite.foreach(g => Ignition.stop(g.name(), true))


[46/46] ignite git commit: Merge branches 'ignite-1282' and 'ignite-1753-1282' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1753-1282

Posted by ak...@apache.org.
Merge branches 'ignite-1282' and 'ignite-1753-1282' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1753-1282

Conflicts:
	modules/yardstick/config/ignite-store-config.xml


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

Branch: refs/heads/ignite-1753-1282
Commit: 59fd5cecc241ec1115c177a7131e3b5e9e5e14a1
Parents: dbf83b9 c00e4ac
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Nov 17 20:34:17 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Nov 17 20:34:17 2015 +0700

----------------------------------------------------------------------
 .../rest/AbstractRestProcessorSelfTest.java     |    5 +-
 .../JettyRestProcessorAbstractSelfTest.java     |  142 +-
 .../java/org/apache/ignite/IgniteCache.java     |    3 +-
 .../eviction/random/RandomEvictionPolicy.java   |    4 +
 .../cache/store/CacheStoreSessionListener.java  |    9 +-
 .../jdbc/CacheJdbcStoreSessionListener.java     |    8 +-
 .../configuration/CacheConfiguration.java       |  121 +-
 .../configuration/ConnectorConfiguration.java   |   61 +-
 .../configuration/IgniteConfiguration.java      |   32 +-
 .../apache/ignite/internal/IgniteKernal.java    |   21 +
 .../org/apache/ignite/internal/IgnitionEx.java  |    3 +
 .../managers/communication/GridIoMessage.java   |    4 +-
 .../communication/GridIoMessageFactory.java     |   10 +-
 .../discovery/GridDiscoveryManager.java         |    5 +-
 .../internal/portable/PortableContext.java      |    4 +-
 .../cache/CacheStoreBalancingWrapper.java       |    5 +-
 .../processors/cache/GridCacheAdapter.java      |   22 +-
 .../processors/cache/GridCacheContext.java      |   29 +-
 .../processors/cache/GridCacheEntryEx.java      |    4 +-
 .../processors/cache/GridCacheIoManager.java    |   19 +-
 .../processors/cache/GridCacheMapEntry.java     |   91 +-
 .../processors/cache/GridCacheMvccManager.java  |  145 +-
 .../GridCachePartitionExchangeManager.java      |  184 ++-
 .../processors/cache/GridCachePreloader.java    |   55 +-
 .../cache/GridCachePreloaderAdapter.java        |   40 +-
 .../processors/cache/GridCacheProcessor.java    |   59 +-
 .../processors/cache/IgniteCacheProxy.java      |    6 +
 .../distributed/GridDistributedCacheEntry.java  |    2 +-
 .../distributed/GridDistributedTxMapping.java   |    8 +-
 .../GridDistributedTxRemoteAdapter.java         |    5 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   11 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   63 +-
 .../distributed/dht/GridDhtLockFuture.java      |    7 +-
 .../dht/GridDhtPartitionsReservation.java       |    2 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   12 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   39 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   78 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   33 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    7 +-
 .../GridDhtPartitionDemandMessage.java          |   14 +-
 .../preloader/GridDhtPartitionDemandPool.java   | 1192 ---------------
 .../dht/preloader/GridDhtPartitionDemander.java | 1408 ++++++++++++++++++
 .../dht/preloader/GridDhtPartitionSupplier.java | 1037 +++++++++++++
 .../GridDhtPartitionSupplyMessageV2.java        |  380 +++++
 .../preloader/GridDhtPartitionSupplyPool.java   |  555 -------
 .../GridDhtPartitionsExchangeFuture.java        |    2 +
 .../dht/preloader/GridDhtPreloader.java         |  282 +++-
 .../near/GridNearOptimisticTxPrepareFuture.java |   11 +-
 .../near/GridNearTransactionalCache.java        |    7 +-
 .../near/GridNearTxFinishFuture.java            |  157 +-
 .../cache/distributed/near/GridNearTxLocal.java |   21 +-
 .../CacheObjectBinaryProcessorImpl.java         |    2 +-
 .../cache/store/CacheOsStoreManager.java        |    2 +-
 .../store/GridCacheStoreManagerAdapter.java     |   29 +-
 .../cache/transactions/IgniteInternalTx.java    |    3 +-
 .../cache/transactions/IgniteTxAdapter.java     |  251 ++--
 .../cache/transactions/IgniteTxHandler.java     |   37 +-
 .../transactions/IgniteTxLocalAdapter.java      |   22 +-
 .../cache/transactions/IgniteTxManager.java     |  171 ++-
 .../datastructures/DataStructuresProcessor.java |    3 +
 .../GridCacheAtomicReferenceImpl.java           |    2 +-
 .../processors/igfs/IgfsMetaManager.java        |    2 +-
 .../handlers/cache/GridCacheCommandHandler.java |  128 +-
 .../handlers/query/QueryCommandHandler.java     |  254 +++-
 .../top/GridTopologyCommandHandler.java         |    4 -
 .../processors/task/GridTaskWorker.java         |    4 +-
 .../GridBoundedConcurrentLinkedHashMap.java     |    7 +-
 .../GridBoundedConcurrentLinkedHashSet.java     |    7 +-
 .../util/GridBoundedConcurrentOrderedMap.java   |   39 +-
 .../internal/util/GridConcurrentFactory.java    |   11 +-
 .../util/GridConcurrentLinkedHashSet.java       |    9 +-
 .../ignite/internal/util/IgniteUuidCache.java   |    6 +-
 .../util/future/GridCompoundFuture.java         |  155 +-
 .../ignite/internal/util/lang/GridTuple4.java   |    2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  360 +++--
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   95 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   37 +
 .../TcpDiscoveryStatusCheckMessage.java         |   11 +
 .../java/org/jsr166/ConcurrentHashMap8.java     |    2 +-
 .../java/org/jsr166/ConcurrentLinkedDeque8.java |    2 +-
 .../org/jsr166/ConcurrentLinkedHashMap.java     |  195 ++-
 .../resources/META-INF/classnames.properties    |  196 ++-
 .../GridCacheAffinityBackupsSelfTest.java       |    8 +
 .../store/GridCacheBalancingStoreSelfTest.java  |  220 ++-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |  145 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    2 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   40 +-
 .../processors/cache/GridCacheTestEntryEx.java  |    3 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |    2 +-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |   20 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   21 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   10 +
 ...achePartitionedMultiNodeFullApiSelfTest.java |    2 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |   68 +
 .../GridCacheRebalancingSyncSelfTest.java       |  506 +++++++
 ...eRebalancingUnmarshallingFailedSelfTest.java |  147 ++
 .../GridCacheReplicatedPreloadSelfTest.java     |   22 +-
 .../GridCachePortableStoreAbstractSelfTest.java |    2 +-
 .../continuous/GridEventConsumeSelfTest.java    |    2 +-
 ...dBoundedConcurrentLinkedHashMapSelfTest.java |    2 +-
 .../GridConcurrentLinkedHashMapSelfTest.java    |   62 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  158 +-
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |   10 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  476 +++++-
 .../spi/discovery/tcp/TestTcpDiscoverySpi.java  |   46 +
 .../TcpDiscoveryMulticastIpFinderSelfTest.java  |   28 +-
 .../testframework/junits/GridAbstractTest.java  |   32 +-
 .../junits/common/GridCommonAbstractTest.java   |   25 +-
 .../testsuites/IgniteCacheTestSuite3.java       |    4 +
 ...rrentLinkedHashMapMultiThreadedSelfTest.java |  104 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   18 +-
 modules/rest-http/pom.xml                       |   11 +
 .../visor/commands/open/VisorOpenCommand.scala  |   14 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    3 +-
 modules/yardstick/README.txt                    |   16 +-
 .../config/benchmark-multicast.properties       |    9 +-
 .../yardstick/IgniteBenchmarkArguments.java     |    2 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |   75 +
 .../IgniteAccountSerializableTxBenchmark.java   |   11 +-
 .../cache/IgniteAccountTxBenchmark.java         |   11 +-
 .../cache/IgniteGetAllPutAllTxBenchmark.java    |   73 +
 .../cache/IgnitePutGetTxBenchmark.java          |   41 +-
 .../yardstick/cache/IgnitePutTxBenchmark.java   |   10 +
 .../cache/IgnitePutTxPrimaryOnlyBenchmark.java  |   65 +
 .../IgnitePutTxSkipLocalBackupBenchmark.java    |   65 +
 .../cache/WaitMapExchangeFinishCallable.java    |   95 ++
 ...IgniteTransactionalWriteInvokeBenchmark.java |   41 +-
 .../IgniteTransactionalWriteReadBenchmark.java  |   41 +-
 modules/zookeeper/pom.xml                       |    2 +-
 131 files changed, 7825 insertions(+), 3423 deletions(-)
----------------------------------------------------------------------



[44/46] ignite git commit: Renaming.

Posted by ak...@apache.org.
Renaming.


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

Branch: refs/heads/ignite-1753-1282
Commit: 6fa22e2e2e00dbb665b3eba24121b756716297d7
Parents: b278fe3
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 17 13:42:40 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 17 13:42:40 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 28 ++++++++++----------
 .../processors/cache/GridCacheProcessor.java    |  4 +--
 .../cache/store/CacheOsStoreManager.java        |  2 +-
 .../GridCachePortableStoreAbstractSelfTest.java |  2 +-
 4 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6fa22e2e/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 9cde3a2..5b246b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -206,9 +206,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default size for onheap SQL row cache size. */
     public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024;
 
-    /** Default value for keep portable in store behavior .*/
+    /** Default value for keep binary in store behavior .*/
     @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"})
-    public static final Boolean DFLT_KEEP_PORTABLE_IN_STORE = new Boolean(true);
+    public static final Boolean DFLT_KEEP_BINARY_IN_STORE = new Boolean(true);
 
     /** Default threshold for concurrent loading of keys from {@link CacheStore}. */
     public static final int DFLT_CONCURRENT_LOAD_ALL_THRESHOLD = 5;
@@ -269,7 +269,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private Factory storeFactory;
 
     /** */
-    private Boolean keepPortableInStore = DFLT_KEEP_PORTABLE_IN_STORE;
+    private Boolean keepBinaryInStore = DFLT_KEEP_BINARY_IN_STORE;
 
     /** */
     private boolean loadPrevVal = DFLT_LOAD_PREV_VAL;
@@ -441,8 +441,8 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         invalidate = cc.isInvalidate();
         isReadThrough = cc.isReadThrough();
         isWriteThrough = cc.isWriteThrough();
-        keepPortableInStore = cc.isKeepPortableInStore() != null ? cc.isKeepPortableInStore() :
-            DFLT_KEEP_PORTABLE_IN_STORE;
+        keepBinaryInStore = cc.isKeepBinaryInStore() != null ? cc.isKeepBinaryInStore() :
+            DFLT_KEEP_BINARY_IN_STORE;
         listenerConfigurations = cc.listenerConfigurations;
         loadPrevVal = cc.isLoadPreviousValue();
         longQryWarnTimeout = cc.getLongQueryWarningTimeout();
@@ -886,34 +886,34 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
     /**
      * Flag indicating that {@link CacheStore} implementation
-     * is working with portable objects instead of Java objects.
-     * Default value of this flag is {@link #DFLT_KEEP_PORTABLE_IN_STORE},
+     * is working with binary objects instead of Java objects.
+     * Default value of this flag is {@link #DFLT_KEEP_BINARY_IN_STORE},
      * because this is recommended behavior from performance standpoint.
      * <p>
      * If set to {@code false}, Ignite will deserialize keys and
-     * values stored in portable format before they are passed
+     * values stored in binary format before they are passed
      * to cache store.
      * <p>
      * Note that setting this flag to {@code false} can simplify
      * store implementation in some cases, but it can cause performance
      * degradation due to additional serializations and deserializations
-     * of portable objects. You will also need to have key and value
+     * of binary objects. You will also need to have key and value
      * classes on all nodes since binary will be deserialized when
      * store is called.
      *
      * @return Keep binary in store flag.
      */
-    public Boolean isKeepPortableInStore() {
-        return keepPortableInStore;
+    public Boolean isKeepBinaryInStore() {
+        return keepBinaryInStore;
     }
 
     /**
      * Sets keep binary in store flag.
      *
-     * @param keepPortableInStore Keep binary in store flag.
+     * @param keepBinaryInStore Keep binary in store flag.
      */
-    public void setKeepPortableInStore(boolean keepPortableInStore) {
-        this.keepPortableInStore = keepPortableInStore;
+    public void setKeepBinaryInStore(boolean keepBinaryInStore) {
+        this.keepBinaryInStore = keepBinaryInStore;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fa22e2e/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 395e827..4fe8d84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1018,9 +1018,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         CacheConfiguration cfg = cacheCtx.config();
 
         // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set.
-        if (cfg.isKeepPortableInStore() && cfg.isKeepPortableInStore() != CacheConfiguration.DFLT_KEEP_PORTABLE_IN_STORE
+        if (cfg.isKeepBinaryInStore() && cfg.isKeepBinaryInStore() != CacheConfiguration.DFLT_KEEP_BINARY_IN_STORE
             && !(ctx.config().getMarshaller() instanceof PortableMarshaller))
-            U.warn(log, "CacheConfiguration.isKeepPortableInStore() configuration property will be ignored because " +
+            U.warn(log, "CacheConfiguration.isKeepBinaryInStore() configuration property will be ignored because " +
                 "PortableMarshaller is not used");
 
         // Start managers.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fa22e2e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
index f7f5f4e..b84908d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
@@ -82,6 +82,6 @@ public class CacheOsStoreManager extends GridCacheStoreManagerAdapter {
 
     /** {@inheritDoc} */
     @Override public boolean configuredConvertPortable() {
-        return !(ctx.config().getMarshaller() instanceof PortableMarshaller && cfg.isKeepPortableInStore());
+        return !(ctx.config().getMarshaller() instanceof PortableMarshaller && cfg.isKeepBinaryInStore());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fa22e2e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
index 1c842e1..13a4c12 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
@@ -59,7 +59,7 @@ public abstract class GridCachePortableStoreAbstractSelfTest extends GridCommonA
         CacheConfiguration cacheCfg = new CacheConfiguration();
 
         cacheCfg.setCacheStoreFactory(singletonFactory(STORE));
-        cacheCfg.setKeepPortableInStore(keepPortableInStore());
+        cacheCfg.setKeepBinaryInStore(keepPortableInStore());
         cacheCfg.setReadThrough(true);
         cacheCfg.setWriteThrough(true);
         cacheCfg.setLoadPreviousValue(true);


[09/46] ignite git commit: Performance optimizations.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
index 5c6cfcf..9fe2690 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
@@ -146,40 +146,41 @@ public class GridConcurrentLinkedHashMapMultiThreadedSelfTest extends GridCommon
     public void testEvictPerSegment() throws Exception {
         info(">>> Test grid concurrent linked hash map...");
 
-        final int maxSize = 1000;
+        int concurLvl = 64;
+        final int maxSize = concurLvl * 30;
+        int diff = (int)(maxSize * 0.1);
 
         ConcurrentLinkedHashMap<Integer, String> linkedMap = new ConcurrentLinkedHashMap<>(
-            32, 0.75f, 64, maxSize, PER_SEGMENT_Q);
+            32, 0.75f, concurLvl, maxSize, PER_SEGMENT_Q);
 
         int keyCnt = 1000000;
 
-        putMultiThreaded(linkedMap, 10, keyCnt, maxSize);
-
-        int diff = 10; // 1% of 1000.
+        Map<String, LinkedList<Integer>> map = putMultiThreaded(
+            linkedMap,
+            10,
+            keyCnt,
+            maxSize * 10); // Intentionally memorize more than maxSize since in this mode LRU is not fair.
 
         assertTrue("Invalid map size: " + linkedMap.size(), U.safeAbs(maxSize - linkedMap.size()) <= diff);
         assertTrue("Invalid map sizex: " + linkedMap.sizex(), U.safeAbs(maxSize - linkedMap.sizex()) <= diff);
 
-//      TODO IGNITE-606 - Need to fix iterators for ConcurrentLinkedHashMap in perSegment mode
-//        LinkedList<Integer> keys = new LinkedList<Integer>(linkedMap.keySet());
-//
-//        while (!keys.isEmpty()) {
-//            boolean found = false;
-//
-//            int key = keys.removeLast();
-//
-//            for (LinkedList<Integer> threadKeys : map.values()) {
-//                if (threadKeys.getLast() == key) {
-//                    threadKeys.removeLast();
-//
-//                    found = true;
-//
-//                    break;
-//                }
-//            }
-//
-//            assertTrue("Key was not found on the top of any thread: " + key, found);
-//        }
+        LinkedList<Integer> keys = new LinkedList<>(linkedMap.keySet());
+
+        while (!keys.isEmpty()) {
+            boolean found = false;
+
+            int key = keys.removeLast();
+
+            for (LinkedList<Integer> threadKeys : map.values()) {
+                if (threadKeys.contains(key)) {
+                    found = true;
+
+                    break;
+                }
+            }
+
+            assertTrue("Key was not found in any thread: " + key, found);
+        }
 
         int min = Integer.MAX_VALUE;
         int max = 0;
@@ -207,40 +208,41 @@ public class GridConcurrentLinkedHashMapMultiThreadedSelfTest extends GridCommon
     public void testEvictPerSegmentOptimizedRemoves() throws Exception {
         info(">>> Test grid concurrent linked hash map...");
 
-        final int maxSize = 1000;
+        int concurLvl = 64;
+        final int maxSize = concurLvl * 30;
+        int diff = (int)(maxSize * 0.1);
 
         ConcurrentLinkedHashMap<Integer, String> linkedMap = new ConcurrentLinkedHashMap<>(
-            32, 0.75f, 64, maxSize, PER_SEGMENT_Q_OPTIMIZED_RMV);
+            32, 0.75f, concurLvl, maxSize, PER_SEGMENT_Q_OPTIMIZED_RMV);
 
         int keyCnt = 1000000;
 
-        putMultiThreaded(linkedMap, 10, keyCnt, maxSize);
-
-        int diff = 10; // 1% of 1000.
+        Map<String, LinkedList<Integer>> map = putMultiThreaded(
+            linkedMap,
+            10,
+            keyCnt,
+            maxSize * 10); // Intentionally memorize more than maxSize since in this mode LRU is not fair.
 
         assertTrue("Invalid map size: " + linkedMap.size(), U.safeAbs(maxSize - linkedMap.size()) <= diff);
         assertTrue("Invalid map sizex: " + linkedMap.sizex(), U.safeAbs(maxSize - linkedMap.sizex()) <= diff);
 
-//      TODO IGNITE-606 - Need to fix iterators for ConcurrentLinkedHashMap in perSegment mode
-//        LinkedList<Integer> keys = new LinkedList<Integer>(linkedMap.keySet());
-//
-//        while (!keys.isEmpty()) {
-//            boolean found = false;
-//
-//            int key = keys.removeLast();
-//
-//            for (LinkedList<Integer> threadKeys : map.values()) {
-//                if (threadKeys.getLast() == key) {
-//                    threadKeys.removeLast();
-//
-//                    found = true;
-//
-//                    break;
-//                }
-//            }
-//
-//            assertTrue("Key was not found on the top of any thread: " + key, found);
-//        }
+        LinkedList<Integer> keys = new LinkedList<>(linkedMap.keySet());
+
+        while (!keys.isEmpty()) {
+            boolean found = false;
+
+            int key = keys.removeLast();
+
+            for (LinkedList<Integer> threadKeys : map.values()) {
+                if (threadKeys.contains(key)) {
+                    found = true;
+
+                    break;
+                }
+            }
+
+            assertTrue("Key was not found in any thread: " + key, found);
+        }
 
         int min = Integer.MAX_VALUE;
         int max = 0;
@@ -558,4 +560,4 @@ public class GridConcurrentLinkedHashMapMultiThreadedSelfTest extends GridCommon
 
         info(">>> put get remove test complete [duration = " + (System.currentTimeMillis() - start) + ']');
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
index ead3a63..9c3389f 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
@@ -19,13 +19,23 @@ package org.apache.ignite.yardstick.cache;
 
 import java.util.Map;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
 
 /**
  * Ignite benchmark that performs transactional put operations.
  */
 public class IgnitePutTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
     /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK"))
+            ignite().compute().broadcast(new WaitMapExchangeFinishCallable());
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
         int key = nextRandom(args.range());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxPrimaryOnlyBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxPrimaryOnlyBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxPrimaryOnlyBenchmark.java
new file mode 100644
index 0000000..21275eb
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxPrimaryOnlyBenchmark.java
@@ -0,0 +1,65 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+/**
+ * Ignite benchmark that performs transactional put operations skipping key if local node is backup.
+ */
+public class IgnitePutTxPrimaryOnlyBenchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK"))
+            ignite().compute().broadcast(new WaitMapExchangeFinishCallable());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key;
+
+        Affinity<Object> aff = ignite().affinity("tx");
+        ClusterNode locNode = ignite().cluster().localNode();
+
+        for (;;) {
+            key = nextRandom(args.range());
+
+            // Exit only if primary.
+            if (aff.isPrimary(locNode, key))
+                break;
+        }
+
+        // Implicit transaction is used.
+        cache.put(key, new SampleValue(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxSkipLocalBackupBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxSkipLocalBackupBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxSkipLocalBackupBenchmark.java
new file mode 100644
index 0000000..63934e6
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxSkipLocalBackupBenchmark.java
@@ -0,0 +1,65 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+/**
+ * Ignite benchmark that performs transactional put operations skipping key if local node is backup.
+ */
+public class IgnitePutTxSkipLocalBackupBenchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK"))
+            ignite().compute().broadcast(new WaitMapExchangeFinishCallable());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key;
+
+        Affinity<Object> aff = ignite().affinity("tx");
+        ClusterNode locNode = ignite().cluster().localNode();
+
+        for (;;) {
+            key = nextRandom(args.range());
+
+            // Skip key if local node is backup.
+            if (!aff.isBackup(locNode, key))
+                break;
+        }
+
+        // Implicit transaction is used.
+        cache.put(key, new SampleValue(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java
new file mode 100644
index 0000000..83c50bd
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/WaitMapExchangeFinishCallable.java
@@ -0,0 +1,95 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+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.preloader.GridDhtPartitionMap;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ *
+ */
+public class WaitMapExchangeFinishCallable implements IgniteCallable<Void> {
+    /** */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** {@inheritDoc} */
+    @Override public Void call() throws Exception {
+        Collection<IgniteInternalCache<?, ?>> cachesx = ((IgniteKernal)ignite).cachesx(null);
+
+        for (IgniteInternalCache<?, ?> cache : cachesx) {
+            try {
+                GridDhtPartitionTopology top = cache.context().isNear() ? cache.context().near().dht().topology() :
+                    cache.context().dht().topology();
+
+                BenchmarkUtils.println("Validating cache: " + cache.name());
+
+                for (;;) {
+                    boolean success = true;
+
+                    if (top.topologyVersion().topologyVersion() == ignite.cluster().topologyVersion()) {
+                        for (Map.Entry<UUID, GridDhtPartitionMap> e : top.partitionMap(true).entrySet()) {
+                            for (Map.Entry<Integer, GridDhtPartitionState> p : e.getValue().entrySet()) {
+                                if (p.getValue() != GridDhtPartitionState.OWNING) {
+                                    BenchmarkUtils.println("Not owning partition [part=" + p.getKey() +
+                                        ", state=" + p.getValue() + ']');
+
+                                    success = false;
+
+                                    break;
+                                }
+                            }
+
+                            if (!success)
+                                break;
+                        }
+                    }
+                    else {
+                        BenchmarkUtils.println("Topology version is different [cache=" + top.topologyVersion() +
+                            ", cluster=" + ignite.cluster().topologyVersion() + ']');
+
+                        success = false;
+                    }
+
+                    if (!success)
+                        Thread.sleep(1000);
+                    else {
+                        BenchmarkUtils.println("Cache state is fine: " + cache.name());
+
+                        break;
+                    }
+                }
+            }
+            catch (RuntimeException e1) {
+                BenchmarkUtils.println("Ignored exception: " + e1);
+            }
+        }
+
+        return null;
+    }
+}


[16/46] ignite git commit: ignite-1863: Allowing to use null in IgniteAtomicReference.compareAndSet operation

Posted by ak...@apache.org.
ignite-1863: Allowing to use null in  IgniteAtomicReference.compareAndSet operation


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

Branch: refs/heads/ignite-1753-1282
Commit: e02b68c93c0b4a2905693d79e4d1583862206ad6
Parents: 3de9d47
Author: Artem Shutak <as...@gridgain.com>
Authored: Tue Nov 10 13:13:31 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Nov 10 13:13:31 2015 +0300

----------------------------------------------------------------------
 .../GridCacheAtomicReferenceImpl.java           |  2 +-
 ...CacheAtomicReferenceApiSelfAbstractTest.java | 20 +++++++++++++++++++-
 2 files changed, 20 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e02b68c9/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index b25e111..c0c38b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -204,7 +204,7 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     private IgnitePredicate<T> wrapperPredicate(final T val) {
         return new IgnitePredicate<T>() {
             @Override public boolean apply(T e) {
-                return val != null && val.equals(e);
+                return F.eq(val, e);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e02b68c9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceApiSelfAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceApiSelfAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceApiSelfAbstractTest.java
index c12da82..278bcf9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceApiSelfAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceApiSelfAbstractTest.java
@@ -109,4 +109,22 @@ public abstract class GridCacheAtomicReferenceApiSelfAbstractTest extends Ignite
 
         assertEquals(null, atomic.get());
     }
-}
\ No newline at end of file
+
+    /**
+     * JUnit.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCompareAndSetNullValue() throws Exception {
+        String atomicName = UUID.randomUUID().toString();
+
+        IgniteAtomicReference<String> atomic = grid(0).atomicReference(atomicName, null, true);
+
+        assertEquals(null, atomic.get());
+
+        boolean success = atomic.compareAndSet(null, "newVal");
+
+        assertTrue(success);
+        assertEquals("newVal", atomic.get());
+    }
+}


[10/46] ignite git commit: Performance optimizations.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 1c82636..eb2ca2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -24,16 +24,14 @@ import java.io.ObjectOutput;
 import java.io.ObjectStreamException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCheckedException;
@@ -47,7 +45,6 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 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.GridCacheFilterFailedException;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
@@ -59,7 +56,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
-import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple;
@@ -199,20 +195,20 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     protected boolean transform;
 
     /** Commit version. */
-    private AtomicReference<GridCacheVersion> commitVer = new AtomicReference<>(null);
-
-    /** Done marker. */
-    protected final AtomicBoolean isDone = new AtomicBoolean(false);
+    private volatile GridCacheVersion commitVer;
 
     /** */
     private AtomicReference<FinalizationStatus> finalizing = new AtomicReference<>(FinalizationStatus.NONE);
 
-    /** Preparing flag. */
-    private AtomicBoolean preparing = new AtomicBoolean();
+    /** Done marker. */
+    protected volatile boolean isDone;
+
+    /** Preparing flag (no need for volatile modifier). */
+    private boolean preparing;
 
     /** */
     @GridToStringInclude
-    private Map<Integer, Set<Integer>> invalidParts = new HashMap<>(3);
+    private Map<Integer, Set<Integer>> invalidParts;
 
     /**
      * Transaction state. Note that state is not protected, as we want to
@@ -230,17 +226,11 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** */
     @GridToStringExclude
-    private AtomicReference<GridFutureAdapter<IgniteInternalTx>> finFut = new AtomicReference<>();
+    private volatile GridFutureAdapter<IgniteInternalTx> finFut;
 
     /** Topology version. */
     @GridToStringInclude
-    protected AtomicReference<AffinityTopologyVersion> topVer = new AtomicReference<>(AffinityTopologyVersion.NONE);
-
-    /** Mutex. */
-    private final Lock lock = new ReentrantLock();
-
-    /** Lock condition. */
-    private final Condition cond = lock.newCondition();
+    protected volatile AffinityTopologyVersion topVer = AffinityTopologyVersion.NONE;
 
     /** */
     protected Map<UUID, Collection<UUID>> txNodes;
@@ -387,37 +377,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     }
 
     /**
-     * Acquires lock.
-     */
-    @SuppressWarnings({"LockAcquiredButNotSafelyReleased"})
-    protected final void lock() {
-        lock.lock();
-    }
-
-    /**
-     * Releases lock.
-     */
-    protected final void unlock() {
-        lock.unlock();
-    }
-
-    /**
-     * Signals all waiters.
-     */
-    protected final void signalAll() {
-        cond.signalAll();
-    }
-
-    /**
-     * Waits for signal.
-     *
-     * @throws InterruptedException If interrupted.
-     */
-    protected final void awaitSignal() throws InterruptedException {
-        cond.await();
-    }
-
-    /**
      * Checks whether near cache should be updated.
      *
      * @return Flag indicating whether near cache should be updated.
@@ -548,7 +507,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public AffinityTopologyVersion topologyVersion() {
-        AffinityTopologyVersion res = topVer.get();
+        AffinityTopologyVersion res = topVer;
 
         if (res.equals(AffinityTopologyVersion.NONE))
             return cctx.exchange().topologyVersion();
@@ -558,16 +517,29 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public AffinityTopologyVersion topologyVersionSnapshot() {
-        AffinityTopologyVersion ret = topVer.get();
+        AffinityTopologyVersion ret = topVer;
 
         return AffinityTopologyVersion.NONE.equals(ret) ? null : ret;
     }
 
     /** {@inheritDoc} */
     @Override public AffinityTopologyVersion topologyVersion(AffinityTopologyVersion topVer) {
-        this.topVer.compareAndSet(AffinityTopologyVersion.NONE, topVer);
+        AffinityTopologyVersion topVer0 = this.topVer;
 
-        return this.topVer.get();
+        if (!AffinityTopologyVersion.NONE.equals(topVer0))
+            return topVer0;
+
+        synchronized (this) {
+            topVer0 = this.topVer;
+
+            if (AffinityTopologyVersion.NONE.equals(topVer0)) {
+                this.topVer = topVer;
+
+                return topVer;
+            }
+
+            return topVer0;
+        }
     }
 
     /** {@inheritDoc} */
@@ -582,7 +554,14 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public boolean markPreparing() {
-        return preparing.compareAndSet(false, true);
+        synchronized (this) {
+            if (preparing)
+                return false;
+
+            preparing = true;
+
+            return true;
+        }
     }
 
     /**
@@ -730,15 +709,18 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public Map<Integer, Set<Integer>> invalidPartitions() {
-        return invalidParts;
+        return invalidParts == null ? Collections.<Integer, Set<Integer>>emptyMap() : invalidParts;
     }
 
     /** {@inheritDoc} */
     @Override public void addInvalidPartition(GridCacheContext<?, ?> cacheCtx, int part) {
+        if (invalidParts == null)
+            invalidParts = new HashMap<>();
+
         Set<Integer> parts = invalidParts.get(cacheCtx.cacheId());
 
         if (parts == null) {
-            parts = new GridLeanSet<>();
+            parts = new HashSet<>();
 
             invalidParts.put(cacheCtx.cacheId(), parts);
         }
@@ -879,32 +861,71 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public boolean done() {
-        return isDone.get();
+        return isDone;
     }
 
     /**
-     * @return Commit version.
+     * @return {@code True} if done flag has been set by this call.
      */
-    @Override public GridCacheVersion commitVersion() {
-        initCommitVersion();
+    private boolean setDone() {
+        boolean isDone0 = isDone;
+
+        if (isDone0)
+            return false;
+
+        synchronized (this) {
+            isDone0 = isDone;
 
-        return commitVer.get();
+            if (isDone0)
+                return false;
+
+            isDone = true;
+
+            return true;
+        }
     }
 
     /**
-     * @param commitVer Commit version.
-     * @return {@code True} if set to not null value.
+     * @return Commit version.
      */
-    @Override public boolean commitVersion(GridCacheVersion commitVer) {
-        return commitVer != null && this.commitVer.compareAndSet(null, commitVer);
+    @Override public GridCacheVersion commitVersion() {
+        GridCacheVersion commitVer0 = commitVer;
+
+        if (commitVer0 != null)
+            return commitVer0;
+
+        synchronized (this) {
+            commitVer0 = commitVer;
+
+            if (commitVer0 != null)
+                return commitVer0;
+
+            commitVer = commitVer0 = xidVer;
+
+            return commitVer0;
+        }
     }
 
     /**
-     *
+     * @param commitVer Commit version.
      */
-    public void initCommitVersion() {
-        if (commitVer.get() == null)
-            commitVer.compareAndSet(null, xidVer);
+    @Override public void commitVersion(GridCacheVersion commitVer) {
+        if (commitVer == null)
+            return;
+
+        GridCacheVersion commitVer0 = this.commitVer;
+
+        if (commitVer0 != null)
+            return;
+
+        synchronized (this) {
+            commitVer0 = this.commitVer;
+
+            if (commitVer0 != null)
+                return;
+
+            this.commitVer = commitVer;
+        }
     }
 
     /**
@@ -916,7 +937,19 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         if (state != ROLLING_BACK && state != ROLLED_BACK && state != COMMITTING && state != COMMITTED)
             rollback();
 
-        awaitCompletion();
+        synchronized (this) {
+            try {
+                while (!done())
+                    wait();
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                if (!done())
+                    throw new IgniteCheckedException("Got interrupted while waiting for transaction to complete: " +
+                        this, e);
+            }
+        }
     }
 
     /** {@inheritDoc} */
@@ -930,29 +963,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         /* No-op. */
     }
 
-    /**
-     * Awaits transaction completion.
-     *
-     * @throws IgniteCheckedException If waiting failed.
-     */
-    protected void awaitCompletion() throws IgniteCheckedException {
-        lock();
-
-        try {
-            while (!done())
-                awaitSignal();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            if (!done())
-                throw new IgniteCheckedException("Got interrupted while waiting for transaction to complete: " + this, e);
-        }
-        finally {
-            unlock();
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public boolean internal() {
         return internal;
@@ -1019,22 +1029,27 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     /** {@inheritDoc} */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
     @Override public IgniteInternalFuture<IgniteInternalTx> finishFuture() {
-        GridFutureAdapter<IgniteInternalTx> fut = finFut.get();
+        GridFutureAdapter<IgniteInternalTx> fut = finFut;
 
         if (fut == null) {
-            fut = new GridFutureAdapter<IgniteInternalTx>() {
-                @Override public String toString() {
-                    return S.toString(GridFutureAdapter.class, this, "tx", IgniteTxAdapter.this);
-                }
-            };
+            synchronized (this) {
+                fut = finFut;
 
-            if (!finFut.compareAndSet(null, fut))
-                fut = finFut.get();
+                if (fut == null) {
+                    fut = new GridFutureAdapter<IgniteInternalTx>() {
+                        @Override public String toString() {
+                            return S.toString(GridFutureAdapter.class, this, "tx", IgniteTxAdapter.this);
+                        }
+                    };
+
+                    finFut = fut;
+                }
+            }
         }
 
         assert fut != null;
 
-        if (isDone.get())
+        if (isDone)
             fut.onDone(this);
 
         return fut;
@@ -1059,9 +1074,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
         boolean notify = false;
 
-        lock();
-
-        try {
+        synchronized (this) {
             prev = this.state;
 
             switch (state) {
@@ -1087,7 +1100,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
                 }
 
                 case UNKNOWN: {
-                    if (isDone.compareAndSet(false, true))
+                    if (setDone())
                         notify = true;
 
                     valid = prev == ROLLING_BACK || prev == COMMITTING;
@@ -1096,7 +1109,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
                 }
 
                 case COMMITTED: {
-                    if (isDone.compareAndSet(false, true))
+                    if (setDone())
                         notify = true;
 
                     valid = prev == COMMITTING;
@@ -1105,7 +1118,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
                 }
 
                 case ROLLED_BACK: {
-                    if (isDone.compareAndSet(false, true))
+                    if (setDone())
                         notify = true;
 
                     valid = prev == ROLLING_BACK;
@@ -1135,8 +1148,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
                 if (log.isDebugEnabled())
                     log.debug("Changed transaction state [prev=" + prev + ", new=" + this.state + ", tx=" + this + ']');
 
-                // Notify of state change.
-                signalAll();
+                notifyAll();
             }
             else {
                 if (log.isDebugEnabled())
@@ -1144,12 +1156,9 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
                         ", tx=" + this + ']');
             }
         }
-        finally {
-            unlock();
-        }
 
         if (notify) {
-            GridFutureAdapter<IgniteInternalTx> fut = finFut.get();
+            GridFutureAdapter<IgniteInternalTx> fut = finFut;
 
             if (fut != null)
                 fut.onDone(this);
@@ -2026,8 +2035,8 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
-        @Override public boolean commitVersion(GridCacheVersion commitVer) {
-            return false;
+        @Override public void commitVersion(GridCacheVersion commitVer) {
+            // No-op.
         }
 
         /** {@inheritDoc} */
@@ -2037,7 +2046,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
         /** {@inheritDoc} */
         @Override public void prepare() throws IgniteCheckedException {
-
+            // No-op.
         }
 
         /** {@inheritDoc} */
@@ -2047,7 +2056,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
         /** {@inheritDoc} */
         @Override public void endVersion(GridCacheVersion endVer) {
-
+            // No-op.
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index d9786a8..570aa48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -876,7 +876,7 @@ public class IgniteTxHandler {
             log.debug("Processing dht tx finish request [nodeId=" + nodeId + ", req=" + req + ']');
 
         if (req.checkCommitted()) {
-            sendReply(nodeId, req, checkDhtRemoteTxCommitted(req.version()));
+            sendReply(nodeId, req, !ctx.tm().addRolledbackTx(null, req.version()));
 
             return;
         }
@@ -896,8 +896,11 @@ public class IgniteTxHandler {
         if (req.replyRequired()) {
             IgniteInternalFuture completeFut;
 
-            IgniteInternalFuture<IgniteInternalTx> dhtFin = dhtTx == null ? null : dhtTx.done() ? null : dhtTx.finishFuture();
-            IgniteInternalFuture<IgniteInternalTx> nearFin = nearTx == null ? null : nearTx.done() ? null : nearTx.finishFuture();
+            IgniteInternalFuture<IgniteInternalTx> dhtFin = dhtTx == null ?
+                null : dhtTx.done() ? null : dhtTx.finishFuture();
+
+            IgniteInternalFuture<IgniteInternalTx> nearFin = nearTx == null ?
+                null : nearTx.done() ? null : nearTx.finishFuture();
 
             if (dhtFin != null && nearFin != null) {
                 GridCompoundFuture fut = new GridCompoundFuture();
@@ -914,8 +917,7 @@ public class IgniteTxHandler {
 
             if (completeFut != null) {
                 completeFut.listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
-                    @Override
-                    public void apply(IgniteInternalFuture<IgniteInternalTx> igniteTxIgniteFuture) {
+                    @Override public void apply(IgniteInternalFuture<IgniteInternalTx> igniteTxIgniteFuture) {
                         sendReply(nodeId, req, true);
                     }
                 });
@@ -928,24 +930,6 @@ public class IgniteTxHandler {
     }
 
     /**
-     * Checks whether DHT remote transaction with given version has been committed. If not, will add version
-     * to rollback version set so that late response will not falsely commit this transaction.
-     *
-     * @param writeVer Write version to check.
-     * @return {@code True} if transaction has been committed, {@code false} otherwise.
-     */
-    public boolean checkDhtRemoteTxCommitted(GridCacheVersion writeVer) {
-        assert writeVer != null;
-
-        boolean committed = true;
-
-        if (ctx.tm().addRolledbackTx(writeVer))
-            committed = false;
-
-        return committed;
-    }
-
-    /**
      * @param nodeId Node ID.
      * @param tx Transaction.
      * @param req Request.
@@ -953,7 +937,8 @@ public class IgniteTxHandler {
     protected void finish(
         UUID nodeId,
         IgniteTxRemoteEx tx,
-        GridDhtTxFinishRequest req) {
+        GridDhtTxFinishRequest req
+    ) {
         // We don't allow explicit locks for transactions and
         // therefore immediately return if transaction is null.
         // However, we may decide to relax this restriction in
@@ -961,9 +946,9 @@ public class IgniteTxHandler {
         if (tx == null) {
             if (req.commit())
                 // Must be some long time duplicate, but we add it anyway.
-                ctx.tm().addCommittedTx(req.version(), null);
+                ctx.tm().addCommittedTx(tx, req.version(), null);
             else
-                ctx.tm().addRolledbackTx(req.version());
+                ctx.tm().addRolledbackTx(tx, req.version());
 
             if (log.isDebugEnabled())
                 log.debug("Received finish request for non-existing transaction (added to completed set) " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 82e5f2a..2c7bf8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -206,21 +206,21 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         int taskNameHash
     ) {
         super(
-            cctx, 
-            xidVer, 
-            implicit, 
-            implicitSingle, 
-            /*local*/true, 
-            sys, 
+            cctx,
+            xidVer,
+            implicit,
+            implicitSingle,
+            /*local*/true,
+            sys,
             plc,
-            concurrency, 
-            isolation, 
+            concurrency,
+            isolation,
             timeout,
             invalidate,
-            storeEnabled, 
-            onePhaseCommit, 
-            txSize, 
-            subjId, 
+            storeEnabled,
+            onePhaseCommit,
+            txSize,
+            subjId,
             taskNameHash
         );
 
@@ -1054,7 +1054,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                             eventNodeId(),
                                             txEntry.nodeId(),
                                             false,
-                                            false,
                                             evt,
                                             metrics,
                                             topVer,
@@ -1072,7 +1071,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                                 nodeId,
                                                 false,
                                                 false,
-                                                false,
                                                 metrics,
                                                 topVer,
                                                 CU.empty0(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 1f51b8a..c2e7dea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -18,17 +18,16 @@
 package org.apache.ignite.internal.processors.cache.transactions;
 
 import java.io.Externalizable;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
-import java.util.LinkedList;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
@@ -62,6 +61,7 @@ import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
@@ -75,6 +75,7 @@ import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionState;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
+import org.jsr166.ConcurrentLinkedHashMap;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_MAX_COMPLETED_TX_COUNT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SLOW_TX_WARN_TIMEOUT;
@@ -95,6 +96,7 @@ import static org.apache.ignite.transactions.TransactionState.PREPARED;
 import static org.apache.ignite.transactions.TransactionState.PREPARING;
 import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK;
 import static org.apache.ignite.transactions.TransactionState.UNKNOWN;
+import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q;
 
 /**
  * Cache transaction manager.
@@ -128,8 +130,18 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     private IgniteTxHandler txHandler;
 
     /** Committed local transactions. */
-    private final GridBoundedConcurrentOrderedMap<GridCacheVersion, Boolean> completedVers =
-        new GridBoundedConcurrentOrderedMap<>(Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT));
+    private final GridBoundedConcurrentOrderedMap<GridCacheVersion, Boolean> completedVersSorted =
+        new GridBoundedConcurrentOrderedMap<>(
+            Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT));
+
+    /** Committed local transactions. */
+    private final ConcurrentLinkedHashMap<GridCacheVersion, Boolean> completedVersHashMap =
+        new ConcurrentLinkedHashMap<>(
+            Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT),
+            0.75f,
+            Runtime.getRuntime().availableProcessors() * 2,
+            Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT),
+            PER_SEGMENT_Q);
 
     /** Transaction finish synchronizer. */
     private GridCacheTxFinishSync txFinishSync;
@@ -298,7 +310,8 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         X.println(">>> Transaction manager memory stats [grid=" + cctx.gridName() + ']');
         X.println(">>>   threadMapSize: " + threadMap.size());
         X.println(">>>   idMap [size=" + idMap.size() + ']');
-        X.println(">>>   completedVersSize: " + completedVers.size());
+        X.println(">>>   completedVersSortedSize: " + completedVersSorted.size());
+        X.println(">>>   completedVersHashMapSize: " + completedVersHashMap.sizex());
     }
 
     /**
@@ -319,7 +332,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * @return Committed versions size.
      */
     public int completedVersionsSize() {
-        return completedVers.size();
+        return completedVersHashMap.size();
     }
 
     /**
@@ -329,7 +342,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      *      {@code false} otherwise.
      */
     public boolean isCompleted(IgniteInternalTx tx) {
-        return completedVers.containsKey(tx.xidVersion());
+        return completedVersHashMap.containsKey(tx.xidVersion());
     }
 
     /**
@@ -770,65 +783,59 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
-     * @param map Collection to copy.
-     * @param expVal Values to copy.
-     * @return Copy of the collection.
+     * @param min Minimum version.
+     * @return Pair [committed, rolledback] - never {@code null}, elements potentially empty,
+     *      but also never {@code null}.
      */
-    private Collection<GridCacheVersion> copyOf(Map<GridCacheVersion, Boolean> map, boolean expVal) {
-        Collection<GridCacheVersion> l = new LinkedList<>();
+    public IgnitePair<Collection<GridCacheVersion>> versions(GridCacheVersion min) {
+        Collection<GridCacheVersion> committed = null;
+        Collection<GridCacheVersion> rolledback = null;
 
-        for (Map.Entry<GridCacheVersion, Boolean> e : map.entrySet()) {
-            if (e.getValue() == expVal)
-                l.add(e.getKey());
-        }
+        for (Map.Entry<GridCacheVersion, Boolean> e : completedVersSorted.tailMap(min, true).entrySet()) {
+            if (e.getValue()) {
+                if (committed == null)
+                    committed = new ArrayList<>();
 
-        return l;
-    }
+                committed.add(e.getKey());
+            }
+            else {
+                if (rolledback == null)
+                    rolledback = new ArrayList<>();
 
-    /**
-     * Gets committed transactions starting from the given version (inclusive). // TODO: GG-4011: why inclusive?
-     *
-     * @param min Start (or minimum) version.
-     * @return Committed transactions starting from the given version (non-inclusive).
-     */
-    public Collection<GridCacheVersion> committedVersions(GridCacheVersion min) {
-        ConcurrentNavigableMap<GridCacheVersion, Boolean> tail
-            = completedVers.tailMap(min, true);
+                rolledback.add(e.getKey());
+            }
+        }
 
-        return F.isEmpty(tail) ? Collections.<GridCacheVersion>emptyList() : copyOf(tail, true);
+        return F.pair(
+            committed == null ? Collections.<GridCacheVersion>emptyList() : committed,
+            rolledback == null ? Collections.<GridCacheVersion>emptyList() : rolledback);
     }
 
     /**
-     * Gets rolledback transactions starting from the given version (inclusive). // TODO: GG-4011: why inclusive?
-     *
-     * @param min Start (or minimum) version.
-     * @return Committed transactions starting from the given version (non-inclusive).
+     * @return Collection of active transactions.
      */
-    public Collection<GridCacheVersion> rolledbackVersions(GridCacheVersion min) {
-        ConcurrentNavigableMap<GridCacheVersion, Boolean> tail
-            = completedVers.tailMap(min, true);
-
-        return F.isEmpty(tail) ? Collections.<GridCacheVersion>emptyList() : copyOf(tail, false);
+    public Collection<IgniteInternalTx> activeTransactions() {
+        return F.concat(false, idMap.values(), nearIdMap.values());
     }
 
     /**
      * @param tx Tx to remove.
      */
     public void removeCommittedTx(IgniteInternalTx tx) {
-        completedVers.remove(tx.xidVersion(), true);
+        completedVersHashMap.remove(tx.xidVersion(), true);
+
+        if (tx.needsCompletedVersions())
+            completedVersSorted.remove(tx.xidVersion(), true);
     }
 
     /**
      * @param tx Committed transaction.
-     * @return If transaction was not already present in committed set.
      */
-    public boolean addCommittedTx(IgniteInternalTx tx) {
-        boolean res = addCommittedTx(tx.xidVersion(), tx.nearXidVersion());
+    public void addCommittedTx(IgniteInternalTx tx) {
+        addCommittedTx(tx, tx.xidVersion(), tx.nearXidVersion());
 
         if (!tx.local() && !tx.near() && tx.onePhaseCommit())
-            addCommittedTx(tx.nearXidVersion(), null);
-
-        return res;
+            addCommittedTx(tx, tx.nearXidVersion(), null);
     }
 
     /**
@@ -836,60 +843,52 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * @return If transaction was not already present in committed set.
      */
     public boolean addRolledbackTx(IgniteInternalTx tx) {
-        return addRolledbackTx(tx.xidVersion());
-    }
-
-    /**
-     * @return Collection of active transactions.
-     */
-    public Collection<IgniteInternalTx> activeTransactions() {
-        return F.concat(false, idMap.values(), nearIdMap.values());
+        return addRolledbackTx(tx, tx.xidVersion());
     }
 
     /**
+     * @param tx Tx.
      * @param xidVer Completed transaction version.
      * @param nearXidVer Optional near transaction ID.
      * @return If transaction was not already present in completed set.
      */
-    public boolean addCommittedTx(GridCacheVersion xidVer, @Nullable GridCacheVersion nearXidVer) {
+    public boolean addCommittedTx(
+        IgniteInternalTx tx,
+        GridCacheVersion xidVer,
+        @Nullable GridCacheVersion nearXidVer
+    ) {
         if (nearXidVer != null)
             xidVer = new CommittedVersion(xidVer, nearXidVer);
 
-        Boolean committed = completedVers.putIfAbsent(xidVer, true);
+        Boolean committed0 = completedVersHashMap.putIfAbsent(xidVer, true);
 
-        if (committed == null || committed) {
-            if (log.isDebugEnabled())
-                log.debug("Added transaction to committed version set: " + xidVer);
+        if (committed0 == null && (tx == null || tx.needsCompletedVersions())) {
+            Boolean b = completedVersSorted.putIfAbsent(xidVer, true);
 
-            return true;
+            assert b == null;
         }
-        else {
-            if (log.isDebugEnabled())
-                log.debug("Transaction is already present in rolled back version set: " + xidVer);
 
-            return false;
-        }
+        return committed0 == null || committed0;
     }
 
     /**
+     * @param tx Tx.
      * @param xidVer Completed transaction version.
      * @return If transaction was not already present in completed set.
      */
-    public boolean addRolledbackTx(GridCacheVersion xidVer) {
-        Boolean committed = completedVers.putIfAbsent(xidVer, false);
+    public boolean addRolledbackTx(
+        IgniteInternalTx tx,
+        GridCacheVersion xidVer
+    ) {
+        Boolean committed0 = completedVersHashMap.putIfAbsent(xidVer, false);
 
-        if (committed == null || !committed) {
-            if (log.isDebugEnabled())
-                log.debug("Added transaction to rolled back version set: " + xidVer);
+        if (committed0 == null && (tx == null || tx.needsCompletedVersions())) {
+            Boolean b = completedVersSorted.putIfAbsent(xidVer, false);
 
-            return true;
+            assert b == null;
         }
-        else {
-            if (log.isDebugEnabled())
-                log.debug("Transaction is already present in committed version set: " + xidVer);
 
-            return false;
-        }
+        return committed0 == null || !committed0;
     }
 
     /**
@@ -903,7 +902,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
 
             assert min != null;
 
-            tx.completedVersions(min, committedVersions(min), rolledbackVersions(min));
+            IgnitePair<Collection<GridCacheVersion>> versPair = versions(min);
+
+            tx.completedVersions(min, versPair.get1(), versPair.get2());
         }
     }
 
@@ -1027,18 +1028,15 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
          * so we don't do it here.
          */
 
-        Boolean committed = completedVers.get(tx.xidVersion());
+        Boolean committed = completedVersHashMap.get(tx.xidVersion());
 
         // 1. Make sure that committed version has been recorded.
         if (!((committed != null && committed) || tx.writeSet().isEmpty() || tx.isSystemInvalidate())) {
             uncommitTx(tx);
 
-            GridCacheVersion first = completedVers.isEmpty() ? null : completedVers.firstKey();
-            GridCacheVersion last = completedVers.isEmpty() ? null : completedVers.lastKey();
-
             throw new IgniteException("Missing commit version (consider increasing " +
-                IGNITE_MAX_COMPLETED_TX_COUNT + " system property) [ver=" + tx.xidVersion() + ", firstVer=" +
-                first + ", lastVer=" + last + ", tx=" + tx.xid() + ']');
+                IGNITE_MAX_COMPLETED_TX_COUNT + " system property) [ver=" + tx.xidVersion() +
+                ", tx=" + tx.getClass().getSimpleName() + ']');
         }
 
         ConcurrentMap<GridCacheVersion, IgniteInternalTx> txIdMap = transactionMap(tx);
@@ -1578,12 +1576,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             return resFut;
         }
 
-        Boolean committed = null;
-
-        for (Map.Entry<GridCacheVersion, Boolean> entry : completedVers.entrySet()) {
-            if (entry.getValue() == null)
-                continue;
+        boolean committed = false;
 
+        for (Map.Entry<GridCacheVersion, Boolean> entry : completedVersHashMap.entrySet()) {
             if (entry.getKey() instanceof CommittedVersion) {
                 CommittedVersion comm = (CommittedVersion)entry.getKey();
 
@@ -1598,7 +1593,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         if (log.isDebugEnabled())
             log.debug("Near transaction committed: " + committed);
 
-        resFut.onDone(committed != null && committed);
+        resFut.onDone(committed);
 
         return resFut;
     }
@@ -1702,7 +1697,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
 
         // Not all transactions were found. Need to scan committed versions to check
         // if transaction was already committed.
-        for (Map.Entry<GridCacheVersion, Boolean> e : completedVers.entrySet()) {
+        for (Map.Entry<GridCacheVersion, Boolean> e : completedVersHashMap.entrySet()) {
             if (!e.getValue())
                 continue;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashMap.java
index 04d1a85..7aa3734 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashMap.java
@@ -116,9 +116,6 @@ public class GridBoundedConcurrentLinkedHashMap<K, V> extends ConcurrentLinkedHa
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        // TODO GG-4788
-        return policy() != SINGLE_Q ?
-            S.toString(GridBoundedConcurrentLinkedHashMap.class, this) :
-            S.toString(GridBoundedConcurrentLinkedHashMap.class, this, "entrySet", keySet());
+        return S.toString(GridBoundedConcurrentLinkedHashMap.class, this, "entrySet", keySet());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashSet.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashSet.java
index a06f2ff..2801839 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentLinkedHashSet.java
@@ -156,9 +156,6 @@ public class GridBoundedConcurrentLinkedHashSet<E> extends GridSetWrapper<E> {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        // TODO GG-4788
-        return ((ConcurrentLinkedHashMap<E, Object>)map()).policy() != SINGLE_Q ?
-            S.toString(GridBoundedConcurrentLinkedHashSet.class, this) :
-            S.toString(GridBoundedConcurrentLinkedHashSet.class, this, "elements", map().keySet());
+        return S.toString(GridBoundedConcurrentLinkedHashSet.class, this, "elements", map().keySet());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java
index b091652..3f6db30 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.util;
 
 import java.util.Comparator;
 import java.util.Map;
-import java.util.NoSuchElementException;
 import java.util.SortedMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -46,7 +45,7 @@ public class GridBoundedConcurrentOrderedMap<K, V> extends ConcurrentSkipListMap
     private static final long serialVersionUID = 0L;
 
     /** Element count. */
-    private final AtomicInteger cnt = new AtomicInteger(0);
+    private final AtomicInteger cnt = new AtomicInteger();
 
     /** Maximum size. */
     private int max;
@@ -168,35 +167,21 @@ public class GridBoundedConcurrentOrderedMap<K, V> extends ConcurrentSkipListMap
     private void onPut() {
         cnt.incrementAndGet();
 
-        int c;
+        IgniteBiInClosure<K, V> lsnr = this.lsnr;
 
-        while ((c = cnt.get()) > max) {
-            // Decrement count.
-            if (cnt.compareAndSet(c, c - 1)) {
-                try {
-                    K key = firstEntry().getKey();
+        int delta = cnt.get() - max;
 
-                    V val;
+        for (int i = 0; i < delta && cnt.get() > max; i++) {
+            Entry<K, V> e = pollFirstEntry();
 
-                    // Make sure that an element is removed.
-                    while ((val = super.remove(firstEntry().getKey())) == null) {
-                        // No-op.
-                    }
+            if (e == null)
+                return;
 
-                    assert val != null;
-
-                    IgniteBiInClosure<K, V> lsnr = this.lsnr;
-
-                    // Listener notification.
-                    if (lsnr != null)
-                        lsnr.apply(key, val);
-                }
-                catch (NoSuchElementException ignored) {
-                    cnt.incrementAndGet();
+            cnt.decrementAndGet();
 
-                    return;
-                }
-            }
+            // Listener notification.
+            if (lsnr != null)
+                lsnr.apply(e.getKey(), e.getValue());
         }
     }
 
@@ -251,4 +236,4 @@ public class GridBoundedConcurrentOrderedMap<K, V> extends ConcurrentSkipListMap
 
         return rmvd;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentFactory.java
index 6e0e876..d1a7bb5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentFactory.java
@@ -18,18 +18,12 @@
 package org.apache.ignite.internal.util;
 
 import java.util.concurrent.ConcurrentMap;
-import org.apache.ignite.IgniteSystemProperties;
 import org.jsr166.ConcurrentHashMap8;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_MAP_CONCURRENCY_LEVEL;
-
 /**
  * Concurrent map factory.
  */
 public class GridConcurrentFactory {
-    /** Default concurrency level. */
-    private static final int CONCURRENCY_LEVEL = IgniteSystemProperties.getInteger(IGNITE_MAP_CONCURRENCY_LEVEL, 256);
-
     /**
      * Ensure singleton.
      */
@@ -43,7 +37,6 @@ public class GridConcurrentFactory {
      * @return New concurrent map.
      */
     public static <K, V> ConcurrentMap<K, V> newMap() {
-        return new ConcurrentHashMap8<>(16 * CONCURRENCY_LEVEL, 0.75f, CONCURRENCY_LEVEL);
+        return new ConcurrentHashMap8<>();
     }
-
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentLinkedHashSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentLinkedHashSet.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentLinkedHashSet.java
index 5a53b4b..0c76787 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentLinkedHashSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridConcurrentLinkedHashSet.java
@@ -24,8 +24,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentLinkedHashMap;
 
-import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.SINGLE_Q;
-
 /**
  * Concurrent linked set implementation.
  */
@@ -123,9 +121,6 @@ public class GridConcurrentLinkedHashSet<E> extends GridSetWrapper<E> {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        // TODO GG-4788
-        return ((ConcurrentLinkedHashMap)map()).policy() != SINGLE_Q ?
-            S.toString(GridConcurrentLinkedHashSet.class, this) :
-            S.toString(GridConcurrentLinkedHashSet.class, this, "elements", map().keySet());
+        return S.toString(GridConcurrentLinkedHashSet.class, this, "elements", map().keySet());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUuidCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUuidCache.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUuidCache.java
index 4ca00d9..d9ffdd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUuidCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUuidCache.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.util;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 
+import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q;
+
 /**
  *
  */
@@ -29,7 +31,7 @@ public final class IgniteUuidCache {
 
     /** Cache. */
     private static final ConcurrentMap<UUID, UUID> cache =
-        new GridBoundedConcurrentLinkedHashMap<>(MAX, 1024, 0.75f, 64);
+        new GridBoundedConcurrentLinkedHashMap<>(MAX, 1024, 0.75f, 64, PER_SEGMENT_Q);
 
     /**
      * Gets cached UUID to preserve memory.
@@ -56,4 +58,4 @@ public final class IgniteUuidCache {
     private IgniteUuidCache() {
         // No-op.
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index 0a6d9aa..31674f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@ -17,11 +17,9 @@
 
 package org.apache.ignite.internal.util.future;
 
+import java.util.ArrayList;
 import java.util.Collection;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicMarkableReference;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -35,7 +33,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteReducer;
 import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentLinkedDeque8;
 
 /**
  * Future composed of multiple inner futures.
@@ -44,33 +41,38 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Futures. */
-    private final ConcurrentLinkedDeque8<IgniteInternalFuture<T>> futs = new ConcurrentLinkedDeque8<>();
+    /** */
+    private static final int INITED = 0b1;
 
-    /** Pending futures. */
-    private final Collection<IgniteInternalFuture<T>> pending = new ConcurrentLinkedDeque8<>();
+    /** */
+    private static final AtomicIntegerFieldUpdater<GridCompoundFuture> flagsUpd =
+        AtomicIntegerFieldUpdater.newUpdater(GridCompoundFuture.class, "flags");
 
-    /** Listener call count. */
-    private final AtomicInteger lsnrCalls = new AtomicInteger();
+    /** */
+    private static final AtomicIntegerFieldUpdater<GridCompoundFuture> lsnrCallsUpd =
+        AtomicIntegerFieldUpdater.newUpdater(GridCompoundFuture.class, "lsnrCalls");
 
-    /** Finished flag. */
-    private final AtomicBoolean finished = new AtomicBoolean();
+    /** Futures. */
+    private final Collection<IgniteInternalFuture<T>> futs = new ArrayList<>();
 
     /** Reducer. */
     @GridToStringInclude
     private IgniteReducer<T, R> rdc;
 
-    /** Initialize flag. */
-    private AtomicBoolean init = new AtomicBoolean(false);
-
-    /** Result with a flag to control if reducer has been called. */
-    private AtomicMarkableReference<R> res = new AtomicMarkableReference<>(null, false);
-
     /** Exceptions to ignore. */
     private Class<? extends Throwable>[] ignoreChildFailures;
 
-    /** Error. */
-    private AtomicReference<Throwable> err = new AtomicReference<>();
+    /**
+     * Updated via {@link #flagsUpd}.
+     *
+     * @see #INITED
+     */
+    @SuppressWarnings("unused")
+    private volatile int flags;
+
+    /** Updated via {@link #lsnrCallsUpd}. */
+    @SuppressWarnings("unused")
+    private volatile int lsnrCalls;
 
     /**
      *
@@ -104,7 +106,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     /** {@inheritDoc} */
     @Override public boolean cancel() throws IgniteCheckedException {
         if (onCancelled()) {
-            for (IgniteInternalFuture<T> fut : futs)
+            for (IgniteInternalFuture<T> fut : futures())
                 fut.cancel();
 
             return true;
@@ -118,8 +120,26 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *
      * @return Collection of futures.
      */
+    private Collection<IgniteInternalFuture<T>> futures(boolean pending) {
+        synchronized (futs) {
+            Collection<IgniteInternalFuture<T>> res = new ArrayList<>(futs.size());
+
+            for (IgniteInternalFuture<T> fut : futs) {
+                if (!pending || !fut.isDone())
+                    res.add(fut);
+            }
+
+            return res;
+        }
+    }
+
+    /**
+     * Gets collection of futures.
+     *
+     * @return Collection of futures.
+     */
     public Collection<IgniteInternalFuture<T>> futures() {
-        return futs;
+        return futures(false);
     }
 
     /**
@@ -128,7 +148,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      * @return Pending futures.
      */
     public Collection<IgniteInternalFuture<T>> pending() {
-        return pending;
+        return futures(true);
     }
 
     /**
@@ -147,7 +167,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      * @return {@code True} if there are pending futures.
      */
     public boolean hasPending() {
-        return !pending.isEmpty();
+        return !pending().isEmpty();
     }
 
     /**
@@ -155,7 +175,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *      {@link #markInitialized()} method is called on future.
      */
     public boolean initialized() {
-        return init.get();
+        return flagSet(INITED);
     }
 
     /**
@@ -166,18 +186,20 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     public void add(IgniteInternalFuture<T> fut) {
         assert fut != null;
 
-        pending.add(fut);
-        futs.add(fut);
+        synchronized (futs) {
+            futs.add(fut);
+        }
 
         fut.listen(new Listener());
 
-        if (isCancelled())
+        if (isCancelled()) {
             try {
                 fut.cancel();
             }
             catch (IgniteCheckedException e) {
                 onDone(e);
             }
+        }
     }
 
     /**
@@ -185,7 +207,8 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *
      * @param futs Futures to add.
      */
-    public void addAll(@Nullable IgniteInternalFuture<T>... futs) {
+    @SafeVarargs
+    public final void addAll(@Nullable IgniteInternalFuture<T>... futs) {
         addAll(F.asList(futs));
     }
 
@@ -195,9 +218,10 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      * @param futs Futures to add.
      */
     public void addAll(@Nullable Iterable<IgniteInternalFuture<T>> futs) {
-        if (futs != null)
+        if (futs != null) {
             for (IgniteInternalFuture<T> fut : futs)
                 add(fut);
+        }
     }
 
     /**
@@ -219,10 +243,34 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     }
 
     /**
+     * @param flag Flag to CAS.
+     * @return {@code True} if CAS succeeds.
+     */
+    private boolean casFlag(int flag) {
+        for (;;) {
+            int flags0 = flags;
+
+            if ((flags0 & flag) != 0)
+                return false;
+
+            if (flagsUpd.compareAndSet(this, flags0, flags0 | flag))
+                return true;
+        }
+    }
+
+    /**
+     * @param flag Flag to check.
+     * @return {@code True} if set.
+     */
+    private boolean flagSet(int flag) {
+        return (flags & flag) != 0;
+    }
+
+    /**
      * Mark this future as initialized.
      */
     public void markInitialized() {
-        if (init.compareAndSet(false, true))
+        if (casFlag(INITED))
             // Check complete to make sure that we take care
             // of all the ignored callbacks.
             checkComplete();
@@ -232,22 +280,14 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      * Check completeness of the future.
      */
     private void checkComplete() {
-        Throwable err = this.err.get();
-
-        boolean ignore = ignoreFailure(err);
-
-        if (init.get() && (res.isMarked() || lsnrCalls.get() == futs.sizex() || (err != null && !ignore))
-            && finished.compareAndSet(false, true)) {
+        if (flagSet(INITED) && !isDone() && lsnrCalls == futuresSize()) {
             try {
-                if (err == null && rdc != null && !res.isMarked())
-                    res.compareAndSet(null, rdc.reduce(), false, true);
+                onDone(rdc != null ? rdc.reduce() : null);
             }
             catch (RuntimeException e) {
                 U.error(null, "Failed to execute compound future reducer: " + this, e);
 
                 onDone(e);
-
-                return;
             }
             catch (AssertionError e) {
                 U.error(null, "Failed to execute compound future reducer: " + this, e);
@@ -256,8 +296,15 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
 
                 throw e;
             }
+        }
+    }
 
-            onDone(res.getReference(), ignore ? null : err);
+    /**
+     * @return Futures size.
+     */
+    private int futuresSize() {
+        synchronized (futs) {
+            return futs.size();
         }
     }
 
@@ -288,7 +335,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
             "cancelled", isCancelled(),
             "err", error(),
             "futs",
-                F.viewReadOnly(futs, new C1<IgniteInternalFuture<T>, String>() {
+                F.viewReadOnly(futures(), new C1<IgniteInternalFuture<T>, String>() {
                     @Override public String apply(IgniteInternalFuture<T> f) {
                         return Boolean.toString(f.isDone());
                     }
@@ -305,14 +352,12 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
 
         /** {@inheritDoc} */
         @Override public void apply(IgniteInternalFuture<T> fut) {
-            pending.remove(fut);
-
             try {
                 T t = fut.get();
 
                 try {
-                    if (rdc != null && !rdc.collect(t) && !res.isMarked())
-                        res.compareAndSet(null, rdc.reduce(), false, true);
+                    if (rdc != null && !rdc.collect(t))
+                        onDone(rdc.reduce());
                 }
                 catch (RuntimeException e) {
                     U.error(null, "Failed to execute compound future reducer: " + this, e);
@@ -331,18 +376,20 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
             }
             catch (IgniteTxOptimisticCheckedException | IgniteFutureCancelledCheckedException |
                 ClusterTopologyCheckedException e) {
-                err.compareAndSet(null, e);
+                if (!ignoreFailure(e))
+                    onDone(e);
             }
             catch (IgniteCheckedException e) {
-                if (!ignoreFailure(e))
+                if (!ignoreFailure(e)) {
                     U.error(null, "Failed to execute compound future reducer: " + this, e);
 
-                err.compareAndSet(null, e);
+                    onDone(e);
+                }
             }
             catch (RuntimeException e) {
                 U.error(null, "Failed to execute compound future reducer: " + this, e);
 
-                err.compareAndSet(null, e);
+                onDone(e);
             }
             catch (AssertionError e) {
                 U.error(null, "Failed to execute compound future reducer: " + this, e);
@@ -353,7 +400,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
                 throw e;
             }
 
-            lsnrCalls.incrementAndGet();
+            lsnrCallsUpd.incrementAndGet(GridCompoundFuture.this);
 
             checkComplete();
         }
@@ -363,4 +410,4 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
             return "Compound future listener: " + GridCompoundFuture.this;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java b/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java
index d93f12e..b3747d7 100644
--- a/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java
+++ b/modules/core/src/main/java/org/jsr166/ConcurrentHashMap8.java
@@ -3805,4 +3805,4 @@ public class ConcurrentHashMap8<K, V>
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/jsr166/ConcurrentLinkedDeque8.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/jsr166/ConcurrentLinkedDeque8.java b/modules/core/src/main/java/org/jsr166/ConcurrentLinkedDeque8.java
index 75db13c..28e38d7 100644
--- a/modules/core/src/main/java/org/jsr166/ConcurrentLinkedDeque8.java
+++ b/modules/core/src/main/java/org/jsr166/ConcurrentLinkedDeque8.java
@@ -1735,4 +1735,4 @@ public class ConcurrentLinkedDeque8<E>
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/main/java/org/jsr166/ConcurrentLinkedHashMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/jsr166/ConcurrentLinkedHashMap.java b/modules/core/src/main/java/org/jsr166/ConcurrentLinkedHashMap.java
index 5b7381e..e8f8e0f 100644
--- a/modules/core/src/main/java/org/jsr166/ConcurrentLinkedHashMap.java
+++ b/modules/core/src/main/java/org/jsr166/ConcurrentLinkedHashMap.java
@@ -17,7 +17,6 @@ import java.util.AbstractSet;
 import java.util.ArrayDeque;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
-import java.util.Deque;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.Hashtable;
@@ -28,6 +27,9 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
 
 import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q;
 import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q_OPTIMIZED_RMV;
@@ -264,12 +266,14 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
         private volatile V val;
 
         /** Reference to a node in queue for fast removal operations. */
+        @GridToStringExclude
         private volatile ConcurrentLinkedDeque8.Node node;
 
         /** Modification count of the map for duplicates exclusion. */
         private volatile int modCnt;
 
         /** Link to the next entry in a bucket */
+        @GridToStringExclude
         private final HashEntry<K, V> next;
 
         /**
@@ -332,6 +336,11 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
         static <K, V> HashEntry<K, V>[] newArray(int i) {
             return new HashEntry[i];
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(HashEntry.class, this, "key", key, "val", val);
+        }
     }
 
     /**
@@ -749,7 +758,7 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
                             recordInsert(e, (ConcurrentLinkedDeque8)segEntryQ);
 
                             if (maxCap > 0)
-                                checkRemoveEldestEntrySegment();
+                                checkRemoveEldestEntrySegment(c);
 
                             break;
 
@@ -757,7 +766,7 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
                             segEntryQ.add(e);
 
                             if (maxCap > 0)
-                                checkRemoveEldestEntrySegment();
+                                checkRemoveEldestEntrySegment(c);
 
                             break;
 
@@ -779,23 +788,21 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
         }
 
         /**
-         *
+         * @param cnt Segment entries count.
          */
-        private void checkRemoveEldestEntrySegment() {
+        private void checkRemoveEldestEntrySegment(int cnt) {
             assert maxCap > 0;
 
-            int rmvCnt = sizex() - maxCap;
-
-            for (int i = 0; i < rmvCnt; i++) {
+            if (cnt - ((maxCap / segments.length) + 1) > 0) {
                 HashEntry<K, V> e0 = segEntryQ.poll();
 
-                if (e0 == null)
-                    break;
-
-                removeLocked(e0.key, e0.hash, null /*no need to compare*/, false);
+                assert e0 != null;
 
-                if (sizex() <= maxCap)
-                    break;
+                removeLocked(
+                    e0.key,
+                    e0.hash,
+                    null /*no need to compare*/,
+                    false);
             }
         }
 
@@ -1812,34 +1819,22 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
          * @param asc {@code True} for ascending iterator.
          */
         HashIterator(boolean asc) {
-            // TODO GG-4788 - Need to fix iterators for ConcurrentLinkedHashMap in perSegment mode
-            if (qPlc != SINGLE_Q)
-                throw new IllegalStateException("Iterators are not supported in 'perSegmentQueue' modes.");
-
             modCnt = ConcurrentLinkedHashMap.this.modCnt.intValue();
 
             // Init delegate.
-            delegate = asc ? entryQ.iterator() : entryQ.descendingIterator();
-
-            advance();
-        }
+            switch (qPlc) {
+                case SINGLE_Q:
+                    delegate = asc ? entryQ.iterator() : entryQ.descendingIterator();
 
-        /**
-         * @return Copy of the queue.
-         */
-        private Deque<HashEntry<K, V>> copyQueue() {
-            int i = entryQ.sizex();
-
-            Deque<HashEntry<K, V>> res = new ArrayDeque<>(i);
-
-            Iterator<HashEntry<K, V>> iter = entryQ.iterator();
+                    break;
 
-            while (iter.hasNext() && i-- >= 0)
-                res.add(iter.next());
+                default:
+                    assert qPlc == PER_SEGMENT_Q || qPlc == PER_SEGMENT_Q_OPTIMIZED_RMV : qPlc;
 
-            assert !iter.hasNext() : "Entries queue has been modified.";
+                    delegate = new HashIteratorDelegate();
+            }
 
-            return res;
+            advance();
         }
 
         /**
@@ -1901,6 +1896,130 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
     }
 
     /**
+     *
+     */
+    private class HashIteratorDelegate implements Iterator<HashEntry<K, V>> {
+        /** */
+        private HashEntry<K, V>[] curTbl;
+
+        /** */
+        private int nextSegIdx;
+
+        /** */
+        private int nextTblIdx;
+
+        /** */
+        private HashEntry<K, V> next;
+
+        /** */
+        private HashEntry<K, V> next0;
+
+        /** */
+        private HashEntry<K, V> cur;
+
+        /**
+         *
+         */
+        public HashIteratorDelegate() {
+            nextSegIdx = segments.length - 1;
+            nextTblIdx = -1;
+
+            advance();
+        }
+
+        /**
+         *
+         */
+        private void advance() {
+            if (next0 != null && advanceInBucket(next0, true))
+                return;
+
+            while (nextTblIdx >= 0) {
+                HashEntry<K, V> bucket = curTbl[nextTblIdx--];
+
+                if (bucket != null && advanceInBucket(bucket, false))
+                    return;
+            }
+
+            while (nextSegIdx >= 0) {
+                int nextSegIdx0 = nextSegIdx--;
+
+                Segment seg = segments[nextSegIdx0];
+
+                curTbl = seg.tbl;
+
+                for (int j = curTbl.length - 1; j >= 0; --j) {
+                    HashEntry<K, V> bucket = curTbl[j];
+
+                    if (bucket != null && advanceInBucket(bucket, false)) {
+                        nextTblIdx = j - 1;
+
+                        return;
+                    }
+                }
+            }
+        }
+
+        /**
+         * @param e Current next.
+         * @return {@code True} if advance succeeded.
+         */
+        @SuppressWarnings( {"unchecked"})
+        private boolean advanceInBucket(@Nullable HashEntry<K, V> e, boolean skipFirst) {
+            if (e == null)
+                return false;
+
+            next0 = e;
+
+            do {
+                if (!skipFirst) {
+                    next = next0;
+
+                    return true;
+                }
+                else
+                    skipFirst = false;
+            }
+            while ((next0 = next0.next) != null);
+
+            assert next0 == null;
+
+            next = null;
+
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return next != null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public HashEntry<K, V> next() {
+            HashEntry<K, V> e = next;
+
+            if (e == null)
+                throw new NoSuchElementException();
+
+            advance();
+
+            return e;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void remove() {
+            if (cur == null)
+                throw new IllegalStateException();
+
+            HashEntry<K, V> e = cur;
+
+            cur = null;
+
+            ConcurrentLinkedHashMap.this.remove(e.key, e.val);
+        }
+    }
+
+    /**
      * Key iterator implementation.
      */
     private final class KeyIterator extends HashIterator implements Iterator<K>, Enumeration<K> {
@@ -2154,13 +2273,17 @@ public class ConcurrentLinkedHashMap<K, V> extends AbstractMap<K, V> implements
          * the fastest &quot;natural&quot; evicts for bounded maps.
          * <p>
          * NOTE: Remove operations on map are slower than with other policies.
+         * <p>
+         * NOTE: Iteration order is not preserved, i.e. iteration goes as if it was ordinary hash map.
          */
         PER_SEGMENT_Q,
 
         /**
          * Instance of {@code GridConcurrentLinkedDequeue} is created for each segment. This gives
          * faster &quot;natural&quot; evicts for bounded queues and better remove operation times.
+         * <p>
+         * NOTE: Iteration order is not preserved, i.e. iteration goes as if it was ordinary hash map.
          */
         PER_SEGMENT_Q_OPTIMIZED_RMV
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
index 50ba241..616fd43 100644
--- a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
@@ -26,12 +26,18 @@ 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.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 affinity function with different number of backups.
  */
 public class GridCacheAffinityBackupsSelfTest extends GridCommonAbstractTest {
+    /** */
+    private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /** Number of backups. */
     private int backups;
 
@@ -45,6 +51,8 @@ public class GridCacheAffinityBackupsSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setCacheMode(CacheMode.PARTITIONED);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 3a530f2..1d79e20 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -4696,7 +4696,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         TransactionIsolation txIsolation)
         throws  Exception
     {
-        log.info("Test tx skip store [concurrency=" + txConcurrency + ", isolation=" + txIsolation + ']');
+        info("Test tx skip store [concurrency=" + txConcurrency + ", isolation=" + txIsolation + ']');
 
         cache.removeAll(data.keySet());
         checkEmpty(cache, cacheSkipStore);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
index 6a0b9ad..19e49f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -32,13 +33,14 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_MAX_COMPLETED_TX_COUNT;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
  *
  */
 public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTest {
     /** */
-    private volatile Integer failedKey;
+    private volatile boolean putFailed;
 
     /** */
     private String maxCompletedTxCount;
@@ -67,6 +69,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
 
         ccfg.setCacheMode(PARTITIONED);
         ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
         cfg.setCacheConfiguration(ccfg);
 
@@ -90,43 +93,48 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
 
         final AtomicInteger keyStart = new AtomicInteger();
 
+        final ConcurrentLinkedDeque<Integer> q = new ConcurrentLinkedDeque<>();
+
         GridTestUtils.runMultiThreaded(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 int start = keyStart.getAndAdd(KEYS_PER_THREAD);
 
-                for (int i = 0; i < KEYS_PER_THREAD && failedKey == null; i++) {
+                for (int i = 0; i < KEYS_PER_THREAD && !putFailed; i++) {
                     int key = start + i;
 
                     try {
                         cache.put(key, 1);
                     }
                     catch (Exception e) {
-                        log.info("Put failed: " + e);
+                        log.info("Put failed [err=" + e + ", i=" + i + ']');
+
+                        putFailed = true;
 
-                        failedKey = key;
+                        q.add(key);
                     }
                 }
 
-
                 return null;
             }
         }, 10, "put-thread");
 
-        assertNotNull("Test failed to provoke 'missing commit version' error.", failedKey);
+        assertTrue("Test failed to provoke 'missing commit version' error.", putFailed);
 
-        log.info("Trying to update " + failedKey);
+        for (Integer key : q) {
+            log.info("Trying to update " + key);
 
-        IgniteCache<Integer, Integer> asyncCache = cache.withAsync();
+            IgniteCache<Integer, Integer> asyncCache = cache.withAsync();
 
-        asyncCache.put(failedKey, 2);
+            asyncCache.put(key, 2);
 
-        IgniteFuture<?> fut = asyncCache.future();
+            IgniteFuture<?> fut = asyncCache.future();
 
-        try {
-            fut.get(5000);
-        }
-        catch (IgniteFutureTimeoutException ignore) {
-            fail("Put failed to finish in 5s.");
+            try {
+                fut.get(5000);
+            }
+            catch (IgniteFutureTimeoutException ignore) {
+                fail("Put failed to finish in 5s: " + key);
+            }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index abb2767..b93acf5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -541,7 +541,6 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         @Nullable IgniteInternalTx tx,
         UUID evtNodeId,
         UUID affNodeId,
-        boolean writeThrough,
         boolean retval,
         boolean evt,
         boolean metrics,
@@ -894,4 +893,4 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     @Override public void onUnlock() {
         // No-op.
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index a2440e2..ad51600 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -513,4 +513,4 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
             assertTrue(((IgniteKernal)ignite).internalCache().context().isNear());
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
index a2308c6..8f5e07b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
@@ -1090,4 +1090,4 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
         assertEquals("Not stopped IDs: " + notStopped, 0, notStopped.size());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedConcurrentLinkedHashMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedConcurrentLinkedHashMapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedConcurrentLinkedHashMapSelfTest.java
index e6dc7e6..8ce7ae3 100644
--- a/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedConcurrentLinkedHashMapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedConcurrentLinkedHashMapSelfTest.java
@@ -52,4 +52,4 @@ public class GridBoundedConcurrentLinkedHashMapSelfTest extends GridCommonAbstra
         assert it.next() == 9;
         assert it.next() == 10;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/modules/core/src/test/java/org/apache/ignite/lang/utils/GridConcurrentLinkedHashMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/utils/GridConcurrentLinkedHashMapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/lang/utils/GridConcurrentLinkedHashMapSelfTest.java
index a09ba15..7bcbd07 100644
--- a/modules/core/src/test/java/org/apache/ignite/lang/utils/GridConcurrentLinkedHashMapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/lang/utils/GridConcurrentLinkedHashMapSelfTest.java
@@ -19,13 +19,18 @@ package org.apache.ignite.lang.utils;
 
 import java.util.Date;
 import java.util.Enumeration;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jsr166.ConcurrentLinkedHashMap;
 
+import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q;
+import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q_OPTIMIZED_RMV;
+
 /**
  * This class tests basic contracts of {@code ConcurrentLinkedHashMap}.
  */
@@ -264,4 +269,59 @@ public class GridConcurrentLinkedHashMapSelfTest extends GridCommonAbstractTest
 
         assert nextVal == -1 : "Unexpected value: " + nextVal;
     }
-}
\ No newline at end of file
+
+    /**
+     *
+     */
+    public void testIterationInPerSegmentModes() {
+        checkIteration(PER_SEGMENT_Q);
+        checkIteration(PER_SEGMENT_Q_OPTIMIZED_RMV);
+    }
+
+    /**
+     * @param plc Policy.
+     */
+    private void checkIteration(ConcurrentLinkedHashMap.QueuePolicy plc) {
+        ConcurrentLinkedHashMap<Integer, Integer> map =
+            new ConcurrentLinkedHashMap<>(10,
+                0.75f,
+                16,
+                0,
+                plc);
+
+        Map<Integer, Integer> map0 = new HashMap<>();
+
+        int cnt = 0;
+
+        for (int i = 0; i < 100_000; i++) {
+            int key = ThreadLocalRandom.current().nextInt(15000);
+            int val = ThreadLocalRandom.current().nextInt(15000);
+
+            Integer rmv0 = map0.put(key, val);
+
+            if (rmv0 == null)
+                cnt++;
+
+            Integer rmv = map.put(key, val);
+
+            assertEquals(rmv0, rmv);
+        }
+
+        int checkCnt = 0;
+
+        for (Map.Entry<Integer, Integer> e : map.entrySet()) {
+            checkCnt++;
+
+            Integer rmv = map0.remove(e.getKey());
+
+            assertNotNull(rmv);
+            assertEquals(rmv, e.getValue());
+        }
+
+        assertEquals(cnt, checkCnt);
+
+        info("Puts count: " + cnt);
+
+        assert map0.isEmpty() : map0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4848a70/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 71f3ee3..c19e718 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
@@ -95,7 +95,7 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  * Super class for all common tests.
  */
 public abstract class GridCommonAbstractTest extends GridAbstractTest {
-    /**Cache peek modes array that consist of only ONHEAP mode. */
+    /** Cache peek modes array that consist of only ONHEAP mode. */
     protected static final CachePeekMode[] ONHEAP_PEEK_MODES = new CachePeekMode[] {CachePeekMode.ONHEAP};
 
     /**
@@ -1087,4 +1087,4 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             }
         }
     }
-}
\ No newline at end of file
+}


[31/46] ignite git commit: IGNITE-1681: loadAll threshold is not configurable for CacheStoreBalancingWrapper

Posted by ak...@apache.org.
IGNITE-1681: loadAll threshold is not configurable for CacheStoreBalancingWrapper


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

Branch: refs/heads/ignite-1753-1282
Commit: 967cfcbb5b87e172a48e619b18e3988f4ef2e428
Parents: 92881e0
Author: Michael Griggs <en...@gmail.com>
Authored: Wed Nov 11 13:45:48 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Nov 11 13:45:48 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 37 ++++++++++++++
 .../cache/CacheStoreBalancingWrapper.java       |  5 +-
 .../store/GridCacheStoreManagerAdapter.java     |  3 +-
 .../store/GridCacheBalancingStoreSelfTest.java  | 53 +++++++++++++++++---
 4 files changed, 88 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/967cfcbb/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index b7276c9..1b8d41c 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -174,9 +174,15 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default size for onheap SQL row cache size. */
     public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024;
 
+    /** Default threshold for concurrent loading of keys from {@link CacheStore}. */
+    public static final int DFLT_CONCURRENT_LOAD_ALL_THRESHOLD = 5;
+
     /** Cache name. */
     private String name;
 
+    /** Threshold for concurrent loading of keys from {@link CacheStore}. */
+    private int storeConcurrentLoadAllThreshold = DFLT_CONCURRENT_LOAD_ALL_THRESHOLD;
+
     /** Rebalance thread pool size. */
     @Deprecated
     private int rebalancePoolSize = DFLT_REBALANCE_THREAD_POOL_SIZE;
@@ -834,6 +840,37 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * Gets the threshold used in cases when values for multiple keys are being loaded from an underlying
+     * {@link CacheStore} in parallel. In the situation when several threads load the same or intersecting set of keys
+     * and the total number of keys to load is less or equal to this threshold then there will be no a second call to
+     * the storage in order to load a key from thread A if the same key is already being loaded by thread B.
+     *
+     * The threshold should be controlled wisely. On the one hand if it's set to a big value then the interaction with
+     * a storage during the load of missing keys will be minimal. On the other hand the big value may result in
+     * significant performance degradation because it is needed to check for every key whether it's being loaded or not.
+     *
+     * When not set, default value is {@link #DFLT_CONCURRENT_LOAD_ALL_THRESHOLD}.
+     *
+     * @return The concurrent load-all threshold.
+     */
+    public int getStoreConcurrentLoadAllThreshold() {
+        return storeConcurrentLoadAllThreshold;
+    }
+
+    /**
+     * Sets the concurrent load-all threshold used for cases when keys' values are being loaded from {@link CacheStore}
+     * in parallel.
+     *
+     * @param storeConcurrentLoadAllThreshold The concurrent load-all threshold.
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration<K, V> setStoreConcurrentLoadAllThreshold(int storeConcurrentLoadAllThreshold) {
+        this.storeConcurrentLoadAllThreshold = storeConcurrentLoadAllThreshold;
+
+        return this;
+    }
+
+    /**
      * Gets key topology resolver to provide mapping from keys to nodes.
      *
      * @return Key topology resolver to provide mapping from keys to nodes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/967cfcbb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java
index 93075f3..8992326 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java
@@ -28,6 +28,7 @@ import javax.cache.integration.CacheLoaderException;
 import javax.cache.integration.CacheWriterException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteBiInClosure;
@@ -39,7 +40,7 @@ import org.jsr166.ConcurrentHashMap8;
  */
 public class CacheStoreBalancingWrapper<K, V> implements CacheStore<K, V> {
     /** */
-    public static final int DFLT_LOAD_ALL_THRESHOLD = 5;
+    public static final int DFLT_LOAD_ALL_THRESHOLD = CacheConfiguration.DFLT_CONCURRENT_LOAD_ALL_THRESHOLD;
 
     /** Delegate store. */
     private CacheStore<K, V> delegate;
@@ -306,4 +307,4 @@ public class CacheStoreBalancingWrapper<K, V> implements CacheStore<K, V> {
             return get().get(key);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/967cfcbb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index dd54da5..6bfafd4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -112,7 +112,8 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
         store = cacheStoreWrapper(ctx, cfgStore, cfg);
 
-        singleThreadGate = store == null ? null : new CacheStoreBalancingWrapper<>(store);
+        singleThreadGate = store == null ? null : new CacheStoreBalancingWrapper<>(store,
+            cfg.getStoreConcurrentLoadAllThreshold());
 
         ThreadLocal<SessionData> sesHolder0 = null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/967cfcbb/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
index 1e3e4b4..bfbb08c 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
@@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import javax.cache.Cache;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.cache.CacheStoreBalancingWrapper;
@@ -127,15 +128,35 @@ public class GridCacheBalancingStoreSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testConcurrentLoad() throws Exception {
-        int threads = 5;
+        CacheConfiguration cfg = new CacheConfiguration();
 
-        final int keys = 50;
+        assertEquals(CacheStoreBalancingWrapper.DFLT_LOAD_ALL_THRESHOLD, cfg.getStoreConcurrentLoadAllThreshold());
 
+        doTestConcurrentLoad(5, 50, CacheStoreBalancingWrapper.DFLT_LOAD_ALL_THRESHOLD);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentLoadCustomThreshold() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setStoreConcurrentLoadAllThreshold(15);
+
+        assertEquals(15, cfg.getStoreConcurrentLoadAllThreshold());
+
+        doTestConcurrentLoad(5, 50, cfg.getStoreConcurrentLoadAllThreshold());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestConcurrentLoad(int threads, final int keys, int threshold) throws Exception {
         final CyclicBarrier beforeBarrier = new CyclicBarrier(threads);
 
         ConcurrentVerifyStore store = new ConcurrentVerifyStore(keys);
 
-        final CacheStoreBalancingWrapper<Integer, Integer> wrapper =new CacheStoreBalancingWrapper<>(store);
+        final CacheStoreBalancingWrapper<Integer, Integer> wrapper = new CacheStoreBalancingWrapper<>(store, threshold);
 
         GridTestUtils.runMultiThreaded(new Runnable() {
             @Override public void run() {
@@ -159,17 +180,35 @@ public class GridCacheBalancingStoreSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testConcurrentLoadAll() throws Exception {
-        int threads = 5;
+        CacheConfiguration cfg = new CacheConfiguration();
 
-        final int threshold = 5;
+        assertEquals(CacheStoreBalancingWrapper.DFLT_LOAD_ALL_THRESHOLD, cfg.getStoreConcurrentLoadAllThreshold());
 
-        final int keysCnt = 100;
+        doTestConcurrentLoadAll(5, CacheStoreBalancingWrapper.DFLT_LOAD_ALL_THRESHOLD, 150);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentLoadAllCustomThreshold() throws Exception {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setStoreConcurrentLoadAllThreshold(15);
 
+        assertEquals(15, cfg.getStoreConcurrentLoadAllThreshold());
+
+        doTestConcurrentLoadAll(5, cfg.getStoreConcurrentLoadAllThreshold(), 150);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestConcurrentLoadAll(int threads, final int threshold, final int keysCnt) throws Exception {
         final CyclicBarrier beforeBarrier = new CyclicBarrier(threads);
 
         ConcurrentVerifyStore store = new ConcurrentVerifyStore(keysCnt);
 
-        final CacheStoreBalancingWrapper<Integer, Integer> wrapper = new CacheStoreBalancingWrapper<>(store);
+        final CacheStoreBalancingWrapper<Integer, Integer> wrapper = new CacheStoreBalancingWrapper<>(store, threshold);
 
         GridTestUtils.runMultiThreaded(new Runnable() {
             @Override public void run() {


[39/46] ignite git commit: Improve IgnitePutGetTxBenchmark

Posted by ak...@apache.org.
Improve IgnitePutGetTxBenchmark


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

Branch: refs/heads/ignite-1753-1282
Commit: 5a116cb264a8834840fde8e5e8a60d06023d6b1a
Parents: c9cea76
Author: ashutak <as...@gridgain.com>
Authored: Fri Nov 13 16:23:56 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Fri Nov 13 16:23:56 2015 +0300

----------------------------------------------------------------------
 .../config/benchmark-multicast.properties       |  4 +-
 .../yardstick/IgniteBenchmarkArguments.java     |  2 +-
 .../cache/IgnitePutGetTxBenchmark.java          | 41 +++++++++++++++-----
 3 files changed, 35 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5a116cb2/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index c508471..85f6c11 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -90,7 +90,9 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutBenchmark -sn IgniteNode -ds ${ver}atomic-put-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutGetBenchmark -sn IgniteNode -ds ${ver}atomic-put-get-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutTxBenchmark -sn IgniteNode -ds ${ver}tx-put-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-optim-repRead-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc PESSIMISTIC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-pessim-repRead-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -txi SERIALIZABLE -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serial-put-get-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds ${ver}sql-query-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds ${ver}sql-query-join-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds ${ver}sql-query-put-1-backup,\

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a116cb2/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 74b1da9..1ecfa0f 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -62,7 +62,7 @@ public class IgniteBenchmarkArguments {
 
     /** */
     @Parameter(names = {"-txc", "--txConcurrency"}, description = "Transaction concurrency")
-    private TransactionConcurrency txConcurrency = TransactionConcurrency.OPTIMISTIC;
+    private TransactionConcurrency txConcurrency = TransactionConcurrency.PESSIMISTIC;
 
     /** */
     @Parameter(names = {"-txi", "--txIsolation"}, description = "Transaction isolation")

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a116cb2/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
index 5afe0b2..3235721 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
@@ -18,28 +18,49 @@
 package org.apache.ignite.yardstick.cache;
 
 import java.util.Map;
+import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
 
 /**
  * Ignite benchmark that performs transactional put and get operations.
  */
 public class IgnitePutGetTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** */
+    private Callable<Void> clo;
+
     /** {@inheritDoc} */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        int key = nextRandom(0, args.range() / 2);
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        txs = ignite().transactions();
 
-        try (Transaction tx = ignite().transactions().txStart()) {
-            Object val = cache.get(key);
+        clo = new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                int key = nextRandom(0, args.range() / 2);
 
-            if (val != null)
-                key = nextRandom(args.range() / 2, args.range());
+                Object val = cache.get(key);
 
-            cache.put(key, new SampleValue(key));
+                if (val != null)
+                    key = nextRandom(args.range() / 2, args.range());
 
-            tx.commit();
-        }
+                cache.put(key, new SampleValue(key));
+
+                return null;
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        doInTransaction(txs, args.txConcurrency(), args.txIsolation(), clo);
 
         return true;
     }


[13/46] ignite git commit: Ignite-1093 Backward compatibility failover.

Posted by ak...@apache.org.
Ignite-1093 Backward compatibility failover.


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

Branch: refs/heads/ignite-1753-1282
Commit: 6ea3b5cbcdfb97ca07e290924068f692a52679fd
Parents: 4ab2bd3
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Nov 9 15:09:36 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Nov 9 19:04:52 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/preloader/GridDhtPartitionDemander.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ea3b5cb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 35cedf9..29ca5f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -1191,7 +1191,7 @@ public class GridDhtPartitionDemander {
 
                     // While.
                     // =====
-                    while (!topologyChanged(fut)) {
+                    while (!fut.isDone() && !topologyChanged(fut)) {
                         SupplyMessage s = poll(msgQ, timeout);
 
                         // If timed out.
@@ -1350,7 +1350,7 @@ public class GridDhtPartitionDemander {
                         }
                     }
                 }
-                while (retry && !topologyChanged(fut));
+                while (retry && !fut.isDone() && !topologyChanged(fut));
             }
             finally {
                 cctx.io().removeOrderedHandler(d.topic());


[25/46] ignite git commit: Warning is writeThrough==false && writeBehind==true

Posted by ak...@apache.org.
Warning is writeThrough==false && writeBehind==true


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

Branch: refs/heads/ignite-1753-1282
Commit: 50ede23a1079271cd21c338968bf936623416a16
Parents: 3de9d47
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Nov 10 16:44:27 2015 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Nov 10 16:44:27 2015 -0800

----------------------------------------------------------------------
 .../store/GridCacheStoreManagerAdapter.java     | 26 ++++++++++++++------
 1 file changed, 19 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/50ede23a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 174e1ce..dd54da5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -188,12 +188,24 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
         CacheConfiguration cfg = cctx.config();
 
-        if (cfgStore != null && !cfg.isWriteThrough() && !cfg.isReadThrough()) {
-            U.quietAndWarn(log,
-                "Persistence store is configured, but both read-through and write-through are disabled. This " +
-                "configuration makes sense if the store implements loadCache method only. If this is the " +
-                "case, ignore this warning. Otherwise, fix the configuration for cache: " + cfg.getName(),
-                "Persistence store is configured, but both read-through and write-through are disabled.");
+        if (cfgStore != null) {
+            if (!cfg.isWriteThrough() && !cfg.isReadThrough()) {
+                U.quietAndWarn(log,
+                    "Persistence store is configured, but both read-through and write-through are disabled. This " +
+                    "configuration makes sense if the store implements loadCache method only. If this is the " +
+                    "case, ignore this warning. Otherwise, fix the configuration for the cache: " + cfg.getName(),
+                    "Persistence store is configured, but both read-through and write-through are disabled " +
+                    "for cache: " + cfg.getName());
+            }
+
+            if (!cfg.isWriteThrough() && cfg.isWriteBehindEnabled()) {
+                U.quietAndWarn(log,
+                    "To enable write-behind mode for the cache store it's also required to set " +
+                    "CacheConfiguration.setWriteThrough(true) property, otherwise the persistence " +
+                    "store will be never updated. Consider fixing configuration for the cache: " + cfg.getName(),
+                    "Write-behind mode for the cache store also requires CacheConfiguration.setWriteThrough(true) " +
+                    "property. Fix configuration for the cache: " + cfg.getName());
+            }
         }
 
         sesLsnrs = CU.startStoreSessionListeners(cctx.kernalContext(), cfg.getCacheStoreSessionListenerFactories());
@@ -1256,4 +1268,4 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             }
         }
     }
-}
\ No newline at end of file
+}


[28/46] ignite git commit: Disabled failing tests.

Posted by ak...@apache.org.
Disabled failing tests.


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

Branch: refs/heads/ignite-1753-1282
Commit: d567d2595cc95c5a27e43d1331b9dd523dc67be7
Parents: 3de9d47
Author: sboikov <sb...@gridgain.com>
Authored: Wed Nov 11 10:19:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Nov 11 10:19:48 2015 +0300

----------------------------------------------------------------------
 .../dht/IgniteCachePutRetryTransactionalSelfTest.java     | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d567d259/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index f61faf2..7655464 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -88,6 +88,16 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void testGetAndPut() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1525");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testInvoke() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1525");
+    }
+
     /**
      * @throws Exception If failed.
      */


[07/46] ignite git commit: IGNITE-1168 REST return sql metadata for all caches in one request. - Fixes #181.

Posted by ak...@apache.org.
IGNITE-1168 REST return sql metadata for all caches in one request. - Fixes #181.

Signed-off-by: Andrey <an...@gridgain.com>


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

Branch: refs/heads/ignite-1753-1282
Commit: 354358cc4ec9a8f15e75d132c7a57d2361b5722b
Parents: 621ecac
Author: Andrey <an...@gridgain.com>
Authored: Mon Nov 9 10:41:04 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Mon Nov 9 10:41:04 2015 +0700

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     | 105 ++++++++-------
 .../discovery/GridDiscoveryManager.java         |   5 +-
 .../handlers/cache/GridCacheCommandHandler.java | 128 +++++++++++++++----
 .../top/GridTopologyCommandHandler.java         |   4 -
 4 files changed, 166 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/354358cc/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index c9c4ced..4c73f78 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -40,15 +40,14 @@ import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlIndexMetadata;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata;
 import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandler;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -919,14 +918,16 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
     }
 
     /**
-     * @param meta Metadata for Ignite cache.
+     * @param metas Metadata for Ignite caches.
      * @throws Exception If failed.
      */
-    private void testMetadata(GridCacheSqlMetadata meta) throws Exception {
+    private void testMetadata(Collection<GridCacheSqlMetadata> metas) throws Exception {
         Map<String, String> params = F.asMap("cmd", GridRestCommand.CACHE_METADATA.key());
 
-        if (meta.cacheName() != null)
-            params.put("cacheName", meta.cacheName());
+        String cacheNameArg = F.first(metas).cacheName();
+
+        if (cacheNameArg != null)
+            params.put("cacheName", cacheNameArg);
 
         String ret = content(params);
 
@@ -935,53 +936,68 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         info("Cache metadata result: " + ret);
 
-        jsonEquals(ret, pattern("\\{.+\\}", true));
+        jsonEquals(ret, pattern("\\[.+\\]", true));
+
+        Collection<Map> results = (Collection)JSONObject.fromObject(ret).get("response");
+
+        assertEquals(metas.size(), results.size());
+        assertEquals(cacheNameArg, F.first(results).get("cacheName"));
+
+        for (Map res : results) {
+            final Object cacheName = res.get("cacheName");
 
-        Map res = (Map)JSONObject.fromObject(ret).get("response");
+            GridCacheSqlMetadata meta = F.find(metas, null, new P1<GridCacheSqlMetadata>() {
+                @Override public boolean apply(GridCacheSqlMetadata meta) {
+                    return F.eq(meta.cacheName(), cacheName);
+                }
+            });
 
-        Collection types = (Collection)res.get("types");
+            assertNotNull("REST return metadata for unexpected cache: " + cacheName, meta);
 
-        assertNotNull(types);
-        assertEqualsCollections(meta.types(), types);
+            Collection types = (Collection)res.get("types");
 
-        Map keyClasses = (Map)res.get("keyClasses");
+            assertNotNull(types);
+            assertEqualsCollections(meta.types(), types);
 
-        assertNotNull(keyClasses);
-        assertTrue(meta.keyClasses().equals(keyClasses));
+            Map keyClasses = (Map)res.get("keyClasses");
 
-        Map valClasses = (Map)res.get("valClasses");
+            assertNotNull(keyClasses);
+            assertTrue(meta.keyClasses().equals(keyClasses));
 
-        assertNotNull(valClasses);
-        assertTrue(meta.valClasses().equals(valClasses));
+            Map valClasses = (Map)res.get("valClasses");
 
-        Map fields = (Map)res.get("fields");
+            assertNotNull(valClasses);
+            assertTrue(meta.valClasses().equals(valClasses));
 
-        assertNotNull(fields);
-        assertTrue(meta.fields().equals(fields));
+            Map fields = (Map)res.get("fields");
 
-        Map indexesByType = (Map)res.get("indexes");
+            assertNotNull(fields);
+            assertTrue(meta.fields().equals(fields));
 
-        assertNotNull(indexesByType);
-        assertEquals(meta.indexes().size(), indexesByType.size());
+            Map indexesByType = (Map)res.get("indexes");
 
-        for (Map.Entry<String, Collection<GridCacheSqlIndexMetadata>> metaIndexes : meta.indexes().entrySet()) {
-            Collection<Map> indexes = (Collection<Map>)indexesByType.get(metaIndexes.getKey());
+            assertNotNull(indexesByType);
+            assertEquals(meta.indexes().size(), indexesByType.size());
 
-            assertNotNull(indexes);
-            assertEquals(metaIndexes.getValue().size(), indexes.size());
+            for (Map.Entry<String, Collection<GridCacheSqlIndexMetadata>> metaIndexes : meta.indexes().entrySet()) {
+                Collection<Map> indexes = (Collection<Map>)indexesByType.get(metaIndexes.getKey());
 
-            for (final GridCacheSqlIndexMetadata metaIdx : metaIndexes.getValue()) {
-                Map idx = F.find(indexes, null, new IgnitePredicate<Map>() {
-                    @Override public boolean apply(Map map) {
-                        return metaIdx.name().equals(map.get("name"));
-                    }
-                });
+                assertNotNull(indexes);
+                assertEquals(metaIndexes.getValue().size(), indexes.size());
 
-                assertNotNull(idx);
+                for (final GridCacheSqlIndexMetadata metaIdx : metaIndexes.getValue()) {
+                    Map idx = F.find(indexes, null, new P1<Map>() {
+                        @Override public boolean apply(Map map) {
+                            return metaIdx.name().equals(map.get("name"));
+                        }
+                    });
 
-                assertEqualsCollections(metaIdx.fields(), (Collection)idx.get("fields"));
-                assertEqualsCollections(metaIdx.descendings(), (Collection)idx.get("descendings"));
-                assertEquals(metaIdx.unique(), idx.get("unique"));
+                    assertNotNull(idx);
+
+                    assertEqualsCollections(metaIdx.fields(), (Collection)idx.get("fields"));
+                    assertEqualsCollections(metaIdx.descendings(), (Collection)idx.get("descendings"));
+                    assertEquals(metaIdx.unique(), idx.get("unique"));
+                }
             }
         }
     }
@@ -990,16 +1006,13 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
      * @throws Exception If failed.
      */
     public void testMetadataLocal() throws Exception {
-        GridCacheProcessor cacheProc = grid(0).context().cache();
+        IgniteCacheProxy<?, ?> cache = F.first(grid(0).context().cache().publicCaches());
 
-        for (IgniteInternalCache<?, ?> cache : cacheProc.caches()) {
-            if (CU.isSystemCache(cache.name()))
-                continue;
+        assertNotNull("Should have configured public cache!", cache);
 
-            GridCacheSqlMetadata meta = F.first(cache.context().queries().sqlMetadata());
+        Collection<GridCacheSqlMetadata> meta = cache.context().queries().sqlMetadata();
 
-            testMetadata(meta);
-        }
+        testMetadata(meta);
     }
 
     /**
@@ -1013,9 +1026,9 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         IgniteCacheProxy<Integer, String> c = (IgniteCacheProxy<Integer, String>)grid(1).createCache(partialCacheCfg);
 
-        GridCacheSqlMetadata meta = F.first(c.context().queries().sqlMetadata());
+        Collection<GridCacheSqlMetadata> metas = c.context().queries().sqlMetadata();
 
-        testMetadata(meta);
+        testMetadata(metas);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/354358cc/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 6aba211..cd2f49c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -1594,7 +1594,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param node Node to check.
-     * @return Cache names accessible on the given node.
+     * @return Public cache names accessible on the given node.
      */
     public Map<String, CacheMode> nodeCaches(ClusterNode node) {
         Map<String, CacheMode> caches = U.newHashMap(registeredCaches.size());
@@ -1604,7 +1604,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             CachePredicate pred = entry.getValue();
 
-            if (pred != null && pred.cacheNode(node))
+            if (!CU.isSystemCache(cacheName) && !CU.isIgfsCache(ctx.config(), cacheName) &&
+                pred != null && pred.cacheNode(node))
                 caches.put(cacheName, pred.cacheMode);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/354358cc/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 1bbc754..65dca08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -18,10 +18,13 @@
 package org.apache.ignite.internal.processors.rest.handlers.cache;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -35,10 +38,18 @@ import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.compute.ComputeJob;
+import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.compute.ComputeTaskAdapter;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
@@ -370,20 +381,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                 }
 
                 case CACHE_METADATA: {
-                    IgniteInternalCache<?, ?> cache = ctx.cache().cache(cacheName);
-
-                    if (cache != null) {
-                        GridCacheSqlMetadata res = F.first(cache.context().queries().sqlMetadata());
-
-                        fut = new GridFinishedFuture<>(new GridRestResponse(res));
-                    }
-                    else {
-                        ClusterGroup prj = ctx.grid().cluster().forDataNodes(cacheName);
-
-                        ctx.task().setThreadContext(TC_NO_FAILOVER, true);
-
-                        fut = ctx.closure().callAsync(BALANCE, new MetadataCommand(cacheName), prj.nodes());
-                    }
+                    fut = ctx.task().execute(MetadataTask.class, cacheName);
 
                     break;
                 }
@@ -903,27 +901,109 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
     }
 
     /** */
-    private static class MetadataCommand implements Callable<GridRestResponse>, Serializable {
+    @GridInternal
+    private static class MetadataTask extends ComputeTaskAdapter<String, GridRestResponse> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** */
-        private final String cacheName;
+        @IgniteInstanceResource
+        private IgniteEx ignite;
 
         /** */
-        @IgniteInstanceResource
-        private Ignite g;
+        private String cacheName;
 
-        /**
-         * @param cacheName Cache name.
-         */
-        private MetadataCommand(String cacheName) {
+        /** {@inheritDoc} */
+        @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+            @Nullable String cacheName) throws IgniteException {
             this.cacheName = cacheName;
+
+            GridDiscoveryManager discovery = ignite.context().discovery();
+
+            boolean sameCaches = true;
+
+            int hash = discovery.nodeCaches(F.first(subgrid)).hashCode();
+
+            for (int i = 1; i < subgrid.size(); i++) {
+                if (hash != discovery.nodeCaches(subgrid.get(i)).hashCode()) {
+                    sameCaches = false;
+
+                    break;
+                }
+            }
+
+            Map<ComputeJob, ClusterNode> map = U.newHashMap(sameCaches ? 1 : subgrid.size());
+
+            if (sameCaches)
+                map.put(new MetadataJob(), ignite.localNode());
+            else {
+                for (ClusterNode node : subgrid)
+                    map.put(new MetadataJob(), node);
+            }
+
+            return map;
         }
 
         /** {@inheritDoc} */
-        @Override public GridRestResponse call() throws Exception {
-            return  new GridRestResponse(F.first(cache(g, cacheName).context().queries().sqlMetadata()));
+        @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+        @Nullable @Override public GridRestResponse reduce(List<ComputeJobResult> results) throws IgniteException {
+            Map<String, GridCacheSqlMetadata> map = new HashMap<>();
+
+            for (ComputeJobResult r : results) {
+                if (!r.isCancelled() && r.getException() == null) {
+                    for (GridCacheSqlMetadata m : r.<Collection<GridCacheSqlMetadata>>getData()) {
+                        if (!map.containsKey(m.cacheName()))
+                            map.put(m.cacheName(), m);
+                    }
+                }
+            }
+
+            Collection<GridCacheSqlMetadata> metas = new ArrayList<>(map.size());
+
+            // Metadata for current cache must be first in list.
+            GridCacheSqlMetadata cacheMeta = map.remove(cacheName);
+
+            if (cacheMeta != null)
+                metas.add(cacheMeta);
+
+            metas.addAll(map.values());
+
+            return new GridRestResponse(metas);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(MetadataTask.class, this);
+        }
+    }
+
+    /** */
+    private static class MetadataJob extends ComputeJobAdapter {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Auto-injected grid instance. */
+        @IgniteInstanceResource
+        private transient IgniteEx ignite;
+
+        /** {@inheritDoc} */
+        @Override public Collection<GridCacheSqlMetadata> execute() {
+            IgniteCacheProxy<?, ?> cache = F.first(ignite.context().cache().publicCaches());
+
+            if (cache == null)
+                return Collections.emptyList();
+
+            try {
+                return cache.context().queries().sqlMetadata();
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(MetadataJob.class, this);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/354358cc/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
index eea29d2..f950ac2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
@@ -44,7 +44,6 @@ import org.apache.ignite.internal.processors.rest.request.GridRestTopologyReques
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.IgnitePortProtocol;
@@ -202,9 +201,6 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
         for (Map.Entry<String, CacheMode> cache : nodeCaches.entrySet()) {
             String cacheName = cache.getKey();
 
-            if (CU.isSystemCache(cacheName) || CU.isIgfsCache(ctx.config(), cacheName))
-                continue;
-
             String mode = cache.getValue().toString();
 
             if (cacheName != null)


[40/46] ignite git commit: ignite-1758 Discovery fixes

Posted by ak...@apache.org.
ignite-1758 Discovery fixes


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

Branch: refs/heads/ignite-1753-1282
Commit: 80147128a3b07f927dec65f0a6934f6782efab5c
Parents: 5a116cb
Author: sboikov <sb...@gridgain.com>
Authored: Tue Nov 17 09:48:58 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Nov 17 09:48:58 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 360 +++++++++++----
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   4 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |  95 ++--
 .../messages/TcpDiscoveryAbstractMessage.java   |  37 ++
 .../TcpDiscoveryStatusCheckMessage.java         |  11 +
 .../tcp/TcpDiscoveryMultiThreadedTest.java      | 158 ++++---
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |  10 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 459 ++++++++++++++++++-
 .../TcpDiscoveryMulticastIpFinderSelfTest.java  |  28 +-
 .../testframework/junits/GridAbstractTest.java  |  29 +-
 10 files changed, 942 insertions(+), 249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/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 0fe2881..ae23d0e 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
@@ -159,6 +159,10 @@ class ServerImpl extends TcpDiscoveryImpl {
     private static final int ENSURED_MSG_HIST_SIZE = getInteger(IGNITE_DISCOVERY_HISTORY_SIZE, 1024 * 10);
 
     /** */
+    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>());
 
@@ -191,10 +195,10 @@ class ServerImpl extends TcpDiscoveryImpl {
     private StatisticsPrinter statsPrinter;
 
     /** Failed nodes (but still in topology). */
-    private Collection<TcpDiscoveryNode> failedNodes = new HashSet<>();
+    private final Collection<TcpDiscoveryNode> failedNodes = new HashSet<>();
 
     /** Leaving nodes (but still in topology). */
-    private Collection<TcpDiscoveryNode> leavingNodes = new HashSet<>();
+    private final Collection<TcpDiscoveryNode> leavingNodes = new HashSet<>();
 
     /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */
     private boolean ipFinderHasLocAddr;
@@ -1080,13 +1084,34 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 openSock = true;
 
+                TcpDiscoveryHandshakeRequest req = new TcpDiscoveryHandshakeRequest(locNodeId);
+
                 // Handshake.
-                spi.writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId), timeoutHelper.nextTimeoutChunk(
-                    spi.getSocketTimeout()));
+                spi.writeToSocket(sock, req, timeoutHelper.nextTimeoutChunk(spi.getSocketTimeout()));
 
                 TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, timeoutHelper.nextTimeoutChunk(
                     ackTimeout0));
 
+                if (msg instanceof TcpDiscoveryJoinRequestMessage) {
+                    boolean ignore = false;
+
+                    synchronized (failedNodes) {
+                        for (TcpDiscoveryNode failedNode : failedNodes) {
+                            if (failedNode.id().equals(res.creatorNodeId())) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Ignore response from node from failed list: " + res);
+
+                                ignore = true;
+
+                                break;
+                            }
+                        }
+                    }
+
+                    if (ignore)
+                        break;
+                }
+
                 if (locNodeId.equals(res.creatorNodeId())) {
                     if (log.isDebugEnabled())
                         log.debug("Handshake response from local node: " + res);
@@ -1104,7 +1129,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 spi.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
 
                 if (debugMode)
-                    debugLog("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
+                    debugLog(msg, "Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
                         ", rmtNodeId=" + res.creatorNodeId() + ']');
 
                 if (log.isDebugEnabled())
@@ -1754,6 +1779,32 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /**
+     * Adds failed nodes specified in the received message to the local failed nodes list.
+     *
+     * @param msg Message.
+     */
+    private void processMessageFailedNodes(TcpDiscoveryAbstractMessage msg) {
+        if (msg.failedNodes() != null) {
+            for (UUID nodeId : msg.failedNodes()) {
+                TcpDiscoveryNode failedNode = ring.node(nodeId);
+
+                if (failedNode != null) {
+                    if (!failedNode.isLocal()) {
+                        boolean added;
+
+                        synchronized (mux) {
+                            added = failedNodes.add(failedNode);
+                        }
+
+                        if (added && log.isDebugEnabled())
+                            log.debug("Added node to failed nodes list [node=" + failedNode + ", msg=" + msg + ']');
+                    }
+                }
+            }
+        }
+    }
+
+    /**
      * Discovery messages history used for client reconnect.
      */
     private class EnsuredMessageHistory {
@@ -2131,10 +2182,28 @@ class ServerImpl extends TcpDiscoveryImpl {
                 log.debug("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
 
             if (debugMode)
-                debugLog("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
+                debugLog(msg, "Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
+
+            if (locNode.internalOrder() == 0) {
+                boolean process = false;
+
+                if (msg instanceof TcpDiscoveryNodeAddedMessage)
+                    process = ((TcpDiscoveryNodeAddedMessage)msg).node().equals(locNode);
+
+                if (!process) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Ignore message, local node order is not initialized [msg=" + msg +
+                            ", locNode=" + locNode + ']');
+                    }
+
+                    return;
+                }
+            }
 
             spi.stats.onMessageProcessingStarted(msg);
 
+            processMessageFailedNodes(msg);
+
             if (msg instanceof TcpDiscoveryJoinRequestMessage)
                 processJoinRequestMessage((TcpDiscoveryJoinRequestMessage)msg);
 
@@ -2200,6 +2269,8 @@ class ServerImpl extends TcpDiscoveryImpl {
             checkHeartbeatsReceiving();
 
             checkPendingCustomMessages();
+
+            checkFailedNodesList();
         }
 
         /**
@@ -2262,50 +2333,50 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             boolean sent = false;
 
-            boolean searchNext = true;
+            boolean newNextNode = false;
 
             UUID locNodeId = getLocalNodeId();
 
             while (true) {
-                if (searchNext) {
-                    TcpDiscoveryNode newNext = ring.nextNode(failedNodes);
-
-                    if (newNext == null) {
-                        if (log.isDebugEnabled())
-                            log.debug("No next node in topology.");
+                TcpDiscoveryNode newNext = ring.nextNode(failedNodes);
 
-                        if (debugMode)
-                            debugLog("No next node in topology.");
+                if (newNext == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("No next node in topology.");
 
-                        if (ring.hasRemoteNodes() && !(msg instanceof TcpDiscoveryConnectionCheckMessage) &&
-                            !(msg instanceof TcpDiscoveryStatusCheckMessage && msg.creatorNodeId().equals(locNodeId))) {
-                            msg.senderNodeId(locNodeId);
+                    if (debugMode)
+                        debugLog(msg, "No next node in topology.");
 
-                            addMessage(msg);
-                        }
+                    if (ring.hasRemoteNodes() && !(msg instanceof TcpDiscoveryConnectionCheckMessage) &&
+                        !(msg instanceof TcpDiscoveryStatusCheckMessage && msg.creatorNodeId().equals(locNodeId))) {
+                        msg.senderNodeId(locNodeId);
 
-                        break;
+                        addMessage(msg);
                     }
 
-                    if (!newNext.equals(next)) {
-                        if (log.isDebugEnabled())
-                            log.debug("New next node [newNext=" + newNext + ", formerNext=" + next +
-                                ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
+                    break;
+                }
 
-                        if (debugMode)
-                            debugLog("New next node [newNext=" + newNext + ", formerNext=" + next +
-                                ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
+                if (!newNext.equals(next)) {
+                    if (log.isDebugEnabled())
+                        log.debug("New next node [newNext=" + newNext + ", formerNext=" + next +
+                            ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
 
-                        U.closeQuiet(sock);
+                    if (debugMode)
+                        debugLog(msg, "New next node [newNext=" + newNext + ", formerNext=" + next +
+                            ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
 
-                        sock = null;
+                    U.closeQuiet(sock);
 
-                        next = newNext;
-                    }
-                    else if (log.isDebugEnabled())
-                        log.debug("Next node remains the same [nextId=" + next.id() +
-                            ", nextOrder=" + next.internalOrder() + ']');
+                    sock = null;
+
+                    next = newNext;
+
+                    newNextNode = true;
                 }
+                else if (log.isDebugEnabled())
+                    log.debug("Next node remains the same [nextId=" + next.id() +
+                        ", nextOrder=" + next.internalOrder() + ']');
 
                 // Flag that shows whether next node exists and accepts incoming connections.
                 boolean nextNodeExists = sock != null;
@@ -2379,8 +2450,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                                             "expected [expectedId=" + next.id() + ", rcvdId=" + nextId + ']');
 
                                     if (debugMode)
-                                        debugLog("Failed to restore ring because next node ID received is not as " +
-                                            "expected [expectedId=" + next.id() + ", rcvdId=" + nextId + ']');
+                                        debugLog(msg, "Failed to restore ring because next node ID received is not " +
+                                            "as expected [expectedId=" + next.id() + ", rcvdId=" + nextId + ']');
 
                                     break;
                                 }
@@ -2401,8 +2472,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                                                     ", rcvd=" + nextOrder + ", id=" + next.id() + ']');
 
                                             if (debugMode)
-                                                debugLog("Failed to restore ring because next node order received " +
-                                                    "is not as expected [expected=" + next.internalOrder() +
+                                                debugLog(msg, "Failed to restore ring because next node order " +
+                                                    "received is not as expected [expected=" + next.internalOrder() +
                                                     ", rcvd=" + nextOrder + ", id=" + next.id() + ']');
 
                                             break;
@@ -2413,7 +2484,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                         log.debug("Initialized connection with next node: " + next.id());
 
                                     if (debugMode)
-                                        debugLog("Initialized connection with next node: " + next.id());
+                                        debugLog(msg, "Initialized connection with next node: " + next.id());
 
                                     errs = null;
 
@@ -2477,13 +2548,20 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                             assert !forceSndPending || msg instanceof TcpDiscoveryNodeLeftMessage;
 
-                            if (failure || forceSndPending) {
+                            boolean sndPending=
+                                (newNextNode && ring.minimumNodeVersion().compareTo(CUSTOM_MSG_ALLOW_JOINING_FOR_VERIFIED_SINCE) >= 0) ||
+                                failure ||
+                                forceSndPending;
+
+                            if (sndPending) {
                                 if (log.isDebugEnabled())
                                     log.debug("Pending messages will be sent [failure=" + failure +
+                                        ", newNextNode=" + newNextNode +
                                         ", forceSndPending=" + forceSndPending + ']');
 
                                 if (debugMode)
-                                    debugLog("Pending messages will be sent [failure=" + failure +
+                                    debugLog(msg, "Pending messages will be sent [failure=" + failure +
+                                        ", newNextNode=" + newNextNode +
                                         ", forceSndPending=" + forceSndPending + ']');
 
                                 for (TcpDiscoveryAbstractMessage pendingMsg : pendingMsgs) {
@@ -2513,7 +2591,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                             ", res=" + res + ']');
 
                                     if (debugMode)
-                                        debugLog("Pending message has been sent to next node [msgId=" + msg.id() +
+                                        debugLog(msg, "Pending message has been sent to next node [msgId=" + msg.id() +
                                             ", pendingMsgId=" + pendingMsg.id() + ", next=" + next.id() +
                                             ", res=" + res + ']');
 
@@ -2540,6 +2618,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                                 if (timeoutHelper == null)
                                     timeoutHelper = new IgniteSpiOperationTimeoutHelper(spi);
 
+                                if (!failedNodes.isEmpty()) {
+                                    for (TcpDiscoveryNode failedNode : failedNodes) {
+                                        assert !failedNode.equals(next) : failedNode;
+
+                                        msg.addFailedNode(failedNode.id());
+                                    }
+                                }
+
                                 writeToSocket(sock, msg, timeoutHelper.nextTimeoutChunk(spi.getSocketTimeout()));
 
                                 spi.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
@@ -2548,15 +2634,17 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                 onMessageExchanged();
 
-                                if (log.isDebugEnabled())
+                                if (log.isDebugEnabled()) {
                                     log.debug("Message has been sent to next node [msg=" + msg +
                                         ", next=" + next.id() +
                                         ", res=" + res + ']');
+                                }
 
-                                if (debugMode)
-                                    debugLog("Message has been sent to next node [msg=" + msg +
+                                if (debugMode) {
+                                    debugLog(msg, "Message has been sent to next node [msg=" + msg +
                                         ", next=" + next.id() +
                                         ", res=" + res + ']');
+                                }
                             }
                             finally {
                                 clearNodeAddedMessage(msg);
@@ -2635,8 +2723,6 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     next = null;
 
-                    searchNext = true;
-
                     errs = null;
                 }
                 else
@@ -2665,25 +2751,30 @@ class ServerImpl extends TcpDiscoveryImpl {
                     msgWorker.addMessage(new TcpDiscoveryNodeFailedMessage(locNodeId, n.id(), n.internalOrder()));
 
                 if (!sent) {
+                    assert next == null : next;
+
                     if (log.isDebugEnabled())
                         log.debug("Pending messages will be resent to local node");
 
                     if (debugMode)
-                        log.debug("Pending messages will be resent to local node");
+                        debugLog(msg, "Pending messages will be resent to local node");
 
                     for (TcpDiscoveryAbstractMessage pendingMsg : pendingMsgs) {
                         prepareNodeAddedMessage(pendingMsg, locNodeId, pendingMsgs.msgs, pendingMsgs.discardId,
                             pendingMsgs.customDiscardId);
 
+                        pendingMsg.senderNodeId(locNodeId);
+
                         msgWorker.addMessage(pendingMsg);
 
                         if (log.isDebugEnabled())
                             log.debug("Pending message has been sent to local node [msg=" + msg.id() +
-                                ", pendingMsgId=" + pendingMsg + ", next=" + next.id() + ']');
+                                ", pendingMsgId=" + pendingMsg + ']');
 
-                        if (debugMode)
-                            debugLog("Pending message has been sent to local node [msg=" + msg.id() +
-                                ", pendingMsgId=" + pendingMsg + ", next=" + next.id() + ']');
+                        if (debugMode) {
+                            debugLog(msg, "Pending message has been sent to local node [msg=" + msg.id() +
+                                ", pendingMsgId=" + pendingMsg + ']');
+                        }
                     }
                 }
 
@@ -3317,15 +3408,17 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (sendMessageToRemotes(msg))
                     sendMessageAcrossRing(msg);
 
-                if (log.isDebugEnabled())
+                if (log.isDebugEnabled()) {
                     log.debug("Local node already has node being added. Passing TcpDiscoveryNodeAddedMessage to " +
-                                  "coordinator for final processing [ring=" + ring + ", node=" + node + ", locNode="
-                                  + locNode + ", msg=" + msg + ']');
+                        "coordinator for final processing [ring=" + ring + ", node=" + node + ", locNode="
+                        + locNode + ", msg=" + msg + ']');
+                }
 
-                if (debugMode)
-                    debugLog("Local node already has node being added. Passing TcpDiscoveryNodeAddedMessage to " +
-                                 "coordinator for final processing [ring=" + ring + ", node=" + node + ", locNode="
-                                 + locNode + ", msg=" + msg + ']');
+                if (debugMode) {
+                    debugLog(msg, "Local node already has node being added. Passing TcpDiscoveryNodeAddedMessage to " +
+                        "coordinator for final processing [ring=" + ring + ", node=" + node + ", locNode="
+                        + locNode + ", msg=" + msg + ']');
+                }
 
                 return;
             }
@@ -3338,7 +3431,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             ", msg=" + msg + ']');
 
                     if (debugMode)
-                        debugLog("Discarding node added message since new node's order is less than " +
+                        debugLog(msg, "Discarding node added message since new node's order is less than " +
                             "max order in ring [ring=" + ring + ", node=" + node + ", locNode=" + locNode +
                             ", msg=" + msg + ']');
 
@@ -3427,6 +3520,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                         spi.onExchange(node.id(), node.id(), data, U.gridClassLoader());
 
                     msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id()));
+
+                    processMessageFailedNodes(msg);
                 }
 
                 if (log.isDebugEnabled())
@@ -3447,6 +3542,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                             spi.gridStartTime = msg.gridStartTime();
 
                             for (TcpDiscoveryNode n : top) {
+                                assert n.internalOrder() < node.internalOrder() :
+                                    "Invalid node [topNode=" + n + ", added=" + node + ']';
+
                                 // Make all preceding nodes and local node visible.
                                 n.visible(true);
                             }
@@ -3500,6 +3598,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                     for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
                         spi.onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader());
                 }
+
+                processMessageFailedNodes(msg);
             }
 
             if (sendMessageToRemotes(msg))
@@ -3733,7 +3833,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 interruptPing(leavingNode);
 
-                assert leftNode != null;
+                assert leftNode != null : msg;
 
                 if (log.isDebugEnabled())
                     log.debug("Removed node from topology: " + leftNode);
@@ -3887,6 +3987,8 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
 
             if (node != null) {
+                assert !node.isLocal() : msg;
+
                 synchronized (mux) {
                     failedNodes.add(node);
                 }
@@ -4036,32 +4138,46 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     return;
                                 }
 
+                                TcpDiscoveryStatusCheckMessage msg0 = msg;
+
+                                if (F.contains(msg.failedNodes(), msg.creatorNodeId())) {
+                                    msg0 = new TcpDiscoveryStatusCheckMessage(msg);
+
+                                    msg0.failedNodes(null);
+
+                                    for (UUID failedNodeId : msg.failedNodes()) {
+                                        if (!failedNodeId.equals(msg.creatorNodeId()))
+                                            msg0.addFailedNode(failedNodeId);
+                                    }
+                                }
+
                                 try {
-                                    trySendMessageDirectly(msg.creatorNode(), msg);
+                                    trySendMessageDirectly(msg0.creatorNode(), msg0);
 
                                     if (log.isDebugEnabled())
                                         log.debug("Responded to status check message " +
-                                            "[recipient=" + msg.creatorNodeId() + ", status=" + msg.status() + ']');
+                                            "[recipient=" + msg0.creatorNodeId() + ", status=" + msg0.status() + ']');
                                 }
                                 catch (IgniteSpiException e) {
                                     if (e.hasCause(SocketException.class)) {
                                         if (log.isDebugEnabled())
                                             log.debug("Failed to respond to status check message (connection " +
-                                                "refused) [recipient=" + msg.creatorNodeId() + ", status=" +
-                                                msg.status() + ']');
+                                                "refused) [recipient=" + msg0.creatorNodeId() + ", status=" +
+                                                msg0.status() + ']');
 
                                         onException("Failed to respond to status check message (connection refused) " +
-                                            "[recipient=" + msg.creatorNodeId() + ", status=" + msg.status() + ']', e);
+                                            "[recipient=" + msg0.creatorNodeId() + ", status=" + msg0.status() + ']', e);
                                     }
-                                    else {
-                                        if (pingNode(msg.creatorNode()))
+                                    else if (!spi.isNodeStopping0()) {
+                                        if (pingNode(msg0.creatorNode()))
                                             // Node exists and accepts incoming connections.
                                             U.error(log, "Failed to respond to status check message [recipient=" +
-                                                msg.creatorNodeId() + ", status=" + msg.status() + ']', e);
-                                        else if (log.isDebugEnabled())
-                                            log.debug("Failed to respond to status check message (did the node " +
-                                                "stop?) [recipient=" + msg.creatorNodeId() + ", status=" + msg.status()
-                                                + ']');
+                                                msg0.creatorNodeId() + ", status=" + msg0.status() + ']', e);
+                                        else if (log.isDebugEnabled()) {
+                                            log.debug("Failed to respond to status check message (did the node stop?)" +
+                                                "[recipient=" + msg0.creatorNodeId() +
+                                                ", status=" + msg0.status() + ']');
+                                        }
                                     }
                                 }
                             }
@@ -4364,27 +4480,42 @@ class ServerImpl extends TcpDiscoveryImpl {
          */
         private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) {
             if (isLocalNodeCoordinator()) {
-                if (!joiningNodes.isEmpty()) {
+                boolean delayMsg;
+
+                assert ring.minimumNodeVersion() != null : ring;
+
+                if (ring.minimumNodeVersion().compareTo(CUSTOM_MSG_ALLOW_JOINING_FOR_VERIFIED_SINCE) >= 0)
+                    delayMsg = msg.topologyVersion() == 0L && !joiningNodes.isEmpty();
+                else
+                    delayMsg = !joiningNodes.isEmpty();
+
+                if (delayMsg) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Delay custom message processing, there are joining nodes [msg=" + msg +
+                            ", joiningNodes=" + joiningNodes + ']');
+                    }
+
                     pendingCustomMsgs.add(msg);
 
                     return;
                 }
 
-                boolean sndNext = !msg.verified();
-
-                if (sndNext) {
+                if (!msg.verified()) {
                     msg.verify(getLocalNodeId());
                     msg.topologyVersion(ring.topologyVersion());
 
-                    if (pendingMsgs.procCustomMsgs.add(msg.id()))
+                    if (pendingMsgs.procCustomMsgs.add(msg.id())) {
                         notifyDiscoveryListener(msg);
-                    else
-                        sndNext = false;
-                }
 
-                if (sndNext && ring.hasRemoteNodes())
-                    sendMessageAcrossRing(msg);
+                        if (sendMessageToRemotes(msg))
+                            sendMessageAcrossRing(msg);
+                        else
+                            processCustomMessage(msg);
+                    }
+                }
                 else {
+                    addMessage(new TcpDiscoveryDiscardMessage(getLocalNodeId(), msg.id(), true));
+
                     spi.stats.onRingMessageReceived(msg);
 
                     DiscoverySpiCustomMessage msgObj = null;
@@ -4401,16 +4532,21 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                         if (nextMsg != null) {
                             try {
-                                addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), nextMsg,
-                                    spi.marsh.marshal(nextMsg)));
+                                TcpDiscoveryCustomEventMessage ackMsg = new TcpDiscoveryCustomEventMessage(
+                                    getLocalNodeId(), nextMsg, spi.marsh.marshal(nextMsg));
+
+                                ackMsg.topologyVersion(msg.topologyVersion());
+
+                                processCustomMessage(ackMsg);
+
+                                if (ackMsg.verified())
+                                    msgHist.add(ackMsg);
                             }
                             catch (IgniteCheckedException e) {
                                 U.error(log, "Failed to marshal discovery custom message.", e);
                             }
                         }
                     }
-
-                    addMessage(new TcpDiscoveryDiscardMessage(getLocalNodeId(), msg.id(), true));
                 }
             }
             else {
@@ -4428,9 +4564,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 if (msg.verified() && state0 == CONNECTED && pendingMsgs.procCustomMsgs.add(msg.id())) {
-                    assert joiningNodes.isEmpty() : "Joining nodes: " + joiningNodes + ", msg=" + msg + ", loc=" + locNode.id() +
-                        ", topver=" + ring.topologyVersion();
-                    assert msg.topologyVersion() == ring.topologyVersion() : "msg: " + msg + ", topver=" + ring.topologyVersion();
+                    assert msg.topologyVersion() == ring.topologyVersion() :
+                        "msg: " + msg + ", topVer=" + ring.topologyVersion();
 
                     notifyDiscoveryListener(msg);
                 }
@@ -4441,6 +4576,38 @@ class ServerImpl extends TcpDiscoveryImpl {
         }
 
         /**
+         * Checks failed nodes list and sends {@link TcpDiscoveryNodeFailedMessage} if failed node
+         * is still in the ring.
+         */
+        private void checkFailedNodesList() {
+            List<TcpDiscoveryNodeFailedMessage> msgs = null;
+
+            synchronized (mux) {
+                for (Iterator<TcpDiscoveryNode> it = failedNodes.iterator(); it.hasNext();) {
+                    TcpDiscoveryNode node = it.next();
+
+                    if (ring.node(node.id()) != null) {
+                        if (msgs == null)
+                            msgs = new ArrayList<>(failedNodes.size());
+
+                        msgs.add(new TcpDiscoveryNodeFailedMessage(getLocalNodeId(), node.id(), node.internalOrder()));
+                    }
+                    else
+                        it.remove();
+                }
+            }
+
+            if (msgs != null) {
+                for (TcpDiscoveryNodeFailedMessage msg : msgs) {
+                    if (log.isDebugEnabled())
+                        log.debug("Add node failed message for node from failed nodes list: " + msg);
+
+                    addMessage(msg);
+                }
+            }
+        }
+
+        /**
          * Checks and flushes custom event messages if no nodes are attempting to join the grid.
          */
         private void checkPendingCustomMessages() {
@@ -4640,10 +4807,10 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     synchronized (mux) {
                         readers.add(reader);
-
-                        reader.start();
                     }
 
+                    reader.start();
+
                     spi.stats.onServerSocketInitialized(U.currentTimeMillis() - tstamp);
                 }
             }
@@ -4861,9 +5028,10 @@ class ServerImpl extends TcpDiscoveryImpl {
                         log.debug("Initialized connection with remote node [nodeId=" + nodeId +
                             ", client=" + req.client() + ']');
 
-                    if (debugMode)
-                        debugLog("Initialized connection with remote node [nodeId=" + nodeId +
+                    if (debugMode) {
+                        debugLog(msg, "Initialized connection with remote node [nodeId=" + nodeId +
                             ", client=" + req.client() + ']');
+                    }
                 }
                 catch (IOException e) {
                     if (log.isDebugEnabled())
@@ -4932,7 +5100,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         spi.stats.onMessageReceived(msg);
 
                         if (debugMode && recordable(msg))
-                            debugLog("Message has been received: " + msg);
+                            debugLog(msg, "Message has been received: " + msg);
 
                         if (msg instanceof TcpDiscoveryConnectionCheckMessage) {
                             spi.writeToSocket(msg, sock, RES_OK, socketTimeout);

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 2786d0b..1aef728 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -37,6 +37,7 @@ import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.IgniteSpiThread;
 import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -99,9 +100,10 @@ abstract class TcpDiscoveryImpl {
     }
 
     /**
+     * @param discoMsg Discovery message.
      * @param msg Message.
      */
-    protected void debugLog(String msg) {
+    protected void debugLog(@Nullable TcpDiscoveryAbstractMessage discoMsg, String msg) {
         assert debugMode;
 
         String msg0 = new SimpleDateFormat("[HH:mm:ss,SSS]").format(new Date(System.currentTimeMillis())) +

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
index 7ca092c..eb0f74a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.util.typedef.PN;
 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.IgniteProductVersion;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Collection;
@@ -88,6 +89,23 @@ public class TcpDiscoveryNodesRing {
     @GridToStringExclude
     private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
 
+    /** */
+    private IgniteProductVersion minNodeVer;
+
+    /**
+     * @return Minimum node version.
+     */
+    public IgniteProductVersion minimumNodeVersion() {
+        rwLock.readLock().lock();
+
+        try {
+            return minNodeVer;
+        }
+        finally {
+            rwLock.readLock().unlock();
+        }
+    }
+
     /**
      * Sets local node.
      *
@@ -225,6 +243,8 @@ public class TcpDiscoveryNodesRing {
             nodeOrder = node.internalOrder();
 
             maxInternalOrder = node.internalOrder();
+
+            initializeMinimumVersion();
         }
         finally {
             rwLock.writeLock().unlock();
@@ -295,6 +315,8 @@ public class TcpDiscoveryNodesRing {
             }
 
             nodeOrder = topVer;
+
+            initializeMinimumVersion();
         }
         finally {
             rwLock.writeLock().unlock();
@@ -341,6 +363,8 @@ public class TcpDiscoveryNodesRing {
                 nodes.remove(rmv);
             }
 
+            initializeMinimumVersion();
+
             return rmv;
         }
         finally {
@@ -372,6 +396,8 @@ public class TcpDiscoveryNodesRing {
             maxInternalOrder = 0;
 
             topVer = 0;
+
+            minNodeVer = locNode.version();
         }
         finally {
             rwLock.writeLock().unlock();
@@ -451,61 +477,8 @@ public class TcpDiscoveryNodesRing {
      * topology contains less than two nodes.
      */
     @Nullable public TcpDiscoveryNode nextNode(@Nullable Collection<TcpDiscoveryNode> excluded) {
-        assert excluded == null || excluded.isEmpty() || !excluded.contains(locNode);
-
-        rwLock.readLock().lock();
-
-        try {
-            Collection<TcpDiscoveryNode> filtered = serverNodes(excluded);
-
-            if (filtered.size() < 2)
-                return null;
-
-            Iterator<TcpDiscoveryNode> iter = filtered.iterator();
-
-            while (iter.hasNext()) {
-                TcpDiscoveryNode node = iter.next();
-
-                if (locNode.equals(node))
-                    break;
-            }
-
-            return iter.hasNext() ? iter.next() : F.first(filtered);
-        }
-        finally {
-            rwLock.readLock().unlock();
-        }
-    }
-
-    /**
-     * Finds previous node in the topology.
-     *
-     * @return Previous node.
-     */
-    @Nullable public TcpDiscoveryNode previousNode() {
-        rwLock.readLock().lock();
-
-        try {
-            if (nodes.size() < 2)
-                return null;
-
-            return previousNode(null);
-        }
-        finally {
-            rwLock.readLock().unlock();
-        }
-    }
-
-    /**
-     * Finds previous node in the topology filtering excluded nodes from search.
-     *
-     * @param excluded Nodes to exclude from the search (optional). If provided,
-     * cannot contain local node.
-     * @return Previous node or {@code null} if all nodes were filtered out or
-     * topology contains less than two nodes.
-     */
-    @Nullable public TcpDiscoveryNode previousNode(@Nullable Collection<TcpDiscoveryNode> excluded) {
-        assert excluded == null || excluded.isEmpty() || !excluded.contains(locNode);
+        assert locNode.internalOrder() > 0 : locNode;
+        assert excluded == null || excluded.isEmpty() || !excluded.contains(locNode) : excluded;
 
         rwLock.readLock().lock();
 
@@ -638,6 +611,18 @@ public class TcpDiscoveryNodesRing {
         });
     }
 
+    /**
+     *
+     */
+    private void initializeMinimumVersion() {
+        minNodeVer = null;
+
+        for (TcpDiscoveryNode node : nodes) {
+            if (minNodeVer == null || node.version().compareTo(minNodeVer) < 0)
+                minNodeVer = node.version();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         rwLock.readLock().lock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/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 875d18e..9cb47af 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
@@ -19,10 +19,15 @@ package org.apache.ignite.spi.discovery.tcp.messages;
 
 import java.io.Externalizable;
 import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.UUID;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Base class to implement discovery messages.
@@ -62,6 +67,10 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable {
     /** Pending message index. */
     private short pendingIdx;
 
+    /** */
+    @GridToStringInclude
+    private Set<UUID> failedNodes;
+
     /**
      * Default no-arg constructor for {@link Externalizable} interface.
      */
@@ -236,6 +245,34 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable {
         return false;
     }
 
+    /**
+     * Adds node ID to the failed nodes list.
+     *
+     * @param nodeId Node ID.
+     */
+    public void addFailedNode(UUID nodeId) {
+        assert nodeId != null;
+
+        if (failedNodes == null)
+            failedNodes = new HashSet<>();
+
+        failedNodes.add(nodeId);
+    }
+
+    /**
+     * @param failedNodes Failed nodes.
+     */
+    public void failedNodes(@Nullable Set<UUID> failedNodes) {
+        this.failedNodes = failedNodes;
+    }
+
+    /**
+     * @return Failed nodes IDs.
+     */
+    @Nullable public Collection<UUID> failedNodes() {
+        return failedNodes;
+    }
+
     /** {@inheritDoc} */
     @Override public final boolean equals(Object obj) {
         if (this == obj)

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java
index 6118f4d..70b0080 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java
@@ -62,6 +62,17 @@ public class TcpDiscoveryStatusCheckMessage extends TcpDiscoveryAbstractMessage
     }
 
     /**
+     * @param msg Message to copy.
+     */
+    public TcpDiscoveryStatusCheckMessage(TcpDiscoveryStatusCheckMessage msg) {
+        super(msg);
+
+        this.creatorNode = msg.creatorNode;
+        this.failedNodeId = msg.failedNodeId;
+        this.status = msg.status;
+    }
+
+    /**
      * Gets creator node.
      *
      * @return Creator node.

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
index 55474dc..5053c2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
@@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -212,6 +213,22 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
     public void testMultiThreadedClientsServersRestart() throws Throwable {
         fail("https://issues.apache.org/jira/browse/IGNITE-1123");
 
+        multiThreadedClientsServersRestart(GRID_CNT, CLIENT_GRID_CNT);
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void _testMultiThreadedServersRestart() throws Throwable {
+        multiThreadedClientsServersRestart(GRID_CNT * 2, 0);
+    }
+
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @throws Exception If any error occurs.
+     */
+    private void multiThreadedClientsServersRestart(int srvs, int clients) throws Throwable {
         final AtomicBoolean done = new AtomicBoolean();
 
         try {
@@ -219,91 +236,95 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
 
             info("Test timeout: " + (getTestTimeout() / (60 * 1000)) + " min.");
 
-            startGridsMultiThreaded(GRID_CNT);
-
-            clientFlagGlobal = true;
+            startGridsMultiThreaded(srvs);
 
-            startGridsMultiThreaded(GRID_CNT, CLIENT_GRID_CNT);
+            IgniteInternalFuture<?> clientFut = null;
 
             final AtomicReference<Throwable> error = new AtomicReference<>();
 
-            final BlockingQueue<Integer> clientStopIdxs = new LinkedBlockingQueue<>();
-
-            for (int i = GRID_CNT; i < GRID_CNT + CLIENT_GRID_CNT; i++)
-                clientStopIdxs.add(i);
+            if (clients > 0) {
+                clientFlagGlobal = true;
 
-            final AtomicInteger clientStartIdx = new AtomicInteger(9000);
+                startGridsMultiThreaded(srvs, clients);
 
-            IgniteInternalFuture<?> fut1 = multithreadedAsync(
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        try {
-                            clientFlagPerThread.set(true);
+                final BlockingQueue<Integer> clientStopIdxs = new LinkedBlockingQueue<>();
 
-                            while (!done.get() && error.get() == null) {
-                                Integer stopIdx = clientStopIdxs.take();
+                for (int i = srvs; i < srvs + clients; i++)
+                    clientStopIdxs.add(i);
 
-                                log.info("Stop client: " + stopIdx);
+                final AtomicInteger clientStartIdx = new AtomicInteger(9000);
 
-                                stopGrid(stopIdx);
+                clientFut = multithreadedAsync(
+                    new Callable<Object>() {
+                        @Override public Object call() throws Exception {
+                            try {
+                                clientFlagPerThread.set(true);
 
                                 while (!done.get() && error.get() == null) {
-                                    // Generate unique name to simplify debugging.
-                                    int startIdx = clientStartIdx.getAndIncrement();
+                                    Integer stopIdx = clientStopIdxs.take();
 
-                                    log.info("Start client: " + startIdx);
+                                    log.info("Stop client: " + stopIdx);
 
-                                    UUID id = UUID.randomUUID();
+                                    stopGrid(stopIdx);
 
-                                    nodeId.set(id);
+                                    while (!done.get() && error.get() == null) {
+                                        // Generate unique name to simplify debugging.
+                                        int startIdx = clientStartIdx.getAndIncrement();
 
-                                    try {
-                                        Ignite ignite = startGrid(startIdx);
+                                        log.info("Start client: " + startIdx);
 
-                                        assertTrue(ignite.configuration().isClientMode());
+                                        UUID id = UUID.randomUUID();
 
-                                        clientStopIdxs.add(startIdx);
+                                        nodeId.set(id);
 
-                                        break;
-                                    }
-                                    catch (Exception e) {
-                                        if (X.hasCause(e, IgniteClientDisconnectedCheckedException.class) ||
-                                            X.hasCause(e, IgniteClientDisconnectedException.class))
-                                            log.info("Client disconnected: " + e);
-                                        else if (X.hasCause(e, ClusterTopologyCheckedException.class))
-                                            log.info("Client failed to start: " + e);
-                                        else {
-                                            if (failedNodes.contains(id) && X.hasCause(e, IgniteSpiException.class))
-                                                log.info("Client failed: " + e);
-                                            else
-                                                throw e;
+                                        try {
+                                            Ignite ignite = startGrid(startIdx);
+
+                                            assertTrue(ignite.configuration().isClientMode());
+
+                                            clientStopIdxs.add(startIdx);
+
+                                            break;
+                                        }
+                                        catch (Exception e) {
+                                            if (X.hasCause(e, IgniteClientDisconnectedCheckedException.class) ||
+                                                X.hasCause(e, IgniteClientDisconnectedException.class))
+                                                log.info("Client disconnected: " + e);
+                                            else if (X.hasCause(e, ClusterTopologyCheckedException.class))
+                                                log.info("Client failed to start: " + e);
+                                            else {
+                                                if (failedNodes.contains(id) && X.hasCause(e, IgniteSpiException.class))
+                                                    log.info("Client failed: " + e);
+                                                else
+                                                    throw e;
+                                            }
                                         }
                                     }
                                 }
                             }
-                        }
-                        catch (Throwable e) {
-                            log.error("Unexpected error: " + e, e);
+                            catch (Throwable e) {
+                                log.error("Unexpected error: " + e, e);
 
-                            error.compareAndSet(null, e);
+                                error.compareAndSet(null, e);
+
+                                return null;
+                            }
 
                             return null;
                         }
-
-                        return null;
-                    }
-                },
-                CLIENT_GRID_CNT,
-                "client-restart");
+                    },
+                    clients,
+                    "client-restart");
+            }
 
             final BlockingQueue<Integer> srvStopIdxs = new LinkedBlockingQueue<>();
 
-            for (int i = 0; i < GRID_CNT; i++)
+            for (int i = 0; i < srvs; i++)
                 srvStopIdxs.add(i);
 
-            final AtomicInteger srvStartIdx = new AtomicInteger(GRID_CNT + CLIENT_GRID_CNT);
+            final AtomicInteger srvStartIdx = new AtomicInteger(srvs + clients);
 
-            IgniteInternalFuture<?> fut2 = multithreadedAsync(
+            IgniteInternalFuture<?> srvFut = multithreadedAsync(
                 new Callable<Object>() {
                     @Override public Object call() throws Exception {
                         try {
@@ -312,6 +333,10 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
                             while (!done.get() && error.get() == null) {
                                 int stopIdx = srvStopIdxs.take();
 
+                                U.sleep(50);
+
+                                Thread.currentThread().setName("stop-server-" + getTestGridName(stopIdx));
+
                                 log.info("Stop server: " + stopIdx);
 
                                 stopGrid(stopIdx);
@@ -319,13 +344,20 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
                                 // Generate unique name to simplify debugging.
                                 int startIdx = srvStartIdx.getAndIncrement();
 
+                                Thread.currentThread().setName("start-server-" + getTestGridName(startIdx));
+
                                 log.info("Start server: " + startIdx);
 
-                                Ignite ignite = startGrid(startIdx);
+                                try {
+                                    Ignite ignite = startGrid(startIdx);
 
-                                assertFalse(ignite.configuration().isClientMode());
+                                    assertFalse(ignite.configuration().isClientMode());
 
-                                srvStopIdxs.add(startIdx);
+                                    srvStopIdxs.add(startIdx);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    log.info("Failed to start: " + e);
+                                }
                             }
                         }
                         catch (Throwable e) {
@@ -339,7 +371,7 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
                         return null;
                     }
                 },
-                GRID_CNT - 1,
+                srvs - 1,
                 "server-restart");
 
             final long timeToExec = getTestTimeout() - 60_000;
@@ -356,8 +388,10 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
 
                     done.set(true);
 
-                    fut1.cancel();
-                    fut2.cancel();
+                    if (clientFut != null)
+                        clientFut.cancel();
+
+                    srvFut.cancel();
 
                     throw err;
                 }
@@ -367,8 +401,10 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
 
             done.set(true);
 
-            fut1.get();
-            fut2.get();
+            if (clientFut != null)
+                clientFut.get();
+
+            srvFut.get();
         }
         finally {
             done.set(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
index 8b94f54..7beeb41 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryRestartTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -196,8 +197,15 @@ public class TcpDiscoveryRestartTest extends GridCommonAbstractTest {
 
         /**
          * @param nodeId Node ID.
+         * @throws Exception If failed.
          */
-        void checkEvents(UUID nodeId) {
+        void checkEvents(final UUID nodeId) throws Exception {
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return joinIds.contains(nodeId) && leftIds.contains(nodeId);
+                }
+            }, 5000);
+
             assertTrue("No join event: " + nodeId, joinIds.contains(nodeId));
 
             assertTrue("No left event: " + nodeId, leftIds.contains(nodeId));

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/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 51d8a2d..379a3a6 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
@@ -21,16 +21,19 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -38,6 +41,8 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteIllegalStateException;
+import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -46,12 +51,16 @@ import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.processors.continuous.StartRoutineAckDiscoveryMessage;
 import org.apache.ignite.internal.processors.port.GridPortRecord;
 import org.apache.ignite.internal.util.io.GridByteArrayOutputStream;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.IgniteSpiException;
@@ -64,8 +73,8 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage;
-import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
@@ -94,7 +103,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     private UUID nodeId;
 
     /** */
-    private TcpDiscoverySpi nodeSpi;
+    private static ThreadLocal<TcpDiscoverySpi> nodeSpi = new ThreadLocal<>();
 
     /**
      * @throws Exception If fails.
@@ -104,15 +113,17 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings({"IfMayBeConditional", "deprecation"})
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi spi = nodeSpi;
+        TcpDiscoverySpi spi = nodeSpi.get();
 
-        if (spi == null)
+        if (spi == null) {
             spi = gridName.contains("testPingInterruptedOnNodeFailedFailingNode") ?
                 new TestTcpDiscoverySpi() : new TcpDiscoverySpi();
+        }
+        else
+            nodeSpi.set(null);
 
         discoMap.put(gridName, spi);
 
@@ -176,6 +187,13 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        discoMap = null;
+
+        super.afterTest();
+    }
+
     /**
      * @throws Exception If any error occurs.
      */
@@ -1202,11 +1220,11 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     private void customEventRace1(final boolean cacheStartFrom1, boolean stopCrd) throws Exception {
         TestCustomEventRaceSpi spi0 = new TestCustomEventRaceSpi();
 
-        nodeSpi = spi0;
+        nodeSpi.set(spi0);
 
         final Ignite ignite0 = startGrid(0);
 
-        nodeSpi = new TestCustomEventRaceSpi();
+        nodeSpi.set(new TestCustomEventRaceSpi());
 
         final Ignite ignite1 = startGrid(1);
 
@@ -1221,7 +1239,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
             @Override public Void call() throws Exception {
                 log.info("Start 2");
 
-                nodeSpi = new TestCustomEventRaceSpi();
+                nodeSpi.set(new TestCustomEventRaceSpi());
 
                 Ignite ignite2 = startGrid(2);
 
@@ -1271,7 +1289,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
 
         assertEquals(1, cache.get(1));
 
-        nodeSpi = new TestCustomEventRaceSpi();
+        nodeSpi.set(new TestCustomEventRaceSpi());
 
         Ignite ignite = startGrid(3);
 
@@ -1314,15 +1332,15 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     private void customEventCoordinatorFailure(boolean twoNodes) throws Exception {
         TestCustomEventCoordinatorFailureSpi spi0 = new TestCustomEventCoordinatorFailureSpi();
 
-        nodeSpi = spi0;
+        nodeSpi.set(spi0);
 
         Ignite ignite0 = startGrid(0);
 
-        nodeSpi = new TestCustomEventCoordinatorFailureSpi();
+        nodeSpi.set(new TestCustomEventCoordinatorFailureSpi());
 
         Ignite ignite1 = startGrid(1);
 
-        nodeSpi = new TestCustomEventCoordinatorFailureSpi();
+        nodeSpi.set(new TestCustomEventCoordinatorFailureSpi());
 
         Ignite ignite2 = twoNodes ? null : startGrid(2);
 
@@ -1366,7 +1384,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
 
         log.info("Try start one more node.");
 
-        nodeSpi = new TestCustomEventCoordinatorFailureSpi();
+        nodeSpi.set(new TestCustomEventCoordinatorFailureSpi());
 
         Ignite ignite = startGrid(twoNodes ? 2 : 3);
 
@@ -1381,6 +1399,421 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Coordinator is added in failed list during node start.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFailedNodes1() throws Exception {
+        try {
+            final int FAIL_ORDER = 3;
+
+            nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+            final Ignite ignite0 = startGrid(0);
+
+            nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+            startGrid(1);
+
+            nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+            Ignite ignite2 = startGrid(2);
+
+            assertEquals(2, ignite2.cluster().nodes().size());
+
+            waitNodeStop(ignite0.name());
+
+            tryCreateCache(2);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Coordinator is added in failed list, concurrent nodes start.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFailedNodes2() throws Exception {
+        try {
+            final int FAIL_ORDER = 3;
+
+            nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+            Ignite ignite0 = startGrid(0);
+
+            nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+            startGrid(1);
+
+            final AtomicInteger nodeIdx = new AtomicInteger(1);
+
+            GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    int idx = nodeIdx.incrementAndGet();
+
+                    nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+                    startGrid(idx);
+
+                    return null;
+                }
+            }, 3, "start-node");
+
+            Ignite ignite2 = ignite(2);
+
+            waitForRemoteNodes(ignite2, 3);
+
+            waitNodeStop(ignite0.name());
+
+            tryCreateCache(4);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Coordinator is added in failed list during node start, test with two nodes.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFailedNodes3() throws Exception {
+        try {
+            nodeSpi.set(new TestFailedNodesSpi(-1));
+
+            Ignite ignite0 = startGrid(0);
+
+            nodeSpi.set(new TestFailedNodesSpi(2));
+
+            Ignite ignite1 = startGrid(1);
+
+            assertEquals(1, ignite1.cluster().nodes().size());
+
+            waitNodeStop(ignite0.name());
+
+            ignite1.getOrCreateCache(new CacheConfiguration<>()).put(1, 1);
+
+            startGrid(2);
+
+            assertEquals(2, ignite1.cluster().nodes().size());
+
+            tryCreateCache(2);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Coordinator is added in failed list during node start, but node detected failure dies before
+     * sending {@link TcpDiscoveryNodeFailedMessage}.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFailedNodes4() throws Exception {
+        try {
+            final int FAIL_ORDER = 3;
+
+            nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+            final Ignite ignite0 = startGrid(0);
+
+            nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER));
+
+            Ignite ignite1 = startGrid(1);
+
+            TestFailedNodesSpi spi = new TestFailedNodesSpi(FAIL_ORDER);
+
+            spi.stopBeforeSndFail = true;
+
+            nodeSpi.set(spi);
+
+            Ignite ignite2 = startGrid(2);
+
+            waitNodeStop(ignite2.name());
+
+            log.info("Try start new node.");
+
+            Ignite ignite3 = startGrid(3);
+
+            waitNodeStop(ignite0.name());
+
+            assertEquals(2, ignite1.cluster().nodes().size());
+            assertEquals(2, ignite3.cluster().nodes().size());
+
+            tryCreateCache(2);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Adds some node in failed list after join process finished.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFailedNodes5() throws Exception {
+        try {
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            for (int iter = 0; iter < 3; iter++) {
+                final int NODES = iter == 0 ? 2 : rnd.nextInt(3, 6);
+
+                for (int i = 0; i < NODES; i++) {
+                    nodeSpi.set(new TestFailedNodesSpi(-1));
+
+                    startGrid(i);
+                }
+
+                Map<Long, Ignite> nodes = new HashMap<>();
+
+                for (int i = 0; i < NODES; i++) {
+                    Ignite ignite = ignite(i);
+
+                    nodes.put(ignite.cluster().localNode().order(), ignite);
+                }
+
+                Ignite ignite = ignite(rnd.nextInt(NODES));
+
+                log.info("Iteration [iter=" + iter + ", nodes=" + NODES + ", failFrom=" + ignite.name() + ']');
+
+                TestFailedNodesSpi spi = (TestFailedNodesSpi)ignite.configuration().getDiscoverySpi();
+
+                spi.failSingleMsg = true;
+
+                long order = ignite.cluster().localNode().order();
+
+                long nextOrder = order == NODES ? 1 : order + 1;
+
+                Ignite failingNode = nodes.get(nextOrder);
+
+                assertNotNull(failingNode);
+
+                waitNodeStop(failingNode.name());
+
+                Ignite newNode = startGrid(NODES);
+
+                assertEquals(NODES, newNode.cluster().nodes().size());
+
+                tryCreateCache(NODES);
+
+                stopAllGrids();
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomEventAckNotSend() throws Exception {
+        try {
+            TestCustomerEventAckSpi spi0 = new TestCustomerEventAckSpi();
+
+            nodeSpi.set(spi0);
+
+            Ignite ignite0 = startGrid(0);
+
+            nodeSpi.set(new TestCustomerEventAckSpi());
+
+            Ignite ignite1 = startGrid(1);
+
+            spi0.stopBeforeSndAck = true;
+
+            ignite1.message().remoteListen("test", new DummyPredicate());
+
+            waitNodeStop(ignite0.name());
+
+            startGrid(2);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @param nodeName Node name.
+     * @throws Exception If failed.
+     */
+    private void waitNodeStop(final String nodeName) throws Exception {
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    Ignition.ignite(nodeName);
+
+                    return false;
+                }
+                catch (IgniteIllegalStateException e) {
+                    return true;
+                }
+            }
+        }, 10_000);
+
+        if (!wait)
+            U.dumpThreads(log);
+
+        assertTrue("Failed to wait for node stop.", wait);
+    }
+
+    /**
+     * @param expNodes Expected nodes number.
+     */
+    private void tryCreateCache(int expNodes) {
+        List<Ignite> allNodes = G.allGrids();
+
+        assertEquals(expNodes, allNodes.size());
+
+        int cntr = 0;
+
+        for (Ignite ignite : allNodes) {
+            CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+            ccfg.setName("cache-" + cntr++);
+
+            log.info("Try create cache [node=" + ignite.name() + ", cache=" + ccfg.getName() + ']');
+
+            ignite.getOrCreateCache(ccfg).put(1, 1);
+        }
+    }
+
+    /**
+     *
+     */
+    static class DummyPredicate implements IgniteBiPredicate<UUID, Object> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID uuid, Object o) {
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCustomerEventAckSpi extends TcpDiscoverySpi {
+        /** */
+        private volatile boolean stopBeforeSndAck;
+
+        /** {@inheritDoc} */
+        @Override protected void writeToSocket(Socket sock,
+            TcpDiscoveryAbstractMessage msg,
+            GridByteArrayOutputStream bout,
+            long timeout) throws IOException, IgniteCheckedException {
+            if (stopBeforeSndAck) {
+                if (msg instanceof TcpDiscoveryCustomEventMessage) {
+                    try {
+                        DiscoveryCustomMessage custMsg = GridTestUtils.getFieldValue(
+                            ((TcpDiscoveryCustomEventMessage)msg).message(marsh), "delegate");
+
+                        if (custMsg instanceof StartRoutineAckDiscoveryMessage) {
+                            log.info("Skip message send and stop node: " + msg);
+
+                            sock.close();
+
+                            GridTestUtils.runAsync(new Callable<Object>() {
+                                @Override public Object call() throws Exception {
+                                    ignite.close();
+
+                                    return null;
+                                }
+                            }, "stop-node");
+
+                            return;
+                        }
+                    }
+                    catch (Throwable e) {
+                        fail("Unexpected error: " + e);
+                    }
+                }
+            }
+
+            super.writeToSocket(sock, msg, bout, timeout);
+        }
+    }
+
+    /**
+     * Simulate scenario when node detects node failure trying to send message, but node still alive.
+     */
+    private static class TestFailedNodesSpi extends TcpDiscoverySpi {
+        /** */
+        private AtomicBoolean failMsg = new AtomicBoolean();
+
+        /** */
+        private int failOrder;
+
+        /** */
+        private boolean stopBeforeSndFail;
+
+        /** */
+        private boolean stop;
+
+        /** */
+        private volatile boolean failSingleMsg;
+
+        /**
+         * @param failOrder Spi fails connection if local node order equals to this order.
+         */
+        TestFailedNodesSpi(int failOrder) {
+            this.failOrder = failOrder;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void writeToSocket(Socket sock,
+            TcpDiscoveryAbstractMessage msg,
+            GridByteArrayOutputStream bout,
+            long timeout) throws IOException, IgniteCheckedException {
+            if (stop)
+                return;
+
+            if (failSingleMsg) {
+                failSingleMsg = false;
+
+                log.info("IO error on message send [locNode=" + locNode + ", msg=" + msg + ']');
+
+                sock.close();
+
+                throw new SocketTimeoutException();
+            }
+
+            if (locNode.internalOrder() == failOrder &&
+                (msg instanceof TcpDiscoveryNodeAddedMessage) &&
+                failMsg.compareAndSet(false, true)) {
+                log.info("IO error on message send [locNode=" + locNode + ", msg=" + msg + ']');
+
+                sock.close();
+
+                throw new SocketTimeoutException();
+            }
+
+            if (stopBeforeSndFail &&
+                locNode.internalOrder() == failOrder &&
+                (msg instanceof TcpDiscoveryNodeFailedMessage)) {
+                stop = true;
+
+                log.info("Skip messages send and stop node [locNode=" + locNode + ", msg=" + msg + ']');
+
+                sock.close();
+
+                GridTestUtils.runAsync(new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        ignite.close();
+
+                        return null;
+                    }
+                }, "stop-node");
+
+                return;
+            }
+
+            super.writeToSocket(sock, msg, bout, timeout);
+        }
+    }
+
+    /**
      *
      */
     private static class TestCustomEventCoordinatorFailureSpi extends TcpDiscoverySpi {

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java
index 97ba5cf..1e710ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java
@@ -58,14 +58,20 @@ public class TcpDiscoveryMulticastIpFinderSelfTest
 
         try {
             ipFinder1 = ipFinder();
+            ipFinder1.setResponseWaitTime(1000);
+            ipFinder1.setAddressRequestAttempts(10);
 
             ipFinder2 = new TcpDiscoveryMulticastIpFinder();
 
+            ipFinder2.setResponseWaitTime(1000);
+            ipFinder2.setAddressRequestAttempts(10);
             ipFinder2.setMulticastGroup(ipFinder1.getMulticastGroup());
             ipFinder2.setMulticastPort(ipFinder1.getMulticastPort());
 
             ipFinder3 = new TcpDiscoveryMulticastIpFinder();
 
+            ipFinder3.setResponseWaitTime(1000);
+            ipFinder3.setAddressRequestAttempts(10);
             ipFinder3.setMulticastGroup(ipFinder1.getMulticastGroup());
             ipFinder3.setMulticastPort(ipFinder1.getMulticastPort());
 
@@ -81,21 +87,13 @@ public class TcpDiscoveryMulticastIpFinderSelfTest
             ipFinder2.initializeLocalAddresses(Collections.singleton(new InetSocketAddress("host2", 1002)));
             ipFinder3.initializeLocalAddresses(Collections.singleton(new InetSocketAddress("host3", 1003)));
 
-            for (int i = 0; i < 5; i++) {
-                Collection<InetSocketAddress> addrs1 = ipFinder1.getRegisteredAddresses();
-                Collection<InetSocketAddress> addrs2 = ipFinder2.getRegisteredAddresses();
-                Collection<InetSocketAddress> addrs3 = ipFinder3.getRegisteredAddresses();
-
-                if (addrs1.size() != 1 || addrs2.size() != 2 || addrs3.size() != 3) {
-                    info("Addrs1: " + addrs1);
-                    info("Addrs2: " + addrs2);
-                    info("Addrs2: " + addrs3);
-
-                    Thread.sleep(1000);
-                }
-                else
-                    break;
-            }
+            Collection<InetSocketAddress> addrs1 = ipFinder1.getRegisteredAddresses();
+            Collection<InetSocketAddress> addrs2 = ipFinder2.getRegisteredAddresses();
+            Collection<InetSocketAddress> addrs3 = ipFinder3.getRegisteredAddresses();
+
+            info("Addrs1: " + addrs1);
+            info("Addrs2: " + addrs2);
+            info("Addrs2: " + addrs3);
 
             assertEquals(1, ipFinder1.getRegisteredAddresses().size());
             assertEquals(2, ipFinder2.getRegisteredAddresses().size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/80147128/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 41d4b4a..3e41979 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -1120,16 +1120,31 @@ public abstract class GridAbstractTest extends TestCase {
         if (gridName != null && gridName.matches(".*\\d")) {
             String idStr = UUID.randomUUID().toString();
 
-            char[] chars = idStr.toCharArray();
+            if (gridName.startsWith(getTestGridName())) {
+                String idxStr = String.valueOf(getTestGridIndex(gridName));
 
-            chars[0] = gridName.charAt(gridName.length() - 1);
-            chars[1] = '0';
+                while (idxStr.length() < 5)
+                    idxStr = '0' + idxStr;
 
-            chars[chars.length - 3] = '0';
-            chars[chars.length - 2] = '0';
-            chars[chars.length - 1] = gridName.charAt(gridName.length() - 1);
+                char[] chars = idStr.toCharArray();
 
-            cfg.setNodeId(UUID.fromString(new String(chars)));
+                for (int i = 0; i < idxStr.length(); i++)
+                    chars[chars.length - idxStr.length() + i] = idxStr.charAt(i);
+
+                cfg.setNodeId(UUID.fromString(new String(chars)));
+            }
+            else {
+                char[] chars = idStr.toCharArray();
+
+                chars[0] = gridName.charAt(gridName.length() - 1);
+                chars[1] = '0';
+
+                chars[chars.length - 3] = '0';
+                chars[chars.length - 2] = '0';
+                chars[chars.length - 1] = gridName.charAt(gridName.length() - 1);
+
+                cfg.setNodeId(UUID.fromString(new String(chars)));
+            }
         }
 
         if (isMultiJvm())


[43/46] ignite git commit: Merge branch ignite-1.5 into ignite-1282

Posted by ak...@apache.org.
Merge branch ignite-1.5 into ignite-1282


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

Branch: refs/heads/ignite-1753-1282
Commit: b278fe3e357badcb26e78891d9edb5f31def512c
Parents: 0ccba79
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 17 13:35:26 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 17 13:35:26 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/preloader/GridDhtPartitionDemander.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b278fe3e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 40d3dc1..7ccb68e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -688,7 +688,7 @@ public class GridDhtPartitionDemander {
                         if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_OBJECT_LOADED) && !cached.isInternal())
                             cctx.events().addEvent(cached.partition(), cached.key(), cctx.localNodeId(),
                                 (IgniteUuid)null, null, EVT_CACHE_REBALANCE_OBJECT_LOADED, entry.value(), true, null,
-                                false, null, null, null);
+                                false, null, null, null, true);
                     }
                     else if (log.isDebugEnabled())
                         log.debug("Rebalancing entry is already in cache (will ignore) [key=" + cached.key() +


[05/46] ignite git commit: Ignite-1093 "Rebalancing with default parameters is very slow" fixes.

Posted by ak...@apache.org.
Ignite-1093 "Rebalancing with default parameters is very slow" fixes.


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

Branch: refs/heads/ignite-1753-1282
Commit: 7dfaa3b062d5020debb5e716395f9b5603ec07ed
Parents: 7573003
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Nov 5 16:15:34 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Nov 5 20:41:17 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   54 +-
 .../configuration/IgniteConfiguration.java      |   32 +-
 .../apache/ignite/internal/IgniteKernal.java    |   21 +
 .../org/apache/ignite/internal/IgnitionEx.java  |    3 +
 .../communication/GridIoMessageFactory.java     |   10 +-
 .../processors/cache/GridCacheIoManager.java    |   19 +-
 .../processors/cache/GridCacheMapEntry.java     |   38 +-
 .../GridCachePartitionExchangeManager.java      |  164 ++-
 .../processors/cache/GridCachePreloader.java    |   55 +-
 .../cache/GridCachePreloaderAdapter.java        |   40 +-
 .../processors/cache/GridCacheProcessor.java    |   55 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   11 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   63 +-
 .../dht/GridDhtPartitionsReservation.java       |    2 +-
 .../GridDhtPartitionDemandMessage.java          |   14 +-
 .../preloader/GridDhtPartitionDemandPool.java   | 1192 ---------------
 .../dht/preloader/GridDhtPartitionDemander.java | 1389 ++++++++++++++++++
 .../dht/preloader/GridDhtPartitionSupplier.java | 1034 +++++++++++++
 .../GridDhtPartitionSupplyMessageV2.java        |  380 +++++
 .../preloader/GridDhtPartitionSupplyPool.java   |  555 -------
 .../GridDhtPartitionsExchangeFuture.java        |    2 +
 .../dht/preloader/GridDhtPreloader.java         |  282 +++-
 .../datastructures/DataStructuresProcessor.java |    3 +
 .../processors/task/GridTaskWorker.java         |    4 +-
 .../ignite/internal/util/lang/GridTuple4.java   |    2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    2 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |    2 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   21 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |   68 +
 .../GridCacheRebalancingSyncSelfTest.java       |  506 +++++++
 ...eRebalancingUnmarshallingFailedSelfTest.java |  147 ++
 .../GridCacheReplicatedPreloadSelfTest.java     |   22 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   17 -
 .../spi/discovery/tcp/TestTcpDiscoverySpi.java  |   46 +
 .../testframework/junits/GridAbstractTest.java  |    3 +-
 .../junits/common/GridCommonAbstractTest.java   |   21 +-
 .../testsuites/IgniteCacheTestSuite3.java       |    4 +
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   18 +-
 38 files changed, 4309 insertions(+), 1992 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 374743f..b7276c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -69,11 +69,15 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private static final long serialVersionUID = 0L;
 
     /** Default size of rebalance thread pool. */
+    @Deprecated
     public static final int DFLT_REBALANCE_THREAD_POOL_SIZE = 2;
 
     /** Default rebalance timeout (ms).*/
     public static final long DFLT_REBALANCE_TIMEOUT = 10000;
 
+    /** Default rebalance batches prefetch count. */
+    public static final long DFLT_REBALANCE_BATCHES_PREFETCH_COUNT = 2;
+
     /** Time in milliseconds to wait between rebalance messages to avoid overloading CPU. */
     public static final long DFLT_REBALANCE_THROTTLE = 0;
 
@@ -174,6 +178,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private String name;
 
     /** Rebalance thread pool size. */
+    @Deprecated
     private int rebalancePoolSize = DFLT_REBALANCE_THREAD_POOL_SIZE;
 
     /** Rebalance timeout. */
@@ -254,6 +259,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Rebalance batch size. */
     private int rebalanceBatchSize = DFLT_REBALANCE_BATCH_SIZE;
 
+    /** Rebalance batches prefetch count. */
+    private long rebalanceBatchesPrefetchCount = DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
+
     /** Off-heap memory size. */
     private long offHeapMaxMem = DFLT_OFFHEAP_MEMORY;
 
@@ -394,9 +402,10 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         name = cc.getName();
         nearCfg = cc.getNearConfiguration();
         nodeFilter = cc.getNodeFilter();
-        rebalanceMode = cc.getRebalanceMode();
+        rebalanceBatchesPrefetchCount = cc.getRebalanceBatchesPrefetchCount();
         rebalanceBatchSize = cc.getRebalanceBatchSize();
         rebalanceDelay = cc.getRebalanceDelay();
+        rebalanceMode = cc.getRebalanceMode();
         rebalanceOrder = cc.getRebalanceOrder();
         rebalancePoolSize = cc.getRebalanceThreadPoolSize();
         rebalanceTimeout = cc.getRebalanceTimeout();
@@ -1036,10 +1045,10 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * {@link CacheRebalanceMode#SYNC SYNC} or {@link CacheRebalanceMode#ASYNC ASYNC} rebalance modes only.
      * <p/>
      * If cache rebalance order is positive, rebalancing for this cache will be started only when rebalancing for
-     * all caches with smaller rebalance order (except caches with rebalance order {@code 0}) will be completed.
+     * all caches with smaller rebalance order will be completed.
      * <p/>
      * Note that cache with order {@code 0} does not participate in ordering. This means that cache with
-     * rebalance order {@code 1} will never wait for any other caches. All caches with order {@code 0} will
+     * rebalance order {@code 0} will never wait for any other caches. All caches with order {@code 0} will
      * be rebalanced right away concurrently with each other and ordered rebalance processes.
      * <p/>
      * If not set, cache order is 0, i.e. rebalancing is not ordered.
@@ -1088,6 +1097,35 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * To gain better rebalancing performance supplier node can provide more than one batch at rebalancing start and
+     * provide one new to each next demand request.
+     *
+     * Gets number of batches generated by supply node at rebalancing start.
+     * Minimum is 1.
+     *
+     * @return batches count
+     */
+    public long getRebalanceBatchesPrefetchCount() {
+        return rebalanceBatchesPrefetchCount;
+    }
+
+    /**
+     * To gain better rebalancing performance supplier node can provide more than one batch at rebalancing start and
+     * provide one new to each next demand request.
+     *
+     * Sets number of batches generated by supply node at rebalancing start.
+     * Minimum is 1.
+     *
+     * @param rebalanceBatchesCnt batches count.
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration<K, V> setRebalanceBatchesPrefetchCount(long rebalanceBatchesCnt) {
+        this.rebalanceBatchesPrefetchCount = rebalanceBatchesCnt;
+
+        return this;
+    }
+
+    /**
      * Flag indicating whether Ignite should use swap storage by default. By default
      * swap is disabled which is defined via {@link #DFLT_SWAP_ENABLED} constant.
      *
@@ -1273,24 +1311,22 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Gets size of rebalancing thread pool. Note that size serves as a hint and implementation
-     * may create more threads for rebalancing than specified here (but never less threads).
-     * <p>
-     * Default value is {@link #DFLT_REBALANCE_THREAD_POOL_SIZE}.
+     * Use {@link IgniteConfiguration#getRebalanceThreadPoolSize()} instead.
      *
      * @return Size of rebalancing thread pool.
      */
+    @Deprecated
     public int getRebalanceThreadPoolSize() {
         return rebalancePoolSize;
     }
 
     /**
-     * Sets size of rebalancing thread pool. Note that size serves as a hint and implementation may create more threads
-     * for rebalancing than specified here (but never less threads).
+     * Use {@link IgniteConfiguration#getRebalanceThreadPoolSize()} instead.
      *
      * @param rebalancePoolSize Size of rebalancing thread pool.
      * @return {@code this} for chaining.
      */
+    @Deprecated
     public CacheConfiguration<K, V> setRebalanceThreadPoolSize(int rebalancePoolSize) {
         this.rebalancePoolSize = rebalancePoolSize;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ecae356..9298c6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -149,6 +149,9 @@ public class IgniteConfiguration {
     /** Default keep alive time for public thread pool. */
     public static final long DFLT_PUBLIC_KEEP_ALIVE_TIME = 0;
 
+    /** Default limit of threads used for rebalance. */
+    public static final int DFLT_REBALANCE_THREAD_POOL_SIZE = 1;
+
     /** Default max queue capacity of public thread pool. */
     public static final int DFLT_PUBLIC_THREADPOOL_QUEUE_CAP = Integer.MAX_VALUE;
 
@@ -354,6 +357,9 @@ public class IgniteConfiguration {
     /** Client mode flag. */
     private Boolean clientMode;
 
+    /** Rebalance thread pool size. */
+    private int rebalanceThreadPoolSize = DFLT_REBALANCE_THREAD_POOL_SIZE;
+
     /** Transactions configuration. */
     private TransactionConfiguration txCfg = new TransactionConfiguration();
 
@@ -500,6 +506,7 @@ public class IgniteConfiguration {
         p2pPoolSize = cfg.getPeerClassLoadingThreadPoolSize();
         pluginCfgs = cfg.getPluginConfigurations();
         pubPoolSize = cfg.getPublicThreadPoolSize();
+        rebalanceThreadPoolSize = cfg.getRebalanceThreadPoolSize();
         segChkFreq = cfg.getSegmentCheckFrequency();
         segPlc = cfg.getSegmentationPolicy();
         segResolveAttempts = cfg.getSegmentationResolveAttempts();
@@ -1331,6 +1338,29 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Gets Max count of threads can be used at rebalancing.
+     * Minimum is 1.
+     * @return count.
+     */
+    public int getRebalanceThreadPoolSize() {
+        return rebalanceThreadPoolSize;
+    }
+
+    /**
+     * Sets Max count of threads can be used at rebalancing.
+     *
+     * Default is {@code 1} which has minimal impact on the operation of the grid.
+     *
+     * @param size Size.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setRebalanceThreadPoolSize(int size) {
+        this.rebalanceThreadPoolSize = size;
+
+        return this;
+    }
+
+    /**
      * Returns a collection of life-cycle beans. These beans will be automatically
      * notified of grid life-cycle events. Use life-cycle beans whenever you
      * want to perform certain logic before and after grid startup and stopping
@@ -2383,4 +2413,4 @@ public class IgniteConfiguration {
     @Override public String toString() {
         return S.toString(IgniteConfiguration.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 8603055..dd129da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -735,6 +735,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ackEnvironmentVariables();
         ackCacheConfiguration();
         ackP2pConfiguration();
+        ackRebalanceConfiguration();
 
         // Run background network diagnostics.
         GridDiagnostic.runBackgroundCheck(gridName, execSvc, log);
@@ -2147,6 +2148,26 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     /**
      *
      */
+    private void ackRebalanceConfiguration() throws IgniteCheckedException {
+        if (cfg.getSystemThreadPoolSize() <= cfg.getRebalanceThreadPoolSize())
+            throw new IgniteCheckedException("Rebalance thread pool size exceed or equals System thread pool size. " +
+                "Change IgniteConfiguration.rebalanceThreadPoolSize property before next start.");
+
+        if (cfg.getRebalanceThreadPoolSize() < 1)
+            throw new IgniteCheckedException("Rebalance thread pool size minimal allowed value is 1. " +
+                "Change IgniteConfiguration.rebalanceThreadPoolSize property before next start.");
+
+        for (CacheConfiguration ccfg : cfg.getCacheConfiguration()){
+            if (ccfg.getRebalanceBatchesPrefetchCount() < 1)
+                throw new IgniteCheckedException("Rebalance batches prefetch count minimal allowed value is 1. " +
+                    "Change CacheConfiguration.rebalanceBatchesPrefetchCount property before next start. " +
+                    "[cache="+ccfg.getName()+"]");
+        }
+    }
+
+    /**
+     *
+     */
     private void ackCacheConfiguration() {
         CacheConfiguration[] cacheCfgs = cfg.getCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 02b28c5..7d2b2dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -2035,6 +2035,7 @@ public class IgnitionEx {
             cache.setAffinity(new RendezvousAffinityFunction(false, 20));
             cache.setNodeFilter(CacheConfiguration.ALL_NODES);
             cache.setStartSize(300);
+            cache.setRebalanceOrder(-2);//Prior to other system caches.
 
             return cache;
         }
@@ -2055,6 +2056,7 @@ public class IgnitionEx {
             cache.setWriteSynchronizationMode(FULL_SYNC);
             cache.setAffinity(new RendezvousAffinityFunction(false, 100));
             cache.setNodeFilter(CacheConfiguration.ALL_NODES);
+            cache.setRebalanceOrder(-1);//Prior to user caches.
 
             return cache;
         }
@@ -2075,6 +2077,7 @@ public class IgnitionEx {
             ccfg.setWriteSynchronizationMode(FULL_SYNC);
             ccfg.setCacheMode(cfg.getCacheMode());
             ccfg.setNodeFilter(CacheConfiguration.ALL_NODES);
+            ccfg.setRebalanceOrder(-1);//Prior to user caches.
 
             if (cfg.getCacheMode() == PARTITIONED)
                 ccfg.setBackups(cfg.getBackups());

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 079015c..ae8c753 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -74,6 +74,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
 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.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleRequest;
@@ -684,7 +685,12 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
-            // [-3..112] - this
+            case 114:
+                msg = new GridDhtPartitionSupplyMessageV2();
+
+                break;
+
+            // [-3..114] - this
             // [120..123] - DR
             // [-4..-22] - SQL
             default:
@@ -722,4 +728,4 @@ public class GridIoMessageFactory implements MessageFactory {
 
         CUSTOM.put(type, c);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 082f330..2334780 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -45,7 +45,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNe
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse;
@@ -269,7 +268,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             unmarshall(nodeId, cacheMsg);
 
             if (cacheMsg.classError() != null)
-                processFailedMessage(nodeId, cacheMsg);
+                processFailedMessage(nodeId, cacheMsg, c);
             else
                 processMessage(nodeId, cacheMsg, c);
         }
@@ -313,7 +312,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param msg Message.
      * @throws IgniteCheckedException If failed.
      */
-    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
+    private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInClosure<UUID, GridCacheMessage> c)
+        throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 
         switch (msg.directType()) {
@@ -412,9 +412,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             break;
 
             case 45: {
-                GridDhtPartitionSupplyMessage req = (GridDhtPartitionSupplyMessage)msg;
-
-                U.error(log, "Supply message cannot be unmarshalled.", req.classError());
+                processMessage(nodeId,msg,c);// Will be handled by Rebalance Demander.
             }
 
             break;
@@ -517,6 +515,12 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
+            case 114: {
+                processMessage(nodeId,msg,c);// Will be handled by Rebalance Demander.
+            }
+
+            break;
+
             default:
                 throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message="
                     + msg + "]");
@@ -528,8 +532,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param msg Message.
      * @param c Closure.
      */
-    private void processMessage(UUID nodeId, GridCacheMessage msg,
-        IgniteBiInClosure<UUID, GridCacheMessage> c) {
+    private void processMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInClosure<UUID, GridCacheMessage> c) {
         try {
             // We will not end up with storing a bunch of new UUIDs
             // in each cache entry, since node ID is stored in NIO session

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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 2111594..ca0995a 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
@@ -456,7 +456,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (cctx.swap().offheapSwapEvict(key, entry, partition(), evictVer)) {
                 assert !hasValueUnlocked() : this;
 
-                obsolete = markObsolete0(obsoleteVer, false);
+                obsolete = markObsolete0(obsoleteVer, false, null);
 
                 assert obsolete : this;
             }
@@ -1364,7 +1364,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             synchronized (this) {
                 // If entry is still removed.
                 if (newVer == ver) {
-                    if (obsoleteVer == null || !(marked = markObsolete0(obsoleteVer, true))) {
+                    if (obsoleteVer == null || !(marked = markObsolete0(obsoleteVer, true, null))) {
                         if (log.isDebugEnabled())
                             log.debug("Entry could not be marked obsolete (it is still used): " + this);
                     }
@@ -2481,7 +2481,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 try {
                     if ((!hasReaders() || readers)) {
                         // markObsolete will clear the value.
-                        if (!(marked = markObsolete0(ver, true))) {
+                        if (!(marked = markObsolete0(ver, true, null))) {
                             if (log.isDebugEnabled())
                                 log.debug("Entry could not be marked obsolete (it is still used): " + this);
 
@@ -2539,7 +2539,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         boolean obsolete;
 
         synchronized (this) {
-            obsolete = markObsolete0(ver, true);
+            obsolete = markObsolete0(ver, true, null);
         }
 
         if (obsolete)
@@ -2572,7 +2572,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         }
                     }
                     else
-                        obsolete = markObsolete0(ver, true);
+                        obsolete = markObsolete0(ver, true, null);
                 }
             }
         }
@@ -2600,7 +2600,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (!this.ver.equals(ver))
                 return false;
 
-            marked = markObsolete0(ver, true);
+            marked = markObsolete0(ver, true, null);
         }
 
         if (marked)
@@ -2623,9 +2623,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      *
      * @param ver Version.
      * @param clear {@code True} to clear.
+     * @param extras Predefined extras.
      * @return {@code True} if entry is obsolete, {@code false} if entry is still used by other threads or nodes.
      */
-    protected final boolean markObsolete0(GridCacheVersion ver, boolean clear) {
+    protected final boolean markObsolete0(GridCacheVersion ver, boolean clear, GridCacheObsoleteEntryExtras extras) {
         assert Thread.holdsLock(this);
 
         if (evictionDisabled()) {
@@ -2646,7 +2647,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (mvcc == null || mvcc.isEmpty(ver)) {
                 obsoleteVer = ver;
 
-                obsoleteVersionExtras(obsoleteVer);
+                obsoleteVersionExtras(obsoleteVer, extras);
 
                 if (clear)
                     value(null);
@@ -2989,7 +2990,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 synchronized (this) {
                     if (checkExpired()) {
-                        rmv = markObsolete0(cctx.versions().next(this.ver), true);
+                        rmv = markObsolete0(cctx.versions().next(this.ver), true, null);
 
                         return null;
                     }
@@ -3465,7 +3466,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             }
                         }
                         else {
-                            if (markObsolete0(obsoleteVer, true))
+                            if (markObsolete0(obsoleteVer, true, null))
                                 obsolete = true; // Success, will return "true".
                         }
                     }
@@ -3793,7 +3794,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                     CacheObject prev = saveOldValueUnlocked(false);
 
-                    if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
+                    if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) {
                         if (swap) {
                             if (!isStartVersion()) {
                                 try {
@@ -3847,7 +3848,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                         CacheObject prevVal = saveValueForIndexUnlocked();
 
-                        if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
+                        if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) {
                             if (swap) {
                                 if (!isStartVersion()) {
                                     try {
@@ -3923,7 +3924,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         GridCacheBatchSwapEntry ret = null;
 
         try {
-            if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
+            if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) {
                 if (!isStartVersion() && hasValueUnlocked()) {
                     if (cctx.offheapTiered() && hasOffHeapPointer()) {
                         if (cctx.swap().offheapEvictionEnabled())
@@ -3982,7 +3983,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     return false;
 
                 if (checkExpired()) {
-                    rmv = markObsolete0(cctx.versions().next(this.ver), true);
+                    rmv = markObsolete0(cctx.versions().next(this.ver), true, null);
 
                     return false;
                 }
@@ -4095,9 +4096,12 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /**
      * @param obsoleteVer Obsolete version.
      */
-    protected void obsoleteVersionExtras(@Nullable GridCacheVersion obsoleteVer) {
-        extras = (extras != null) ? extras.obsoleteVersion(obsoleteVer) : obsoleteVer != null ?
-            new GridCacheObsoleteEntryExtras(obsoleteVer) : null;
+    protected void obsoleteVersionExtras(@Nullable GridCacheVersion obsoleteVer, GridCacheObsoleteEntryExtras ext) {
+        extras = (extras != null) ?
+            extras.obsoleteVersion(obsoleteVer) :
+            obsoleteVer != null ?
+                (ext != null) ? ext : new GridCacheObsoleteEntryExtras(obsoleteVer) :
+                null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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 c8ee6e3..479a0b6 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
@@ -23,10 +23,12 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.Queue;
+import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -49,9 +51,11 @@ 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.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
 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;
@@ -65,8 +69,10 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter;
 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.CI2;
 import org.apache.ignite.internal.util.typedef.F;
 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;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
@@ -75,6 +81,7 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
+import org.jsr166.ConcurrentLinkedDeque8;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE;
@@ -85,6 +92,7 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader.DFLT_PRELOAD_RESEND_TIMEOUT;
@@ -132,6 +140,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /** */
     private GridFutureAdapter<?> reconnectExchangeFut;
 
+    /** */
+    private final Queue<Callable<Boolean>> rebalanceQ = new ConcurrentLinkedDeque8<>();
+
     /**
      * Partition map futures.
      * This set also contains already completed exchange futures to address race conditions when coordinator
@@ -309,6 +320,33 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         exchWorker.futQ.addFirst(fut);
 
+        if (!cctx.kernalContext().clientNode()) {
+            for (int cnt = 0; cnt < cctx.gridConfig().getRebalanceThreadPoolSize(); cnt++) {
+                final int idx = cnt;
+
+                cctx.io().addOrderedHandler(rebalanceTopic(cnt), new CI2<UUID, GridCacheMessage>() {
+                    @Override public void apply(final UUID id, final GridCacheMessage m) {
+                        if (!enterBusy())
+                            return;
+
+                        try {
+                            if (m instanceof GridDhtPartitionSupplyMessageV2)
+                                cctx.cacheContext(m.cacheId).preloader().handleSupplyMessage(
+                                    idx, id, (GridDhtPartitionSupplyMessageV2)m);
+                            else if (m instanceof GridDhtPartitionDemandMessage)
+                                cctx.cacheContext(m.cacheId).preloader().handleDemandMessage(
+                                    idx, id, (GridDhtPartitionDemandMessage)m);
+                            else
+                                log.error("Unsupported message type: " + m.getClass().getName());
+                        }
+                        finally {
+                            leaveBusy();
+                        }
+                    }
+                });
+            }
+        }
+
         new IgniteThread(cctx.gridName(), "exchange-worker", exchWorker).start();
 
         if (reconnect) {
@@ -368,6 +406,14 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         }
     }
 
+    /**
+     * @param idx Index.
+     * @return Topic for index.
+     */
+    public static Object rebalanceTopic(int idx) {
+        return TOPIC_CACHE.topic("Rebalance", idx);
+    }
+
     /** {@inheritDoc} */
     @Override protected void onKernalStop0(boolean cancel) {
         cctx.gridEvents().removeLocalEventListener(discoLsnr);
@@ -392,6 +438,11 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         for (AffinityReadyFuture f : readyFuts.values())
             f.onDone(stopErr);
 
+        if (!cctx.kernalContext().clientNode()) {
+            for (int cnt = 0; cnt < cctx.gridConfig().getRebalanceThreadPoolSize(); cnt++)
+                cctx.io().removeOrderedHandler(rebalanceTopic(cnt));
+        }
+
         U.cancel(exchWorker);
 
         if (log.isDebugEnabled())
@@ -1015,6 +1066,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         }
 
         dumpPendingObjects();
+
+        for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+            cacheCtx.preloader().dumpDebugInfo();
+        }
     }
 
     /**
@@ -1127,9 +1182,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
             boolean startEvtFired = false;
 
+            int cnt = 0;
+
+            IgniteInternalFuture asyncStartFut = null;
+
             while (!isCancelled()) {
                 GridDhtPartitionsExchangeFuture exchFut = null;
 
+                cnt++;
+
                 try {
                     boolean preloadFinished = true;
 
@@ -1244,12 +1305,111 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     }
 
                     if (assignsMap != null) {
+                        int size = assignsMap.size();
+
+                        rebalanceQ.clear();
+
+                        NavigableMap<Integer, List<Integer>> orderMap = new TreeMap<>();
+
                         for (Map.Entry<Integer, GridDhtPreloaderAssignments> e : assignsMap.entrySet()) {
                             int cacheId = e.getKey();
 
                             GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
 
-                            cacheCtx.preloader().addAssignments(e.getValue(), forcePreload);
+                            int order = cacheCtx.config().getRebalanceOrder();
+
+                            if (orderMap.get(order) == null)
+                                orderMap.put(order, new ArrayList<Integer>(size));
+
+                            orderMap.get(order).add(cacheId);
+                        }
+
+                        Callable<Boolean> marshR = null;
+                        List<Callable<Boolean>> orderedRs = new ArrayList<>(size);
+
+                        //Ordered rebalance scheduling.
+                        for (Integer order : orderMap.keySet()) {
+                            for (Integer cacheId : orderMap.get(order)) {
+                                GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
+
+                                List<String> waitList = new ArrayList<>(size - 1);
+
+                                for (List<Integer> cIds : orderMap.headMap(order).values()) {
+                                    for (Integer cId : cIds) {
+                                        waitList.add(cctx.cacheContext(cId).name());
+                                    }
+                                }
+
+                                Callable<Boolean> r = cacheCtx.preloader().addAssignments(
+                                    assignsMap.get(cacheId), forcePreload, waitList, cnt);
+
+                                if (r != null) {
+                                    U.log(log, "Cache rebalancing scheduled: [cache=" + cacheCtx.name() +
+                                        ", waitList=" + waitList.toString() + "]");
+
+                                    if (cacheId == CU.cacheId(GridCacheUtils.MARSH_CACHE_NAME))
+                                        marshR = r;
+                                    else
+                                        orderedRs.add(r);
+                                }
+                            }
+                        }
+
+                        if (asyncStartFut != null)
+                            asyncStartFut.get(); // Wait for thread stop.
+
+                        rebalanceQ.addAll(orderedRs);
+
+                        if (marshR != null || !rebalanceQ.isEmpty()) {
+                            if (futQ.isEmpty()) {
+                                U.log(log, "Starting caches rebalancing [top=" + exchFut.topologyVersion() + "]");
+
+                                if (marshR != null)
+                                    try {
+                                        marshR.call(); //Marshaller cache rebalancing launches in sync way.
+                                    }
+                                    catch (Exception ex) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Failed to send initial demand request to node");
+
+                                        continue;
+                                    }
+
+                                final GridFutureAdapter fut = new GridFutureAdapter();
+
+                                asyncStartFut = fut;
+
+                                cctx.kernalContext().closure().callLocalSafe(new GPC<Boolean>() {
+                                    @Override public Boolean call() {
+                                        try {
+                                            while (true) {
+                                                Callable<Boolean> r = rebalanceQ.poll();
+
+                                                if (r == null)
+                                                    return false;
+
+                                                if (!r.call())
+                                                    return false;
+                                            }
+                                        }
+                                        catch (Exception ex) {
+                                            if (log.isDebugEnabled())
+                                                log.debug("Failed to send initial demand request to node");
+
+                                            return false;
+                                        }
+                                        finally {
+                                            fut.onDone();
+                                        }
+                                    }
+                                }, /*system pool*/ true);
+                            }
+                            else {
+                                U.log(log, "Obsolete exchange, skipping rebalancing [top=" + exchFut.topologyVersion() + "]");
+                            }
+                        }
+                        else {
+                            U.log(log, "Nothing scheduled, skipping rebalancing [top=" + exchFut.topologyVersion() + "]");
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 1edaef2..1658a89 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -18,9 +18,14 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -90,8 +95,11 @@ public interface GridCachePreloader {
      *
      * @param assignments Assignments to add.
      * @param forcePreload Force preload flag.
+     * @param caches Rebalancing of these caches will be finished before this started.
+     * @param cnt Counter.
      */
-    public void addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload);
+    public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
+        Collection<String> caches, int cnt);
 
     /**
      * @param p Preload predicate.
@@ -115,6 +123,17 @@ public interface GridCachePreloader {
     public IgniteInternalFuture<?> syncFuture();
 
     /**
+     * @return Future which will complete when preloading finishes on current topology.
+     *
+     * Future result is {@code true} in case rebalancing successfully finished at current topology.
+     * Future result is {@code false} in case rebalancing cancelled or finished with missed partitions and will be
+     * restarted at current or pending topology.
+     *
+     * Note that topology change creates new futures and finishes previous.
+     */
+    public IgniteInternalFuture<Boolean> rebalanceFuture();
+
+    /**
      * Requests that preloader sends the request for the key.
      *
      * @param keys Keys to request.
@@ -134,7 +153,39 @@ public interface GridCachePreloader {
     public void unwindUndeploys();
 
     /**
+     * Handles Supply message.
+     *
+     * @param idx Index.
+     * @param id Node Id.
+     * @param s Supply message.
+     */
+    public void handleSupplyMessage(int idx, UUID id, final GridDhtPartitionSupplyMessageV2 s);
+
+    /**
+     * Handles Demand message.
+     *
+     * @param idx Index.
+     * @param id Node Id.
+     * @param d Demand message.
+     */
+    public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage d);
+
+    /**
+     * Evicts partition asynchronously.
+     *
+     * @param part Partition.
+     */
+    public void evictPartitionAsync(GridDhtLocalPartition part);
+
+    /**
+     * Handles new topology.
+     *
+     * @param topVer Topology version.
+     */
+    public void onTopologyChanged(AffinityTopologyVersion topVer);
+
+    /**
      * Dumps debug information.
      */
     public void dumpDebugInfo();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index 4ec6749..9c0e9f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -18,11 +18,16 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -36,7 +41,7 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     /** Cache context. */
     protected final GridCacheContext<?, ?> cctx;
 
-    /** Logger.*/
+    /** Logger. */
     protected final IgniteLogger log;
 
     /** Affinity. */
@@ -113,12 +118,28 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Boolean> rebalanceFuture() {
+        return new GridFinishedFuture<>(true);
+    }
+
+    /** {@inheritDoc} */
     @Override public void unwindUndeploys() {
         cctx.deploy().unwind(cctx);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Object> request(Collection<KeyCacheObject> keys, AffinityTopologyVersion topVer) {
+    @Override public void handleSupplyMessage(int idx, UUID id, GridDhtPartitionSupplyMessageV2 s) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage d) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Object> request(Collection<KeyCacheObject> keys,
+        AffinityTopologyVersion topVer) {
         return new GridFinishedFuture<>();
     }
 
@@ -143,7 +164,18 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
-    @Override public void addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload) {
+    @Override public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
+        Collection<String> caches, int cnt) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void evictPartitionAsync(GridDhtLocalPartition part) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onTopologyChanged(AffinityTopologyVersion topVer) {
         // No-op.
     }
 
@@ -151,4 +183,4 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     @Override public void dumpDebugInfo() {
         // No-op.
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/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 b2bb6ff..f54f63e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -31,9 +31,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
-import java.util.NavigableMap;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -98,7 +96,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManag
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.F0;
-import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -160,12 +157,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Map of proxies. */
     private final Map<String, IgniteCacheProxy<?, ?>> jCacheProxies;
 
-    /** Map of preload finish futures grouped by preload order. */
-    private final NavigableMap<Integer, IgniteInternalFuture<?>> preloadFuts;
-
-    /** Maximum detected rebalance order. */
-    private int maxRebalanceOrder;
-
     /** Caches stop sequence. */
     private final Deque<String> stopSeq;
 
@@ -207,8 +198,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         caches = new ConcurrentHashMap<>();
         jCacheProxies = new ConcurrentHashMap<>();
-        preloadFuts = new TreeMap<>();
-
         stopSeq = new LinkedList<>();
     }
 
@@ -378,10 +367,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             U.warn(log, "AffinityFunction configuration parameter will be ignored for local cache [cacheName=" +
                 U.maskName(cc.getName()) + ']');
 
-        if (cc.getRebalanceMode() != CacheRebalanceMode.NONE) {
-            assertParameter(cc.getRebalanceThreadPoolSize() > 0, "rebalanceThreadPoolSize > 0");
+        if (cc.getRebalanceMode() != CacheRebalanceMode.NONE)
             assertParameter(cc.getRebalanceBatchSize() > 0, "rebalanceBatchSize > 0");
-        }
 
         if (cc.getCacheMode() == PARTITIONED || cc.getCacheMode() == REPLICATED) {
             if (cc.getAtomicityMode() == ATOMIC && cc.getWriteSynchronizationMode() == FULL_ASYNC)
@@ -591,8 +578,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     "Deployment mode for cache is not CONTINUOUS or SHARED.");
         }
 
-        maxRebalanceOrder = validatePreloadOrder(ctx.config().getCacheConfiguration());
-
         ctx.discovery().setCustomEventListener(DynamicCacheChangeBatch.class,
             new CustomEventListener<DynamicCacheChangeBatch>() {
                 @Override public void onCustomEvent(ClusterNode snd,
@@ -820,31 +805,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
             mgr.onKernalStart(false);
 
-        for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
-            GridCacheAdapter cache = e.getValue();
-
-            if (maxRebalanceOrder > 0) {
-                CacheConfiguration cfg = cache.configuration();
-
-                int order = cfg.getRebalanceOrder();
-
-                if (order > 0 && order != maxRebalanceOrder && cfg.getCacheMode() != LOCAL) {
-                    GridCompoundFuture fut = (GridCompoundFuture)preloadFuts.get(order);
-
-                    if (fut == null) {
-                        fut = new GridCompoundFuture<>();
-
-                        preloadFuts.put(order, fut);
-                    }
-
-                    fut.add(cache.preloader().syncFuture());
-                }
-            }
-        }
-
-        for (IgniteInternalFuture<?> fut : preloadFuts.values())
-            ((GridCompoundFuture<Object, Object>)fut).markInitialized();
-
         for (GridCacheAdapter<?, ?> cache : caches.values())
             onKernalStart(cache);
 
@@ -2791,19 +2751,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Gets preload finish future for preload-ordered cache with given order. I.e. will get compound preload future
-     * with maximum order less than {@code order}.
-     *
-     * @param order Cache order.
-     * @return Compound preload future or {@code null} if order is minimal order found.
-     */
-    @Nullable public IgniteInternalFuture<?> orderedPreloadFuture(int order) {
-        Map.Entry<Integer, IgniteInternalFuture<?>> entry = preloadFuts.lowerEntry(order);
-
-        return entry == null ? null : entry.getValue();
-    }
-
-    /**
      * @param spaceName Space name.
      * @param keyBytes Key bytes.
      * @param valBytes Value bytes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index 1b2d834..392ad6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException;
+import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.lang.GridPlainRunnable;
@@ -554,7 +555,11 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
      * @return {@code True} if entry was not being used, passed the filter and could be removed.
      * @throws IgniteCheckedException If failed to remove from swap.
      */
-    public boolean clearInternal(GridCacheVersion ver, boolean swap) throws IgniteCheckedException {
+    public boolean clearInternal(
+        GridCacheVersion ver,
+        boolean swap,
+        GridCacheObsoleteEntryExtras extras
+    ) throws IgniteCheckedException {
         boolean rmv = false;
 
         try {
@@ -563,7 +568,7 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
 
                 // Call markObsolete0 to avoid recursive calls to clear if
                 // we are clearing dht local partition (onMarkedObsolete should not be called).
-                if (!markObsolete0(ver, false)) {
+                if (!markObsolete0(ver, false, extras)) {
                     if (log.isDebugEnabled())
                         log.debug("Entry could not be marked obsolete (it is still used or has readers): " + this);
 
@@ -819,4 +824,4 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
             return S.toString(ReaderId.class, this);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 749d06a..1516ee4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -17,6 +17,17 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.AtomicStampedReference;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -27,10 +38,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedExceptio
 import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader;
+import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.GridCircularBuffer;
-import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -38,7 +49,6 @@ 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.GPC;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -46,18 +56,6 @@ import org.jetbrains.annotations.NotNull;
 import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.LongAdder8;
 
-import javax.cache.CacheException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.atomic.AtomicStampedReference;
-import java.util.concurrent.locks.ReentrantLock;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_UNLOADED;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.EVICTED;
@@ -286,7 +284,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
         }
 
         // Attempt to evict.
-        tryEvict(true);
+        tryEvict();
     }
 
     /**
@@ -411,7 +409,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
             // Decrement reservations.
             if (state.compareAndSet(s, s, reservations, --reservations)) {
-                tryEvict(true);
+                tryEvict();
 
                 break;
             }
@@ -477,10 +475,10 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
     /**
      * @param updateSeq Update sequence.
-     * @return Future for evict attempt.
      */
-    IgniteInternalFuture<Boolean> tryEvictAsync(boolean updateSeq) {
+    void tryEvictAsync(boolean updateSeq) {
         if (map.isEmpty() && !GridQueryProcessor.isEnabled(cctx.config()) &&
+            state.getReference() == RENTING && state.getStamp() == 0 &&
             state.compareAndSet(RENTING, EVICTED, 0, 0)) {
             if (log.isDebugEnabled())
                 log.debug("Evicted partition: " + this);
@@ -497,15 +495,9 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
             ((GridDhtPreloader)cctx.preloader()).onPartitionEvicted(this, updateSeq);
 
             clearDeferredDeletes();
-
-            return new GridFinishedFuture<>(true);
         }
-
-        return cctx.closures().callLocalSafe(new GPC<Boolean>() {
-            @Override public Boolean call() {
-                return tryEvict(true);
-            }
-        }, /*system pool*/ true);
+        else
+            cctx.preloader().evictPartitionAsync(this);
     }
 
     /**
@@ -521,12 +513,11 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     }
 
     /**
-     * @param updateSeq Update sequence.
-     * @return {@code True} if entry has been transitioned to state EVICTED.
+     *
      */
-    boolean tryEvict(boolean updateSeq) {
+    public void tryEvict() {
         if (state.getReference() != RENTING || state.getStamp() != 0 || groupReserved())
-            return false;
+            return;
 
         // Attempt to evict partition entries from cache.
         clearAll();
@@ -545,14 +536,10 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
             rent.onDone();
 
-            ((GridDhtPreloader)cctx.preloader()).onPartitionEvicted(this, updateSeq);
+            ((GridDhtPreloader)cctx.preloader()).onPartitionEvicted(this, true);
 
             clearDeferredDeletes();
-
-            return true;
         }
-
-        return false;
     }
 
     /**
@@ -592,7 +579,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      *
      */
     void onUnlock() {
-        tryEvict(true);
+        tryEvict();
     }
 
     /**
@@ -640,6 +627,8 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                 it = F.concat(it, unswapIt);
         }
 
+        GridCacheObsoleteEntryExtras extras = new GridCacheObsoleteEntryExtras(clearVer);
+
         try {
             while (it.hasNext()) {
                 GridDhtCacheEntry cached = null;
@@ -647,7 +636,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                 try {
                     cached = it.next();
 
-                    if (cached.clearInternal(clearVer, swap)) {
+                    if (cached.clearInternal(clearVer, swap, extras)) {
                         map.remove(cached.key(), cached);
 
                         if (!cached.isInternal()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
index 756326e..d12247e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
@@ -239,7 +239,7 @@ public class GridDhtPartitionsReservation implements GridReservable {
     }
 
     /**
-     * Must be checked in {@link GridDhtLocalPartition#tryEvict(boolean)}.
+     * Must be checked in {@link GridDhtLocalPartition#tryEvict()}.
      * If returns {@code true} this reservation object becomes invalid and partitions
      * can be evicted or at least cleared.
      * Also this means that after returning {@code true} here method {@link #reserve()} can not

http://git-wip-us.apache.org/repos/asf/ignite/blob/7dfaa3b0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
index 7609d98..53c3d90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
@@ -68,8 +68,6 @@ public class GridDhtPartitionDemandMessage extends GridCacheMessage {
      * @param topVer Topology version.
      */
     GridDhtPartitionDemandMessage(long updateSeq, @NotNull AffinityTopologyVersion topVer, int cacheId) {
-        assert updateSeq > 0;
-
         this.cacheId = cacheId;
         this.updateSeq = updateSeq;
         this.topVer = topVer;
@@ -116,6 +114,13 @@ public class GridDhtPartitionDemandMessage extends GridCacheMessage {
     }
 
     /**
+     * @param updateSeq Update sequence.
+     */
+    void updateSequence(long updateSeq) {
+        this.updateSeq = updateSeq;
+    }
+
+    /**
      * @return Update sequence.
      */
     long updateSequence() {
@@ -325,7 +330,8 @@ public class GridDhtPartitionDemandMessage extends GridCacheMessage {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridDhtPartitionDemandMessage.class, this, "partCnt", parts.size(), "super",
-            super.toString());
+        return S.toString(GridDhtPartitionDemandMessage.class, this,
+            "partCnt", parts != null ? parts.size() : 0,
+            "super", super.toString());
     }
 }


[36/46] ignite git commit: ignite-1.5 Minor fix in JDBC POJO store.

Posted by ak...@apache.org.
ignite-1.5 Minor fix in JDBC POJO store.


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

Branch: refs/heads/ignite-1753-1282
Commit: 5887ae4d7a318089bb8b19e56cbecc782be1ba76
Parents: 1f5a409
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Nov 13 10:44:35 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Nov 13 10:44:35 2015 +0700

----------------------------------------------------------------------
 .../store/jdbc/CacheAbstractJdbcStore.java      | 34 +++++++++++---------
 1 file changed, 19 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5887ae4d/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
index 6e27b9a..6e19234 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
@@ -354,22 +354,26 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
         Transaction tx = ses.transaction();
 
         if (tx != null) {
-            Connection conn = ses.<String, Connection>properties().remove(ATTR_CONN_PROP);
+            Map<String, Connection> sesProps = ses.properties();
 
-            assert conn != null;
+            Connection conn = sesProps.get(ATTR_CONN_PROP);
 
-            try {
-                if (commit)
-                    conn.commit();
-                else
-                    conn.rollback();
-            }
-            catch (SQLException e) {
-                throw new CacheWriterException(
-                    "Failed to end transaction [xid=" + tx.xid() + ", commit=" + commit + ']', e);
-            }
-            finally {
-                U.closeQuiet(conn);
+            if (conn != null) {
+                sesProps.remove(ATTR_CONN_PROP);
+
+                try {
+                    if (commit)
+                        conn.commit();
+                    else
+                        conn.rollback();
+                }
+                catch (SQLException e) {
+                    throw new CacheWriterException(
+                            "Failed to end transaction [xid=" + tx.xid() + ", commit=" + commit + ']', e);
+                }
+                finally {
+                    U.closeQuiet(conn);
+                }
             }
 
             if (log.isDebugEnabled())
@@ -1821,4 +1825,4 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
             }
         }
     }
-}
\ No newline at end of file
+}


[19/46] ignite git commit: IGNITE-1681: Dogpile effect tests for CacheStoreBalancingWrapper

Posted by ak...@apache.org.
IGNITE-1681: Dogpile effect tests for CacheStoreBalancingWrapper


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

Branch: refs/heads/ignite-1753-1282
Commit: d70f7eda0492857ffd4879c311c814867552070e
Parents: 7ba2efb
Author: Andrey Gura <ag...@gridgain.com>
Authored: Tue Nov 10 13:59:38 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Nov 10 13:59:38 2015 +0300

----------------------------------------------------------------------
 .../store/GridCacheBalancingStoreSelfTest.java  | 181 ++++++++++++++++++-
 1 file changed, 180 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d70f7eda/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
index d41a441..1e3e4b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
@@ -17,10 +17,14 @@
 
 package org.apache.ignite.cache.store;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -29,8 +33,10 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import javax.cache.Cache;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.cache.CacheStoreBalancingWrapper;
 import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -118,6 +124,81 @@ public class GridCacheBalancingStoreSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentLoad() throws Exception {
+        int threads = 5;
+
+        final int keys = 50;
+
+        final CyclicBarrier beforeBarrier = new CyclicBarrier(threads);
+
+        ConcurrentVerifyStore store = new ConcurrentVerifyStore(keys);
+
+        final CacheStoreBalancingWrapper<Integer, Integer> wrapper =new CacheStoreBalancingWrapper<>(store);
+
+        GridTestUtils.runMultiThreaded(new Runnable() {
+            @Override public void run() {
+                for (int i = 0; i < keys; i++) {
+                    try {
+                        beforeBarrier.await();
+                    }
+                    catch (InterruptedException | BrokenBarrierException e) {
+                        throw new RuntimeException(e);
+                    }
+
+                    info("Load key: " + i);
+
+                    wrapper.load(i);
+                }
+            }
+        }, threads, "load-thread");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentLoadAll() throws Exception {
+        int threads = 5;
+
+        final int threshold = 5;
+
+        final int keysCnt = 100;
+
+        final CyclicBarrier beforeBarrier = new CyclicBarrier(threads);
+
+        ConcurrentVerifyStore store = new ConcurrentVerifyStore(keysCnt);
+
+        final CacheStoreBalancingWrapper<Integer, Integer> wrapper = new CacheStoreBalancingWrapper<>(store);
+
+        GridTestUtils.runMultiThreaded(new Runnable() {
+            @Override public void run() {
+                for (int i = 0; i < keysCnt; i += threshold) {
+                    try {
+                        beforeBarrier.await();
+                    }
+                    catch (InterruptedException | BrokenBarrierException e) {
+                        throw new RuntimeException(e);
+                    }
+
+                    List<Integer> keys = new ArrayList<>(threshold);
+
+                    for (int j = i; j < i + threshold; j++)
+                        keys.add(j);
+
+                    info("Load keys: " + keys);
+
+                    wrapper.loadAll(keys, new IgniteBiInClosure<Integer, Integer>() {
+                        @Override public void apply(Integer integer, Integer integer2) {
+                            // No-op.
+                        }
+                    });
+                }
+            }
+        }, threads, "load-thread");
+    }
+
+    /**
      *
      */
     private static class VerifyStore implements CacheStore<Integer, Integer> {
@@ -204,4 +285,102 @@ public class GridCacheBalancingStoreSelfTest extends GridCommonAbstractTest {
             // No-op.
         }
     }
-}
\ No newline at end of file
+
+    /**
+     *
+     */
+    private static class ConcurrentVerifyStore implements CacheStore<Integer, Integer> {
+
+        /** Cnts. */
+        private final AtomicInteger[] cnts;
+
+        /**
+         */
+        private ConcurrentVerifyStore(int keys) {
+            this.cnts = new AtomicInteger[keys];
+
+            for (int i = 0; i < keys; i++)
+                cnts[i] = new AtomicInteger();
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public Integer load(Integer key) {
+            try {
+                U.sleep(500);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                throw new RuntimeException(e);
+            }
+
+            assertEquals("Redundant load call.", 1, cnts[key].incrementAndGet());
+
+            return key;
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, @Nullable Object... args) {
+            // No-op.
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public Map<Integer, Integer> loadAll(Iterable<? extends Integer> keys) {
+            try {
+                U.sleep(500);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                e.printStackTrace();
+            }
+
+            Map<Integer, Integer> loaded = new HashMap<>();
+
+            for (Integer key : keys) {
+                assertEquals("Redundant loadAll call.", 1, cnts[key].incrementAndGet());
+
+                loaded.put(key, key);
+            }
+
+            return loaded;
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry) {
+            // No-op.
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void writeAll(Collection<Cache.Entry<? extends Integer, ? extends Integer>> entries) {
+            // No-op.
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void delete(Object key) {
+            // No-op.
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void deleteAll(Collection<?> keys) {
+            // No-op.
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void sessionEnd(boolean commit) {
+            // No-op.
+        }
+    }
+}


[32/46] ignite git commit: Added tx-getAll PutAll benchmarks

Posted by ak...@apache.org.
Added tx-getAll PutAll benchmarks


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

Branch: refs/heads/ignite-1753-1282
Commit: 847b616f1a15756db8876fc8f88b0629318f14c3
Parents: 967cfcb
Author: ashutak <as...@gridgain.com>
Authored: Wed Nov 11 18:35:02 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Nov 11 18:35:02 2015 +0300

----------------------------------------------------------------------
 .../config/benchmark-multicast.properties       |  5 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  | 75 ++++++++++++++++++++
 .../IgniteAccountSerializableTxBenchmark.java   | 11 ++-
 .../cache/IgniteAccountTxBenchmark.java         | 11 ++-
 .../cache/IgniteGetAllPutAllTxBenchmark.java    | 73 +++++++++++++++++++
 ...IgniteTransactionalWriteInvokeBenchmark.java | 41 +----------
 .../IgniteTransactionalWriteReadBenchmark.java  | 41 +----------
 7 files changed, 166 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/847b616f/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index 536ccf4..577d611 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -110,5 +110,8 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryPutOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-put-offheap-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllBenchmark -sn IgniteNode -ds ${ver}atomic-putAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putAll-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-optimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-pessimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC -txi SERIALIZABLE -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds tx-opt-serializable-getAllPutAll-1-backup,\
 "

http://git-wip-us.apache.org/repos/asf/ignite/blob/847b616f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
new file mode 100644
index 0000000..0b7568b
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
@@ -0,0 +1,75 @@
+/*
+ * 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.yardstick;
+
+import java.util.concurrent.Callable;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cluster.ClusterTopologyException;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.apache.ignite.transactions.TransactionOptimisticException;
+import org.apache.ignite.transactions.TransactionRollbackException;
+
+/**
+ * Utils.
+ */
+public class IgniteBenchmarkUtils {
+    /**
+     * Utility class constructor.
+     */
+    private IgniteBenchmarkUtils() {
+        // No-op.
+    }
+
+    /**
+     * @param igniteTx Ignite transaction.
+     * @param txConcurrency Transaction concurrency.
+     * @param clo Closure.
+     * @return Result of closure execution.
+     * @throws Exception
+     */
+    public static <T> T doInTransaction(IgniteTransactions igniteTx, TransactionConcurrency txConcurrency,
+        TransactionIsolation txIsolation,  Callable<T> clo) throws Exception {
+        while (true) {
+            try (Transaction tx = igniteTx.txStart(txConcurrency, txIsolation)) {
+                T res = clo.call();
+
+                tx.commit();
+
+                return res;
+            }
+            catch (CacheException e) {
+                if (e.getCause() instanceof ClusterTopologyException) {
+                    ClusterTopologyException topEx = (ClusterTopologyException)e.getCause();
+
+                    topEx.retryReadyFuture().get();
+                }
+                else
+                    throw e;
+            }
+            catch (ClusterTopologyException e) {
+                e.retryReadyFuture().get();
+            }
+            catch (TransactionRollbackException | TransactionOptimisticException ignore) {
+                // Safe to retry right away.
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/847b616f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java
index 32e7653..ecb6727 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountSerializableTxBenchmark.java
@@ -31,21 +31,20 @@ import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
  *
  */
 public class IgniteAccountSerializableTxBenchmark extends IgniteAccountTxAbstractBenchmark {
-    /** */
-    private static final int ACCOUNT_NUMBER = 3;
-
     /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
         Set<Integer> accountIds = new HashSet<>();
 
-        while (accountIds.size() < ACCOUNT_NUMBER)
+        int accNum = args.batch();
+
+        while (accountIds.size() < accNum)
             accountIds.add(nextRandom(args.range()));
 
         while (true) {
             try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
                 Map<Integer, Account> accounts = (Map)cache.getAll(accountIds);
 
-                if (accounts.size() != ACCOUNT_NUMBER)
+                if (accounts.size() != accNum)
                     throw new Exception("Failed to find accounts: " + accountIds);
 
                 Integer fromId = accountIds.iterator().next();
@@ -78,4 +77,4 @@ public class IgniteAccountSerializableTxBenchmark extends IgniteAccountTxAbstrac
 
         return true;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/847b616f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java
index 78e675e..de332c1 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteAccountTxBenchmark.java
@@ -30,21 +30,20 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  *
  */
 public class IgniteAccountTxBenchmark extends IgniteAccountTxAbstractBenchmark {
-    /** */
-    private static final int ACCOUNT_NUMBER = 3;
-
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
         Set<Integer> accountIds = new TreeSet<>();
 
-        while (accountIds.size() < ACCOUNT_NUMBER)
+        int accNum = args.batch();
+
+        while (accountIds.size() < accNum)
             accountIds.add(nextRandom(args.range()));
 
         try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) {
             Map<Integer, Account> accounts = (Map)cache.getAll(accountIds);
 
-            if (accounts.size() != ACCOUNT_NUMBER)
+            if (accounts.size() != accNum)
                 throw new Exception("Failed to find accounts: " + accountIds);
 
             Integer fromId = accountIds.iterator().next();
@@ -71,4 +70,4 @@ public class IgniteAccountTxBenchmark extends IgniteAccountTxAbstractBenchmark {
 
         return true;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/847b616f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAllPutAllTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAllPutAllTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAllPutAllTxBenchmark.java
new file mode 100644
index 0000000..a8f6d7b
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAllPutAllTxBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
+
+/**
+ * Ignite benchmark that performs transactional putAll operations.
+ */
+public class IgniteGetAllPutAllTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Integer> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        txs = ignite().transactions();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        final ThreadRange r = threadRange();
+
+        doInTransaction(txs, args.txConcurrency(), args.txIsolation(), new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                SortedMap<Integer, Integer> vals = new TreeMap<>();
+
+                for (int i = 0; i < args.batch(); i++) {
+                    int key = r.nextRandom();
+
+                    vals.put(key, key);
+                }
+
+                cache.getAll(vals.keySet());
+
+                cache.putAll(vals);
+
+                return null;
+            }
+        });
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Integer> cache() {
+        return ignite().cache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/847b616f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
index 1a8ee14..2c4050b 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteInvokeBenchmark.java
@@ -22,17 +22,13 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
-import javax.cache.CacheException;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.cache.CacheEntryProcessor;
-import org.apache.ignite.cluster.ClusterTopologyException;
-import org.apache.ignite.transactions.Transaction;
-import org.apache.ignite.transactions.TransactionRollbackException;
 
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
 import static org.yardstickframework.BenchmarkUtils.println;
 
 /**
@@ -63,7 +59,7 @@ public class IgniteTransactionalWriteInvokeBenchmark extends IgniteFailoverAbstr
 
         final int scenario = nextRandom(2);
 
-        return doInTransaction(ignite(), new Callable<Boolean>() {
+        return doInTransaction(ignite().transactions(), PESSIMISTIC, REPEATABLE_READ, new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
                 final int timeout = args.cacheOperationTimeoutMillis();
 
@@ -127,39 +123,6 @@ public class IgniteTransactionalWriteInvokeBenchmark extends IgniteFailoverAbstr
         });
     }
 
-    /**
-     * @param ignite Ignite instance.
-     * @param clo Closure.
-     * @return Result of closure execution.
-     * @throws Exception
-     */
-    public static <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
-        while (true) {
-            try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                T res = clo.call();
-
-                tx.commit();
-
-                return res;
-            }
-            catch (CacheException e) {
-                if (e.getCause() instanceof ClusterTopologyException) {
-                    ClusterTopologyException topEx = (ClusterTopologyException)e.getCause();
-
-                    topEx.retryReadyFuture().get();
-                }
-                else
-                    throw e;
-            }
-            catch (ClusterTopologyException e) {
-                e.retryReadyFuture().get();
-            }
-            catch (TransactionRollbackException ignore) {
-                // Safe to retry right away.
-            }
-        }
-    }
-
     /** {@inheritDoc} */
     @Override protected String cacheName() {
         return "tx-write-invoke";

http://git-wip-us.apache.org/repos/asf/ignite/blob/847b616f/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteReadBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteReadBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteReadBenchmark.java
index c962749..f278e8a 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteReadBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/failover/IgniteTransactionalWriteReadBenchmark.java
@@ -22,14 +22,10 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
-import javax.cache.CacheException;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cluster.ClusterTopologyException;
-import org.apache.ignite.transactions.Transaction;
-import org.apache.ignite.transactions.TransactionRollbackException;
 
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
 import static org.yardstickframework.BenchmarkUtils.println;
 
 /**
@@ -51,7 +47,7 @@ public class IgniteTransactionalWriteReadBenchmark extends IgniteFailoverAbstrac
         for (int i = 0; i < keys.length; i++)
             keys[i] = "key-" + k + "-" + i;
 
-        return doInTransaction(ignite(), new Callable<Boolean>() {
+        return doInTransaction(ignite().transactions(), PESSIMISTIC, REPEATABLE_READ, new Callable<Boolean>() {
             @Override public Boolean call() throws Exception {
                 Map<String, Long> map = new HashMap<>();
 
@@ -101,39 +97,6 @@ public class IgniteTransactionalWriteReadBenchmark extends IgniteFailoverAbstrac
         });
     }
 
-    /**
-     * @param ignite Ignite instance.
-     * @param clo Closure.
-     * @return Result of closure execution.
-     * @throws Exception
-     */
-    public static <T> T doInTransaction(Ignite ignite, Callable<T> clo) throws Exception {
-        while (true) {
-            try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                T res = clo.call();
-
-                tx.commit();
-
-                return res;
-            }
-            catch (CacheException e) {
-                if (e.getCause() instanceof ClusterTopologyException) {
-                    ClusterTopologyException topEx = (ClusterTopologyException)e.getCause();
-
-                    topEx.retryReadyFuture().get();
-                }
-                else
-                    throw e;
-            }
-            catch (ClusterTopologyException e) {
-                e.retryReadyFuture().get();
-            }
-            catch (TransactionRollbackException ignore) {
-                // Safe to retry right away.
-            }
-        }
-    }
-
     /** {@inheritDoc} */
     @Override protected String cacheName() {
         return "tx-write-read";


[35/46] ignite git commit: ignite-1.5 Added descriptions for missing options and sort them alphabetically.

Posted by ak...@apache.org.
ignite-1.5 Added descriptions for missing options and sort them alphabetically.


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

Branch: refs/heads/ignite-1753-1282
Commit: 1f5a409cc138b7df396bcc78ab166b0b366ad7d6
Parents: 55c227a
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Nov 12 10:00:05 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Nov 12 10:00:05 2015 +0700

----------------------------------------------------------------------
 modules/yardstick/README.txt | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f5a409c/modules/yardstick/README.txt
----------------------------------------------------------------------
diff --git a/modules/yardstick/README.txt b/modules/yardstick/README.txt
index 8cbe3cf..0b8e678 100644
--- a/modules/yardstick/README.txt
+++ b/modules/yardstick/README.txt
@@ -54,21 +54,23 @@ and command line arguments for running Yardstick scripts.
 
 The following Ignite benchmark properties can be defined in the benchmark configuration:
 
-* `-nn <num>` or `--nodeNumber <num>` - Number of nodes (automatically set in `benchmark.properties`), used to wait for the specified number of nodes to start
 * `-b <num>` or `--backups <num>` - Number of backups for every key
 * `-cfg <path>` or `--Config <path>` - Path to Ignite configuration file
-* `-sm <mode>` or `-syncMode <mode>` - Synchronization mode (defined in `CacheWriteSynchronizationMode`)
+* `-cs` or `--cacheStore` - Enable or disable cache store readThrough, writeThrough
 * `-cl` or `--client` - Client flag
 * `-nc` or `--nearCache` - Near cache flag
-* `-wom <mode>` or `--writeOrderMode <mode>` - Write order mode for ATOMIC caches (defined in `CacheAtomicWriteOrderMode`)
-* `-txc <value>` or `--txConcurrency <value>` - Cache transaction concurrency control, either `OPTIMISTIC` or `PESSIMISTIC` (defined in `CacheTxConcurrency`)
-* `-txi <value>` or `--txIsolation <value>` - Cache transaction isolation (defined in `CacheTxIsolation`)
+* `-nn <num>` or `--nodeNumber <num>` - Number of nodes (automatically set in `benchmark.properties`), used to wait for the specified number of nodes to start
+* `-sm <mode>` or `-syncMode <mode>` - Synchronization mode (defined in `CacheWriteSynchronizationMode`)
 * `-ot` or `--offheapTiered` - Flag indicating whether tiered off-heap mode is on
 * `-ov` or `--offheapValuesOnly` - Flag indicating whether off-heap mode is on and only cache values are stored off-heap
-* `-rtp <num>`  or `--restPort <num>` - REST TCP port, indicates that a Ignite node is ready to process Ignite Clients
+* `-r <num>` or `--range` - Range of keys that are randomly generated for cache operations
 * `-rth <host>` or `--restHost <host>` - REST TCP host
+* `-rtp <num>` or `--restPort <num>` - REST TCP port, indicates that a Ignite node is ready to process Ignite Clients
 * `-ss` or `--syncSend` - Flag indicating whether synchronous send is used in `TcpCommunicationSpi`
-* `-r <num>` or `--range` - Range of keys that are randomly generated for cache operations
+* `-txc <value>` or `--txConcurrency <value>` - Cache transaction concurrency control, either `OPTIMISTIC` or `PESSIMISTIC` (defined in `CacheTxConcurrency`)
+* `-txi <value>` or `--txIsolation <value>` - Cache transaction isolation (defined in `CacheTxIsolation`)
+* `-wb` or `--writeBehind` - Enable or disable writeBehind for cache store
+* `-wom <mode>` or `--writeOrderMode <mode>` - Write order mode for ATOMIC caches (defined in `CacheAtomicWriteOrderMode`)
 
 For example if we need to run 2 `IgniteNode` servers on localhost with `PutBenchmark` benchmark on localhost,
 with number of backups set to 1, synchronization mode set to `PRIMARY_SYNC`, then the following configuration


[27/46] ignite git commit: Merge remote-tracking branch 'apache/ignite-1.5' into ignite-1.5

Posted by ak...@apache.org.
Merge remote-tracking branch 'apache/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1753-1282
Commit: 388a8921032af9e055a021e6f859c030eaaa5ea8
Parents: dbadb45 cb28819
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Nov 10 16:48:14 2015 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Nov 10 16:48:14 2015 -0800

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |  34 ++--
 .../dht/preloader/GridDhtPartitionDemander.java |  25 ++-
 .../GridCacheAtomicReferenceImpl.java           |   2 +-
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../org/apache/ignite/stream/StreamAdapter.java |   2 +-
 .../store/GridCacheBalancingStoreSelfTest.java  | 181 ++++++++++++++++++-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |  20 +-
 modules/rest-http/pom.xml                       |  11 ++
 .../yardstick/config/ignite-store-config.xml    |  31 ++--
 modules/zookeeper/pom.xml                       |   2 +-
 10 files changed, 265 insertions(+), 45 deletions(-)
----------------------------------------------------------------------



[26/46] ignite git commit: Fixed typos

Posted by ak...@apache.org.
Fixed typos


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

Branch: refs/heads/ignite-1753-1282
Commit: dbadb45a09dd2a8a2e7e153b9862f71419ea5ecc
Parents: 50ede23
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Nov 10 16:01:46 2015 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Nov 10 16:45:07 2015 -0800

----------------------------------------------------------------------
 .../ignite/cache/store/CacheStoreSessionListener.java       | 9 +++++----
 .../cache/store/jdbc/CacheJdbcStoreSessionListener.java     | 8 ++++----
 2 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dbadb45a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
index 8215756..9496d20 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
@@ -35,11 +35,12 @@ import org.apache.ignite.configuration.IgniteConfiguration;
  * rollback when session is finished.
  * <p>
  * Cache store session listener allows to implement this and other
- * scenarios providing to callback methods:
+ * scenarios providing two callback methods:
  * <ul>
  *     <li>
  *         {@link #onSessionStart(CacheStoreSession)} - called
- *         before any store operation within a session is invoked.
+ *         when a session is created prior to all operations
+ *         within his session.
  *     </li>
  *     <li>
  *         {@link #onSessionEnd(CacheStoreSession, boolean)} - called
@@ -47,7 +48,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
  *     </li>
  * </ul>
  * <h2>Implementations</h2>
- * Ignites provides several out-of-the-box implementations
+ * Ignite provides several out-of-the-box implementations
  * of session listener (refer to individual JavaDocs for more
  * details):
  * <ul>
@@ -130,4 +131,4 @@ public interface CacheStoreSessionListener {
      *      should commit, {@code false} for rollback.
      */
     public void onSessionEnd(CacheStoreSession ses, boolean commit);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dbadb45a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
index 2c70d26..6be237e 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
@@ -37,12 +37,12 @@ import org.apache.ignite.lifecycle.LifecycleAware;
  * back) it when session ends.
  * <p>
  * The connection is saved as a store session
-  * {@link CacheStoreSession#attachment() attachment}.
+ * {@link CacheStoreSession#attachment() attachment}.
  * The listener guarantees that the connection will be
  * available for any store operation. If there is an
  * ongoing cache transaction, all operations within this
  * transaction will be committed or rolled back only when
- * session ends.
+ * the session ends.
  * <p>
  * As an example, here is how the {@link CacheStore#write(Cache.Entry)}
  * method can be implemented if {@link CacheJdbcStoreSessionListener}
@@ -133,11 +133,11 @@ public class CacheJdbcStoreSessionListener implements CacheStoreSessionListener,
                     conn.rollback();
             }
             catch (SQLException e) {
-                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+                throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
             }
             finally {
                 U.closeQuiet(conn);
             }
         }
     }
-}
\ No newline at end of file
+}


[37/46] ignite git commit: IGNITE-1890 Fixed Visor Console log command help.

Posted by ak...@apache.org.
IGNITE-1890 Fixed Visor Console log command help.


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

Branch: refs/heads/ignite-1753-1282
Commit: 7a403644b60771b2bb76238f1c6d955bea9a608a
Parents: 5887ae4
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Nov 13 16:59:27 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Nov 13 16:59:27 2015 +0700

----------------------------------------------------------------------
 .../src/main/scala/org/apache/ignite/visor/visor.scala            | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7a403644/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index 28ef0d7..986089b 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -507,8 +507,7 @@ object visor extends VisorTag {
                 "If logging is already stopped - it's no-op."
             ),
             "-dl" -> Seq(
-                "Disables collecting of job and task fail events, licence violation events, cache rebalance events" +
-                    " from remote nodes."
+                "Disables collecting of job and task fail events, cache rebalance events from remote nodes."
             )
         ),
         examples = Seq(


[41/46] ignite git commit: Added advanced tests for GridCacheLoadOnlyStoreAdapterSelfTest

Posted by ak...@apache.org.
Added advanced tests for GridCacheLoadOnlyStoreAdapterSelfTest


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

Branch: refs/heads/ignite-1753-1282
Commit: d54fcbedf9fdc110de8e73387a6796852b0ff42c
Parents: 8014712
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Nov 17 11:56:01 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Nov 17 11:56:01 2015 +0300

----------------------------------------------------------------------
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  | 145 +++++++++++++++++--
 1 file changed, 135 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d54fcbed/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
index 70fae9c..461f562 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.cache.store;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
+import javax.cache.integration.CacheLoaderException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.T2;
@@ -33,8 +34,8 @@ public class GridCacheLoadOnlyStoreAdapterSelfTest extends GridCacheAbstractSelf
     /** Expected loadAll arguments, hardcoded on call site for convenience. */
     private static final Integer[] EXP_ARGS = {1, 2, 3};
 
-    /** Test input size. */
-    private static final int INPUT_SIZE = 100;
+    /** Store to use. */
+    private CacheLoadOnlyStoreAdapter store;
 
     /** {@inheritDoc} */
     @Override protected int gridCount() {
@@ -42,11 +43,33 @@ public class GridCacheLoadOnlyStoreAdapterSelfTest extends GridCacheAbstractSelf
     }
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
         CacheConfiguration cfg = super.cacheConfiguration(gridName);
 
-        cfg.setCacheStoreFactory(singletonFactory(new TestStore()));
+        assertNotNull(store);
+
+        cfg.setCacheStoreFactory(singletonFactory(store));
         cfg.setReadThrough(true);
         cfg.setWriteThrough(true);
         cfg.setLoadPreviousValue(true);
@@ -58,20 +81,74 @@ public class GridCacheLoadOnlyStoreAdapterSelfTest extends GridCacheAbstractSelf
      * @throws Exception If failed.
      */
     public void testStore() throws Exception {
-        jcache().localLoadCache(null, 1, 2, 3);
+        try {
+            int inputSize = 100;
+
+            store = new TestStore(inputSize);
+
+            startGrids(gridCount());
+
+            awaitPartitionMapExchange();
+
+            jcache().localLoadCache(null, 1, 2, 3);
+
+            int cnt = 0;
+
+            for (int i = 0; i < gridCount(); i++)
+                cnt += jcache(i).localSize();
+
+            assertEquals(inputSize - (inputSize / 10), cnt);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStoreSmallQueueSize() throws Exception {
+        try {
+            int inputSize = 1500;
+
+            store = new ParallelTestStore(inputSize);
+
+            store.setBatchSize(1);
+            store.setBatchQueueSize(1);
+            store.setThreadsCount(2);
+
+            startGrids(gridCount());
+
+            awaitPartitionMapExchange();
+
+            jcache().localLoadCache(null, 1, 2, 3);
 
-        int cnt = 0;
+            int cnt = 0;
 
-        for (int i = 0; i < gridCount(); i++)
-            cnt += jcache(i).localSize();
+            for (int i = 0; i < gridCount(); i++)
+                cnt += jcache(i).localSize();
 
-        assertEquals(INPUT_SIZE - (INPUT_SIZE/10), cnt);
+            assertEquals(inputSize, cnt);
+        }
+        finally {
+            stopAllGrids();
+        }
     }
 
     /**
      *
      */
     private static class TestStore extends CacheLoadOnlyStoreAdapter<Integer, String, String> {
+        /** */
+        private final int inputSize;
+
+        /**
+         * @param inputSize Input size.
+         */
+        public TestStore(int inputSize) {
+            this.inputSize = inputSize;
+        }
+
         /** {@inheritDoc} */
         @Override protected Iterator<String> inputIterator(@Nullable Object... args) {
             assertNotNull(args);
@@ -81,7 +158,7 @@ public class GridCacheLoadOnlyStoreAdapterSelfTest extends GridCacheAbstractSelf
                 private int i = -1;
 
                 @Override public boolean hasNext() {
-                    return i < INPUT_SIZE;
+                    return i < inputSize;
                 }
 
                 @Override public String next() {
@@ -111,4 +188,52 @@ public class GridCacheLoadOnlyStoreAdapterSelfTest extends GridCacheAbstractSelf
             return i % 10 == 0 ? null : new T2<>(i, p[1]);
         }
     }
-}
\ No newline at end of file
+
+    /**
+     *
+     */
+    private static class ParallelTestStore extends CacheLoadOnlyStoreAdapter<Integer, String, String> {
+        /** */
+        private final int inputSize;
+
+        /**
+         * @param inputSize Input size.
+         */
+        public ParallelTestStore(int inputSize) {
+            this.inputSize = inputSize;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Iterator<String> inputIterator(@Nullable Object... args) throws CacheLoaderException {
+            return new Iterator<String>() {
+                private int i;
+
+                @Override public boolean hasNext() {
+                    return i < inputSize;
+                }
+
+                @Override public String next() {
+                    if (!hasNext())
+                        throw new NoSuchElementException();
+
+                    String res = i + "=str" + i;
+
+                    i++;
+
+                    return res;
+                }
+
+                @Override public void remove() {
+                    // No-op.
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override protected IgniteBiTuple<Integer, String> parse(String rec, @Nullable Object... args) {
+            String[] p = rec.split("=");
+
+            return new T2<>(Integer.parseInt(p[0]), p[1]);
+        }
+    }
+}