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/06/23 15:44:59 UTC

incubator-ignite git commit: # ignite-648: skip part of tests for multi jvm

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648 783a9f388 -> ada57fc79


# ignite-648: skip part of tests for multi jvm


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

Branch: refs/heads/ignite-648
Commit: ada57fc79d425642c89ab7f7a372bf29f3d55570
Parents: 783a9f3
Author: ashutak <as...@gridgain.com>
Authored: Tue Jun 23 16:45:20 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jun 23 16:45:20 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java | 161 +++++++++++--------
 .../framework/IgniteCacheProcessProxy.java      |   3 +
 .../multijvm/framework/IgniteProcessProxy.java  |  14 +-
 3 files changed, 104 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada57fc7/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 db38285..e896b55 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
@@ -371,6 +371,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testRemoveAllSkipStore() throws Exception {
+        if (isMultiJvm())
+            return;
+
         IgniteCache<String, Integer> jcache = jcache();
 
         jcache.putAll(F.asMap("1", 1, "2", 2, "3", 3));
@@ -1741,12 +1744,15 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         assertEquals((Integer)1, cache.getAndPutIfAbsent("key2", 3));
 
         // Check db.
-        putToStore("key3", 3);
+        if (!isMultiJvm()) {
+            putToStore("key3", 3);
 
-        assertEquals((Integer)3, cache.getAndPutIfAbsent("key3", 4));
+            assertEquals((Integer)3, cache.getAndPutIfAbsent("key3", 4));
+
+            assertEquals((Integer)3, cache.get("key3"));
+        }
 
         assertEquals((Integer)1, cache.get("key2"));
-        assertEquals((Integer)3, cache.get("key3"));
 
         cache.localEvict(Collections.singleton("key2"));
 
@@ -1810,11 +1816,13 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         assertEquals((Integer)1, cacheAsync.<Integer>future().get());
 
         // Check db.
-        putToStore("key3", 3);
+        if (!isMultiJvm()) {
+            putToStore("key3", 3);
 
-        cacheAsync.getAndPutIfAbsent("key3", 4);
+            cacheAsync.getAndPutIfAbsent("key3", 4);
 
-        assertEquals((Integer)3, cacheAsync.<Integer>future().get());
+            assertEquals((Integer)3, cacheAsync.<Integer>future().get());
+        }
 
         cache.localEvict(Collections.singleton("key2"));
 
@@ -1857,9 +1865,11 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         assertFalse(cache.putIfAbsent("key2", 3));
 
         // Check db.
-        putToStore("key3", 3);
+        if (!isMultiJvm()) {
+            putToStore("key3", 3);
 
-        assertFalse(cache.putIfAbsent("key3", 4));
+            assertFalse(cache.putIfAbsent("key3", 4));
+        }
 
         cache.localEvict(Collections.singleton("key2"));
 
@@ -1928,11 +1938,13 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         assertFalse(cacheAsync.<Boolean>future().get());
 
         // Check db.
-        putToStore("key3", 3);
+        if (!isMultiJvm()) {
+            putToStore("key3", 3);
 
-        cacheAsync.putIfAbsent("key3", 4);
+            cacheAsync.putIfAbsent("key3", 4);
 
-        assertFalse(cacheAsync.<Boolean>future().get());
+            assertFalse(cacheAsync.<Boolean>future().get());
+        }
 
         cache.localEvict(Arrays.asList("key2"));
 
@@ -1944,9 +1956,11 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
             assertFalse(cacheAsync.<Boolean>future().get());
 
-            cacheAsync.putIfAbsent("key3", 4);
+            if (!isMultiJvm()) {
+                cacheAsync.putIfAbsent("key3", 4);
 
-            assertFalse(cacheAsync.<Boolean>future().get());
+                assertFalse(cacheAsync.<Boolean>future().get());
+            }
 
             if (tx != null)
                 tx.commit();
@@ -1957,7 +1971,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         }
 
         assertEquals((Integer)1, cache.get("key2"));
-        assertEquals((Integer)3, cache.get("key3"));
+
+        if (!isMultiJvm())
+            assertEquals((Integer)3, cache.get("key3"));
     }
 
     /**
@@ -2026,11 +2042,13 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         assert cache.get("key") == 4;
 
-        putToStore("key2", 5);
+        if (!isMultiJvm()) {
+            putToStore("key2", 5);
 
-        info("key2 5 -> 6");
+            info("key2 5 -> 6");
 
-        assert cache.replace("key2", 5, 6);
+            assert cache.replace("key2", 5, 6);
+        }
 
         for (int i = 0; i < gridCount(); i++) {
             info("Peek key on grid [i=" + i + ", nodeId=" + grid(i).localNode().id() +
@@ -2040,7 +2058,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 ", peekVal=" + grid(i).cache(null).localPeek("key2", ONHEAP) + ']');
         }
 
-        assertEquals((Integer)6, cache.get("key2"));
+        if (!isMultiJvm())
+            assertEquals((Integer)6, cache.get("key2"));
 
         cache.localEvict(Collections.singleton("key"));
 
@@ -2082,11 +2101,13 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         assert cache.get("key") == 4;
 
-        putToStore("key2", 5);
+        if (!isMultiJvm()) {
+            putToStore("key2", 5);
 
-        assert cache.replace("key2", 6);
+            assert cache.replace("key2", 6);
 
-        assertEquals((Integer)6, cache.get("key2"));
+            assertEquals((Integer)6, cache.get("key2"));
+        }
 
         cache.localEvict(Collections.singleton("key"));
 
@@ -2156,13 +2177,15 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         assert cache.get("key") == 4;
 
-        putToStore("key2", 5);
+        if (!isMultiJvm()) {
+            putToStore("key2", 5);
 
-        cacheAsync.replace("key2", 5, 6);
+            cacheAsync.replace("key2", 5, 6);
 
-        assert cacheAsync.<Boolean>future().get();
+            assert cacheAsync.<Boolean>future().get();
 
-        assertEquals((Integer)6, cache.get("key2"));
+            assertEquals((Integer)6, cache.get("key2"));
+        }
 
         cache.localEvict(Collections.singleton("key"));
 
@@ -2216,13 +2239,15 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         assert cache.get("key") == 4;
 
-        putToStore("key2", 5);
+        if (!isMultiJvm()) {
+            putToStore("key2", 5);
 
-        cacheAsync.replace("key2", 6);
+            cacheAsync.replace("key2", 6);
 
-        assert cacheAsync.<Boolean>future().get();
+            assert cacheAsync.<Boolean>future().get();
 
-        assert cache.get("key2") == 6;
+            assert cache.get("key2") == 6;
+        }
 
         cache.localEvict(Collections.singleton("key"));
 
@@ -2453,7 +2478,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
             asyncCache0.removeAll();
 
-            asyncCache0.future().get();
+            Thread.sleep(1000);
+
+            asyncCache0.future().get(); // --
         }
         else
             jcache(gridCount() > 1 ? 1 : 0).removeAll();
@@ -3113,9 +3140,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testTtlNoTx() throws Exception {
-        if (isMultiJvm())
-            fail("TODO implement multi jvm support.");
-
         checkTtl(false, false);
     }
 
@@ -3123,9 +3147,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testTtlNoTxOldEntry() throws Exception {
-        if (isMultiJvm())
-            fail("TODO implement multi jvm support.");
-
         checkTtl(false, true);
     }
 
@@ -3203,18 +3224,20 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         for (int i = 0; i < gridCount(); i++) {
             if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key)) {
-                GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
+                if (!isMultiJvmAndNodeIsRemote(i)) {
+                    GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
 
-                if (cache.context().isNear())
-                    cache = cache.context().near().dht();
+                    if (cache.context().isNear())
+                        cache = cache.context().near().dht();
 
-                GridCacheEntryEx curEntry = cache.peekEx(key);
+                    GridCacheEntryEx curEntry = cache.peekEx(key);
 
-                assertEquals(ttl, curEntry.ttl());
+                    assertEquals(ttl, curEntry.ttl());
 
-                assert curEntry.expireTime() > startTime;
+                    assert curEntry.expireTime() > startTime;
 
-                expireTimes[i] = curEntry.expireTime();
+                    expireTimes[i] = curEntry.expireTime();
+                }
             }
         }
 
@@ -3236,18 +3259,20 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         for (int i = 0; i < gridCount(); i++) {
             if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key)) {
-                GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
+                if (!isMultiJvmAndNodeIsRemote(i)) {
+                    GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
 
-                if (cache.context().isNear())
-                    cache = cache.context().near().dht();
+                    if (cache.context().isNear())
+                        cache = cache.context().near().dht();
 
-                GridCacheEntryEx curEntry = cache.peekEx(key);
+                    GridCacheEntryEx curEntry = cache.peekEx(key);
 
-                assertEquals(ttl, curEntry.ttl());
+                    assertEquals(ttl, curEntry.ttl());
 
-                assert curEntry.expireTime() > startTime;
+                    assert curEntry.expireTime() > startTime;
 
-                expireTimes[i] = curEntry.expireTime();
+                    expireTimes[i] = curEntry.expireTime();
+                }
             }
         }
 
@@ -3269,18 +3294,20 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         for (int i = 0; i < gridCount(); i++) {
             if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key)) {
-                GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
+                if (!isMultiJvmAndNodeIsRemote(i)) {
+                    GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
 
-                if (cache.context().isNear())
-                    cache = cache.context().near().dht();
+                    if (cache.context().isNear())
+                        cache = cache.context().near().dht();
 
-                GridCacheEntryEx curEntry = cache.peekEx(key);
+                    GridCacheEntryEx curEntry = cache.peekEx(key);
 
-                assertEquals(ttl, curEntry.ttl());
+                    assertEquals(ttl, curEntry.ttl());
 
-                assert curEntry.expireTime() > startTime;
+                    assert curEntry.expireTime() > startTime;
 
-                expireTimes[i] = curEntry.expireTime();
+                    expireTimes[i] = curEntry.expireTime();
+                }
             }
         }
 
@@ -3306,15 +3333,17 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         for (int i = 0; i < gridCount(); i++) {
             if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key)) {
-                GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
+                if (!isMultiJvmAndNodeIsRemote(i)) {
+                    GridCacheAdapter<String, Integer> cache = internalCache(jcache(i));
 
-                if (cache.context().isNear())
-                    cache = cache.context().near().dht();
+                    if (cache.context().isNear())
+                        cache = cache.context().near().dht();
 
-                GridCacheEntryEx curEntry = cache.peekEx(key);
+                    GridCacheEntryEx curEntry = cache.peekEx(key);
 
-                assertEquals(ttl, curEntry.ttl());
-                assertEquals(expireTimes[i], curEntry.expireTime());
+                    assertEquals(ttl, curEntry.ttl());
+                    assertEquals(expireTimes[i], curEntry.expireTime());
+                }
             }
         }
 
@@ -4025,7 +4054,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         checkIteratorRemove(cache, entries);
 
-        checkIteratorEmpty(cache);
+        if(!isMultiJvm())
+            checkIteratorEmpty(cache);
     }
 
     /**
@@ -4383,6 +4413,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testWithSkipStore() throws Exception {
+        if(isMultiJvm())
+            return;
+
         IgniteCache<String, Integer> cache = grid(0).cache(null);
 
         IgniteCache<String, Integer> cacheSkipStore = cache.withSkipStore();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada57fc7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
index f8cd8a3..27b2262 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteCacheProcessProxy.java
@@ -517,6 +517,9 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
 
     /** {@inheritDoc} */
     @Override public <T> T unwrap(final Class<T> clazz) {
+        if (Ignite.class.equals(clazz))
+            return (T)igniteProxy;
+
         try {
             return (T)compute.call(new IgniteCallable<Object>() {
                 @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada57fc7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
index 367a662..e2751e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/framework/IgniteProcessProxy.java
@@ -28,7 +28,6 @@ import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.*;
@@ -59,9 +58,6 @@ public class IgniteProcessProxy implements IgniteEx {
     /** Grid id. */
     private final UUID id = UUID.randomUUID();
 
-    /** Compute. */
-    private transient final IgniteCompute compute;
-
     /** Remote ignite instance started latch. */
     private transient final CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
 
@@ -117,8 +113,6 @@ public class IgniteProcessProxy implements IgniteEx {
         assert rmtNodeStartedLatch.await(30, TimeUnit.SECONDS): "Remote node with id=" + id + " didn't join.";
 
         gridProxies.put(cfg.getGridName(), this);
-
-        compute = locJvmGrid.compute(locJvmGrid.cluster().forNodeId(id));
     }
 
     /**
@@ -243,11 +237,11 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public ClusterNode localNode() {
-        return F.first(compute.broadcast(new IgniteClosureX<Object, ClusterNode>() {
-            @Override public ClusterNode applyx(Object o) {
+        return remoteCompute().call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
                 return ((IgniteEx)Ignition.ignite(id)).localNode();
             }
-        }, null));
+        });
     }
 
     /** {@inheritDoc} */
@@ -463,7 +457,7 @@ public class IgniteProcessProxy implements IgniteEx {
      * @return {@link IgniteCompute} instance to communicate with remote node.
      */
     public IgniteCompute remoteCompute() {
-        ClusterGroup grp = localJvmGrid().cluster().forNodeId(id);
+        ClusterGroup grp = locJvmGrid.cluster().forNodeId(id);
 
         if (grp.nodes().isEmpty())
             throw new IllegalStateException("Could not found node with id=" + id + ".");