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/22 19:02:20 UTC

[2/6] incubator-ignite git commit: # ignite-648: get vs grid methods (fix some tests automatically)

# ignite-648: get vs grid methods (fix some tests automatically)


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

Branch: refs/heads/ignite-648
Commit: 9ed8a4e627256a7add543326b24b42ae4b39050f
Parents: 2e8c0fe
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 15:57:27 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 15:57:27 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java | 73 ++----------------
 .../cache/GridCacheAbstractSelfTest.java        | 13 ++--
 .../multijvm/framework/IgniteProcessProxy.java  | 25 +++----
 .../testframework/junits/GridAbstractTest.java  | 78 +++++++++++++++++++-
 4 files changed, 96 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/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 9314b70..292773c 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
@@ -27,7 +27,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.affinity.*;
-import org.apache.ignite.internal.processors.cache.multijvm.framework.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -35,8 +34,6 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.swapspace.inmemory.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
@@ -65,11 +62,6 @@ import static org.apache.ignite.transactions.TransactionState.*;
  * Full API cache test.
  */
 public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstractSelfTest {
-    /** Ip finder for TCP discovery. */
-    public static final TcpDiscoveryIpFinder LOCAL_IP_FINDER = new TcpDiscoveryVmIpFinder(false) {{
-        setAddresses(Collections.singleton("127.0.0.1:47500..47509"));
-    }};
-
     /** Increment processor for invoke operations. */
     public static final EntryProcessor<String, Integer, String> INCR_PROCESSOR = new EntryProcessor<String, Integer, String>() {
         @Override public String process(MutableEntry<String, Integer> e, Object... args) {
@@ -118,18 +110,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /** */
     private Map<String, CacheConfiguration[]> cacheCfgMap;
 
-    /** All nodes join latch (for multi JVM mode). */
-    private CountDownLatch allNodesJoinLatch;
-
-    /** Node join listener (for multi JVM mode). */
-    private final IgnitePredicate<Event> nodeJoinLsnr = new IgnitePredicate<Event>() {
-        @Override public boolean apply(Event evt) {
-            allNodesJoinLatch.countDown();
-
-            return true;
-        }
-    };
-
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 1;
@@ -156,14 +136,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         if (memoryMode() == OFFHEAP_TIERED || memoryMode() == OFFHEAP_VALUES)
             cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 
-        if (isMultiJvm()) {
-            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(LOCAL_IP_FINDER);
-
-            cfg.setLocalEventListeners(new HashMap<IgnitePredicate<? extends Event>, int[]>() {{
-                put(nodeJoinLsnr, new int[] {EventType.EVT_NODE_JOINED});
-            }});
-        }
-
         return cfg;
     }
 
@@ -180,10 +152,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /** {@inheritDoc} */
+    // TODO review. See IgnitionEx.grid(entry.getKey()).
     @Override protected void beforeTestsStarted() throws Exception {
-        if (isMultiJvm())
-            allNodesJoinLatch = new CountDownLatch(gridCount() - 1);
-
         if (cacheStartType() == CacheStartMode.STATIC)
             super.beforeTestsStarted();
         else {
@@ -194,11 +164,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
                 for (Map.Entry<String, CacheConfiguration[]> entry : cacheCfgMap.entrySet()) {
                     Ignite ignite;
-
-                    if (isMultiJvm())
-                        ignite = IgniteProcessProxy.grid(entry.getKey());
-                    else
-                        ignite = IgnitionEx.grid(entry.getKey());
+                    
+                    ignite = IgnitionEx.grid(entry.getKey());
 
                     for (CacheConfiguration cfg : entry.getValue())
                         ignite.createCache(cfg);
@@ -229,21 +196,11 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         for (int i = 0; i < gridCount(); i++)
             info("Grid " + i + ": " + grid(i).localNode().id());
-
-        if (isMultiJvm())
-            assert allNodesJoinLatch.await(5, TimeUnit.SECONDS);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        IgniteProcessProxy.killAll();
-
-        super.afterTestsStopped();
     }
 
     /** {@inheritDoc} */
     @Override protected Ignite startGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
-        if (!isMultiJvm() || gridName.endsWith("0")) {
+        if (!isMultiJvmAndNodeIsRemote(gridName)) {
             if (cacheCfgMap == null)
                 return super.startGrid(gridName, ctx);
 
@@ -256,27 +213,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             return IgnitionEx.start(optimize(cfg), ctx);
         }
 
-        return new IgniteProcessProxy(optimize(getConfiguration(gridName)), log, grid(0));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteEx grid(int idx) {
-        if (!isMultiJvm() || idx == 0)
-            return super.grid(idx);
-
-        return IgniteProcessProxy.get(getTestGridName(idx));
-    }
-
-    /**
-     * @param idx Index of grid.
-     * @return Default cache.
-     */
-    @SuppressWarnings({"unchecked"})
-    @Override protected IgniteCache<String, Integer> jcache(int idx) {
-        if (!isMultiJvm() || idx == 0)
-            return super.jcache(idx);
-
-        return IgniteProcessProxy.get(getTestGridName(idx)).cache(null);
+        return startRemoteGrid(gridName, ctx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index b6e56d5..9168c67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -60,11 +60,6 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     /** VM ip finder for TCP discovery. */
     protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
-    /**
-     * @return Grids count to start.
-     */
-    protected abstract int gridCount();
-
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
         return TEST_TIMEOUT;
@@ -354,7 +349,11 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"unchecked"})
     @Override protected IgniteCache<String, Integer> jcache(int idx) {
-        return ignite(idx).cache(null);
+        if (!isMultiJvmAndNodeIsRemote(idx))
+            return ignite(idx).cache(null);
+        else
+            return IgniteProcessProxy.get(getTestGridName(idx)).cache(null);
+
     }
 
     /**
@@ -367,7 +366,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
         else {
 //            ((IgniteProcessProxy)grid(idx)).remoteInternalCache();
 
-            // TODO
+            // TODO refix it.
             final UUID id = ((IgniteProcessProxy)grid(idx)).getId();
 
             return new GridCacheContext<String, Integer>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/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 538f316..71c6689 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
@@ -123,10 +123,16 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /**
      * @param gridName Grid name.
-     * @return Instance by name or <code>null</code>.
+     * @return Instance by name or exception wiil be thrown.
      */
     public static IgniteProcessProxy get(String gridName) {
-        return gridProxies.get(gridName);
+        IgniteProcessProxy res = gridProxies.get(gridName);
+
+        if (res == null)
+            throw new IgniteIllegalStateException("Grid instance was not properly started " +
+                "or was already stopped: " + gridName);
+
+        return res;
     }
 
     /**
@@ -447,22 +453,13 @@ public class IgniteProcessProxy implements IgniteEx {
         return null; // TODO: CODE: implement.
     }
 
+    /**
+     * @return Jvm process in which grid node started.
+     */
     public GridJavaProcess getProcess() {
         return proc;
     }
 
-    public static Ignite grid(@Nullable String name) {
-//        IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
-
-        Ignite res = gridProxies.get(name);
-
-        if (res == null)
-            throw new IgniteIllegalStateException("Grid instance was not properly started " +
-                "or was already stopped: " + name);
-
-        return res;
-    }
-
     // TODO delete or use.
 //    public <K, V> GridCacheAdapter<K, V> remoteInternalCache() {
 //        return (GridCacheAdapter<K, V>)compute.call(new MyCallable(id));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ed8a4e6/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 fb5ca51..990ea76 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
@@ -23,6 +23,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.multijvm.framework.*;
 import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -33,7 +34,9 @@ import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.spi.checkpoint.sharedfs.*;
 import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.config.*;
 import org.apache.ignite.testframework.junits.logger.*;
@@ -72,6 +75,11 @@ public abstract class GridAbstractTest extends TestCase {
     /** Null name for execution map. */
     private static final String NULL_NAME = UUID.randomUUID().toString();
 
+    /** Ip finder for TCP discovery. */
+    public static final TcpDiscoveryIpFinder LOCAL_IP_FINDER = new TcpDiscoveryVmIpFinder(false) {{
+        setAddresses(Collections.singleton("127.0.0.1:47500..47509"));
+    }};
+
     /** */
     private static final long DFLT_TEST_TIMEOUT = 5 * 60 * 1000;
 
@@ -96,6 +104,18 @@ public abstract class GridAbstractTest extends TestCase {
     /** Starting grid name. */
     protected static ThreadLocal<String> startingGrid = new ThreadLocal<>();
 
+    /** All nodes join latch (for multi JVM mode). */
+    private CountDownLatch allNodesJoinLatch;
+
+    /** Node join listener (for multi JVM mode). */
+    private final IgnitePredicate<Event> nodeJoinLsnr = new IgnitePredicate<Event>() {
+        @Override public boolean apply(Event evt) {
+            allNodesJoinLatch.countDown();
+
+            return true;
+        }
+    };
+
     /**
      *
      */
@@ -421,7 +441,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected void afterTestsStopped() throws Exception {
-        // No-op.
+        IgniteProcessProxy.killAll();
     }
 
     /** {@inheritDoc} */
@@ -464,7 +484,13 @@ public abstract class GridAbstractTest extends TestCase {
             }
 
             try {
+                if (isMultiJvm() && gridCount() > 0)
+                    allNodesJoinLatch = new CountDownLatch(gridCount() - 1);
+
                 beforeTestsStarted();
+
+                if (isMultiJvm() && gridCount() > 0)
+                    assert allNodesJoinLatch.await(5, TimeUnit.SECONDS);
             }
             catch (Exception | Error t) {
                 t.printStackTrace();
@@ -668,6 +694,19 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * Starts new grid at another JVM with given name.
+     *
+     * @param gridName Grid name.
+     * @param ctx Spring context.
+     * @return Started grid.
+     * @throws Exception If failed.
+     */
+    // TODO review. Is it okey that ctx doesn't used?
+    protected Ignite startRemoteGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
+        return new IgniteProcessProxy(optimize(getConfiguration(gridName)), log, grid(0));
+    }
+
+    /**
      * Optimizes configuration to achieve better test performance.
      *
      * @param cfg Configuration.
@@ -821,7 +860,10 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid instance.
      */
     protected IgniteEx grid(String name) {
-        return (IgniteEx)G.ignite(name);
+        if (!isMultiJvmAndNodeIsRemote(name))
+            return (IgniteEx)G.ignite(name);
+        else
+            return IgniteProcessProxy.get(name);
     }
 
     /**
@@ -831,7 +873,11 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid instance.
      */
     protected IgniteEx grid(int idx) {
-        return (IgniteEx)G.ignite(getTestGridName(idx));
+        if (!isMultiJvmAndNodeIsRemote(idx))
+            return (IgniteEx)G.ignite(getTestGridName(idx));
+        else
+            return IgniteProcessProxy.get(getTestGridName(idx));
+
     }
 
     /**
@@ -839,7 +885,12 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Ignite instance.
      */
     protected Ignite ignite(int idx) {
-        return G.ignite(getTestGridName(idx));
+        String gridName = getTestGridName(idx);
+
+        if (!isMultiJvmAndNodeIsRemote(idx))
+            return G.ignite(gridName);
+        else
+            return IgniteProcessProxy.get(gridName);
     }
 
     /**
@@ -847,6 +898,7 @@ public abstract class GridAbstractTest extends TestCase {
      *
      * @return Grid for given test.
      */
+    // TODO isMultyJvm.
     protected IgniteEx grid() {
         return (IgniteEx)G.ignite(getTestGridName());
     }
@@ -855,6 +907,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @param node Node.
      * @return Ignite instance with given local node.
      */
+    // TODO isMultyJvm.
     protected final Ignite grid(ClusterNode node) {
         return G.ignite(node.id());
     }
@@ -869,6 +922,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid Started grid.
      * @throws Exception If failed.
      */
+    // TODO isMultyJvm.
     protected Ignite startGrid(String gridName, String springCfgPath) throws Exception {
         return startGrid(gridName, loadConfiguration(springCfgPath));
     }
@@ -883,6 +937,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid Started grid.
      * @throws Exception If failed.
      */
+    // TODO isMultyJvm.
     protected Ignite startGrid(String gridName, IgniteConfiguration cfg) throws Exception {
         cfg.setGridName(gridName);
 
@@ -1015,6 +1070,14 @@ public abstract class GridAbstractTest extends TestCase {
             cfg.setNodeId(UUID.fromString(new String(chars)));
         }
 
+        if (isMultiJvm()) {
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(LOCAL_IP_FINDER);
+
+            cfg.setLocalEventListeners(new HashMap<IgnitePredicate<? extends Event>, int[]>() {{
+                put(nodeJoinLsnr, new int[] {EventType.EVT_NODE_JOINED});
+            }});
+        }
+
         return cfg;
     }
 
@@ -1286,6 +1349,13 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * @return Grids count to start.
+     */
+    protected int gridCount() {
+        return 0;
+    }
+
+    /**
      * Gets flag whether nodes will run in one jvm or in separate jvms.
      *
      * @return <code>True</code> to run nodes in separate jvms.