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/05/19 07:37:37 UTC

[1/9] ignite git commit: IGNITE-4842 Now containsKey() respects isReadFromBackup() flag.

Repository: ignite
Updated Branches:
  refs/heads/ignite-5075 e88450df1 -> 4841b5587


IGNITE-4842 Now containsKey() respects isReadFromBackup() flag.


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

Branch: refs/heads/ignite-5075
Commit: d84fd29d924ea1f81ce2bfdb7f8e42c7b85b4c05
Parents: 6be8d97
Author: dkarachentsev <dk...@gridgain.com>
Authored: Thu May 18 19:11:08 2017 +0300
Committer: dkarachentsev <dk...@gridgain.com>
Committed: Thu May 18 19:11:08 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   4 +-
 .../cache/IgniteCacheContainsKeyAtomicTest.java | 103 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 3 files changed, 108 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d84fd29d/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 2820bb7..307c5cf 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
@@ -635,7 +635,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         return (IgniteInternalFuture)getAsync(
             key,
-            /*force primary*/false,
+            /*force primary*/ !ctx.config().isReadFromBackup(),
             /*skip tx*/false,
             /*subj id*/null,
             /*task name*/null,
@@ -664,7 +664,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         return getAllAsync(
             keys,
-            /*force primary*/false,
+            /*force primary*/ !ctx.config().isReadFromBackup(),
             /*skip tx*/false,
             /*subj id*/null,
             /*task name*/null,

http://git-wip-us.apache.org/repos/asf/ignite/blob/d84fd29d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.java
new file mode 100644
index 0000000..981d245
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheContainsKeyAtomicTest.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Verifies that containsKey() works as expected on atomic cache.
+ */
+public class IgniteCacheContainsKeyAtomicTest extends GridCacheAbstractSelfTest {
+    /** Cache name. */
+    public static final String CACHE_NAME = "replicated";
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        IgniteCache cache = ignite(0).cache(CACHE_NAME);
+
+        if (cache != null)
+            cache.clear();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testContainsPutIfAbsent() throws Exception {
+        checkPutIfAbsent(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testContainsPutIfAbsentAll() throws Exception {
+        checkPutIfAbsent(true);
+    }
+
+    /**
+     * @param all Check for set of keys.
+     * @throws Exception If failed.
+     */
+    private void checkPutIfAbsent(final boolean all) throws Exception {
+        Ignite srv = ignite(0);
+
+        final IgniteCache<Integer, Integer> cache1 = srv.getOrCreateCache(replicatedCache());
+        final IgniteCache<Integer, Integer> cache2 = ignite(1).getOrCreateCache(replicatedCache());
+
+        final AtomicInteger fails = new AtomicInteger(0);
+
+        GridTestUtils.runMultiThreaded(new Runnable() {
+            @Override public void run() {
+                for (int i = 0; i < 100; i++) {
+                    if (!cache1.putIfAbsent(i, i)) {
+                        if (all ? !cache2.containsKeys(Collections.singleton(i)) : !cache2.containsKey(i))
+                            fails.incrementAndGet();
+                    }
+                }
+            }
+        }, 100, "put-if-abs");
+
+        assertEquals(0, fails.get());
+    }
+
+    /**
+     * @return replicated cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> replicatedCache() {
+        return new CacheConfiguration<Integer, Integer>(CACHE_NAME)
+            .setAtomicityMode(ATOMIC)
+            .setWriteSynchronizationMode(FULL_SYNC)
+            .setReadFromBackup(false) // containsKey() must respect this flag
+            .setCacheMode(REPLICATED);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d84fd29d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index e423098..6370a10 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicPeekModesTes
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicReplicatedPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationDefaultTemplateTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationTemplateTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheContainsKeyAtomicTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDynamicStopSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughSingleNodeTest;
@@ -326,6 +327,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheAtomicPrimarySyncBackPressureTest.class);
 
+        suite.addTestSuite(IgniteCacheContainsKeyAtomicTest.class);
+
         return suite;
     }
 }
\ No newline at end of file


[7/9] ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-5075
Commit: 9193c58b6b3c9c5ab794a2523afa0121d41b0a1d
Parents: d9a30e8 7402ea1
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 19 10:27:50 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 19 10:27:50 2017 +0300

----------------------------------------------------------------------
 .../node/VisorAffinityTopologyVersion.java      | 87 ++++++++++++++++++++
 .../visor/node/VisorNodeDataCollectorJob.java   |  6 ++
 .../node/VisorNodeDataCollectorJobResult.java   | 38 +++++++++
 .../visor/node/VisorNodeDataCollectorTask.java  |  4 +
 .../node/VisorNodeDataCollectorTaskResult.java  | 28 ++++++-
 5 files changed, 162 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[2/9] ignite git commit: master - Improve internal javadoc

Posted by sb...@apache.org.
master - Improve internal javadoc


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

Branch: refs/heads/ignite-5075
Commit: a20fe961e90c1f9b43f0a0771b3ef33b795e4006
Parents: d84fd29
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu May 18 20:54:17 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu May 18 20:54:17 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheSharedContext.java   | 17 ++++++++++++++++-
 .../preloader/GridDhtPartitionsExchangeFuture.java |  6 ++++++
 2 files changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a20fe961/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 55f3c42..1df74ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -41,6 +41,9 @@ import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.database.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
 import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
@@ -714,7 +717,19 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
-     * Waits for partition locks and transactions release.
+     * Captures all ongoing operations that we need to wait before we can proceed to the next topology version.
+     * This method must be called only after
+     * {@link GridDhtPartitionTopology#updateTopologyVersion(GridDhtPartitionExchangeId, GridDhtPartitionsExchangeFuture, long, boolean)}
+     * method is called so that all new updates will wait to switch to the new version.
+     * This method will capture:
+     * <ul>
+     *     <li>All non-released cache locks</li>
+     *     <li>All non-committed transactions (local and remote)</li>
+     *     <li>All pending atomic updates</li>
+     *     <li>All pending DataStreamer updates</li>
+     * </ul>
+     *
+     * Captured updates are wrapped in a future that will be completed once pending objects are released.
      *
      * @param topVer Topology version.
      * @return {@code true} if waiting was successful.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a20fe961/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 b4cb3c1..544f847 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
@@ -837,6 +837,12 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
+     * The main purpose of this method is to wait for all ongoing updates (transactional and atomic), initiated on
+     * the previous topology version, to finish to prevent inconsistencies during rebalancing and to prevent two
+     * different simultaneous owners of the same lock.
+     * For the exact list of the objects being awaited for see
+     * {@link GridCacheSharedContext#partitionReleaseFuture(AffinityTopologyVersion)} javadoc.
+     *
      * @throws IgniteCheckedException If failed.
      */
     private void waitPartitionRelease() throws IgniteCheckedException {


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

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


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

Branch: refs/heads/ignite-5075
Commit: 4841b5587ce7cb3b76e7c9541c9ac106d6b1c2d8
Parents: 214263e 9e88f97
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 19 10:36:43 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 19 10:36:43 2017 +0300

----------------------------------------------------------------------
 .../ignite/configuration/TopologyValidator.java | 10 +--
 .../processors/cache/ClusterCachesInfo.java     | 14 +++-
 .../node/VisorAffinityTopologyVersion.java      | 87 ++++++++++++++++++++
 .../visor/node/VisorNodeDataCollectorJob.java   |  6 ++
 .../node/VisorNodeDataCollectorJobResult.java   | 38 +++++++++
 .../visor/node/VisorNodeDataCollectorTask.java  |  4 +
 .../node/VisorNodeDataCollectorTaskResult.java  | 28 ++++++-
 .../processors/cache/CacheGetFromJobTest.java   |  5 ++
 .../testsuites/IgniteCacheTestSuite3.java       |  2 +-
 .../IgniteTopologyValidatorTestSuit.java        | 44 ----------
 .../IgniteTopologyValidatorTestSuite.java       | 50 +++++++++++
 11 files changed, 235 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/ignite/blob/4841b558/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------


[4/9] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-5075

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


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

Branch: refs/heads/ignite-5075
Commit: 214263ec55213a5073fb2ab034f588a5e51381f4
Parents: e88450d 86465f5
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 19 09:51:42 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 19 09:51:42 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   4 +-
 .../cache/GridCacheSharedContext.java           |  17 ++-
 .../GridDhtPartitionsExchangeFuture.java        |   6 ++
 .../cache/IgniteCacheContainsKeyAtomicTest.java | 103 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 .../frontend/app/primitives/dropdown/index.pug  |   2 +-
 6 files changed, 131 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/214263ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------


[8/9] ignite git commit: Fixed validator suite name.

Posted by sb...@apache.org.
Fixed validator suite name.


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

Branch: refs/heads/ignite-5075
Commit: 9e88f979b20acd8333351ceea58d67182ee06d60
Parents: 9193c58
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 19 10:36:12 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 19 10:36:12 2017 +0300

----------------------------------------------------------------------
 .../ignite/configuration/TopologyValidator.java | 10 ++--
 .../testsuites/IgniteCacheTestSuite3.java       |  2 +-
 .../IgniteTopologyValidatorTestSuit.java        | 44 -----------------
 .../IgniteTopologyValidatorTestSuite.java       | 50 ++++++++++++++++++++
 4 files changed, 56 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9e88f979/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
index abd69be..320be10 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/TopologyValidator.java
@@ -17,12 +17,11 @@
 
 package org.apache.ignite.configuration;
 
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cluster.ClusterNode;
-
-import javax.cache.CacheException;
 import java.io.Serializable;
 import java.util.Collection;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
 
 /**
  * Topology validator is used to verify that cluster topology is valid for further cache operations.
@@ -57,8 +56,9 @@ import java.util.Collection;
 public interface TopologyValidator extends Serializable {
     /**
      * Validates topology.
+     *
      * @param nodes Collection of nodes.
      * @return {@code true} in case topology is valid for specific cache, otherwise {@code false}
      */
-    boolean validate(Collection<ClusterNode> nodes);
+    public boolean validate(Collection<ClusterNode> nodes);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e88f979/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 d0b74d6..222ac30 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
@@ -180,7 +180,7 @@ public class IgniteCacheTestSuite3 extends TestSuite {
         suite.addTest(IgniteCacheMetricsSelfTestSuite.suite());
 
         // Topology validator.
-        suite.addTest(IgniteTopologyValidatorTestSuit.suite());
+        suite.addTest(IgniteTopologyValidatorTestSuite.suite());
 
         // Eviction.
         suite.addTest(IgniteCacheEvictionSelfTestSuite.suite());

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e88f979/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuit.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuit.java
deleted file mode 100644
index 8c4cd11..0000000
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuit.java
+++ /dev/null
@@ -1,44 +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.testsuites;
-
-import junit.framework.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-/**
- * Topology validator test suite.
- */
-public class IgniteTopologyValidatorTestSuit extends TestSuite {
-    /**
-     * @return Topology validator tests suite.
-     * @throws Exception If failed.
-     */
-    public static TestSuite suite() throws Exception {
-        TestSuite suite = new TestSuite("Topology validator Test Suite");
-
-        suite.addTest(new TestSuite(IgniteTopologyValidatorNearPartitionedAtomicCacheTest.class));
-        suite.addTest(new TestSuite(IgniteTopologyValidatorNearPartitionedTxCacheTest.class));
-        suite.addTest(new TestSuite(IgniteTopologyValidatorPartitionedAtomicCacheTest.class));
-        suite.addTest(new TestSuite(IgniteTopologyValidatorPartitionedTxCacheTest.class));
-        suite.addTest(new TestSuite(IgniteTopologyValidatorReplicatedAtomicCacheTest.class));
-        suite.addTest(new TestSuite(IgniteTopologyValidatorReplicatedTxCacheTest.class));
-        suite.addTest(new TestSuite(IgniteTopologyValidatorGridSplitCacheTest.class));
-
-        return suite;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e88f979/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuite.java
new file mode 100644
index 0000000..58c754e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteTopologyValidatorTestSuite.java
@@ -0,0 +1,50 @@
+/*
+ *  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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.IgniteTopologyValidatorGridSplitCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteTopologyValidatorNearPartitionedAtomicCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteTopologyValidatorNearPartitionedTxCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteTopologyValidatorPartitionedAtomicCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteTopologyValidatorPartitionedTxCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteTopologyValidatorReplicatedAtomicCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteTopologyValidatorReplicatedTxCacheTest;
+
+/**
+ * Topology validator test suite.
+ */
+public class IgniteTopologyValidatorTestSuite extends TestSuite {
+    /**
+     * @return Topology validator tests suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Topology validator Test Suite");
+
+        suite.addTest(new TestSuite(IgniteTopologyValidatorNearPartitionedAtomicCacheTest.class));
+        suite.addTest(new TestSuite(IgniteTopologyValidatorNearPartitionedTxCacheTest.class));
+        suite.addTest(new TestSuite(IgniteTopologyValidatorPartitionedAtomicCacheTest.class));
+        suite.addTest(new TestSuite(IgniteTopologyValidatorPartitionedTxCacheTest.class));
+        suite.addTest(new TestSuite(IgniteTopologyValidatorReplicatedAtomicCacheTest.class));
+        suite.addTest(new TestSuite(IgniteTopologyValidatorReplicatedTxCacheTest.class));
+        suite.addTest(new TestSuite(IgniteTopologyValidatorGridSplitCacheTest.class));
+
+        return suite;
+    }
+}
\ No newline at end of file


[5/9] ignite git commit: Added Affinity topology version and Pending exchanges to Visor data collector task.

Posted by sb...@apache.org.
Added Affinity topology version and Pending exchanges to Visor data collector task.


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

Branch: refs/heads/ignite-5075
Commit: 7402ea11051eb328702ca85d12477accfd22337c
Parents: 86465f5
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri May 19 14:18:23 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri May 19 14:18:23 2017 +0700

----------------------------------------------------------------------
 .../node/VisorAffinityTopologyVersion.java      | 87 ++++++++++++++++++++
 .../visor/node/VisorNodeDataCollectorJob.java   |  6 ++
 .../node/VisorNodeDataCollectorJobResult.java   | 38 +++++++++
 .../visor/node/VisorNodeDataCollectorTask.java  |  4 +
 .../node/VisorNodeDataCollectorTaskResult.java  | 28 ++++++-
 5 files changed, 162 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7402ea11/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAffinityTopologyVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAffinityTopologyVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAffinityTopologyVersion.java
new file mode 100644
index 0000000..83b2fb1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAffinityTopologyVersion.java
@@ -0,0 +1,87 @@
+/*
+ * 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.visor.node;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
+
+/**
+ * Data transfer object for {@link AffinityTopologyVersion}
+ */
+public class VisorAffinityTopologyVersion extends VisorDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private long topVer;
+
+    /** */
+    private int minorTopVer;
+
+    /**
+     * Default constructor.
+     */
+    public VisorAffinityTopologyVersion() {
+        // No-op.
+    }
+
+    /**
+     * Create data transfer object for affinity topology version.
+     *
+     * @param affTopVer Affinity topology version.
+     */
+    public VisorAffinityTopologyVersion(AffinityTopologyVersion affTopVer) {
+        topVer = affTopVer.topologyVersion();
+        minorTopVer = affTopVer.minorTopologyVersion();
+    }
+
+    /**
+     * @return Topology version.
+     */
+    public long getTopologyVersion() {
+        return topVer;
+    }
+
+    /**
+     * @return Minor topology version.
+     */
+    public int getMinorTopologyVersion() {
+        return minorTopVer;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeLong(topVer);
+        out.writeInt(minorTopVer);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        topVer = in.readLong();
+        minorTopVer = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorAffinityTopologyVersion.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7402ea11/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index 125e219..573cc83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -25,6 +25,7 @@ import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.util.ipc.IpcServerEndpoint;
@@ -242,6 +243,11 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         VisorNodeDataCollectorTaskArg arg) {
         res.setGridName(ignite.name());
 
+        GridCachePartitionExchangeManager<Object, Object> exchange = ignite.context().cache().context().exchange();
+
+        res.setReadyAffinityVersion(new VisorAffinityTopologyVersion(exchange.readyAffinityVersion()));
+        res.setHasPendingExchange(exchange.hasPendingExchange());
+
         res.setTopologyVersion(ignite.cluster().topologyVersion());
 
         long start0 = U.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7402ea11/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
index 41f9468..761d0ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
@@ -70,6 +70,12 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /** Errors count. */
     private long errCnt;
 
+    /** Topology version of latest completed partition exchange. */
+    private VisorAffinityTopologyVersion readyTopVer;
+
+    /** Whether pending exchange future exists. */
+    private boolean hasPendingExchange;
+
     /**
      * Default constructor.
      */
@@ -203,6 +209,34 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         this.errCnt = errCnt;
     }
 
+    /**
+     * @return Topology version of latest completed partition exchange.
+     */
+    public VisorAffinityTopologyVersion getReadyAffinityVersion() {
+        return readyTopVer;
+    }
+
+    /**
+     * @param readyTopVer Topology version of latest completed partition exchange.
+     */
+    public void setReadyAffinityVersion(VisorAffinityTopologyVersion readyTopVer) {
+        this.readyTopVer = readyTopVer;
+    }
+
+    /**
+     * @return Whether pending exchange future exists.
+     */
+    public boolean isHasPendingExchange() {
+        return hasPendingExchange;
+    }
+
+    /**
+     * @param hasPendingExchange Whether pending exchange future exists.
+     */
+    public void setHasPendingExchange(boolean hasPendingExchange) {
+        this.hasPendingExchange = hasPendingExchange;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, gridName);
@@ -216,6 +250,8 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         U.writeCollection(out, igfsEndpoints);
         out.writeObject(igfssEx);
         out.writeLong(errCnt);
+        out.writeObject(readyTopVer);
+        out.writeBoolean(hasPendingExchange);
     }
 
     /** {@inheritDoc} */
@@ -231,6 +267,8 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         igfsEndpoints = U.readList(in);
         igfssEx = (Throwable)in.readObject();
         errCnt = in.readLong();
+        readyTopVer = (VisorAffinityTopologyVersion)in.readObject();
+        hasPendingExchange = in.readBoolean();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7402ea11/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index a317ffe..2790dec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -113,5 +113,9 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
 
         if (jobRes.getIgfssEx() != null)
             taskRes.getIgfssEx().put(nid, new VisorExceptionWrapper(jobRes.getIgfssEx()));
+
+        taskRes.getReadyAffinityVersions().put(nid, jobRes.getReadyAffinityVersion());
+
+        taskRes.getPendingExchanges().put(nid, jobRes.isHasPendingExchange());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7402ea11/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
index 669cc80..8d1fe8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
@@ -81,6 +81,12 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
     /** Exceptions caught during collecting IGFS from nodes. */
     private Map<UUID, VisorExceptionWrapper> igfssEx = new HashMap<>();
 
+    /** Topology version of latest completed partition exchange from nodes. */
+    private Map<UUID, VisorAffinityTopologyVersion> readyTopVers = new HashMap<>();
+
+    /** Whether pending exchange future exists from nodes. */
+    private Map<UUID, Boolean> pendingExchanges = new HashMap<>();
+
     /**
      * Default constructor.
      */
@@ -103,7 +109,9 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
             cachesEx.isEmpty() &&
             igfss.isEmpty() &&
             igfsEndpoints.isEmpty() &&
-            igfssEx.isEmpty();
+            igfssEx.isEmpty() &&
+            readyTopVers.isEmpty() &&
+            pendingExchanges.isEmpty();
     }
 
     /**
@@ -204,6 +212,20 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         return errCnts;
     }
 
+    /**
+     * @return Topology version of latest completed partition exchange from nodes.
+     */
+    public Map<UUID, VisorAffinityTopologyVersion> getReadyAffinityVersions() {
+        return readyTopVers;
+    }
+
+    /**
+     * @return Whether pending exchange future exists from nodes.
+     */
+    public Map<UUID, Boolean> getPendingExchanges() {
+        return pendingExchanges;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         out.writeBoolean(active);
@@ -219,6 +241,8 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         U.writeMap(out, igfss);
         U.writeMap(out, igfsEndpoints);
         U.writeMap(out, igfssEx);
+        U.writeMap(out, readyTopVers);
+        U.writeMap(out, pendingExchanges);
     }
 
     /** {@inheritDoc} */
@@ -236,6 +260,8 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         igfss = U.readMap(in);
         igfsEndpoints = U.readMap(in);
         igfssEx = U.readMap(in);
+        readyTopVers = U.readMap(in);
+        pendingExchanges = U.readMap(in);
     }
 
     /** {@inheritDoc} */


[3/9] ignite git commit: IGNITE-5230 Fixed dropdown trigger.

Posted by sb...@apache.org.
IGNITE-5230 Fixed dropdown trigger.


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

Branch: refs/heads/ignite-5075
Commit: 86465f585d82c1eec3466b8af3d648ec57040de2
Parents: a20fe96
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri May 19 09:19:14 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri May 19 09:19:14 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/primitives/dropdown/index.pug | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86465f58/modules/web-console/frontend/app/primitives/dropdown/index.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/dropdown/index.pug b/modules/web-console/frontend/app/primitives/dropdown/index.pug
index 76fe2ed..7015433 100644
--- a/modules/web-console/frontend/app/primitives/dropdown/index.pug
+++ b/modules/web-console/frontend/app/primitives/dropdown/index.pug
@@ -24,7 +24,7 @@ mixin ignite-form-field-bsdropdown(label, model, name, disabled, required, optio
 
             bs-dropdown=''
 
-            data-trigger='hover focus click'
+            data-trigger='hover focus'
             data-placement='bottom-right'
             data-container='self'
             data-animation=''


[6/9] ignite git commit: Fixed CacheGetFromJobTest failure (wait for correct exchange version for cache start operation).

Posted by sb...@apache.org.
Fixed CacheGetFromJobTest failure (wait for correct exchange version for cache start operation).


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

Branch: refs/heads/ignite-5075
Commit: d9a30e8bc285672418f800b527d0aa8e7dd3a4ff
Parents: 86465f5
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 19 10:27:24 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 19 10:27:24 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/ClusterCachesInfo.java  | 14 ++++++++++++--
 .../processors/cache/CacheGetFromJobTest.java         |  5 +++++
 2 files changed, 17 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a30e8b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 28ec600..7150cf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -350,8 +350,18 @@ class ClusterCachesInfo {
                 if (!needExchange && desc != null) {
                     if (desc.clientCacheStartVersion() != null)
                         waitTopVer = desc.clientCacheStartVersion();
-                    else
-                        waitTopVer = desc.startTopologyVersion();
+                    else {
+                        AffinityTopologyVersion nodeStartVer =
+                            new AffinityTopologyVersion(ctx.discovery().localNode().order(), 0);
+
+                        if (desc.startTopologyVersion() != null)
+                            waitTopVer = desc.startTopologyVersion();
+                        else
+                            waitTopVer = desc.receivedFromStartVersion();
+
+                        if (waitTopVer == null || nodeStartVer.compareTo(waitTopVer) > 0)
+                            waitTopVer = nodeStartVer;
+                    }
                 }
             }
             else if (req.globalStateChange())

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a30e8b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetFromJobTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetFromJobTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetFromJobTest.java
index a48f342..7c9eeec 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetFromJobTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetFromJobTest.java
@@ -41,6 +41,11 @@ public class CacheGetFromJobTest extends GridCacheAbstractSelfTest {
         stopAllGrids();
     }
 
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 5 * 60_000;
+    }
+
     /**
      * @throws Exception If failed.
      */