You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/02/01 07:30:34 UTC

[1/9] ignite git commit: ignite-db-x fix rebalance logic, add print partition state in common abstract test.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3477-compact f110b247e -> f9f105571


ignite-db-x fix rebalance logic, add print partition state in common abstract test.


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

Branch: refs/heads/ignite-3477-compact
Commit: b9bca8ca6e88bb7cf1b9e5c5370fffbba59487a4
Parents: e15cbe8
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Tue Jan 31 16:43:38 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Tue Jan 31 16:43:38 2017 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionDemander.java |  58 ++++----
 .../near/IgniteCacheNearOnlyTxTest.java         |   3 +-
 .../TxOptimisticDeadlockDetectionTest.java      |  12 +-
 .../junits/common/GridCommonAbstractTest.java   | 149 ++++++++++++++++---
 4 files changed, 172 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b9bca8ca/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 204e4da..04a9d1a 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
@@ -434,33 +434,51 @@ public class GridDhtPartitionDemander {
     }
 
     /**
-     * @param fut Future.
+     * @param fut Rebalance future.
      * @param assigns Assignments.
      * @throws IgniteCheckedException If failed.
-     * @return Partitions were requested.
      */
     private void requestPartitions(
         RebalanceFuture fut,
         GridDhtPreloaderAssignments assigns
     ) throws IgniteCheckedException {
+        assert fut != null;
+
         if (topologyChanged(fut)) {
             fut.cancel();
 
             return;
         }
 
+        synchronized (fut) {
+            if (fut.isDone())
+                return;
+
+            // Must add all remaining node before send first request, for avoid race between add remaining node and
+            // processing response, see checkIsDone(boolean).
+            for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
+                UUID nodeId = e.getKey().id();
+
+                Collection<Integer> parts= e.getValue().partitions();
+
+                assert parts != null : "Partitions are null [cache=" + cctx.name() + ", fromNode=" + nodeId + "]";
+
+                fut.remaining.put(nodeId, new T2<>(U.currentTimeMillis(), parts));
+            }
+        }
+
         for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
             final ClusterNode node = e.getKey();
 
-            final GridDhtPartitionDemandMessage d = e.getValue();
+            final CacheConfiguration cfg = cctx.config();
 
-            final Collection<Integer> parts = d.partitions();
+            final Collection<Integer> parts = fut.remaining.get(node.id()).get2();
 
-            fut.appendPartitions(node.id(), parts);
+            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 [mode=" + cctx.config().getRebalanceMode() +
+                U.log(log, "Starting rebalancing [mode=" + cfg.getRebalanceMode() +
                     ", fromNode=" + node.id() + ", partitionsCount=" + parts.size() +
                     ", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]");
 
@@ -488,12 +506,14 @@ public class GridDhtPartitionDemander {
                         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 (fut.isDone())
+                                return;
+
+                            // 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())
@@ -505,7 +525,7 @@ public class GridDhtPartitionDemander {
             }
             else {
                 U.log(log, "Starting rebalancing (old api) [cache=" + cctx.name() +
-                    ", mode=" + cctx.config().getRebalanceMode() +
+                    ", mode=" + cfg.getRebalanceMode() +
                     ", fromNode=" + node.id() +
                     ", partitionsCount=" + parts.size() +
                     ", topology=" + fut.topologyVersion() +
@@ -937,18 +957,6 @@ public class GridDhtPartitionDemander {
         }
 
         /**
-         * @param nodeId Node id.
-         * @param parts Parts.
-         */
-        private void appendPartitions(UUID nodeId, Collection<Integer> parts) {
-            synchronized (this) {
-                assert parts != null : "Partitions are null [cache=" + cctx.name() + ", fromNode=" + nodeId + "]";
-
-                remaining.put(nodeId, new T2<>(U.currentTimeMillis(), parts));
-            }
-        }
-
-        /**
          * Cancels this future.
          *
          * @return {@code True}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9bca8ca/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
index 1d80ac1..a3c2622 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
@@ -98,8 +98,7 @@ public class IgniteCacheNearOnlyTxTest extends IgniteCacheAbstractTest {
             log.info("Iteration: " + i);
 
             futs.add(GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-                @Override
-                public Object call() throws Exception {
+                @Override public Object call() throws Exception {
                     int val = idx.getAndIncrement();
 
                     IgniteCache<Integer, Integer> cache = ignite1.cache(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9bca8ca/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
index f6a06c2..c484d09 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
@@ -59,6 +59,7 @@ import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
@@ -77,6 +78,9 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  *
  */
 public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
+    /** Ip finder. */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** Cache name. */
     private static final String CACHE_NAME = "cache";
 
@@ -97,9 +101,11 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (isDebug()) {
-            TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
 
+        if (isDebug()) {
             discoSpi.failureDetectionTimeoutEnabled(false);
 
             cfg.setDiscoverySpi(discoSpi);
@@ -111,6 +117,8 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(client);
 
+        cfg.setDiscoverySpi(discoSpi);
+
         // Test spi blocks message send, this can cause hang with striped pool.
         cfg.setStripedPoolSize(-1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9bca8ca/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 469930e..acedfe8 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
@@ -19,6 +19,7 @@ package org.apache.ignite.testframework.junits.common;
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -26,6 +27,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -59,6 +61,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
@@ -80,6 +83,7 @@ import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.PA;
+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.U;
 import org.apache.ignite.lang.IgniteFuture;
@@ -132,7 +136,6 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      *
      * @param clsK Key class.
      * @param clsV Value class.
-     *
      * @return cache instance
      */
     protected <K, V> IgniteCache<K, V> jcache(CacheConfiguration ccfg, Class<K> clsK, Class<V> clsV) {
@@ -147,7 +150,6 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param ccfg Cache configuration.
      * @param clsK Key class.
      * @param clsV Value class.
-     *
      * @return cache instance
      */
     protected <K, V> IgniteCache<K, V> jcache(Ignite ig, CacheConfiguration ccfg, Class<K> clsK, Class<V> clsV) {
@@ -163,7 +165,6 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param name Cache name.
      * @param clsK Key class.
      * @param clsV Value class.
-     *
      * @return cache instance
      */
     @SuppressWarnings("unchecked")
@@ -186,7 +187,6 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param ig Ignite.
      * @param ccfg Cache configuration.
      * @param name Cache name.
-     *
      * @return cache instance
      */
     @SuppressWarnings("unchecked")
@@ -286,7 +286,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param cache Cache.
      * @return DHT cache.
      */
-    protected static <K, V> GridDhtCacheAdapter<K, V> dht(IgniteCache<K,V> cache) {
+    protected static <K, V> GridDhtCacheAdapter<K, V> dht(IgniteCache<K, V> cache) {
         return nearEnabled(cache) ? near(cache).dht() :
             ((IgniteKernal)cache.unwrap(Ignite.class)).<K, V>internalCache(cache.getName()).context().dht();
     }
@@ -336,7 +336,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param cache Cache.
      * @return {@code True} if near cache is enabled.
      */
-    protected static <K, V> boolean nearEnabled(final IgniteCache<K,V> cache) {
+    protected static <K, V> boolean nearEnabled(final IgniteCache<K, V> cache) {
         CacheConfiguration cfg = GridAbstractTest.executeOnLocalOrRemoteJvm(cache,
             new TestCacheCallable<K, V, CacheConfiguration>() {
                 private static final long serialVersionUID = 0L;
@@ -361,7 +361,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param cache Cache.
      * @return Near cache.
      */
-    protected static <K, V> GridNearCacheAdapter<K, V> near(IgniteCache<K,V> cache) {
+    protected static <K, V> GridNearCacheAdapter<K, V> near(IgniteCache<K, V> cache) {
         return ((IgniteKernal)cache.unwrap(Ignite.class)).<K, V>internalCache(cache.getName()).context().near();
     }
 
@@ -369,7 +369,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param cache Cache.
      * @return Colocated cache.
      */
-    protected static <K, V> GridDhtColocatedCache<K, V> colocated(IgniteCache<K,V> cache) {
+    protected static <K, V> GridDhtColocatedCache<K, V> colocated(IgniteCache<K, V> cache) {
         return ((IgniteKernal)cache.unwrap(Ignite.class)).<K, V>internalCache(cache.getName()).context().colocated();
     }
 
@@ -505,20 +505,23 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      */
     @SuppressWarnings("BusyWait")
     protected void awaitPartitionMapExchange() throws InterruptedException {
-        awaitPartitionMapExchange(false, false, null);
+        awaitPartitionMapExchange(false, false, null, false);
     }
 
     /**
      * @param waitEvicts If {@code true} will wait for evictions finished.
      * @param waitNode2PartUpdate If {@code true} will wait for nodes node2part info update finished.
      * @param nodes Optional nodes.
+     * @param printPartState If {@code true} will print partition state if evictions not happened.
      * @throws InterruptedException If interrupted.
      */
     @SuppressWarnings("BusyWait")
-    protected void awaitPartitionMapExchange(boolean waitEvicts,
+    protected void awaitPartitionMapExchange(
+        boolean waitEvicts,
         boolean waitNode2PartUpdate,
-        @Nullable Collection<ClusterNode> nodes)
-        throws InterruptedException {
+        @Nullable Collection<ClusterNode> nodes,
+        boolean printPartState
+    ) throws InterruptedException {
         long timeout = 30_000;
 
         long startTime = -1;
@@ -569,18 +572,18 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                 Collection<ClusterNode> affNodes =
                                     dht.context().affinity().assignment(readyVer).idealAssignment().get(p);
 
-                                int exp = affNodes.size();
+                                int affNodesCnt = affNodes.size();
 
                                 GridDhtTopologyFuture topFut = top.topologyVersionFuture();
 
                                 Collection<ClusterNode> owners = (topFut != null && topFut.isDone()) ?
                                     top.nodes(p, AffinityTopologyVersion.NONE) : Collections.<ClusterNode>emptyList();
 
-                                int actual = owners.size();
+                                int ownerNodesCnt = owners.size();
 
                                 GridDhtLocalPartition loc = top.localPartition(p, readyVer, false);
 
-                                if (affNodes.size() != owners.size() || !affNodes.containsAll(owners) ||
+                                if (affNodesCnt != ownerNodesCnt || !affNodes.containsAll(owners) ||
                                     (waitEvicts && loc != null && loc.state() != GridDhtPartitionState.OWNING)) {
                                     LT.warn(log(), "Waiting for topology map update [" +
                                         "grid=" + g.name() +
@@ -588,8 +591,8 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                         ", cacheId=" + dht.context().cacheId() +
                                         ", topVer=" + top.topologyVersion() +
                                         ", p=" + p +
-                                        ", affNodesCnt=" + exp +
-                                        ", ownersCnt=" + actual +
+                                        ", affNodesCnt=" + affNodesCnt +
+                                        ", ownersCnt=" + ownerNodesCnt +
                                         ", affNodes=" + F.nodeIds(affNodes) +
                                         ", owners=" + F.nodeIds(owners) +
                                         ", topFut=" + topFut +
@@ -617,6 +620,9 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                 if (System.currentTimeMillis() - start > timeout) {
                                     U.dumpThreads(log);
 
+                                    if (printPartState)
+                                        printPartitionState(c);
+
                                     throw new IgniteException("Timeout of waiting for topology map update [" +
                                         "grid=" + g.name() +
                                         ", cache=" + cfg.getName() +
@@ -685,6 +691,107 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     }
 
     /**
+     * @param c Cache proxy.
+     *
+     * Print partitionState for cache.
+     */
+    public void printPartitionState(IgniteCache<?, ?> c) {
+        IgniteCacheProxy<?, ?> cache = (IgniteCacheProxy<?, ?>)c;
+
+        CacheConfiguration cfg = cache.context().config();
+
+        if (cfg == null)
+            return;
+
+        StringBuilder sb = new StringBuilder();
+
+        for (Ignite ig : G.allGrids()) {
+            IgniteKernal k = ((IgniteKernal)ig);
+
+            IgniteInternalFuture<?> syncFut = k.internalCache(cfg.getName())
+                .preloader()
+                .syncFuture();
+
+            sb.append("nodeId = ")
+                .append(k.context().localNodeId())
+                .append(" preload sync future isDone=")
+                .append(syncFut.isDone())
+                .append("\n");
+        }
+
+        for (Ignite ig : G.allGrids()) {
+            IgniteKernal k = ((IgniteKernal)ig);
+
+            IgniteInternalFuture<?> f = k.internalCache(cfg.getName())
+                .preloader()
+                .rebalanceFuture();
+
+            try {
+                sb.append("nodeId = ").append(k.context().localNodeId())
+                    .append(" rebalance future isDone=").append(f.isDone())
+                    .append(" res=").append(f.isDone() ? f.get() : "N/A")
+                    .append(" topVer=").append(U.field(f, "topVer"))
+                    .append("\n");
+
+                Map<UUID, T2<Long, Collection<Integer>>> remaining = U.field(f, "remaining");
+
+                sb.append("remaining:\n");
+
+                if (remaining.isEmpty())
+                    sb.append("empty\n");
+                else
+                    for (Map.Entry<UUID, T2<Long, Collection<Integer>>> e : remaining.entrySet())
+                        sb.append("uuid=").append(e.getKey())
+                            .append(" startTime=").append(e.getValue().getKey())
+                            .append(" parts=").append(Arrays.toString(e.getValue().getValue().toArray()))
+                            .append("\n");
+
+            }
+            catch (Throwable e) {
+                log.error(e.getMessage());
+            }
+        }
+
+        for (Ignite g : G.allGrids()) {
+            IgniteKernal g0 = (IgniteKernal)g;
+
+            sb.append("grid=").append(g0.name())
+                .append(" localNodeId=").append(g0.localNode().id())
+                .append("\n");
+
+            GridDhtPartitionTopology top = dht(cache).topology();
+
+            for (int p = 0; p < cfg.getAffinity().partitions(); p++) {
+                GridDhtLocalPartition part = top.localPartition(p, AffinityTopologyVersion.NONE, false);
+
+                sb.append("local part=");
+
+                if (part != null)
+                    sb.append(p).append(" state=").append(part.state());
+                else
+                    sb.append(p).append(" is null");
+
+                sb.append("\n");
+
+                for (UUID nodeId : F.nodeIds(g0.context().discovery().allNodes())) {
+                    if (!nodeId.equals(g0.localNode().id()))
+                        sb.append(" nodeId=")
+                            .append(nodeId)
+                            .append(" part=")
+                            .append(p)
+                            .append(" state=")
+                            .append(top.partitionState(nodeId, p))
+                            .append("\n");
+                }
+            }
+
+            sb.append("\n");
+        }
+
+        log.info("dump partition state for " + cfg.getName() + ":\n" + sb.toString());
+    }
+
+    /**
      * @param ignite Node.
      */
     public void dumpCacheDebugInfo(Ignite ignite) {
@@ -806,7 +913,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param iterable Iterator
      * @return Set
      */
-    protected <K, V> Set<Cache.Entry<K, V>> entrySet(Iterable<Cache.Entry<K, V>> iterable){
+    protected <K, V> Set<Cache.Entry<K, V>> entrySet(Iterable<Cache.Entry<K, V>> iterable) {
         Set<Cache.Entry<K, V>> set = new HashSet<>();
 
         for (Cache.Entry<K, V> entry : iterable)
@@ -1146,7 +1253,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
         Collection<ClusterNode> nodes = aff.mapKeyToPrimaryAndBackups(key);
 
-        assertTrue("Expected more than one node for key [key=" + key + ", nodes=" + nodes +']', nodes.size() > 1);
+        assertTrue("Expected more than one node for key [key=" + key + ", nodes=" + nodes + ']', nodes.size() > 1);
 
         Iterator<ClusterNode> it = nodes.iterator();
 
@@ -1171,7 +1278,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
         Collection<ClusterNode> nodes = aff.mapKeyToPrimaryAndBackups(key);
 
-        assertTrue("Expected more than one node for key [key=" + key + ", nodes=" + nodes +']', nodes.size() > 1);
+        assertTrue("Expected more than one node for key [key=" + key + ", nodes=" + nodes + ']', nodes.size() > 1);
 
         Iterator<ClusterNode> it = nodes.iterator();
 
@@ -1299,7 +1406,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
         if (ok)
             info("Deleted OK: " + file.getAbsolutePath() +
-                (size >=0 ? "(" + IgniteUtils.readableSize(size, false) + ")" : ""));
+                (size >= 0 ? "(" + IgniteUtils.readableSize(size, false) + ")" : ""));
 
         return ok;
     }


[3/9] ignite git commit: ignite-db-x minor update 2

Posted by sb...@apache.org.
ignite-db-x minor update 2


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

Branch: refs/heads/ignite-3477-compact
Commit: 087c4062879a5c77c194f1e43a3b6873afcb1694
Parents: 86ea2c7
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Tue Jan 31 17:07:34 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Tue Jan 31 17:07:34 2017 +0300

----------------------------------------------------------------------
 .../junits/common/GridCommonAbstractTest.java      | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/087c4062/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 9c4d954..1fc105c 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
@@ -505,7 +505,22 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      */
     @SuppressWarnings("BusyWait")
     protected void awaitPartitionMapExchange() throws InterruptedException {
-        awaitPartitionMapExchange(false, false, null, false);
+        awaitPartitionMapExchange(false, false, null);
+    }
+
+    /**
+     * @param waitEvicts If {@code true} will wait for evictions finished.
+     * @param waitNode2PartUpdate If {@code true} will wait for nodes node2part info update finished.
+     * @param nodes Optional nodes.
+     * @throws InterruptedException If interrupted.
+     */
+    @SuppressWarnings("BusyWait")
+    protected void awaitPartitionMapExchange(
+        boolean waitEvicts,
+        boolean waitNode2PartUpdate,
+        @Nullable Collection<ClusterNode> nodes
+    ) throws InterruptedException {
+        awaitPartitionMapExchange(waitEvicts, waitNode2PartUpdate, nodes, false);
     }
 
     /**


[8/9] ignite git commit: ignite-3477

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


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

Branch: refs/heads/ignite-3477-compact
Commit: e4237978c083252820221ceb2f1c76138e7fe30a
Parents: f110b24
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 1 10:29:45 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 1 10:29:45 2017 +0300

----------------------------------------------------------------------
 .../cache/distributed/near/IgniteCacheNearOnlyTxTest.java         | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e4237978/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
index 1d80ac1..a3c2622 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheNearOnlyTxTest.java
@@ -98,8 +98,7 @@ public class IgniteCacheNearOnlyTxTest extends IgniteCacheAbstractTest {
             log.info("Iteration: " + i);
 
             futs.add(GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-                @Override
-                public Object call() throws Exception {
+                @Override public Object call() throws Exception {
                     int val = idx.getAndIncrement();
 
                     IgniteCache<Integer, Integer> cache = ignite1.cache(null);


[5/9] ignite git commit: Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x

Posted by sb...@apache.org.
Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x


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

Branch: refs/heads/ignite-3477-compact
Commit: ae97143e86073e600c78c5d82452cf8bc956dd91
Parents: 587a05e 9773ea7
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 31 17:59:55 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jan 31 17:59:55 2017 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionDemander.java |  58 ++++---
 .../ignite/internal/util/IgniteUtils.java       |  16 ++
 .../near/IgniteCacheNearOnlyTxTest.java         |   3 +-
 .../TxOptimisticDeadlockDetectionTest.java      |  12 +-
 .../junits/common/GridCommonAbstractTest.java   | 170 ++++++++++++++++---
 5 files changed, 210 insertions(+), 49 deletions(-)
----------------------------------------------------------------------



[6/9] ignite git commit: ignite-db-x fix testFirstFilteredEvent

Posted by sb...@apache.org.
ignite-db-x fix testFirstFilteredEvent


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

Branch: refs/heads/ignite-3477-compact
Commit: 8dd2181f5f914677b18dd43ca24025f471c7f081
Parents: 9773ea7
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Tue Jan 31 18:53:58 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Tue Jan 31 18:53:58 2017 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryFailoverAbstractSelfTest.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd2181f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index bd980be..8e6f406 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -260,15 +260,15 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
                 qryClnCache.put(key, -1);
 
             qryClnCache.put(keys.get(0), 100);
-        }
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return lsnr.evts.size() == 1;
-            }
-        }, 5000);
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return lsnr.evts.size() == 1;
+                }
+            }, 5000);
 
-        assertEquals(lsnr.evts.size(), 1);
+            assertEquals(lsnr.evts.size(), 1);
+        }
     }
 
     /**


[9/9] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-3477' into ignite-3477-compact

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


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

Branch: refs/heads/ignite-3477-compact
Commit: f9f10557189d08c15c41179bdfc3cefbb3b821b7
Parents: e423797 954d017
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 1 10:29:59 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 1 10:29:59 2017 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionDemander.java |  58 ++++---
 .../ignite/internal/util/IgniteUtils.java       |  16 ++
 ...ContinuousQueryFailoverAbstractSelfTest.java |  14 +-
 .../TxOptimisticDeadlockDetectionTest.java      |  12 +-
 .../junits/common/GridCommonAbstractTest.java   | 170 ++++++++++++++++---
 5 files changed, 216 insertions(+), 54 deletions(-)
----------------------------------------------------------------------



[4/9] ignite git commit: ignite-db-x cosmetic refactoring

Posted by sb...@apache.org.
ignite-db-x cosmetic refactoring


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

Branch: refs/heads/ignite-3477-compact
Commit: 9773ea74d8f8827feeea6552509c46c23bf54d76
Parents: 087c406
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Tue Jan 31 17:58:45 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Tue Jan 31 17:58:45 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       | 16 +++++++++++
 .../junits/common/GridCommonAbstractTest.java   | 30 +++++++++++++-------
 2 files changed, 35 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9773ea74/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index b077666..1be9f75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -7961,6 +7961,22 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Check that field exist.
+     *
+     * @param obj Object.
+     * @param fieldName Field name.
+     * @return Boolean flag.
+     */
+    public static boolean hasField(Object obj, String fieldName){
+        try {
+            field(obj, fieldName);
+
+            return true;
+        }catch (IgniteException e){
+            return false;
+        }
+    }
+    /**
      * Gets object field offset.
      *
      * @param cls Object class.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9773ea74/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 1fc105c..48189f7 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
@@ -720,6 +720,8 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
         StringBuilder sb = new StringBuilder();
 
+        sb.append("----preload sync futures----\n");
+
         for (Ignite ig : G.allGrids()) {
             IgniteKernal k = ((IgniteKernal)ig);
 
@@ -727,13 +729,15 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                 .preloader()
                 .syncFuture();
 
-            sb.append("nodeId = ")
+            sb.append("nodeId=")
                 .append(k.context().localNodeId())
-                .append(" preload sync future isDone=")
+                .append(" isDone=")
                 .append(syncFut.isDone())
                 .append("\n");
         }
 
+        sb.append("----rebalance futures----\n");
+
         for (Ignite ig : G.allGrids()) {
             IgniteKernal k = ((IgniteKernal)ig);
 
@@ -742,21 +746,23 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                 .rebalanceFuture();
 
             try {
-                sb.append("nodeId = ").append(k.context().localNodeId())
-                    .append(" rebalance future isDone=").append(f.isDone())
+                sb.append("nodeId=").append(k.context().localNodeId())
+                    .append(" isDone=").append(f.isDone())
                     .append(" res=").append(f.isDone() ? f.get() : "N/A")
-                    .append(" topVer=").append(U.field(f, "topVer"))
+                    .append(" topVer=")
+                    .append((U.hasField(f, "topVer") ?
+                        U.field(f, "topVer") : "[unknown] may be it is finished future"))
                     .append("\n");
 
                 Map<UUID, T2<Long, Collection<Integer>>> remaining = U.field(f, "remaining");
 
-                sb.append("remaining:\n");
+                sb.append("remaining:");
 
                 if (remaining.isEmpty())
                     sb.append("empty\n");
                 else
                     for (Map.Entry<UUID, T2<Long, Collection<Integer>>> e : remaining.entrySet())
-                        sb.append("uuid=").append(e.getKey())
+                        sb.append("\nuuid=").append(e.getKey())
                             .append(" startTime=").append(e.getValue().getKey())
                             .append(" parts=").append(Arrays.toString(e.getValue().getValue().toArray()))
                             .append("\n");
@@ -767,11 +773,13 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             }
         }
 
+        sb.append("----partition state----\n");
+
         for (Ignite g : G.allGrids()) {
             IgniteKernal g0 = (IgniteKernal)g;
 
-            sb.append("grid=").append(g0.name())
-                .append(" localNodeId=").append(g0.localNode().id())
+            sb.append("localNodeId=").append(g0.localNode().id())
+                .append(" grid=").append(g0.name())
                 .append("\n");
 
             GridDhtPartitionTopology top = dht(cache).topology();
@@ -790,7 +798,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
                 for (UUID nodeId : F.nodeIds(g0.context().discovery().allNodes())) {
                     if (!nodeId.equals(g0.localNode().id()))
-                        sb.append(" nodeId=")
+                        sb.append(" nodeId = ")
                             .append(nodeId)
                             .append(" part=")
                             .append(p)
@@ -803,7 +811,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             sb.append("\n");
         }
 
-        log.info("dump partition state for " + cfg.getName() + ":\n" + sb.toString());
+        log.info("dump partitions state for <" + cfg.getName() + ">:\n" + sb.toString());
     }
 
     /**


[2/9] ignite git commit: ignite-db-x minor update

Posted by sb...@apache.org.
ignite-db-x minor update


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

Branch: refs/heads/ignite-3477-compact
Commit: 86ea2c7cd26c03ae59cdf9974253b68f5bfabdf3
Parents: b9bca8c
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Tue Jan 31 16:46:17 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Tue Jan 31 16:46:17 2017 +0300

----------------------------------------------------------------------
 .../ignite/testframework/junits/common/GridCommonAbstractTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86ea2c7c/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 acedfe8..9c4d954 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
@@ -695,7 +695,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      *
      * Print partitionState for cache.
      */
-    public void printPartitionState(IgniteCache<?, ?> c) {
+    protected void printPartitionState(IgniteCache<?, ?> c) {
         IgniteCacheProxy<?, ?> cache = (IgniteCacheProxy<?, ?>)c;
 
         CacheConfiguration cfg = cache.context().config();


[7/9] ignite git commit: Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x

Posted by sb...@apache.org.
Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x


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

Branch: refs/heads/ignite-3477-compact
Commit: 954d0173428103b521964ce23469bed681645959
Parents: ae97143 8dd2181
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 31 20:09:22 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jan 31 20:09:22 2017 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryFailoverAbstractSelfTest.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------