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

[1/8] incubator-ignite git commit: # ignite-648: concurrent map

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648-failover d0b8b0dd9 -> a3f9dc17d


# ignite-648: concurrent map


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

Branch: refs/heads/ignite-648-failover
Commit: 04aaa6ed51e8ec731e3eb37ee9ce7001973d22ee
Parents: efb8239
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 17:52:09 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 17:52:09 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/04aaa6ed/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 6da478d..7c1769a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -43,7 +43,7 @@ import java.util.concurrent.*;
 @SuppressWarnings("TransientFieldInNonSerializableClass")
 public class IgniteProcessProxy implements IgniteEx {
     /** Grid proxies. */
-    private static final transient Map<String, IgniteProcessProxy> gridProxies = new HashMap<>();
+    private static final transient ConcurrentMap<String, IgniteProcessProxy> gridProxies = new ConcurrentHashMap<>();
 
     /** Jvm process with ignite instance. */
     private final transient GridJavaProcess proc;


[2/8] incubator-ignite git commit: # ignite-648: fix test + internalCache unsupported msg

Posted by sb...@apache.org.
# ignite-648: fix test + internalCache unsupported msg


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

Branch: refs/heads/ignite-648-failover
Commit: 84c7d61928770473992d57c6f4f0d35a22987289
Parents: 04aaa6e
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 19:16:28 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 19:16:28 2015 +0300

----------------------------------------------------------------------
 .../testframework/junits/common/GridCommonAbstractTest.java  | 4 ++++
 .../junits/multijvm/IgniteCacheProcessProxy.java             | 8 ++++++--
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/84c7d619/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 01cb240..4764de0 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
@@ -117,6 +117,10 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Cache.
      */
     protected <K, V> GridCacheAdapter<K, V> internalCache(IgniteCache<K, V> cache) {
+        if (cache instanceof IgniteCacheProcessProxy)
+            throw new UnsupportedOperationException("Oparetion can't be supported automatically for multi jvm " +
+                "(send closure instead).");
+
         return ((IgniteKernal)cache.unwrap(Ignite.class)).internalCache(cache.getName());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/84c7d619/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index e23739d..07ed950 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -168,8 +168,12 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isLocalLocked(K key, boolean byCurrThread) {
-        throw new UnsupportedOperationException("Method should be supported.");
+    @Override public boolean isLocalLocked(final K key, final boolean byCurrThread) {
+        return compute.call(new IgniteCallable<Boolean>() {
+            @Override public Boolean call() throws Exception {
+                return cache().isLocalLocked(key, byCurrThread);
+            }
+        });
     }
 
     /** {@inheritDoc} */


[4/8] incubator-ignite git commit: # ignite-648: testLocalClearKeys

Posted by sb...@apache.org.
# ignite-648: testLocalClearKeys


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

Branch: refs/heads/ignite-648-failover
Commit: 335a3f9c44829652966623d03016e670c7d0b0fc
Parents: c52bb1b
Author: ashutak <as...@gridgain.com>
Authored: Thu Jul 2 14:00:04 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu Jul 2 14:00:04 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/335a3f9c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 9b9c86a..7ea73a4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -4362,6 +4362,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testLocalClearKeys() throws Exception {
+        if (isMultiJvm()) // TODO: IGNITE-648, test should be fixed and enabled.
+            return;
+
         Map<String, List<String>> keys = addKeys();
 
         Ignite g = grid(0);


[6/8] incubator-ignite git commit: # ignite-648: fix tests

Posted by sb...@apache.org.
# ignite-648: fix tests


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

Branch: refs/heads/ignite-648-failover
Commit: 37fca3ba2b4e52fe461996144ef709f759d521a4
Parents: 44692e3
Author: ashutak <as...@gridgain.com>
Authored: Thu Jul 2 18:39:39 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu Jul 2 18:39:39 2015 +0300

----------------------------------------------------------------------
 .../near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java  |  2 +-
 .../GridCachePartitionedMultiNodeCounterSelfTest.java    |  3 +++
 .../GridCachePartitionedMultiNodeFullApiSelfTest.java    |  3 +++
 .../ignite/testframework/junits/GridAbstractTest.java    | 11 +++++++++--
 4 files changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37fca3ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
index 60168e1..3d9364b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
@@ -125,7 +125,7 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
 
         return F.view(super.affinityNodes(), new P1<ClusterNode>() {
             @Override public boolean apply(ClusterNode n) {
-                return !F.eq(G.ignite(n.id()).name(), grid(nearIdx).name());
+                return !F.eq(grid(n).name(), grid(nearIdx).name());
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37fca3ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
index 5d00e1e..0ee28e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
@@ -537,6 +537,9 @@ public class GridCachePartitionedMultiNodeCounterSelfTest extends GridCommonAbst
      * @throws Exception If failed.
      */
     private void checkNearAndPrimaryMultiNode(int gridCnt) throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
         Affinity<String> aff = affinity(grid(0).<String, Integer>cache(null));
 
         Collection<ClusterNode> affNodes = aff.mapKeyToPrimaryAndBackups(CNTR_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37fca3ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index df29a5a..814ad06 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -327,6 +327,9 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
      */
     @SuppressWarnings("unchecked")
     public void testNearDhtKeySize() throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
         List<String> keys = new ArrayList<>(5);
 
         info("Generating keys for test...");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37fca3ba/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index fc7cc88..2a48fe6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -909,8 +909,15 @@ public abstract class GridAbstractTest extends TestCase {
     protected final Ignite grid(ClusterNode node) {
         if (!isMultiJvm())
             return G.ignite(node.id());
-        else
-            return IgniteProcessProxy.ignite(node.id());
+        else {
+            try {
+                return IgniteProcessProxy.ignite(node.id());
+            }
+            catch (Exception ignore) {
+                // A hack if it is local grid.
+                return G.ignite(node.id());
+            }
+        }
     }
 
     /**


[7/8] incubator-ignite git commit: # ignite-648: fix tests

Posted by sb...@apache.org.
# ignite-648: fix tests


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

Branch: refs/heads/ignite-648-failover
Commit: f53094f454ef33b584969fc911d2cdc4e9ebc6a0
Parents: 37fca3b
Author: ashutak <as...@gridgain.com>
Authored: Thu Jul 2 20:27:24 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu Jul 2 20:27:24 2015 +0300

----------------------------------------------------------------------
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |  3 ++
 .../near/GridCacheNearTxMultiNodeSelfTest.java  | 13 +++---
 .../testframework/junits/GridAbstractTest.java  | 44 ++++++++++++--------
 .../junits/multijvm/IgniteProcessProxy.java     | 18 ++++++++
 4 files changed, 56 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f53094f4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java
index ac93adb..20146b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java
@@ -114,6 +114,9 @@ public class GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest extends GridCacheNe
 
     /** {@inheritDoc} */
     @Override public void testEvictExpired() throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
         IgniteCache<String, Integer> cache = jcache();
 
         String key = primaryKeysForCache(cache, 1).get(0);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f53094f4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
index ef0ab48..e39a2ef 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
@@ -88,6 +88,9 @@ public class GridCacheNearTxMultiNodeSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings( {"unchecked"})
     public void testTxCleanup() throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
         backups = 1;
 
         Ignite ignite = startGrids(GRID_CNT);
@@ -104,9 +107,9 @@ public class GridCacheNearTxMultiNodeSelfTest extends GridCommonAbstractTest {
             assert backupNode != otherNode;
             assert priNode != otherNode;
 
-            Ignite priIgnite = G.ignite(priNode.id());
-            Ignite backupIgnite = G.ignite(backupNode.id());
-            Ignite otherIgnite = G.ignite(otherNode.id());
+            final Ignite priIgnite = grid(priNode);
+            Ignite backupIgnite = grid(backupNode);
+            Ignite otherIgnite = grid(otherNode);
 
             List<Ignite> ignites = F.asList(otherIgnite, priIgnite, backupIgnite);
 
@@ -152,8 +155,8 @@ public class GridCacheNearTxMultiNodeSelfTest extends GridCommonAbstractTest {
                 tx.close();
             }
 
-            G.stop(priIgnite.name(), true);
-            G.stop(backupIgnite.name(), true);
+            stopGrid(priIgnite.name(), true);
+            stopGrid(backupIgnite.name(), true);
 
             Ignite newIgnite = startGrid(GRID_CNT);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f53094f4/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 2a48fe6..732af54 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -572,6 +572,9 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected final Ignite startGridsMultiThreaded(int init, int cnt) throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
         assert init >= 0;
         assert cnt > 0;
 
@@ -746,13 +749,16 @@ public abstract class GridAbstractTest extends TestCase {
     @SuppressWarnings({"deprecation"})
     protected void stopGrid(@Nullable String gridName, boolean cancel) {
         try {
-            Ignite ignite = G.ignite(gridName);
+            Ignite ignite = grid(gridName);
 
             assert ignite != null : "Ignite returned null grid for name: " + gridName;
 
             info(">>> Stopping grid [name=" + ignite.name() + ", id=" + ignite.cluster().localNode().id() + ']');
 
-            G.stop(gridName, cancel);
+            if (!isMultiJvmAndNodeIsRemote(gridName))
+                G.stop(gridName, cancel);
+            else
+                IgniteProcessProxy.stop(gridName, cancel);
         }
         catch (IllegalStateException ignored) {
             // Ignore error if grid already stopped.
@@ -775,25 +781,29 @@ public abstract class GridAbstractTest extends TestCase {
      * @param cancel Cancel flag.
      */
     protected void stopAllGrids(boolean cancel) {
-        IgniteProcessProxy.killAll(); // In multi jvm case.
-
-        Collection<Ignite> clients = new ArrayList<>();
-        Collection<Ignite> srvs = new ArrayList<>();
+        try {
+            Collection<Ignite> clients = new ArrayList<>();
+            Collection<Ignite> srvs = new ArrayList<>();
+
+            for (Ignite g : G.allGrids()) {
+                if (g.configuration().getDiscoverySpi().isClientMode())
+                    clients.add(g);
+                else
+                    srvs.add(g);
+            }
 
-        for (Ignite g : G.allGrids()) {
-            if (g.configuration().getDiscoverySpi().isClientMode())
-                clients.add(g);
-            else
-                srvs.add(g);
-        }
+            for (Ignite g : clients)
+                stopGrid(g.name(), cancel);
 
-        for (Ignite g : clients)
-            stopGrid(g.name(), cancel);
+            for (Ignite g : srvs)
+                stopGrid(g.name(), cancel);
 
-        for (Ignite g : srvs)
-            stopGrid(g.name(), cancel);
+            assert G.allGrids().isEmpty();
+        }
+        finally {
+            IgniteProcessProxy.killAll(); // In multi jvm case.
+        }
 
-        assert G.allGrids().isEmpty();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f53094f4/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 7c1769a..7ac652a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -151,6 +151,24 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /**
+     * @param gridName Grid name.
+     * @param cancel Cacnel flag.
+     */
+    public static void stop(final String gridName, final boolean cancel) {
+        IgniteProcessProxy proxy = gridProxies.get(gridName);
+
+        if (proxy != null) {
+            proxy.remoteCompute().run(new IgniteRunnable() {
+                @Override public void run() {
+                    G.stop(gridName, cancel);
+                }
+            });
+
+            gridProxies.remove(gridName, proxy);
+        }
+    }
+
+    /**
      * For usage in closures.
      *
      * @return Ignite instance.


[3/8] incubator-ignite git commit: # ignite-648: testPeekPartitionedModes

Posted by sb...@apache.org.
# ignite-648: testPeekPartitionedModes


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

Branch: refs/heads/ignite-648-failover
Commit: c52bb1b3772ffd0cbe0e5a5bc2a9b3c7fa8cf490
Parents: 84c7d61
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 19:21:15 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 19:21:15 2015 +0300

----------------------------------------------------------------------
 ...GridCachePartitionedMultiNodeFullApiSelfTest.java | 15 +++++++++++++--
 1 file changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c52bb1b3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 0232efe..df29a5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -256,8 +256,19 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
 
             boolean nearEnabled = nearEnabled(c);
 
-            if (nearEnabled)
-                assertTrue(((IgniteKernal)ignite(i)).internalCache().context().isNear());
+            if (nearEnabled) {
+                if (!isMultiJvmAndNodeIsRemote(i))
+                    assertTrue(((IgniteKernal)ignite(i)).internalCache().context().isNear());
+                else {
+                    final int iCopy = i;
+
+                    ((IgniteProcessProxy)grid(i)).remoteCompute().run(new IgniteRunnable() {
+                        @Override public void run() {
+                            assertTrue(((IgniteKernal)ignite(iCopy)).internalCache().context().isNear());
+                        }
+                    });
+                }
+            }
 
             Integer nearPeekVal = nearEnabled ? 1 : null;
 


[5/8] incubator-ignite git commit: # ignite-648: disable tests

Posted by sb...@apache.org.
# ignite-648: disable tests


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

Branch: refs/heads/ignite-648-failover
Commit: 44692e32eff1a0d247f59e54379747dee794b55d
Parents: 335a3f9
Author: ashutak <as...@gridgain.com>
Authored: Thu Jul 2 15:29:23 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu Jul 2 15:29:23 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java          |  6 +++---
 .../near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java  | 11 +++++++++++
 .../GridCachePartitionedMultiNodeCounterSelfTest.java    |  3 +++
 3 files changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44692e32/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 7ea73a4..4e31771 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -3187,7 +3187,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @param oldEntry {@code True} to check TTL on old entry, {@code false} on new.
      * @throws Exception If failed.
      */
-    private void checkTtl(boolean inTx, boolean oldEntry) throws Exception {
+    protected void checkTtl(boolean inTx, boolean oldEntry) throws Exception {
         if (memoryMode() == OFFHEAP_TIERED)
             return;
 
@@ -4362,8 +4362,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testLocalClearKeys() throws Exception {
-        if (isMultiJvm()) // TODO: IGNITE-648, test should be fixed and enabled.
-            return;
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
 
         Map<String, List<String>> keys = addKeys();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44692e32/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
index d40e9e3..60168e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java
@@ -137,6 +137,14 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
         return nearIdx == 0 ? jcache(1) : jcache(0);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void checkTtl(boolean inTx, boolean oldEntry) throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
+        super.checkTtl(inTx, oldEntry);
+    }
+
     /**
      * @return For the purpose of this test returns the near-only instance.
      */
@@ -211,6 +219,9 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
      * @throws Exception If failed.
      */
     private void checkReaderTtl(boolean inTx) throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
         int ttl = 1000;
 
         final ExpiryPolicy expiry = new TouchedExpiryPolicy(new Duration(TimeUnit.MILLISECONDS, ttl));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44692e32/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
index fd735c3..5d00e1e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
@@ -225,6 +225,9 @@ public class GridCachePartitionedMultiNodeCounterSelfTest extends GridCommonAbst
      * @throws Exception If failed.
      */
     private void checkNearAndPrimary(int gridCnt, int priThreads, int nearThreads) throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-648");
+
         assert gridCnt > 0;
         assert priThreads >= 0;
         assert nearThreads >= 0;


[8/8] incubator-ignite git commit: Merge branch 'ignite-648' into ignite-648-failover

Posted by sb...@apache.org.
Merge branch 'ignite-648' into ignite-648-failover

Conflicts:
	modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java


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

Branch: refs/heads/ignite-648-failover
Commit: a3f9dc17d16a6d2bc7bb9e8a708b55ce2f534b66
Parents: d0b8b0d f53094f
Author: ashutak <as...@gridgain.com>
Authored: Thu Jul 2 20:40:38 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Thu Jul 2 20:40:38 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java |  5 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |  3 ++
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java | 13 ++++-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  | 13 +++--
 ...achePartitionedMultiNodeCounterSelfTest.java |  6 +++
 ...achePartitionedMultiNodeFullApiSelfTest.java | 18 ++++++-
 .../testframework/junits/GridAbstractTest.java  | 55 +++++++++++++-------
 .../junits/common/GridCommonAbstractTest.java   |  4 ++
 .../multijvm/IgniteCacheProcessProxy.java       |  8 ++-
 .../junits/multijvm/IgniteProcessProxy.java     | 18 +++++++
 10 files changed, 113 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a3f9dc17/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index d05c859,732af54..cd94bb9
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@@ -775,26 -781,29 +781,30 @@@ public abstract class GridAbstractTest 
       * @param cancel Cancel flag.
       */
      protected void stopAllGrids(boolean cancel) {
-         Collection<Ignite> clients = new ArrayList<>();
-         Collection<Ignite> srvs = new ArrayList<>();
- 
-         for (Ignite g : G.allGrids()) {
-             if (g.configuration().getDiscoverySpi().isClientMode())
-                 clients.add(g);
-             else
-                 srvs.add(g);
-         }
+         try {
+             Collection<Ignite> clients = new ArrayList<>();
+             Collection<Ignite> srvs = new ArrayList<>();
+ 
+             for (Ignite g : G.allGrids()) {
+                 if (g.configuration().getDiscoverySpi().isClientMode())
+                     clients.add(g);
+                 else
+                     srvs.add(g);
+             }
  
-         for (Ignite g : clients)
-             stopGrid(g.name(), cancel);
+             for (Ignite g : clients)
+                 stopGrid(g.name(), cancel);
  
-         for (Ignite g : srvs)
-             stopGrid(g.name(), cancel);
+             for (Ignite g : srvs)
+                 stopGrid(g.name(), cancel);
  
-         if (isMultiJvm())
+             assert G.allGrids().isEmpty();
+         }
+         finally {
+             IgniteProcessProxy.killAll(); // In multi jvm case.
 +            IgniteNodeRunner.killAll();
+         }
  
-         assert G.allGrids().isEmpty();
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a3f9dc17/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------