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/24 17:19:19 UTC

incubator-ignite git commit: # ignite-648: self-review

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648 a7ad0eaff -> 03aea8c63


# ignite-648: self-review


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

Branch: refs/heads/ignite-648
Commit: 03aea8c631219c911ccc4c5242e9f48f566a77db
Parents: a7ad0ea
Author: ashutak <as...@gridgain.com>
Authored: Wed Jun 24 18:19:45 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jun 24 18:19:45 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java | 95 +++++++++++++------
 .../cache/GridCacheAbstractSelfTest.java        | 14 +--
 .../testframework/junits/GridAbstractTest.java  | 39 +++++---
 .../junits/multijvm/AffinityProcessProxy.java   |  1 +
 .../multijvm/IgniteCacheProcessProxy.java       | 31 +++---
 .../multijvm/IgniteEventsProcessProxy.java      | 33 +++----
 .../junits/multijvm/IgniteNodeRunner.java       | 15 +--
 .../junits/multijvm/IgniteProcessProxy.java     | 99 ++++++++++----------
 8 files changed, 184 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/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 bb2b03d..b73a209 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
@@ -36,6 +36,7 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.swapspace.inmemory.*;
 import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.multijvm.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
@@ -61,6 +62,7 @@ import static org.apache.ignite.transactions.TransactionState.*;
 /**
  * Full API cache test.
  */
+@SuppressWarnings("TransientFieldInNonSerializableClass")
 public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstractSelfTest {
     /** Increment processor for invoke operations. */
     public static final EntryProcessor<String, Integer, String> INCR_PROCESSOR = new EntryProcessor<String, Integer, String>() {
@@ -152,7 +154,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /** {@inheritDoc} */
-    // TODO review. See IgnitionEx.grid(entry.getKey()).
     @Override protected void beforeTestsStarted() throws Exception {
         if (cacheStartType() == CacheStartMode.STATIC)
             super.beforeTestsStarted();
@@ -163,9 +164,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 super.beforeTestsStarted();
 
                 for (Map.Entry<String, CacheConfiguration[]> entry : cacheCfgMap.entrySet()) {
-                    Ignite ignite;
-
-                    ignite = IgnitionEx.grid(entry.getKey());
+                    Ignite ignite = grid(entry.getKey());
 
                     for (CacheConfiguration cfg : entry.getValue())
                         ignite.createCache(cfg);
@@ -200,20 +199,19 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
     /** {@inheritDoc} */
     @Override protected Ignite startGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
-        if (!isMultiJvmAndNodeIsRemote(gridName)) {
-            if (cacheCfgMap == null)
-                return super.startGrid(gridName, ctx);
+        if (cacheCfgMap == null)
+            return super.startGrid(gridName, ctx);
 
-            IgniteConfiguration cfg = getConfiguration(gridName);
+        IgniteConfiguration cfg = getConfiguration(gridName);
 
-            cacheCfgMap.put(gridName, cfg.getCacheConfiguration());
+        cacheCfgMap.put(gridName, cfg.getCacheConfiguration());
 
-            cfg.setCacheConfiguration();
+        cfg.setCacheConfiguration();
 
+        if (!isMultiJvmAndNodeIsRemote(gridName))
             return IgnitionEx.start(optimize(cfg), ctx);
-        }
-
-        return startRemoteGrid(gridName, ctx);
+        else
+            return startRemoteGrid(gridName, optimize(cfg), ctx);
     }
 
     /** {@inheritDoc} */
@@ -258,7 +256,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         int size = 10;
 
-        Map<String, Integer> map = new HashMap<>();
+        final Map<String, Integer> map = new HashMap<>();
 
         for (int i = 0; i < size; i++)
             map.put("key" + i, i);
@@ -303,7 +301,25 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 assertEquals("Incorrect key size on cache #" + i, sum, jcache(i).localSize(ALL));
             }
             else {
-                // TODO add multi jvm support.
+                final int finalI = i;
+                final UUID id = grid(i).localNode().id();
+
+                ((IgniteProcessProxy)grid(i)).remoteCompute().run(new IgniteRunnable() {
+                    @Override public void run() {
+                        Ignite grid = Ignition.ignite(id);
+
+                        GridCacheContext<String, Integer> ctx =
+                            ((IgniteKernal)grid).<String, Integer>internalCache().context();
+
+                        int sum = 0;
+
+                        for (String key : map.keySet())
+                            if (ctx.affinity().localNode(key, new AffinityTopologyVersion(ctx.discovery().topologyVersion())))
+                                sum++;
+
+                        assertEquals("Incorrect key size on cache #" + finalI, sum, grid.cache(null).localSize(ALL));
+                    }
+                });
             }
         }
 
@@ -2326,7 +2342,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testRemoveLoad() throws Exception {
-        int cnt = 10;
+        final int cnt = 10;
 
         Set<String> keys = new HashSet<>();
 
@@ -2480,9 +2496,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
             asyncCache0.removeAll();
 
-            Thread.sleep(1000);
-
-            asyncCache0.future().get(); // --
+            asyncCache0.future().get();
         }
         else
             jcache(gridCount() > 1 ? 1 : 0).removeAll();
@@ -2983,16 +2997,14 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testPeekTxRemoveOptimistic() throws Exception {
-        if (!isMultiJvm()) // Transactions are not supported in multi JVM mode.
-            checkPeekTxRemove(OPTIMISTIC);
+        checkPeekTxRemove(OPTIMISTIC);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPeekTxRemovePessimistic() throws Exception {
-        if (!isMultiJvm()) // Transactions are not supported in multi JVM mode.
-            checkPeekTxRemove(PESSIMISTIC);
+        checkPeekTxRemove(PESSIMISTIC);
     }
 
     /**
@@ -3134,7 +3146,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testTtlTx() throws Exception {
-        if (txShouldBeUsed() && !isMultiJvm())
+        if (txShouldBeUsed())
             checkTtl(true, false);
     }
 
@@ -3841,7 +3853,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @param keys Expected keys.
      * @throws Exception If failed.
      */
-    protected void checkSize(Collection<String> keys) throws Exception {
+    protected void checkSize(final Collection<String> keys) throws Exception {
         if (memoryMode() == OFFHEAP_TIERED)
             return;
 
@@ -3872,7 +3884,36 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                     assertEquals("Incorrect size on cache #" + i, size, jcache(i).localSize(ALL));
                 }
                 else {
-                    // TODO add multi jvm support.
+                    final UUID id = grid(i).localNode().id();
+
+                    final int finalI = i;
+                    ((IgniteProcessProxy)grid(i)).remoteCompute().run(new IgniteRunnable() {
+                        @Override public void run() {
+                            Ignite grid = Ignition.ignite(id);
+
+                            GridCacheContext<String, Integer> ctx =
+                                ((IgniteKernal)grid).<String, Integer>internalCache().context();
+
+                            if (ctx.cache().configuration().getMemoryMode() == OFFHEAP_TIERED)
+                                return;
+
+                            int size = 0;
+
+                            for (String key : keys) {
+                                if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx())) {
+                                    GridCacheEntryEx e =
+                                        ctx.isNear() ? ctx.near().dht().peekEx(key) : ctx.cache().peekEx(key);
+
+                                    assert e != null : "Entry is null [idx=" + finalI + ", key=" + key + ", ctx=" + ctx + ']';
+                                    assert !e.deleted() : "Entry is deleted: " + e;
+
+                                    size++;
+                                }
+                            }
+
+                            assertEquals("Incorrect size on cache #" + finalI, size, grid.cache(null).localSize(ALL));
+                        }
+                    });
                 }
             }
         }
@@ -3935,7 +3976,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         UUID nodeId = node.id();
 
         for (int i = 0; i < gridCount(); i++) {
-            if (context(i).localNodeId().equals(nodeId))
+            if (grid(i).localNode().id().equals(nodeId))
                 return ignite(i);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/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 8807348..2a84be9 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
@@ -371,18 +371,8 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     protected GridCacheContext<String, Integer> context(final int idx) {
         if (!isMultiJvmAndNodeIsRemote(idx))
             return ((IgniteKernal)grid(idx)).<String, Integer>internalCache().context();
-        else {
-//            ((IgniteProcessProxy)grid(idx)).remoteInternalCache();
-
-            // TODO refix it.
-            final UUID id = ((IgniteProcessProxy)grid(idx)).getId();
-
-            return new GridCacheContext<String, Integer>() {
-                @Override public UUID localNodeId() {
-                    return id;
-                }
-            };
-        }
+        else
+            throw new UnsupportedOperationException("Operation cant be supported for multi jvm mode.");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/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 f329d98..8802362 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
@@ -484,13 +484,17 @@ public abstract class GridAbstractTest extends TestCase {
             }
 
             try {
-                if (isMultiJvm() && gridCount() > 0)
+                if (isMultiJvm()) {
+                    if (gridCount() < 2)
+                        throw new IllegalStateException("Grid count have to be more 1 in milti jvm mode.");
+
                     allNodesJoinLatch = new CountDownLatch(gridCount() - 1);
+                }
 
                 beforeTestsStarted();
 
-                if (isMultiJvm() && gridCount() > 0)
-                    assert allNodesJoinLatch.await(5, TimeUnit.SECONDS);
+                if (isMultiJvm())
+                    assert allNodesJoinLatch.await(20, TimeUnit.SECONDS);
             }
             catch (Exception | Error t) {
                 t.printStackTrace();
@@ -694,7 +698,7 @@ public abstract class GridAbstractTest extends TestCase {
             }
         }
         else
-            return startRemoteGrid(gridName, ctx);
+            return startRemoteGrid(gridName, null, ctx);
     }
 
     /**
@@ -705,11 +709,15 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Started grid.
      * @throws Exception If failed.
      */
-    protected Ignite startRemoteGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
+    protected Ignite startRemoteGrid(String gridName, IgniteConfiguration cfg, GridSpringResourceContext ctx)
+        throws Exception {
         if (ctx != null)
             throw new UnsupportedOperationException("Starting of grid at another jvm by context doesn't supported.");
 
-        return new IgniteProcessProxy(optimize(getConfiguration(gridName)), log, grid(0));
+        if (cfg == null)
+            cfg = optimize(getConfiguration(gridName));
+
+        return new IgniteProcessProxy(cfg, log, grid(0));
     }
 
     /**
@@ -906,18 +914,22 @@ public abstract class GridAbstractTest extends TestCase {
      *
      * @return Grid for given test.
      */
-    // TODO isMultyJvm.
     protected IgniteEx grid() {
-        return (IgniteEx)G.ignite(getTestGridName());
+        if (!isMultiJvm())
+            return (IgniteEx)G.ignite(getTestGridName());
+        else
+            throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /**
      * @param node Node.
      * @return Ignite instance with given local node.
      */
-    // TODO isMultyJvm.
     protected final Ignite grid(ClusterNode node) {
-        return G.ignite(node.id());
+        if (!isMultiJvm())
+            return G.ignite(node.id());
+        else
+            throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /**
@@ -930,7 +942,6 @@ 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));
     }
@@ -945,11 +956,13 @@ 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);
 
-        return G.start(cfg);
+        if (!isMultiJvmAndNodeIsRemote(gridName))
+            return G.start(cfg);
+        else
+            return startRemoteGrid(gridName, cfg, null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
index 2b699b6..1c816f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
@@ -28,6 +28,7 @@ import java.util.*;
 /**
  * Proxy class for affinity at another jvm.
  */
+@SuppressWarnings("TransientFieldInNonSerializableClass")
 public class AffinityProcessProxy<K> implements Affinity<K> {
     /** Compute. */
     private final transient IgniteCompute compute;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/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 d5ef32b..a478db0 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
@@ -38,6 +38,7 @@ import java.util.concurrent.locks.*;
 /**
  * Ignite cache proxy for ignite instance at another JVM.
  */
+@SuppressWarnings("TransientFieldInNonSerializableClass")
 public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     /** Compute. */
     private final transient IgniteCompute compute;
@@ -75,6 +76,20 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
         compute = proxy.remoteCompute();
     }
 
+    /**
+     * Returns cache instance. Method to be called from closure at another JVM.
+     *
+     * @return Cache.
+     */
+    private IgniteCache<Object, Object> cache() {
+        IgniteCache cache = Ignition.ignite(gridId).cache(cacheName);
+
+        if (isAsync)
+            cache = cache.withAsync();
+
+        return cache;
+    }
+
     /** {@inheritDoc} */
     @Override public IgniteCache<K, V> withAsync() {
         return new IgniteCacheProcessProxy<>(cacheName, true, igniteProxy);
@@ -238,20 +253,6 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
         });
     }
 
-    /**
-     * Returns cache instance. Method to be called from closure at another JVM.
-     *
-     * @return Cache.
-     */
-    private IgniteCache<Object, Object> cache() {
-        IgniteCache cache = Ignition.ignite(gridId).cache(cacheName);
-
-        if (isAsync)
-            cache = cache.withAsync();
-
-        return cache;
-    }
-
     /** {@inheritDoc} */
     @Override public Map<K, V> getAll(final Set<? extends K> keys) {
         return (Map<K, V>)compute.call(new IgniteCallable<Object>() {
@@ -399,7 +400,7 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
 
                 cache.removeAll();
 
-                if(isAsync)
+                if (isAsync)
                     cache.future().get();
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
index 6f9a0cb..eb09965 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
@@ -28,6 +28,7 @@ import java.util.*;
 /**
  * Ignite events proxy for ignite instance at another JVM.
  */
+@SuppressWarnings("TransientFieldInNonSerializableClass")
 public class IgniteEventsProcessProxy implements IgniteEvents {
     /** Ignite proxy. */
     private final transient IgniteProcessProxy igniteProxy;
@@ -53,47 +54,47 @@ public class IgniteEventsProcessProxy implements IgniteEvents {
 
     /** {@inheritDoc} */
     @Override public ClusterGroup clusterGroup() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> List<T> remoteQuery(IgnitePredicate<T> p, long timeout,
         @Nullable int... types) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> UUID remoteListen(@Nullable IgniteBiPredicate<UUID, T> locLsnr,
         @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> UUID remoteListen(int bufSize, long interval, boolean autoUnsubscribe,
         @Nullable IgniteBiPredicate<UUID, T> locLsnr, @Nullable IgnitePredicate<T> rmtFilter,
         @Nullable int... types) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void stopRemoteListen(UUID opId) throws IgniteException {
-        // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> T waitForLocal(@Nullable IgnitePredicate<T> filter,
         @Nullable int... types) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> Collection<T> localQuery(IgnitePredicate<T> p, @Nullable int... types) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void recordLocal(Event evt) {
-        // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -107,41 +108,41 @@ public class IgniteEventsProcessProxy implements IgniteEvents {
 
     /** {@inheritDoc} */
     @Override public boolean stopLocalListen(IgnitePredicate<? extends Event> lsnr, @Nullable int... types) {
-        return false; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void enableLocal(int... types) {
-        // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void disableLocal(int... types) {
-        // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public int[] enabledEvents() {
-        return new int[0]; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isEnabled(int type) {
-        return false; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteEvents withAsync() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isAsync() {
-        return false; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <R> IgniteFuture<R> future() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
index 677d383..0dec37b 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
@@ -45,20 +45,13 @@ public class IgniteNodeRunner {
      * @throws Exception If failed.
      */
     public static void main(String[] args) throws Exception {
-        try {
-            X.println(GridJavaProcess.PID_MSG_PREFIX + U.jvmPid());
+        X.println(GridJavaProcess.PID_MSG_PREFIX + U.jvmPid());
 
-            X.println("Starting Ignite Node... Args=" + Arrays.toString(args));
+        X.println("Starting Ignite Node... Args=" + Arrays.toString(args));
 
-            IgniteConfiguration cfg = readCfgFromFileAndDeleteFile(args[0]);
+        IgniteConfiguration cfg = readCfgFromFileAndDeleteFile(args[0]);
 
-            Ignition.start(cfg);
-        }
-        catch (Throwable e) {
-            e.printStackTrace();
-
-            System.exit(1);
-        }
+        Ignition.start(cfg);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03aea8c6/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 96026c4..51b910a 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
@@ -39,27 +39,28 @@ import java.util.concurrent.*;
 /**
  * Ignite proxy for ignite instance at another JVM.
  */
+@SuppressWarnings("TransientFieldInNonSerializableClass")
 public class IgniteProcessProxy implements IgniteEx {
     /** Grid proxies. */
-    private transient static final Map<String, IgniteProcessProxy> gridProxies = new HashMap<>();
+    private static final transient Map<String, IgniteProcessProxy> gridProxies = new HashMap<>();
 
     /** Jvm process with ignite instance. */
-    private transient final GridJavaProcess proc;
+    private final transient GridJavaProcess proc;
 
     /** Configuration. */
-    private transient final IgniteConfiguration cfg;
+    private final transient IgniteConfiguration cfg;
 
     /** Local jvm grid. */
-    private transient final Ignite locJvmGrid;
+    private final transient Ignite locJvmGrid;
 
     /** Logger. */
-    private transient final IgniteLogger log;
+    private final transient IgniteLogger log;
 
     /** Grid id. */
     private final UUID id = UUID.randomUUID();
 
     /** Remote ignite instance started latch. */
-    private transient final CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
+    private final transient CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
 
     /**
      * @param cfg Configuration.
@@ -176,53 +177,53 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex(@Nullable String name) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteInternalCache<?, ?>> cachesx(
         @Nullable IgnitePredicate<? super IgniteInternalCache<?, ?>>... p) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean eventUserRecordable(int type) {
-        return false; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean allEventsUserRecordable(int[] types) {
-        return false; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isJmxRemoteEnabled() {
-        return false; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isRestartEnabled() {
-        return false; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public IgniteFileSystem igfsx(@Nullable String name) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Hadoop hadoop() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -232,7 +233,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Nullable @Override public String latestVersion() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -246,27 +247,27 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public GridKernalContext context() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCompute compute() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCompute compute(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteMessaging message() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteMessaging message(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -276,85 +277,85 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public IgniteEvents events(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteServices services() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteServices services(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public ExecutorService executorService() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public ExecutorService executorService(ClusterGroup grp) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteProductVersion version() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteScheduler scheduler() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
-        // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName,
         NearCacheConfiguration<K, V> nearCfg) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -374,61 +375,61 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFileSystem fileSystem(String name) {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFileSystem> fileSystems() {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteAtomicReference<T> atomicReference(String name, @Nullable T initVal,
         boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public <T, S> IgniteAtomicStamped<T, S> atomicStamped(String name, @Nullable T initVal, @Nullable S initStamp,
         boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
         boolean create) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteQueue<T> queue(String name, int cap,
         @Nullable CollectionConfiguration cfg) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteSet<T> set(String name, @Nullable CollectionConfiguration cfg) throws IgniteException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException {
-        return null; // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation doesn't supported yet.");
     }
 
     /** {@inheritDoc} */